Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

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

Closed
wants to merge 5 commits into from

Conversation

nishkamravi2
Copy link
Contributor

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.

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@pwendell
Copy link
Contributor

pwendell commented Jun 7, 2014

@rxin can you take a quick look at this? From what I can tell we only ever use this at a per-partition granularity, so I don't anticipate any performance issues. But it would be good to have someone else look at it.

Jenkins, test this please.

@rxin
Copy link
Contributor

rxin commented Jun 7, 2014

Thanks. Can we synchronized only the JobConf creation, i.e.

      val newJobConf = new JobConf(broadcastedConf.value.value)

and add an inline comment linking to the spark/hadoop jira issue

@nishkamravi2
Copy link
Contributor Author

Thanks rxin. Modified the fix. Synchronized the block so as to retain immutable newJobConf.

@nishkamravi2
Copy link
Contributor Author

@rxin does this look ok?

@rxin
Copy link
Contributor

rxin commented Jun 13, 2014

LGTM.

@rxin
Copy link
Contributor

rxin commented Jun 13, 2014

Jenkins, test this please.

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15758/

@rxin
Copy link
Contributor

rxin commented Jun 13, 2014

Merging this in master & branch-1.0.

asfgit pushed a commit that referenced this pull request Jun 13, 2014
…56, 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

(cherry picked from commit 70c8116)
Signed-off-by: Reynold Xin <rxin@apache.org>
@asfgit asfgit closed this in 70c8116 Jun 13, 2014
pdeyhim pushed a commit to pdeyhim/spark-1 that referenced this pull request Jun 25, 2014
…56, 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 apache#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
@colorant
Copy link
Contributor

colorant commented Jul 1, 2014

It seems that this workaround not works for me on Hadoop 2.2.0, I still hit into this problem from within the synchronized block with the latest trunk code:

java.util.ConcurrentModificationException (java.util.ConcurrentModificationException}
java.util.HashMap$HashIterator.nextEntry(HashMap.java:793)
java.util.HashMap$KeyIterator.next(HashMap.java:828)
java.util.AbstractCollection.addAll(AbstractCollection.java:305)
java.util.HashSet.(HashSet.java:100)
org.apache.hadoop.conf.Configuration.(Configuration.java:554)
org.apache.hadoop.mapred.JobConf.(JobConf.java:439)
org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:144)
org.apache.spark.rdd.HadoopRDD$$anon$1.(HadoopRDD.scala:189)
org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:184)
org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:93)
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:261)
org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:261)
org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
org.apache.spark.rdd.FlatMappedRDD.compute(FlatMappedRDD.scala:33)
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:261)
org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:261)
org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:59)
org.apache.spark.rdd.RDD.iterator(RDD.scala:226)
org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:112)
org.apache.spark.scheduler.Task.run(Task.scala:51)
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:187)
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
java.lang.Thread.run(Thread.java:662)

@rxin
Copy link
Contributor

rxin commented Jul 1, 2014

@colorant if you can look into it and submit a fix, that'd be great!
Thanks for reporting this.

@colorant
Copy link
Contributor

colorant commented Jul 1, 2014

@rxin correct me if I am wrong.

The problem here is that the broadcastedConf is in per task HadoopRDD, synchronized on the method or on the broadcastedConf itself is good within this task. while when you call braodcastedConf.value.value, you actually return the value saved in the memory store,( when memory is enough and with deserialize approaching) this conf object should be the same one per Executor? say when getconf across task, you don't prevent to get the same conf object. and pass this conf object to JobConf(conf) lead to this problem.

If I am right, then, broadcastedConf.value.value.synchronized might solve this problem?

I am not 100% sure those reference across task staffs did work as I described above. What do you think about it? I will try to modify the code and see if it works, If this is true, I can do a quick pull request then

@colorant
Copy link
Contributor

colorant commented Jul 1, 2014

@rxin, PR at #1273 , I tried for around 10 batches of job with that patch, do not see this problem happen again. without this patch, on my nodes, it do happen from time to time, say every 1-3 jobs will meet this problem.

@nishkamravi2
Copy link
Contributor Author

@rxin, @colorant Sorry, the emails went to my spam folder. The fix looks better than the one before.

xiliu82 pushed a commit to xiliu82/spark that referenced this pull request Sep 4, 2014
…56, 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 apache#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
wangyum pushed a commit that referenced this pull request May 26, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants