You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by tg...@apache.org on 2012/11/28 20:23:53 UTC

svn commit: r1414875 - in /hadoop/common/branches/branch-0.23/hadoop-mapreduce-project: ./ hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/ hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org...

Author: tgraves
Date: Wed Nov 28 19:23:52 2012
New Revision: 1414875

URL: http://svn.apache.org/viewvc?rev=1414875&view=rev
Log:
merge -r 1414872:1414873 from trunk. FIXES: MAPREDUCE-4817

Modified:
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt?rev=1414875&r1=1414874&r2=1414875&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt Wed Nov 28 19:23:52 2012
@@ -20,6 +20,9 @@ Release 0.23.6 - UNRELEASED
     MAPREDUCE-4825. JobImpl.finished doesn't expect ERROR as a final job state
     (jlowe via bobby)
 
+    MAPREDUCE-4817. Hardcoded task ping timeout kills tasks localizing large 
+    amounts of data (tgraves)
+
 Release 0.23.5 - UNRELEASED
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java?rev=1414875&r1=1414874&r2=1414875&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java Wed Nov 28 19:23:52 2012
@@ -274,7 +274,6 @@ public class TaskAttemptListenerImpl ext
   @Override
   public boolean ping(TaskAttemptID taskAttemptID) throws IOException {
     LOG.info("Ping from " + taskAttemptID.toString());
-    taskHeartbeatHandler.pinged(TypeConverter.toYarn(taskAttemptID));
     return true;
   }
 

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java?rev=1414875&r1=1414874&r2=1414875&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java Wed Nov 28 19:23:52 2012
@@ -46,33 +46,22 @@ import org.apache.hadoop.yarn.service.Ab
 public class TaskHeartbeatHandler extends AbstractService {
   
   private static class ReportTime {
-    private long lastPing;
     private long lastProgress;
     
     public ReportTime(long time) {
       setLastProgress(time);
     }
     
-    public synchronized void setLastPing(long time) {
-      lastPing = time;
-    }
-    
     public synchronized void setLastProgress(long time) {
       lastProgress = time;
-      lastPing = time;
-    }
-    
-    public synchronized long getLastPing() {
-      return lastPing;
     }
-    
+
     public synchronized long getLastProgress() {
       return lastProgress;
     }
   }
   
   private static final Log LOG = LogFactory.getLog(TaskHeartbeatHandler.class);
-  private static final int PING_TIMEOUT = 5 * 60 * 1000;
   
   //thread which runs periodically to see the last time since a heartbeat is
   //received from a task.
@@ -127,14 +116,6 @@ public class TaskHeartbeatHandler extend
     }
   }
 
-  public void pinged(TaskAttemptId attemptID) {
-    //only put for the registered attempts
-      //TODO throw an exception if the task isn't registered.
-      ReportTime time = runningAttempts.get(attemptID);
-      if(time != null) {
-        time.setLastPing(clock.getTime());
-      }
-    }
   
   public void register(TaskAttemptId attemptID) {
     runningAttempts.put(attemptID, new ReportTime(clock.getTime()));
@@ -159,10 +140,8 @@ public class TaskHeartbeatHandler extend
           Map.Entry<TaskAttemptId, ReportTime> entry = iterator.next();
           boolean taskTimedOut = (taskTimeOut > 0) && 
               (currentTime > (entry.getValue().getLastProgress() + taskTimeOut));
-          boolean pingTimedOut =
-              (currentTime > (entry.getValue().getLastPing() + PING_TIMEOUT));
-              
-          if(taskTimedOut || pingTimedOut) {
+           
+          if(taskTimedOut) {
             // task is lost, remove from the list and raise lost event
             iterator.remove();
             eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry