[ 
https://issues.apache.org/jira/browse/MAPREDUCE-1372?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12799118#action_12799118
 ] 

Amareshwari Sriramadasu commented on MAPREDUCE-1372:
----------------------------------------------------

The code corresponding to the exception:
{code}
2012:    Collection<Node> nodesAtMaxLevel = jobtracker.getNodesAtMaxLevel();
2013:
2014:    // get the node parent at max level
2015:    Node nodeParentAtMaxLevel = 
2016:     (node == null) ? null : JobTracker.getParentNode(node, maxLevel - 1);
2017:   
2018:    for (Node parent : nodesAtMaxLevel) {
2019:
{code}

Add to nodesAtMaxLevel Map happens from two methods: 
JobTracker.addNewTracker()(with JobTracker lock) and JobInProgress.createCache 
(with JobInProgress lock).

Solution is to make JobTracker.nodesAtMaxLevel a SynchronizedMap and add 
synchronization for the iterations. Thoughts?

> ConcurrentModificationException in JobInProgress
> ------------------------------------------------
>
>                 Key: MAPREDUCE-1372
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-1372
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: jobtracker
>    Affects Versions: 0.20.1
>            Reporter: Amareshwari Sriramadasu
>            Priority: Blocker
>             Fix For: 0.21.0
>
>
> We have seen the following  ConcurrentModificationException in one of our 
> clusters
> {noformat}
> java.io.IOException: java.util.ConcurrentModificationException
>         at java.util.HashMap$HashIterator.nextEntry(HashMap.java:793)
>         at java.util.HashMap$KeyIterator.next(HashMap.java:828)
>         at 
> org.apache.hadoop.mapred.JobInProgress.findNewMapTask(JobInProgress.java:2018)
>         at 
> org.apache.hadoop.mapred.JobInProgress.obtainNewMapTask(JobInProgress.java:1077)
>         at 
> org.apache.hadoop.mapred.CapacityTaskScheduler$MapSchedulingMgr.obtainNewTask(CapacityTaskScheduler.java:796)
>         at 
> org.apache.hadoop.mapred.CapacityTaskScheduler$TaskSchedulingMgr.getTaskFromQueue(CapacityTaskScheduler.java:589)
>         at 
> org.apache.hadoop.mapred.CapacityTaskScheduler$TaskSchedulingMgr.assignTasks(CapacityTaskScheduler.java:677)
>         at 
> org.apache.hadoop.mapred.CapacityTaskScheduler$TaskSchedulingMgr.access$500(CapacityTaskScheduler.java:348)
>         at 
> org.apache.hadoop.mapred.CapacityTaskScheduler.addMapTask(CapacityTaskScheduler.java:1397)
>         at 
> org.apache.hadoop.mapred.CapacityTaskScheduler.assignTasks(CapacityTaskScheduler.java:1349)
>         at org.apache.hadoop.mapred.JobTracker.heartbeat(JobTracker.java:2976)
>         at sun.reflect.GeneratedMethodAccessor5.invoke(Unknown Source)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:508)
>         at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:959)
>         at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:955)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:396)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:953)
> {noformat}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to