[ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13073441#comment-13073441 ]
Subroto Sanyal commented on MAPREDUCE-2209: ------------------------------------------- Hi Amar, The order of locking: ||Thread Name||State||Locked||Waiting to Lock|| |TaskLauncher(TaskLauncher.run localizeJob())|Runnable(Downloading the jar file)|*rjob* 0x0000002afce2d260| | |Map-events(MapEventsFetcherThread.run() reducesInShuffle())|Blocked|*runningJobs* 0x0000002a9eefe1f8|Waiting to lock 0x0000002afce2d260 which is held in previous row| |IPC Server handler(GetMapEventsThread.run() getMapCompletionEvents() TaskUmbilicalProtocol.getMapCompletionEvents())|Blocked|*TaskTracker* 0x0000002a9eac1de8|Waiting to Lock 0x0000002a9eefe1f8 which is held in previous row| |Main Thread (TaskTracker.offerService())|Blocked| |Waiting to Lock 0x0000002a9eac1de8 which is held in previous row| As per my understanding the lock on *rjob* in the method *void org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskInProgress tip) throws IOException* is aquired to make sure that no two task of same job try to localize the job. I think making the download of HDFS resources asynchrounus (in a queue) will not solve the purpose as before the Task Execution starts, the said resources should be available in Local File System. Instead to aquiring the lock on *rjob*, we can aquire the lock on more granular object say *"rjob.localized"*(changing localized to Object from primitive). This will solve the purpose of aquiring lock and at the same time we will not block the other threads. > TaskTracker's heartbeat hang for several minutes when copying large job.jar > from HDFS > ------------------------------------------------------------------------------------- > > Key: MAPREDUCE-2209 > URL: https://issues.apache.org/jira/browse/MAPREDUCE-2209 > Project: Hadoop Map/Reduce > Issue Type: Bug > Environment: hadoop version: 0.19.1 > Reporter: Liyin Liang > Priority: Blocker > > If a job's jar file is very large, e.g 200m+, the TaskTracker's heartbeat > hang for several minutes when localizing the job. The jstack of related > threads are as follows: > {code:borderStyle=solid} > "TaskLauncher for task" daemon prio=10 tid=0x0000002b05ee5000 nid=0x1adf > runnable [0x0000000042e56000] > java.lang.Thread.State: RUNNABLE > at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) > at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215) > at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65) > at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) > - locked <0x0000002afc892ec8> (a sun.nio.ch.Util$1) > - locked <0x0000002afc892eb0> (a > java.util.Collections$UnmodifiableSet) > - locked <0x0000002afc8927d8> (a sun.nio.ch.EPollSelectorImpl) > at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) > at > org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:260) > at > org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:155) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:150) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:123) > at java.io.BufferedInputStream.fill(BufferedInputStream.java:218) > at java.io.BufferedInputStream.read(BufferedInputStream.java:237) > - locked <0x0000002afce26158> (a java.io.BufferedInputStream) > at java.io.DataInputStream.readShort(DataInputStream.java:295) > at > org.apache.hadoop.hdfs.DFSClient$BlockReader.newBlockReader(DFSClient.java:1304) > at > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.blockSeekTo(DFSClient.java:1556) > - locked <0x0000002afce26218> (a > org.apache.hadoop.hdfs.DFSClient$DFSInputStream) > at > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1673) > - locked <0x0000002afce26218> (a > org.apache.hadoop.hdfs.DFSClient$DFSInputStream) > at java.io.DataInputStream.read(DataInputStream.java:83) > at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:47) > at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:85) > at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:209) > at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:142) > at > org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1214) > at > org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1195) > at > org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:824) > - locked <0x0000002afce2d260> (a > org.apache.hadoop.mapred.TaskTracker$RunningJob) > at > org.apache.hadoop.mapred.TaskTracker.startNewTask(TaskTracker.java:1745) > at > org.apache.hadoop.mapred.TaskTracker.access$1200(TaskTracker.java:103) > at > org.apache.hadoop.mapred.TaskTracker$TaskLauncher.run(TaskTracker.java:1710) > "Map-events fetcher for all reduce tasks on > tracker_r01a08025:localhost/127.0.0.1:50050" daemon prio=10 > tid=0x0000002b05ef8000 > nid=0x1ada waiting for monitor entry [0x0000000042d55000] > java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.hadoop.mapred.TaskTracker$MapEventsFetcherThread.reducesInShuffle(TaskTracker.java:582) > - waiting to lock <0x0000002afce2d260> (a > org.apache.hadoop.mapred.TaskTracker$RunningJob) > at > org.apache.hadoop.mapred.TaskTracker$MapEventsFetcherThread.run(TaskTracker.java:617) > - locked <0x0000002a9eefe1f8> (a java.util.TreeMap) > "IPC Server handler 2 on 50050" daemon prio=10 tid=0x0000002b050eb000 > nid=0x1ab0 waiting for monitor entry [0x000000004234b000] > java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.hadoop.mapred.TaskTracker.getMapCompletionEvents(TaskTracker.java:2684) > - waiting to lock <0x0000002a9eefe1f8> (a java.util.TreeMap) > - locked <0x0000002a9eac1de8> (a org.apache.hadoop.mapred.TaskTracker) > 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:481) > at org.apache.hadoop.ipc.Server$Handler.run(Server.java:894) > "main" prio=10 tid=0x0000000040113800 nid=0x197d waiting for monitor entry > [0x000000004022a000] > java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.hadoop.mapred.TaskTracker.transmitHeartBeat(TaskTracker.java:1196) > - waiting to lock <0x0000002a9eac1de8> (a > org.apache.hadoop.mapred.TaskTracker) > at > org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:1068) > at org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:1799) > at org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:2898) > {code} -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira