Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 1.0.0, 1.0.1, 1.0.2, 1.0.3
    • Fix Version/s: 1.1.2
    • Component/s: None
    • Labels:
      None
    • Target Version/s:
    • Hadoop Flags:
      Reviewed
    • Release Note:
      Fixed a bug in TaskTracker's heartbeat to keep it under control.
    1. 4478.diff
      0.8 kB
      Liyin Liang
    2. MAPREDUCE-4478.patch
      0.7 kB
      Suresh Srinivas

      Issue Links

        Activity

        Transition Time In Source Status Execution Times Last Executer Last Execution Date
        Open Open Resolved Resolved
        128d 15h 19m 1 Vinod Kumar Vavilapalli 30/Nov/12 20:49
        Resolved Resolved Closed Closed
        95d 13h 6m 1 Matt Foley 06/Mar/13 09:55
        Matt Foley made changes -
        Status Resolved [ 5 ] Closed [ 6 ]
        Hide
        Matt Foley added a comment -

        Closed upon successful release of 1.1.2.

        Show
        Matt Foley added a comment - Closed upon successful release of 1.1.2.
        Suresh Srinivas made changes -
        Fix Version/s 1.1.1 [ 12321660 ]
        Hide
        Suresh Srinivas added a comment -

        Realized it was already merged to branch-1.1. Thx Vinod for merging.

        Show
        Suresh Srinivas added a comment - Realized it was already merged to branch-1.1. Thx Vinod for merging.
        Suresh Srinivas made changes -
        Fix Version/s 1.1.2 [ 12323594 ]
        Target Version/s 1.1.2 [ 12323594 ]
        Hide
        Suresh Srinivas added a comment -

        I merged the change to branch-1.1 to be picked up for 1.1.2.

        Show
        Suresh Srinivas added a comment - I merged the change to branch-1.1 to be picked up for 1.1.2.
        Vinod Kumar Vavilapalli made changes -
        Status Open [ 1 ] Resolved [ 5 ]
        Hadoop Flags Reviewed [ 10343 ]
        Release Note Fixed a bug in TaskTracker's heartbeat to keep it under control.
        Fix Version/s 1.1.1 [ 12321660 ]
        Resolution Fixed [ 1 ]
        Hide
        Vinod Kumar Vavilapalli added a comment -

        Just committed this to branch-1 and branch-1.1. Thanks Liyin!

        Thanks for the patch update, Suresh.

        Show
        Vinod Kumar Vavilapalli added a comment - Just committed this to branch-1 and branch-1.1. Thanks Liyin! Thanks for the patch update, Suresh.
        Hide
        Vinod Kumar Vavilapalli added a comment -

        Great catch Liyin! And patch looks good. Checking it in.

        Show
        Vinod Kumar Vavilapalli added a comment - Great catch Liyin! And patch looks good. Checking it in.
        Suresh Srinivas made changes -
        Attachment MAPREDUCE-4478.patch [ 12555552 ]
        Hide
        Suresh Srinivas added a comment -

        Updated patch that addresses Luke's comment.

        Show
        Suresh Srinivas added a comment - Updated patch that addresses Luke's comment.
        Suresh Srinivas made changes -
        Assignee Liyin Liang [ liangly ]
        Hide
        Luke Lu added a comment -

        The synchronized is not necessary is finishedCount is already an AtomicInteger. Otherwise the patch lgtm.

        Show
        Luke Lu added a comment - The synchronized is not necessary is finishedCount is already an AtomicInteger. Otherwise the patch lgtm.
        Liyin Liang made changes -
        Attachment 4478.diff [ 12537810 ]
        Hide
        Liyin Liang added a comment -

        Attach a patch to fix this bug. I don't know whether the synchronized is necessary.

        Show
        Liyin Liang added a comment - Attach a patch to fix this bug. I don't know whether the synchronized is necessary.
        Liyin Liang made changes -
        Link This issue relates to MAPREDUCE-2355 [ MAPREDUCE-2355 ]
        Hide
        Liyin Liang added a comment -

        There are two configuration items to control the TaskTracker's heartbeat interval. One is mapreduce.tasktracker.outofband.heartbeat. The other is mapreduce.tasktracker.outofband.heartbeat.damper. If we set mapreduce.tasktracker.outofband.heartbeat with true and set mapreduce.tasktracker.outofband.heartbeat.damper with default value (1000000), TaskTracker may send heartbeat without any interval.

        The code to control heartbeat interval is as follows:

        long now = System.currentTimeMillis();
                
                // accelerate to account for multiple finished tasks up-front
                long remaining = 
                  (lastHeartbeat + getHeartbeatInterval(finishedCount.get())) - now;
                while (remaining > 0) {
                  // sleeps for the wait time or 
                  // until there are *enough* empty slots to schedule tasks
                  synchronized (finishedCount) {
                    finishedCount.wait(remaining);
                    
                    // Recompute
                    now = System.currentTimeMillis();
                    remaining = 
                      (lastHeartbeat + getHeartbeatInterval(finishedCount.get())) - now;
                    
                    if (remaining <= 0) {
                      // Reset count 
                      finishedCount.set(0);
                      break;
                    }
                  }
                }
        

        During the first time computing, if finishedCount is more than zero, getHeartbeatInterval(finishedCount.get()) will return zero. Then remaining will be less than or equal with zero. In this case, the while loop will be skipped. So finishedCount will never be set with zero.

        Show
        Liyin Liang added a comment - There are two configuration items to control the TaskTracker's heartbeat interval. One is mapreduce.tasktracker.outofband.heartbeat . The other is mapreduce.tasktracker.outofband.heartbeat.damper . If we set mapreduce.tasktracker.outofband.heartbeat with true and set mapreduce.tasktracker.outofband.heartbeat.damper with default value (1000000), TaskTracker may send heartbeat without any interval. The code to control heartbeat interval is as follows: long now = System .currentTimeMillis(); // accelerate to account for multiple finished tasks up-front long remaining = (lastHeartbeat + getHeartbeatInterval(finishedCount.get())) - now; while (remaining > 0) { // sleeps for the wait time or // until there are *enough* empty slots to schedule tasks synchronized (finishedCount) { finishedCount.wait(remaining); // Recompute now = System .currentTimeMillis(); remaining = (lastHeartbeat + getHeartbeatInterval(finishedCount.get())) - now; if (remaining <= 0) { // Reset count finishedCount.set(0); break ; } } } During the first time computing, if finishedCount is more than zero, getHeartbeatInterval(finishedCount.get()) will return zero. Then remaining will be less than or equal with zero. In this case, the while loop will be skipped. So finishedCount will never be set with zero.
        Liyin Liang made changes -
        Field Original Value New Value
        Project Hadoop HDFS [ 12310942 ] Hadoop Map/Reduce [ 12310941 ]
        Key HDFS-3722 MAPREDUCE-4478
        Affects Version/s 1.0.3 [ 12320250 ]
        Affects Version/s 1.0.2 [ 12320047 ]
        Affects Version/s 1.0.1 [ 12319503 ]
        Affects Version/s 1.0.0 [ 12318240 ]
        Affects Version/s 1.0.0 [ 12318243 ]
        Affects Version/s 1.0.1 [ 12319502 ]
        Affects Version/s 1.0.2 [ 12320051 ]
        Affects Version/s 1.0.3 [ 12320249 ]
        Liyin Liang created issue -

          People

          • Assignee:
            Liyin Liang
            Reporter:
            Liyin Liang
          • Votes:
            0 Vote for this issue
            Watchers:
            12 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development