-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
Conversation
…extFiles The prefix "file:" is missing in the string inserted as key in HashMap
Can one of the admins verify this patch? |
@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. |
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 |
Thanks rxin. Modified the fix. Synchronized the block so as to retain immutable newJobConf. |
@rxin does this look ok? |
LGTM. |
Jenkins, test this please. |
Merged build triggered. |
Merged build started. |
Merged build finished. All automated tests passed. |
All automated tests passed. |
Merging this in master & branch-1.0. |
…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>
…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
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} |
@colorant if you can look into it and submit a fix, that'd be great! |
@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 |
…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
…g disabled by planner (#1000)
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.