You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-issues@hadoop.apache.org by "Liyin Liang (JIRA)" <ji...@apache.org> on 2010/12/03 10:17:37 UTC

[jira] Created: (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

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.


[jira] [Updated] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Subroto Sanyal (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Subroto Sanyal updated MAPREDUCE-2209:
--------------------------------------

    Affects Version/s: 0.23.0
               Status: Patch Available  (was: Open)

The solution of reducing the lock will ensure the task tracker will keep on breathing.
Reducing the lock level in the method *TaskTracker::getMapCompletionEvents* will not target the thread blocking *MapEventsFetcherThread* and *GetMapEventsThread*. The threads will be still blocked.

Attaching a patch for the fix.

> 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
>    Affects Versions: 0.23.0
>         Environment: hadoop version: 0.19.1
>            Reporter: Liyin Liang
>            Priority: Blocker
>         Attachments: 2209-1.diff, MAPREDUCE-2209.patch
>
>
> 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

        

[jira] [Commented] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Subroto Sanyal (JIRA)" <ji...@apache.org>.
    [ 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

        

[jira] [Updated] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Arun C Murthy (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Arun C Murthy updated MAPREDUCE-2209:
-------------------------------------

    Assignee: Liyin Liang
      Status: Open  (was: Patch Available)

Sorry to come in late, the patch has gone stale. Can you please rebase? Thanks.

Given this is not an issue with MRv2 should we still commit this? I'm happy to, but not sure it's useful. Thanks.

> 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
>    Affects Versions: 0.23.0
>         Environment: hadoop version: 0.19.1
>            Reporter: Liyin Liang
>            Assignee: Liyin Liang
>            Priority: Blocker
>         Attachments: 2209-1.diff, MAPREDUCE-2209.patch
>
>
> 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

        

[jira] [Commented] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Arun C Murthy (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13081084#comment-13081084 ] 

Arun C Murthy commented on MAPREDUCE-2209:
------------------------------------------

Subroto/Liyin - In general the community is moving forward with a complete re-architecture of Hadoop MapReduce from 0.23 onwards: MAPREDUCE-279 for more details.

Thus, you might be better off focussing your efforts either on hadoop-0.20.203 and beyond (the classic mapreduce) or, better yet, helping out with MAPREDUCE-279 (new mapreduce). Given your valuable experience on 'classic' mapreduce, your help on MAPREDUCE-279 would be very welcome. Thanks!

> 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
>    Affects Versions: 0.23.0
>         Environment: hadoop version: 0.19.1
>            Reporter: Liyin Liang
>            Priority: Blocker
>         Attachments: 2209-1.diff, MAPREDUCE-2209.patch
>
>
> 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

        

[jira] Commented: (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Amar Kamat (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12970536#action_12970536 ] 

Amar Kamat commented on MAPREDUCE-2209:
---------------------------------------

Hi Liyin,
Your analysis is correct. The locking order (indirect) is causing the heartbeat to block. The main cause of this issue is a DFS call inside a lock. We removed lot of such cases in JobTracker by removing DFS calls from under locks. One such way to do it would be to queue up the DFS requests. This will help in this case because TaskLauncher is running as a thread  and hence starting a new task isn't inline, i.e its asynchronous.

> 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.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Liyin Liang (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Commented: (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12966606#action_12966606 ] 

Todd Lipcon commented on MAPREDUCE-2209:
----------------------------------------

I noticed you marked this as Affects 0.19.1. Do you have reason to believe this is still an issue in trunk? 0.19 is >2 years old at this point.

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


[jira] [Updated] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Subroto Sanyal (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Subroto Sanyal updated MAPREDUCE-2209:
--------------------------------------

    Attachment:     (was: MAPREDUCE-2209.patch)

> 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
>    Affects Versions: 0.23.0
>         Environment: hadoop version: 0.19.1
>            Reporter: Liyin Liang
>            Priority: Blocker
>         Attachments: 2209-1.diff
>
>
> 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

        

[jira] [Updated] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Harsh J (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Harsh J updated MAPREDUCE-2209:
-------------------------------

    Priority: Minor  (was: Blocker)

(This isn't a blocker, resetting severity)
                
> 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
>    Affects Versions: 0.23.0
>         Environment: hadoop version: 0.19.1
>            Reporter: Liyin Liang
>            Assignee: Liyin Liang
>            Priority: Minor
>         Attachments: 2209-1.diff, MAPREDUCE-2209.patch
>
>
> 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.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Liyin Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13080715#comment-13080715 ] 

Liyin Liang commented on MAPREDUCE-2209:
----------------------------------------

Hi Subroto,
Your analysis is great and your patch looks good to me. However, I found another issue MAPREDUCE-2364 which is duplicated with this one. What's more, their solution is mostly the same with your patch. I think one of them should be close as duplicate.

> 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
>    Affects Versions: 0.23.0
>         Environment: hadoop version: 0.19.1
>            Reporter: Liyin Liang
>            Priority: Blocker
>         Attachments: 2209-1.diff, MAPREDUCE-2209.patch
>
>
> 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

        

[jira] Updated: (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Liyin Liang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Liyin Liang updated MAPREDUCE-2209:
-----------------------------------

    Description: 
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}

  was:
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)



> 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.
-
You can reply to this email to add a comment to the issue online.


[jira] [Updated] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Subroto Sanyal (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Subroto Sanyal updated MAPREDUCE-2209:
--------------------------------------

    Attachment: MAPREDUCE-2209.patch

> 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
>    Affects Versions: 0.23.0
>         Environment: hadoop version: 0.19.1
>            Reporter: Liyin Liang
>            Priority: Blocker
>         Attachments: 2209-1.diff, MAPREDUCE-2209.patch
>
>
> 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

        

[jira] [Updated] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Liyin Liang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Liyin Liang updated MAPREDUCE-2209:
-----------------------------------

    Attachment: 2209-1.diff

> 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
>         Attachments: 2209-1.diff
>
>
> 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

        

[jira] [Updated] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Subroto Sanyal (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Subroto Sanyal updated MAPREDUCE-2209:
--------------------------------------

    Attachment: MAPREDUCE-2209.patch

> 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
>         Attachments: 2209-1.diff, MAPREDUCE-2209.patch
>
>
> 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

        

[jira] [Commented] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Liyin Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13073442#comment-13073442 ] 

Liyin Liang commented on MAPREDUCE-2209:
----------------------------------------

Hi Subroto,
  In fact,we have fixed this issuethrough reducing the lock of _TaskTracker::getMapCompletionEvents()_. And it works well in our 1500 nodes product cluster.I will attach a diff file for 0.19.

> 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
>         Attachments: 2209-1.diff
>
>
> 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

       

[jira] [Commented] (MAPREDUCE-2209) TaskTracker's heartbeat hang for several minutes when copying large job.jar from HDFS

Posted by "Subroto Sanyal (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/MAPREDUCE-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13080743#comment-13080743 ] 

Subroto Sanyal commented on MAPREDUCE-2209:
-------------------------------------------

Thankx Liyin for pointing out the issue MAPREDUCE-2364

> 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
>    Affects Versions: 0.23.0
>         Environment: hadoop version: 0.19.1
>            Reporter: Liyin Liang
>            Priority: Blocker
>         Attachments: 2209-1.diff, MAPREDUCE-2209.patch
>
>
> 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