Details

    • Type: Bug
    • Status: Open
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: None
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      All maps had been completed successfully. I had only one reduce task during which
      TaskTracker infinitely outputs:

      07/05/15 19:35:41 INFO mapred.TaskTracker: task_0001_r_000000_0 0.16666667% reduce > copy (4 of 8 at 0.00 MB/s) >
      07/05/15 19:35:42 INFO mapred.TaskTracker: task_0001_r_000000_0 0.16666667% reduce > copy (4 of 8 at 0.00 MB/s) >
      07/05/15 19:35:43 INFO mapred.TaskTracker: task_0001_r_000000_0 0.16666667% reduce > copy (4 of 8 at 0.00 MB/s) >
      07/05/15 19:35:44 INFO mapred.TaskTracker: task_0001_r_000000_0 0.16666667% reduce > copy (4 of 8 at 0.00 MB/s) >
      07/05/15 19:35:45 INFO mapred.TaskTracker: task_0001_r_000000_0 0.16666667% reduce > copy (4 of 8 at 0.00 MB/s) >

      JobTracker does not log anything about task task_0001_r_000000_0 except for
      07/05/15 19:49:01 INFO mapred.JobTracker: Adding task 'task_0001_r_000000_0' to tip tip_0001_r_000000, for tracker 'tracker_my-host.com:50050'

      1. DataNode1.log
        6 kB
        Konstantin Shvachko
      2. DataNode2.log
        7 kB
        Konstantin Shvachko
      3. hadoop-1374-yunhong.tar.gz
        4 kB
        Yunhong Gu
      4. JobTracker.log
        11 kB
        Konstantin Shvachko
      5. NameNode.log
        4 kB
        Konstantin Shvachko
      6. TaskTracker1.log
        14 kB
        Konstantin Shvachko
      7. TaskTracker2.log
        9 kB
        Konstantin Shvachko
      8. TestDFSIO.log
        0.9 kB

        Issue Links

          Activity

          Hide
          owen.omalley Owen O'Malley added a comment -

          It should be noted that this has only been seen on windows.

          Show
          owen.omalley Owen O'Malley added a comment - It should be noted that this has only been seen on windows.
          Hide
          shv Konstantin Shvachko added a comment -

          Not really. The first time I have seen it on a pure windows cluster.
          But this log is from a linux machine that was running a TaskTracker.
          It should be noted that the JobTracker was running on windows though.
          But it is not producing any logs while the TaskTracker looping,
          so I suspect this could be a general problem.

          Show
          shv Konstantin Shvachko added a comment - Not really. The first time I have seen it on a pure windows cluster. But this log is from a linux machine that was running a TaskTracker. It should be noted that the JobTracker was running on windows though. But it is not producing any logs while the TaskTracker looping, so I suspect this could be a general problem.
          Hide
          owen.omalley Owen O'Malley added a comment -

          Ok, I got this reproducing on Linux. It looks like it is related to the event log array out of bounds exception. In particular, I get:

          >bin/hadoop job -events job_0001
          Exception in thread "main" java.lang.ArrayIndexOutOfBoundsException: 2
          at org.apache.hadoop.mapred.JobClient.run(JobClient.java:748)
          at org.apache.hadoop.util.ToolBase.doMain(ToolBase.java:189)
          at org.apache.hadoop.mapred.JobClient.main(JobClient.java:806)

          Show
          owen.omalley Owen O'Malley added a comment - Ok, I got this reproducing on Linux. It looks like it is related to the event log array out of bounds exception. In particular, I get: >bin/hadoop job -events job_0001 Exception in thread "main" java.lang.ArrayIndexOutOfBoundsException: 2 at org.apache.hadoop.mapred.JobClient.run(JobClient.java:748) at org.apache.hadoop.util.ToolBase.doMain(ToolBase.java:189) at org.apache.hadoop.mapred.JobClient.main(JobClient.java:806)
          Hide
          acmurthy Arun C Murthy added a comment -

          Owen, I'm not sure these are related... the code in question:

          <code>
          else if ("-events".equals(argv[i])) {
          listEvents(argv[i+1], Integer.parseInt(argv[i+2]), Integer.parseInt(argv[i+3])); i += 3;
          }
          </code>

          So incomplete args to
          $ bin/hadoop job -events job_0001<fromeventid> <numEvents>
          could be the reason for the ArrayIndexOutOfBoundsException you noticed? smile

          Show
          acmurthy Arun C Murthy added a comment - Owen, I'm not sure these are related... the code in question: <code> else if ("-events".equals(argv [i] )) { listEvents(argv [i+1] , Integer.parseInt(argv [i+2] ), Integer.parseInt(argv [i+3] )); i += 3; } </code> So incomplete args to $ bin/hadoop job -events job_0001<fromeventid> <numEvents> could be the reason for the ArrayIndexOutOfBoundsException you noticed? smile
          Hide
          devaraj Devaraj Das added a comment -

          Konstantin, could you pls check whether you still have the task logs for the task task_0001_r_000000_0 on the linux node. If so, please check what exceptions you see there. If not, is it possible to reproduce this and get the logs ?
          Could it be because of some problem in the jetty server on the node from where this hung reducer is trying to get map outputs from? That's HADOOP-1158.

          Show
          devaraj Devaraj Das added a comment - Konstantin, could you pls check whether you still have the task logs for the task task_0001_r_000000_0 on the linux node. If so, please check what exceptions you see there. If not, is it possible to reproduce this and get the logs ? Could it be because of some problem in the jetty server on the node from where this hung reducer is trying to get map outputs from? That's HADOOP-1158 .
          Hide
          devaraj Devaraj Das added a comment -

          Just clarifying that post HADOOP-968, tasktracker logs wouldn't have the logs for the shuffle phase. Those will appear in the syslogs for the reduce tasks. The tasktracker logs will just have the shuffle status messages (no exception traces, etc.).

          Show
          devaraj Devaraj Das added a comment - Just clarifying that post HADOOP-968 , tasktracker logs wouldn't have the logs for the shuffle phase. Those will appear in the syslogs for the reduce tasks. The tasktracker logs will just have the shuffle status messages (no exception traces, etc.).
          Hide
          acmurthy Arun C Murthy added a comment -

          The 'ant test' seems to run fine on both linux and cygwin (sans contrib/hbase as noted by HADOOP-1423) and I can't think of any way to reproduce this yet... hence I'm marking this to fix for 0.14.0.

          Konstantin, I'm not sure how much of this has to do with the heterogenous environment you have... I'd appreciate if you could attach logs when run into this again (JT/TT and tasklogs). Thanks!

          Show
          acmurthy Arun C Murthy added a comment - The 'ant test' seems to run fine on both linux and cygwin (sans contrib/hbase as noted by HADOOP-1423 ) and I can't think of any way to reproduce this yet... hence I'm marking this to fix for 0.14.0. Konstantin, I'm not sure how much of this has to do with the heterogenous environment you have... I'd appreciate if you could attach logs when run into this again (JT/TT and tasklogs). Thanks!
          Hide
          shv Konstantin Shvachko added a comment -

          I've attached all logs.
          This a single node cluster. I am running TestDFSIO, not the unit tests.
          I do not know how Owen derived this is related to ArrayIndexOutOfBoundsException.
          I don't see any exceptions at all.

          I do not think we can release our distributed software if it does not run on a single node.

          Show
          shv Konstantin Shvachko added a comment - I've attached all logs. This a single node cluster. I am running TestDFSIO, not the unit tests. I do not know how Owen derived this is related to ArrayIndexOutOfBoundsException. I don't see any exceptions at all. I do not think we can release our distributed software if it does not run on a single node.
          Hide
          owen.omalley Owen O'Malley added a comment -

          What I observed was that when running the unit tests under Windows (especially TestMiniMRLocalFS) is that all of the maps except 1 finish. The last map is at 100%, but the state is RUNNING. Clearly the reduce keeps saying he is waiting to copy one file over and over again. It is in fact an infinite loop because of HADOOP-1431, which I filed as a blocker on 0.13. What isn't clear is why the stuck map is getting stuck. I am still investigating that. But the relevant fact is that it is the map completion, not the shuffle, that is the problem.

          Show
          owen.omalley Owen O'Malley added a comment - What I observed was that when running the unit tests under Windows (especially TestMiniMRLocalFS) is that all of the maps except 1 finish. The last map is at 100%, but the state is RUNNING. Clearly the reduce keeps saying he is waiting to copy one file over and over again. It is in fact an infinite loop because of HADOOP-1431 , which I filed as a blocker on 0.13. What isn't clear is why the stuck map is getting stuck. I am still investigating that. But the relevant fact is that it is the map completion, not the shuffle, that is the problem.
          Hide
          owen.omalley Owen O'Malley added a comment -

          One more set of information:
          1. In the web/ui the task shows as running both in the jobtracker AND tasktracker.
          2. The logs from the unit test show that the task tracker got both the final status update (100%) and the done for the stuck map:

              [junit] 2007-05-24 21:04:03,744 INFO  mapred.TaskTracker (TaskTracker.java:reportProgress(1286)) - task_0001_m_000009_0 1.0% Generated 99901 samples.
              [junit] 2007-05-24 21:04:03,744 INFO  mapred.TaskTracker (TaskTracker.java:reportDone(1336)) - Task task_0001_m_000009_0 is done.
          

          I've looked through the locking in the TaskTracker for the status messages and it looked right to me. I must be missing something.

          Show
          owen.omalley Owen O'Malley added a comment - One more set of information: 1. In the web/ui the task shows as running both in the jobtracker AND tasktracker. 2. The logs from the unit test show that the task tracker got both the final status update (100%) and the done for the stuck map: [junit] 2007-05-24 21:04:03,744 INFO mapred.TaskTracker (TaskTracker.java:reportProgress(1286)) - task_0001_m_000009_0 1.0% Generated 99901 samples. [junit] 2007-05-24 21:04:03,744 INFO mapred.TaskTracker (TaskTracker.java:reportDone(1336)) - Task task_0001_m_000009_0 is done. I've looked through the locking in the TaskTracker for the status messages and it looked right to me. I must be missing something.
          Hide
          nidaley Nigel Daley added a comment -

          Sounds similar to HADOOP-1332 where a 'done' message appears, but no 'completed' message.

          Show
          nidaley Nigel Daley added a comment - Sounds similar to HADOOP-1332 where a 'done' message appears, but no 'completed' message.
          Hide
          acmurthy Arun C Murthy added a comment -

          Konstantin, as per your attached logs one of your task-trackers was 'lost' (it takes 10mins to declare it to be 'lost'), the tasks were rescheduled to the other tracker and your job completed fine (as per the jobtracker logs)...

          Ok, I've racked my brains on this one and let me try and explain what I think is happening and potentially one short-term fix to ease our lives... fasten your seat-belts please:

          a) MapTask completes and we see the 'done' message from TaskTracker:reportDone
          b) However TaskTracker.reportDone only notes that the task is done by setting a boolean (but does not mark the TaskInProgress.runstate as SUCCEEDED).
          c) The child jvm, for whatever reason (maybe a windows peculiarity) doesn't 'exit' (might be due to stray non-daemon threads etc.). Thus TaskRunner.runChild's process.waitFor is hung, and hence TaskRunner.run cannot call TaskTracker.reportTaskFinished which is the place which sets TaskInProgress.runstate to SUCCEEDED.
          d) 10 mins later TaskTracker.markUnresponsiveTasks marks this task as 'unresponsive' and kills it. However this might be too late since the junit test case is killed for (possibly) over-running it's 15mins limit and we have a failed test case.

          Phew! Hope that makes sense, it looks like we might have to figure out why the child-jvm isn't exiting in the first place. So far other than that there isn't a bug IMO.

          One option is to reduce those timeouts from 10mins to 3/5mins for the test-cases and things should swim along fine for now, while we continue to try and figure out this one for 0.14.0 or 0.13.1 if possible... does that sound reasonable? Nigel?

          Show
          acmurthy Arun C Murthy added a comment - Konstantin, as per your attached logs one of your task-trackers was 'lost' (it takes 10mins to declare it to be 'lost'), the tasks were rescheduled to the other tracker and your job completed fine (as per the jobtracker logs)... Ok, I've racked my brains on this one and let me try and explain what I think is happening and potentially one short-term fix to ease our lives... fasten your seat-belts please: a) MapTask completes and we see the 'done' message from TaskTracker:reportDone b) However TaskTracker.reportDone only notes that the task is done by setting a boolean (but does not mark the TaskInProgress.runstate as SUCCEEDED ). c) The child jvm, for whatever reason (maybe a windows peculiarity) doesn't 'exit' (might be due to stray non-daemon threads etc.). Thus TaskRunner.runChild 's process.waitFor is hung, and hence TaskRunner.run cannot call TaskTracker.reportTaskFinished which is the place which sets TaskInProgress.runstate to SUCCEEDED . d) 10 mins later TaskTracker.markUnresponsiveTasks marks this task as 'unresponsive' and kills it. However this might be too late since the junit test case is killed for (possibly) over-running it's 15mins limit and we have a failed test case. Phew! Hope that makes sense, it looks like we might have to figure out why the child-jvm isn't exiting in the first place. So far other than that there isn't a bug IMO. One option is to reduce those timeouts from 10mins to 3/5mins for the test-cases and things should swim along fine for now, while we continue to try and figure out this one for 0.14.0 or 0.13.1 if possible... does that sound reasonable? Nigel?
          Hide
          shv Konstantin Shvachko added a comment -

          The TaskTracker was lost because I killed it since it was spewing status report every second.
          So the problem happend before that. And it looks like it is not reflected in the log.
          This is not windows related, I am running all these things on a linux machine.
          I'll try to debug it later today since nobody else can reproduce it.

          Show
          shv Konstantin Shvachko added a comment - The TaskTracker was lost because I killed it since it was spewing status report every second. So the problem happend before that. And it looks like it is not reflected in the log. This is not windows related, I am running all these things on a linux machine. I'll try to debug it later today since nobody else can reproduce it.
          Hide
          shv Konstantin Shvachko added a comment -

          Debugging TaskTracker being in the loop on Linux.
          It falls into loop when all 8 maps are done. The reduce will never finish.
          All 2 nodes are sending heartbeats every 10 secs, nobody is dying.
          This is what WebUI showes for the bad task-tracker

          Running tasks
          Task Attempts Status Progress Errors
          task_0001_r_000000_1 RUNNING 16.66%

          Non-Running Tasks
          Task Attempts Status
          task_0001_m_000004_0 SUCCEEDED
          task_0001_m_000007_0 SUCCEEDED
          task_0001_m_000003_0 SUCCEEDED
          task_0001_m_000006_0 SUCCEEDED

          I put a breakpoint in org.apache.hadoop.ipc.Server.Handler.run() where the calls are proccessed, at
          value = call(call.param); // make the call

          I see it is processing only the following 3 calls.

          ping(task_0001_r_000000_1) from 66.22.15.15:58122
          progress(task_0001_r_000000_1, 0.16666667, reduce > copy (4 of 8 at 0.00 MB/s) > , SHUFFLE, org.apache.hadoop.mapred.Counters@b0f534) from 66.22.15.15:58122
          getMapCompletionEvents(job_0001, 8, 50) from 66.22.15.15:58122

          Hope this helps.

          Show
          shv Konstantin Shvachko added a comment - Debugging TaskTracker being in the loop on Linux. It falls into loop when all 8 maps are done. The reduce will never finish. All 2 nodes are sending heartbeats every 10 secs, nobody is dying. This is what WebUI showes for the bad task-tracker Running tasks Task Attempts Status Progress Errors task_0001_r_000000_1 RUNNING 16.66% Non-Running Tasks Task Attempts Status task_0001_m_000004_0 SUCCEEDED task_0001_m_000007_0 SUCCEEDED task_0001_m_000003_0 SUCCEEDED task_0001_m_000006_0 SUCCEEDED I put a breakpoint in org.apache.hadoop.ipc.Server.Handler.run() where the calls are proccessed, at value = call(call.param); // make the call I see it is processing only the following 3 calls. ping(task_0001_r_000000_1) from 66.22.15.15:58122 progress(task_0001_r_000000_1, 0.16666667, reduce > copy (4 of 8 at 0.00 MB/s) > , SHUFFLE, org.apache.hadoop.mapred.Counters@b0f534) from 66.22.15.15:58122 getMapCompletionEvents(job_0001, 8, 50) from 66.22.15.15:58122 Hope this helps.
          Hide
          devaraj Devaraj Das added a comment -

          Regarding the calls the reduce task makes, they all look good. It would be very nice to have a look at the tasklog of task_0001_r_000000_1 (btw from the taskid string it looks like this is the second attempt for the same reduce TIP; the first attempt failed) since they have all what the reduce task is doing in its life.
          Could you please attach the task logs whenever you see this happening. Also, in your test app do you have just one reduce? (The task logs can be found at $LOG_DIR/userlogs/<taskid>/).
          Another thing worth doing is to set the value of mapred.userlog.purgesplits to false so that we don't lose the task logs.
          <property>
          <name>mapred.userlog.purgesplits</name>
          <value>false</value>
          <description>Should the splits be purged disregarding the user-log size limit.
          </description>
          </property>
          The task logs will hopefully give some insights.

          Show
          devaraj Devaraj Das added a comment - Regarding the calls the reduce task makes, they all look good. It would be very nice to have a look at the tasklog of task_0001_r_000000_1 (btw from the taskid string it looks like this is the second attempt for the same reduce TIP; the first attempt failed) since they have all what the reduce task is doing in its life. Could you please attach the task logs whenever you see this happening. Also, in your test app do you have just one reduce? (The task logs can be found at $LOG_DIR/userlogs/<taskid>/). Another thing worth doing is to set the value of mapred.userlog.purgesplits to false so that we don't lose the task logs. <property> <name>mapred.userlog.purgesplits</name> <value>false</value> <description>Should the splits be purged disregarding the user-log size limit. </description> </property> The task logs will hopefully give some insights.
          Hide
          owen.omalley Owen O'Malley added a comment -

          This seems to be a very rare bug. It should not block the 0.13 release. Possibly once it has been identified and fixed we can put it into a 13.1 release.

          Show
          owen.omalley Owen O'Malley added a comment - This seems to be a very rare bug. It should not block the 0.13 release. Possibly once it has been identified and fixed we can put it into a 13.1 release.
          Hide
          mike.smith Mike Smith added a comment -

          I do have the similar problem on 0.12.3. My last reducer stucks in almost end of copying the map output and it stays there for ever. But, when I kill that node and the task is assigned to another node, the task will finish successfully. The stucked tasktracker keeps sending this log:

          2007-06-02 13:33:13,335 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Got 3 known map output location(s); scheduling...
          2007-06-02 13:33:13,335 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Scheduled 0 of 3 known outputs (3 slow hosts and 0 dup hosts)
          2007-06-02 13:33:14,034 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) >
          2007-06-02 13:33:15,044 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) >
          2007-06-02 13:33:16,054 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) >
          2007-06-02 13:33:17,064 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) >
          2007-06-02 13:33:18,074 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) >
          2007-06-02 13:33:18,343 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Need 3 map output(s)
          2007-06-02 13:33:18,344 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Need 3 map output location(s)
          2007-06-02 13:33:18,345 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Got 0 new map outputs from jobtracker and 0 map outputs from previous failures

          Show
          mike.smith Mike Smith added a comment - I do have the similar problem on 0.12.3. My last reducer stucks in almost end of copying the map output and it stays there for ever. But, when I kill that node and the task is assigned to another node, the task will finish successfully. The stucked tasktracker keeps sending this log: 2007-06-02 13:33:13,335 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Got 3 known map output location(s); scheduling... 2007-06-02 13:33:13,335 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Scheduled 0 of 3 known outputs (3 slow hosts and 0 dup hosts) 2007-06-02 13:33:14,034 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) > 2007-06-02 13:33:15,044 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) > 2007-06-02 13:33:16,054 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) > 2007-06-02 13:33:17,064 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) > 2007-06-02 13:33:18,074 INFO org.apache.hadoop.mapred.TaskTracker: task_0023_r_000085_0 0.33295453% reduce > copy (2637 of 2640 at 0.00 MB/s) > 2007-06-02 13:33:18,343 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Need 3 map output(s) 2007-06-02 13:33:18,344 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Need 3 map output location(s) 2007-06-02 13:33:18,345 INFO org.apache.hadoop.mapred.TaskRunner: task_0023_r_000085_0 Got 0 new map outputs from jobtracker and 0 map outputs from previous failures
          Hide
          mike.smith Mike Smith added a comment -

          As Kostantin mentioed, all maps are done and only the last reducer stucks and task will not fail after 10 min, because jobtracker still gets the heartbeats. By the way, here is the core dump when the reducers stucks and the whole cluster runs in linux:

          Full thread dump Java HotSpot(TM) Client VM (1.5.0_11-b03 mixed mode):

          "IPC Client connection to jobtracker_address/jobtracker_address:50002" daemon prio=1 tid=0x096f29d8 nid=0x76cc in Object.wait() [0x4cfdc000..0x4cfdcf40]
          at java.lang.Object.wait(Native Method)
          at java.lang.Object.wait(Object.java:474)
          at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:213)

          • locked <0x4f13b5c0> (a org.apache.hadoop.ipc.Client$Connection)
            at org.apache.hadoop.ipc.Client$Connection.run(Client.java:252)

          "SocketListener0-19" prio=1 tid=0x0a61ccf8 nid=0x49ed in Object.wait() [0x4b08a000..0x4b08af40]
          at java.lang.Object.wait(Native Method)
          at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522)

          • locked <0x567ba450> (a org.mortbay.util.ThreadPool$PoolThread)

          "Map output copy reporter for task task_0023_r_000085_0" daemon prio=1 tid=0x0a3c9ca8 nid=0x49c2 waiting on condition [0x4b28e000..0x4b28f040]
          at java.lang.Thread.sleep(Native Method)
          at org.apache.hadoop.mapred.ReduceTaskRunner$PingTimer.run(ReduceTaskRunner.java:189)

          "MapOutputCopier task_0023_r_000085_0.294" prio=1 tid=0x0971a398 nid=0x49c0 in Object.wait() [0x4af07000..0x4af07f40]
          at java.lang.Object.wait(Native Method)
          at java.lang.Object.wait(Object.java:474)
          at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267)

          • locked <0x567557b8> (a java.util.ArrayList)

          "MapOutputCopier task_0023_r_000085_0.292" prio=1 tid=0x096ff088 nid=0x49be in Object.wait() [0x4af88000..0x4af88e40]
          at java.lang.Object.wait(Native Method)
          at java.lang.Object.wait(Object.java:474)
          at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267)

          • locked <0x567557b8> (a java.util.ArrayList)

          "MapOutputCopier task_0023_r_000085_0.290" prio=1 tid=0x09bd99f8 nid=0x49bc in Object.wait() [0x4c1ad000..0x4c1ad140]
          at java.lang.Object.wait(Native Method)
          at java.lang.Object.wait(Object.java:474)
          at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267)

          • locked <0x567557b8> (a java.util.ArrayList)

          "MapOutputCopier task_0023_r_000085_0.288" prio=1 tid=0x0a314ce0 nid=0x49ba in Object.wait() [0x4bca2000..0x4bca3040]
          at java.lang.Object.wait(Native Method)
          at java.lang.Object.wait(Object.java:474)
          at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267)

          • locked <0x567557b8> (a java.util.ArrayList)

          "MapOutputCopier task_0023_r_000085_0.286" prio=1 tid=0x093d2c18 nid=0x49b8 in Object.wait() [0x4d15f000..0x4d15ff40]
          at java.lang.Object.wait(Native Method)
          at java.lang.Object.wait(Object.java:474)
          at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267)

          • locked <0x567557b8> (a java.util.ArrayList)

          "Thread-27723" prio=1 tid=0x093d29a8 nid=0x49b7 waiting on condition [0x4b99c000..0x4b99cec0]
          at java.lang.Thread.sleep(Native Method)
          at org.apache.hadoop.mapred.ReduceTaskRunner.prepare(ReduceTaskRunner.java:572)
          at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:143)

          "MultiThreadedHttpConnectionManager cleanup" daemon prio=1 tid=0x0860d368 nid=0x7e66 in Object.wait() [0x4d1e0000..0x4d1e1040]
          at java.lang.Object.wait(Native Method)
          at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)

          • locked <0x56418ca8> (a java.lang.ref.ReferenceQueue$Lock)
            at org.apache.commons.httpclient.MultiThreadedHttpConnectionManager$ReferenceQueueThread.run(MultiThreadedHttpConnectionManager.java:1082)

          "org.apache.hadoop.io.ObjectWritable Connection Culler" daemon prio=1 tid=0x085c4fb0 nid=0x7e64 waiting on condition [0x4d2e2000..0x4d2e2f40]
          at java.lang.Thread.sleep(Native Method)
          at org.apache.hadoop.ipc.Client$ConnectionCuller.run(Client.java:401)

          "IPC Server handler 1 on 50050" daemon prio=1 tid=0x085bbaf0 nid=0x7e63 in Object.wait() [0x4d363000..0x4d363ec0]
          at java.lang.Object.wait(Native Method)
          at org.apache.hadoop.ipc.Server$Handler.run(Server.java:533)

          • locked <0x5640dcd8> (a java.util.LinkedList)

          "IPC Server handler 0 on 50050" daemon prio=1 tid=0x085c3fc8 nid=0x7e62 in Object.wait() [0x4d3e4000..0x4d3e4e40]
          at java.lang.Object.wait(Native Method)
          at org.apache.hadoop.ipc.Server$Handler.run(Server.java:533)

          • locked <0x5640dcd8> (a java.util.LinkedList)

          "IPC Server listener on 50050" daemon prio=1 tid=0x085c3ab0 nid=0x7e61 runnable [0x4d465000..0x4d4661c0]
          at sun.nio.ch.PollArrayWrapper.poll0(Native Method)
          at sun.nio.ch.PollArrayWrapper.poll(PollArrayWrapper.java:100)
          at sun.nio.ch.PollSelectorImpl.doSelect(PollSelectorImpl.java:56)
          at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)

          • locked <0x5640de48> (a sun.nio.ch.Util$1)
          • locked <0x5640de58> (a java.util.Collections$UnmodifiableSet)
          • locked <0x5640ddf8> (a sun.nio.ch.PollSelectorImpl)
            at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
            at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)
            at org.apache.hadoop.ipc.Server$Listener.run(Server.java:253)

          "Acceptor ServerSocket[addr=0.0.0.0/0.0.0.0,port=0,localport=50060]" prio=1 tid=0x0859f278 nid=0x7e5f runnable [0x4d56f000..0x4d5700c0]
          at java.net.PlainSocketImpl.socketAccept(Native Method)
          at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384)

          • locked <0x564028b8> (a java.net.SocksSocketImpl)
            at java.net.ServerSocket.implAccept(ServerSocket.java:450)
            at java.net.ServerSocket.accept(ServerSocket.java:421)
            at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432)
            at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631)

          "SessionScavenger" daemon prio=1 tid=0x08599fc8 nid=0x7e5e waiting on condition [0x4d5f0000..0x4d5f1040]
          at java.lang.Thread.sleep(Native Method)
          at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587)

          "taskCleanup" daemon prio=1 tid=0x082f4f60 nid=0x7e5b waiting on condition [0x4d701000..0x4d701fc0]
          at sun.misc.Unsafe.park(Native Method)
          at java.util.concurrent.locks.LockSupport.park(LockSupport.java:118)
          at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1767)
          at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:359)
          at org.apache.hadoop.mapred.TaskTracker$1.run(TaskTracker.java:181)
          at java.lang.Thread.run(Thread.java:595)

          "Low Memory Detector" daemon prio=1 tid=0x080c99b8 nid=0x7e59 runnable [0x00000000..0x00000000]

          "CompilerThread0" daemon prio=1 tid=0x080c8408 nid=0x7e58 waiting on condition [0x00000000..0x4e76b828]

          "Signal Dispatcher" daemon prio=1 tid=0x080c7580 nid=0x7e57 waiting on condition [0x00000000..0x00000000]

          "Finalizer" daemon prio=1 tid=0x080bcf00 nid=0x7e53 in Object.wait() [0x4e86e000..0x4e86e140]
          at java.lang.Object.wait(Native Method)
          at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)

          • locked <0x562d2798> (a java.lang.ref.ReferenceQueue$Lock)
            at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
            at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)

          "Reference Handler" daemon prio=1 tid=0x080bc238 nid=0x7e52 in Object.wait() [0x4e8ee000..0x4e8ef0c0]
          at java.lang.Object.wait(Native Method)
          at java.lang.Object.wait(Object.java:474)
          at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)

          • locked <0x562d2818> (a java.lang.ref.Reference$Lock)

          "main" prio=1 tid=0x0805dc50 nid=0x7e3b in Object.wait() [0xbfc72000..0xbfc725c8]
          at java.lang.Object.wait(Native Method)
          at org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:509)

          • locked <0x56311618> (a [I)
            at org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:899)
            at org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:1589)

          "VM Thread" prio=1 tid=0x080b9540 nid=0x7e51 runnable

          "VM Periodic Task Thread" prio=1 tid=0x080cae98 nid=0x7e5a waiting on condition

          Show
          mike.smith Mike Smith added a comment - As Kostantin mentioed, all maps are done and only the last reducer stucks and task will not fail after 10 min, because jobtracker still gets the heartbeats. By the way, here is the core dump when the reducers stucks and the whole cluster runs in linux: Full thread dump Java HotSpot(TM) Client VM (1.5.0_11-b03 mixed mode): "IPC Client connection to jobtracker_address/jobtracker_address:50002" daemon prio=1 tid=0x096f29d8 nid=0x76cc in Object.wait() [0x4cfdc000..0x4cfdcf40] at java.lang.Object.wait(Native Method) at java.lang.Object.wait(Object.java:474) at org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:213) locked <0x4f13b5c0> (a org.apache.hadoop.ipc.Client$Connection) at org.apache.hadoop.ipc.Client$Connection.run(Client.java:252) "SocketListener0-19" prio=1 tid=0x0a61ccf8 nid=0x49ed in Object.wait() [0x4b08a000..0x4b08af40] at java.lang.Object.wait(Native Method) at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:522) locked <0x567ba450> (a org.mortbay.util.ThreadPool$PoolThread) "Map output copy reporter for task task_0023_r_000085_0" daemon prio=1 tid=0x0a3c9ca8 nid=0x49c2 waiting on condition [0x4b28e000..0x4b28f040] at java.lang.Thread.sleep(Native Method) at org.apache.hadoop.mapred.ReduceTaskRunner$PingTimer.run(ReduceTaskRunner.java:189) "MapOutputCopier task_0023_r_000085_0.294" prio=1 tid=0x0971a398 nid=0x49c0 in Object.wait() [0x4af07000..0x4af07f40] at java.lang.Object.wait(Native Method) at java.lang.Object.wait(Object.java:474) at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267) locked <0x567557b8> (a java.util.ArrayList) "MapOutputCopier task_0023_r_000085_0.292" prio=1 tid=0x096ff088 nid=0x49be in Object.wait() [0x4af88000..0x4af88e40] at java.lang.Object.wait(Native Method) at java.lang.Object.wait(Object.java:474) at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267) locked <0x567557b8> (a java.util.ArrayList) "MapOutputCopier task_0023_r_000085_0.290" prio=1 tid=0x09bd99f8 nid=0x49bc in Object.wait() [0x4c1ad000..0x4c1ad140] at java.lang.Object.wait(Native Method) at java.lang.Object.wait(Object.java:474) at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267) locked <0x567557b8> (a java.util.ArrayList) "MapOutputCopier task_0023_r_000085_0.288" prio=1 tid=0x0a314ce0 nid=0x49ba in Object.wait() [0x4bca2000..0x4bca3040] at java.lang.Object.wait(Native Method) at java.lang.Object.wait(Object.java:474) at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267) locked <0x567557b8> (a java.util.ArrayList) "MapOutputCopier task_0023_r_000085_0.286" prio=1 tid=0x093d2c18 nid=0x49b8 in Object.wait() [0x4d15f000..0x4d15ff40] at java.lang.Object.wait(Native Method) at java.lang.Object.wait(Object.java:474) at org.apache.hadoop.mapred.ReduceTaskRunner$MapOutputCopier.run(ReduceTaskRunner.java:267) locked <0x567557b8> (a java.util.ArrayList) "Thread-27723" prio=1 tid=0x093d29a8 nid=0x49b7 waiting on condition [0x4b99c000..0x4b99cec0] at java.lang.Thread.sleep(Native Method) at org.apache.hadoop.mapred.ReduceTaskRunner.prepare(ReduceTaskRunner.java:572) at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:143) "MultiThreadedHttpConnectionManager cleanup" daemon prio=1 tid=0x0860d368 nid=0x7e66 in Object.wait() [0x4d1e0000..0x4d1e1040] at java.lang.Object.wait(Native Method) at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116) locked <0x56418ca8> (a java.lang.ref.ReferenceQueue$Lock) at org.apache.commons.httpclient.MultiThreadedHttpConnectionManager$ReferenceQueueThread.run(MultiThreadedHttpConnectionManager.java:1082) "org.apache.hadoop.io.ObjectWritable Connection Culler" daemon prio=1 tid=0x085c4fb0 nid=0x7e64 waiting on condition [0x4d2e2000..0x4d2e2f40] at java.lang.Thread.sleep(Native Method) at org.apache.hadoop.ipc.Client$ConnectionCuller.run(Client.java:401) "IPC Server handler 1 on 50050" daemon prio=1 tid=0x085bbaf0 nid=0x7e63 in Object.wait() [0x4d363000..0x4d363ec0] at java.lang.Object.wait(Native Method) at org.apache.hadoop.ipc.Server$Handler.run(Server.java:533) locked <0x5640dcd8> (a java.util.LinkedList) "IPC Server handler 0 on 50050" daemon prio=1 tid=0x085c3fc8 nid=0x7e62 in Object.wait() [0x4d3e4000..0x4d3e4e40] at java.lang.Object.wait(Native Method) at org.apache.hadoop.ipc.Server$Handler.run(Server.java:533) locked <0x5640dcd8> (a java.util.LinkedList) "IPC Server listener on 50050" daemon prio=1 tid=0x085c3ab0 nid=0x7e61 runnable [0x4d465000..0x4d4661c0] at sun.nio.ch.PollArrayWrapper.poll0(Native Method) at sun.nio.ch.PollArrayWrapper.poll(PollArrayWrapper.java:100) at sun.nio.ch.PollSelectorImpl.doSelect(PollSelectorImpl.java:56) at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) locked <0x5640de48> (a sun.nio.ch.Util$1) locked <0x5640de58> (a java.util.Collections$UnmodifiableSet) locked <0x5640ddf8> (a sun.nio.ch.PollSelectorImpl) at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84) at org.apache.hadoop.ipc.Server$Listener.run(Server.java:253) "Acceptor ServerSocket [addr=0.0.0.0/0.0.0.0,port=0,localport=50060] " prio=1 tid=0x0859f278 nid=0x7e5f runnable [0x4d56f000..0x4d5700c0] at java.net.PlainSocketImpl.socketAccept(Native Method) at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:384) locked <0x564028b8> (a java.net.SocksSocketImpl) at java.net.ServerSocket.implAccept(ServerSocket.java:450) at java.net.ServerSocket.accept(ServerSocket.java:421) at org.mortbay.util.ThreadedServer.acceptSocket(ThreadedServer.java:432) at org.mortbay.util.ThreadedServer$Acceptor.run(ThreadedServer.java:631) "SessionScavenger" daemon prio=1 tid=0x08599fc8 nid=0x7e5e waiting on condition [0x4d5f0000..0x4d5f1040] at java.lang.Thread.sleep(Native Method) at org.mortbay.jetty.servlet.AbstractSessionManager$SessionScavenger.run(AbstractSessionManager.java:587) "taskCleanup" daemon prio=1 tid=0x082f4f60 nid=0x7e5b waiting on condition [0x4d701000..0x4d701fc0] at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:118) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1767) at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:359) at org.apache.hadoop.mapred.TaskTracker$1.run(TaskTracker.java:181) at java.lang.Thread.run(Thread.java:595) "Low Memory Detector" daemon prio=1 tid=0x080c99b8 nid=0x7e59 runnable [0x00000000..0x00000000] "CompilerThread0" daemon prio=1 tid=0x080c8408 nid=0x7e58 waiting on condition [0x00000000..0x4e76b828] "Signal Dispatcher" daemon prio=1 tid=0x080c7580 nid=0x7e57 waiting on condition [0x00000000..0x00000000] "Finalizer" daemon prio=1 tid=0x080bcf00 nid=0x7e53 in Object.wait() [0x4e86e000..0x4e86e140] at java.lang.Object.wait(Native Method) at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116) locked <0x562d2798> (a java.lang.ref.ReferenceQueue$Lock) at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132) at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159) "Reference Handler" daemon prio=1 tid=0x080bc238 nid=0x7e52 in Object.wait() [0x4e8ee000..0x4e8ef0c0] at java.lang.Object.wait(Native Method) at java.lang.Object.wait(Object.java:474) at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116) locked <0x562d2818> (a java.lang.ref.Reference$Lock) "main" prio=1 tid=0x0805dc50 nid=0x7e3b in Object.wait() [0xbfc72000..0xbfc725c8] at java.lang.Object.wait(Native Method) at org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:509) locked <0x56311618> (a [I) at org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:899) at org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:1589) "VM Thread" prio=1 tid=0x080b9540 nid=0x7e51 runnable "VM Periodic Task Thread" prio=1 tid=0x080cae98 nid=0x7e5a waiting on condition
          Hide
          devaraj Devaraj Das added a comment -

          Mike, since you are using 0.12.3, there is a good amount of chance that you are hitting HADOOP-1152 or HADOOP-1183 (HADOOP-1270 addresses HADOOP-1183 under the covers). Both these fixes will be there in the 0.13 release. This is not to say that there could not be other issues.

          Show
          devaraj Devaraj Das added a comment - Mike, since you are using 0.12.3, there is a good amount of chance that you are hitting HADOOP-1152 or HADOOP-1183 ( HADOOP-1270 addresses HADOOP-1183 under the covers). Both these fixes will be there in the 0.13 release. This is not to say that there could not be other issues.
          Hide
          owen.omalley Owen O'Malley added a comment -

          Konstantin, can you still reproduce this one still?

          Show
          owen.omalley Owen O'Malley added a comment - Konstantin, can you still reproduce this one still?
          Hide
          shv Konstantin Shvachko added a comment -

          Yes, unfortunately I can still see this with the current trunk.

          Show
          shv Konstantin Shvachko added a comment - Yes, unfortunately I can still see this with the current trunk.
          Hide
          devaraj Devaraj Das added a comment -

          Konstantin, could you please give it a shot again with the current trunk.

          Show
          devaraj Devaraj Das added a comment - Konstantin, could you please give it a shot again with the current trunk.
          Hide
          lilyco Yunhong Gu added a comment -

          Hi,

          I can still reproduce this problem with 0.15.2 and 0.15.3. If you need any more information, I can post here. I can even help to debug with your instructions. It is very easy to reproduce on my servers.

          Show
          lilyco Yunhong Gu added a comment - Hi, I can still reproduce this problem with 0.15.2 and 0.15.3. If you need any more information, I can post here. I can even help to debug with your instructions. It is very easy to reproduce on my servers.
          Hide
          acmurthy Arun C Murthy added a comment -

          Please upload the JT/TT and task logs (as a tgz), thanks!

          Show
          acmurthy Arun C Murthy added a comment - Please upload the JT/TT and task logs (as a tgz), thanks!
          Hide
          acmurthy Arun C Murthy added a comment -

          Forgot to add: we have features which enable a reduce to give feedback to the JT about maps from which it is failing to fetch outputs; which leads to the map being killed and being restarted (HADOOP-1158). In 0.15.0 it takes a bit of time for the errant map to get killed and re-run somwhere else, so you might just have to wait for sometime (at times upto 30mins) before this occurs... 0.16.0 has improves upon that further (HADOOP-1984).

          Show
          acmurthy Arun C Murthy added a comment - Forgot to add: we have features which enable a reduce to give feedback to the JT about maps from which it is failing to fetch outputs; which leads to the map being killed and being restarted ( HADOOP-1158 ). In 0.15.0 it takes a bit of time for the errant map to get killed and re-run somwhere else, so you might just have to wait for sometime (at times upto 30mins) before this occurs... 0.16.0 has improves upon that further ( HADOOP-1984 ).
          Hide
          lilyco Yunhong Gu added a comment -

          log file added (hadoop-1374.tar.gz)

          The program "mrbench" eventually stopped, as you said, after about 30 minutes.

          Show
          lilyco Yunhong Gu added a comment - log file added (hadoop-1374.tar.gz) The program "mrbench" eventually stopped, as you said, after about 30 minutes.
          Hide
          lilyco Yunhong Gu added a comment -

          I think I found the problem. It is caused by failed hostname resolving, so that either JT or TT cannot get data from another node (because no network connection can be set up), so it is blocked, for about 30 minutes.

          Show
          lilyco Yunhong Gu added a comment - I think I found the problem. It is caused by failed hostname resolving, so that either JT or TT cannot get data from another node (because no network connection can be set up), so it is blocked, for about 30 minutes.
          Hide
          michaelsembwever mck added a comment -

          Seems similar to HADOOP-3362
          I solved it (or something looking similar to me) by opening port 50060 in the firewall between nodes.

          Show
          michaelsembwever mck added a comment - Seems similar to HADOOP-3362 I solved it (or something looking similar to me) by opening port 50060 in the firewall between nodes.
          Hide
          jinseng jinseng lin added a comment -

          I encounter this problem too. My cluster relies on the /etc/hosts file to resolve host name and my fault causing this problem is that there is a wrong record in the /etc/hosts file.
          After correcting the wrong record and then the problem is solved.

          Show
          jinseng jinseng lin added a comment - I encounter this problem too. My cluster relies on the /etc/hosts file to resolve host name and my fault causing this problem is that there is a wrong record in the /etc/hosts file. After correcting the wrong record and then the problem is solved.

            People

            • Assignee:
              acmurthy Arun C Murthy
              Reporter:
              shv Konstantin Shvachko
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:

                Development