Uploaded image for project: 'Hadoop Map/Reduce'
  1. Hadoop Map/Reduce
  2. MAPREDUCE-6190

If a task stucks before its first heartbeat, it never timeouts and the MR job becomes stuck

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.6.0, 2.7.0, 2.8.0, 2.9.0, 3.0.0, 3.1.1
    • 3.3.0
    • None
    • None
    • Reviewed
    • Added "mapreduce.task.stuck.timeout-ms" parameter to timeout a task before sending the task's first heartbeat. The default value is 600000 milliseconds (10 minutes).

    Description

      Trying to figure out a weird issue we started seeing on our CDH5.1.0 cluster with map reduce jobs on YARN.

      We had a job stuck for hours because one of the mappers never started up fully. Basically, the map task had 2 attempts, the first one failed and the AM tried to schedule a second one and the second attempt was stuck on STATE: STARTING, STATUS: NEW. A node never got assigned and the task along with the job was stuck indefinitely.

      The AM logs had this being logged again and again:

      2014-12-09 19:25:12,347 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Ramping down 0
      2014-12-09 19:25:13,352 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Received completed container container_1408745633994_450952_02_003807
      2014-12-09 19:25:13,352 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Reduce preemption successful attempt_1408745633994_450952_r_000048_1000
      2014-12-09 19:25:13,352 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Ramping down all scheduled reduces:0
      2014-12-09 19:25:13,352 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Going to preempt 1
      2014-12-09 19:25:13,353 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Preempting attempt_1408745633994_450952_r_000050_1000
      2014-12-09 19:25:13,353 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Recalculating schedule, headroom=0
      2014-12-09 19:25:13,353 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: completedMapPercent 0.99968 totalMemLimit:1722880 finalMapMemLimit:2560 finalReduceMemLimit:1720320 netScheduledMapMem:2560 netScheduledReduceMem:1722880
      2014-12-09 19:25:13,353 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Ramping down 0
      2014-12-09 19:25:13,353 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: After Scheduling: PendingReds:77 ScheduledMaps:1 ScheduledReds:0 AssignedMaps:0 AssignedReds:673 CompletedMaps:3124 CompletedReds:0 ContAlloc:4789 ContRel:798 HostLocal:2944 RackLocal:155
      2014-12-09 19:25:14,353 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Before Scheduling: PendingReds:78 ScheduledMaps:1 ScheduledReds:0 AssignedMaps:0 AssignedReds:673 CompletedMaps:3124 CompletedReds:0 ContAlloc:4789 ContRel:798 HostLocal:2944 RackLocal:155
      2014-12-09 19:25:14,359 INFO [RMCommunicator Allocator] org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Recalculating schedule, headroom=0
      

      On killing the task manually, the AM started up the task again, scheduled and ran it successfully completing the task and the job with it.

      Some quick code grepping led us here:
      http://grepcode.com/file/repo1.maven.org/maven2/org.apache.hadoop/hadoop-mapreduce-client-app/2.3.0/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java#397

      But still dont quite understand why this would happen once in a while and why the job would suddenly be ok once the stuck task is manually killed.

      Note: Other jobs succeed on the cluster while this job is stuck.

      Attachments

        1. MAPREDUCE-6190.005.patch
          11 kB
          Zhaohui Xin
        2. MAPREDUCE-6190.004.patch
          11 kB
          Zhaohui Xin
        3. MAPREDUCE-6190.003.patch
          11 kB
          Zhaohui Xin
        4. MAPREDUCE-6190.002.patch
          9 kB
          Zhaohui Xin
        5. MAPREDUCE-6190.001.patch
          5 kB
          Zhaohui Xin

        Issue Links

          Activity

            People

              uranus Zhaohui Xin
              amalhotra159 Ankit Malhotra
              Votes:
              5 Vote for this issue
              Watchers:
              20 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: