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

Amareshwari Sriramadasu commented on MAPREDUCE-1895:
----------------------------------------------------

Here is the stacktrace for the above scenario :
{noformat}
"TaskLauncher for MAP tasks" daemon prio=10 tid=0xaf51f800 nid=0x70ce in 
Object.wait()
[0xaf6ad000..0xaf6adf30]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:485)
        at org.apache.hadoop.ipc.Client.call(Client.java:937)
        - locked <0xee8d2218> (a org.apache.hadoop.ipc.Client$Call)
        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:223)
        at $Proxy7.getFileInfo(Unknown Source)
        at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
        at $Proxy7.getFileInfo(Unknown Source)
        at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:676)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:507)
        at org.apache.hadoop.fs.FileSystem.isFile(FileSystem.java:700)
        at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:218)
        at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:157)
        at org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1255)
        at org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1236)
        at 
org.apache.hadoop.mapred.TaskTracker.localizeJobJarFile(TaskTracker.java:1171)
        at 
org.apache.hadoop.mapred.TaskTracker.localizeJobFiles(TaskTracker.java:1046)
        at 
org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:954)
        - locked <0xba4b6f40> (a 
org.apache.hadoop.mapred.TaskTracker$RunningJob)
        at 
org.apache.hadoop.mapred.TaskTracker.startNewTask(TaskTracker.java:2165)
        at 
org.apache.hadoop.mapred.TaskTracker$TaskLauncher.run(TaskTracker.java:2130)

"Map-events fetcher for all reduce tasks on 
tracker_gsgd40932.gold.ygrid.yahoo.com:localhost/127.0.0.1:50542"
daemon prio=10 tid=0xaf597800 nid=0x70c9 waiting for monitor entry 
[0xaefe1000..0xaefe2130]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at 
org.apache.hadoop.mapred.TaskTracker$MapEventsFetcherThread.reducesInShuffle(TaskTracker.java:777)
        - waiting to lock <0xba4b6f40> (a 
org.apache.hadoop.mapred.TaskTracker$RunningJob)
        at 
org.apache.hadoop.mapred.TaskTracker$MapEventsFetcherThread.run(TaskTracker.java:812)
        - locked <0xb4f02fe8> (a java.util.TreeMap)

"main" prio=10 tid=0x0805ac00 nid=0x70a2 waiting for monitor entry 
[0xf7fbb000..0xf7fbc1f8]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at 
org.apache.hadoop.mapred.TaskTracker.removeTaskFromJob(TaskTracker.java:449)
        - waiting to lock <0xb4f02fe8> (a java.util.TreeMap)
        at org.apache.hadoop.mapred.TaskTracker.purgeTask(TaskTracker.java:1882)
        at 
org.apache.hadoop.mapred.TaskTracker.markUnresponsiveTasks(TaskTracker.java:1737)
        - locked <0xb4ea44b8> (a org.apache.hadoop.mapred.TaskTracker)
        at 
org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:1501)
        at org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:2236)
        at org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:3414)
{noformat}

> MapEventFetcherThread should not iterate over jobs that are not localized
> -------------------------------------------------------------------------
>
>                 Key: MAPREDUCE-1895
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-1895
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: tasktracker
>            Reporter: Amareshwari Sriramadasu
>
> We have seen a scenario of lost trackers on our clusters because of the 
> following:
> TaskLauncher has locked a TaskTracker$RunningJob and doing localizeJob, which 
> involves DFS operations. Map-event
> fetcher has locked TaskTracker.runningJobs map and is waiting to lock the 
> RunningJob object. TaskTracker offerService
> is waiting to lock TaskTracker.runningJobs map, thus failing to send 
> heartbeats in 10 minutes. 
> So, I think map-event fetcher should circuit jobs that are not localized.

-- 
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