[
https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12969238#action_12969238
]
Liyin Liang commented on MAPREDUCE-2209:
----------------------------------------
I setup a cluster with the latest version 0.21.0. To simulate the large
job.jar problem, let TaskLauncher thread sleep 100 seconds just before download
job.jar in localizeJobJarFile function. Then the heartbeat of some TT will
hang for almost 100 seconds. Basically, the jstack is the same with 0.19:
{code:borderStyle=solid}
"TaskLauncher for MAP tasks" daemon prio=10 tid=0x00002aab3145a800 nid=0x3fe8
waiting on condition [0x00000000440b3000..0x00000000440b3a10]
java.lang.Thread.State: TIMED_WAITING (sleeping)
at java.lang.Thread.sleep(Native Method)
at
org.apache.hadoop.mapred.TaskTracker.localizeJobJarFile(TaskTracker.java:1150)
at
org.apache.hadoop.mapred.TaskTracker.localizeJobFiles(TaskTracker.java:1074)
at
org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:977)
- locked <0x00002aaab3a86f10> (a
org.apache.hadoop.mapred.TaskTracker$RunningJob)
at
org.apache.hadoop.mapred.TaskTracker.startNewTask(TaskTracker.java:2248)
at
org.apache.hadoop.mapred.TaskTracker$TaskLauncher.run(TaskTracker.java:2213)
"Map-events fetcher for all reduce tasks on
tracker_hd2:localhost.localdomain/127.0.0.1:36128" daemon prio=10 tid=0x00002aab
31451c00 nid=0x3fde waiting for monitor entry
[0x0000000041a40000..0x0000000041a40d90]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.hadoop.mapred.TaskTracker$MapEventsFetcherThread.reducesInShuffle(TaskTracker.java:800)
- waiting to lock <0x00002aaab3a86f10> (a
org.apache.hadoop.mapred.TaskTracker$RunningJob)
at
org.apache.hadoop.mapred.TaskTracker$MapEventsFetcherThread.run(TaskTracker.java:834)
- locked <0x00002aaab38ee1b8> (a java.util.TreeMap)
"IPC Server handler 0 on 36128" daemon prio=10 tid=0x000000004368ac00
nid=0x3fc8 waiting for monitor entry [0x00000000425f6000..0x00000000425
f7c90]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.hadoop.mapred.TaskTracker.getMapCompletionEvents(TaskTracker.java:3254)
- waiting to lock <0x00002aaab38ee1b8> (a java.util.TreeMap)
- locked <0x00002aaab37f1708> (a org.apache.hadoop.mapred.TaskTracker)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:597)
at
org.apache.hadoop.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:342)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1350)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1346)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:742)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1344)
"main" prio=10 tid=0x0000000042fff400 nid=0x3f91 waiting for monitor entry
[0x0000000041ef0000..0x0000000041ef0ed0]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.hadoop.mapred.TaskTracker.transmitHeartBeat(TaskTracker.java:1535)
- waiting to lock <0x00002aaab37f1708> (a
org.apache.hadoop.mapred.TaskTracker)
at
org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:1433)
at org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:2330)
at org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:3462)
{code}
lock order of relative threads:
TaskLauncher(localizeJobJarFile): locked RunningJob
Map-events fetcher: locked
runningJobs waiting to lock RunningJob
IPC Server handler(getMapCompletionEvents): locked TaskTracker waiting to
lock runningJobs
main(transmitHeartBeat): waiting to
lock TaskTracker
So, TaskTracker is locked indirectly when downloading job.jar.
> 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:
> "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)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.