From 5108700230fd70b995e76598f49bdf328c971e77 Mon Sep 17 00:00:00 2001 From: nravi Date: Tue, 3 Jun 2014 15:25:22 -0700 Subject: [PATCH] Fix in Spark for the Concurrent thread modification issue (SPARK-1097, HADOOP-10456) --- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 6547755764dcf..8346d25a29d49 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -126,7 +126,7 @@ class HadoopRDD[K, V]( private val createTime = new Date() // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. - protected def getJobConf(): JobConf = { + protected def getJobConf(): JobConf = synchronized { val conf: Configuration = broadcastedConf.value.value if (conf.isInstanceOf[JobConf]) { // A user-broadcasted JobConf was provided to the HadoopRDD, so always use it.