Skip to content
Snippets Groups Projects
Commit 70c8116c authored by nravi's avatar nravi Committed by Reynold Xin
Browse files

Workaround in Spark for ConcurrentModification issue (JIRA Hadoop-10456, Spark-1097)

This fix has gone into Hadoop 2.4.1. For developers using <  2.4.1, it would be good to have a workaround in Spark as well.

Fix has been tested for performance as well, no regressions found.

Author: nravi <nravi@c1704.halxg.cloudera.com>

Closes #1000 from nishkamravi2/master and squashes the following commits:

eb663ca [nravi] Merge branch 'master' of https://github.com/apache/spark
df2aeb1 [nravi] Improved fix for ConcurrentModificationIssue (Spark-1097, Hadoop-10456)
6b840f0 [nravi] Undo the fix for SPARK-1758 (the problem is fixed)
5108700 [nravi] Fix in Spark for the Concurrent thread modification issue (SPARK-1097, HADOOP-10456)
681b36f [nravi] Fix for SPARK-1758: failing test org.apache.spark.JavaAPISuite.wholeTextFiles
parent b3736e3d
No related branches found
No related tags found
No related merge requests found
......@@ -139,10 +139,13 @@ class HadoopRDD[K, V](
// Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the
// local process. The local cache is accessed through HadoopRDD.putCachedMetadata().
// The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects.
val newJobConf = new JobConf(broadcastedConf.value.value)
initLocalJobConfFuncOpt.map(f => f(newJobConf))
HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf)
newJobConf
// synchronize to prevent ConcurrentModificationException (Spark-1097, Hadoop-10456)
broadcastedConf.synchronized {
val newJobConf = new JobConf(broadcastedConf.value.value)
initLocalJobConfFuncOpt.map(f => f(newJobConf))
HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf)
newJobConf
}
}
}
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment