You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by dd...@apache.org on 2009/02/26 07:53:09 UTC

svn commit: r748040 - in /hadoop/core/branches/branch-0.19: ./ src/mapred/org/apache/hadoop/mapred/

Author: ddas
Date: Thu Feb 26 06:53:08 2009
New Revision: 748040

URL: http://svn.apache.org/viewvc?rev=748040&view=rev
Log:
HADOOP-5233. Committing this to the 0.19 branch.

Modified:
    hadoop/core/branches/branch-0.19/CHANGES.txt
    hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/JobInProgress.java
    hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/Task.java
    hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskInProgress.java
    hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskStatus.java
    hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskTracker.java

Modified: hadoop/core/branches/branch-0.19/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/CHANGES.txt?rev=748040&r1=748039&r2=748040&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/CHANGES.txt (original)
+++ hadoop/core/branches/branch-0.19/CHANGES.txt Thu Feb 26 06:53:08 2009
@@ -16,6 +16,11 @@
     HADOOP-5241. Fixes a bug in disk-space resource estimation. Makes the estimation
     formula linear where blowUp = Total-Output/Total-Input. (Sharad Agarwal via ddas)
 
+    HADOOP-5233. Addresses the three issues - Race condition in updating
+    status, NPE in TaskTracker task localization when the conf file is missing
+    (HADOOP-5234) and NPE in handling KillTaskAction of a cleanup task (HADOOP-5235).
+    (Amareshwari Sriramadasu via ddas)
+
 Release 0.19.1 - 2009-02-23
 
   INCOMPATIBLE CHANGES

Modified: hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/JobInProgress.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/JobInProgress.java?rev=748040&r1=748039&r2=748040&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/JobInProgress.java (original)
+++ hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/JobInProgress.java Thu Feb 26 06:53:08 2009
@@ -768,6 +768,8 @@
         } else {
           reduceCleanupTasks.add(taskid);
         }
+        // Remove the task entry from jobtracker
+        jobtracker.removeTaskEntry(taskid);
       }
       //For a failed task update the JT datastructures. 
       else if (state == TaskStatus.State.FAILED ||

Modified: hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/Task.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/Task.java?rev=748040&r1=748039&r2=748040&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/Task.java (original)
+++ hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/Task.java Thu Feb 26 06:53:08 2009
@@ -411,8 +411,7 @@
               if (sendProgress) {
                 // we need to send progress update
                 updateCounters();
-                taskStatus.statusUpdate(getState(),
-                                        taskProgress.get(),
+                taskStatus.statusUpdate(taskProgress.get(),
                                         taskProgress.toString(), 
                                         counters);
                 taskFound = umbilical.statusUpdate(taskId, taskStatus);
@@ -680,8 +679,7 @@
   private void sendLastUpdate(TaskUmbilicalProtocol umbilical) 
   throws IOException {
     // send a final status report
-    taskStatus.statusUpdate(getState(),
-                            taskProgress.get(),
+    taskStatus.statusUpdate(taskProgress.get(),
                             taskProgress.toString(), 
                             counters);
     statusUpdate(umbilical);

Modified: hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskInProgress.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskInProgress.java?rev=748040&r1=748039&r2=748040&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskInProgress.java (original)
+++ hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskInProgress.java Thu Feb 26 06:53:08 2009
@@ -910,7 +910,6 @@
       t.setTaskCleanupTask();
       t.setState(taskStatuses.get(taskid).getRunState());
       cleanupTasks.put(taskid, taskTracker);
-      jobtracker.removeTaskEntry(taskid);
     }
     t.setConf(conf);
     LOG.debug("Launching task with skipRanges:"+failedRanges.getSkipRanges());

Modified: hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskStatus.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskStatus.java?rev=748040&r1=748039&r2=748040&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskStatus.java (original)
+++ hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskStatus.java Thu Feb 26 06:53:08 2009
@@ -54,7 +54,7 @@
   private long finishTime; 
   private long outputSize;
     
-  private Phase phase = Phase.STARTING; 
+  private volatile Phase phase = Phase.STARTING; 
   private Counters counters;
   private boolean includeCounters;
   private SortedRanges.Range nextRecordRange = new SortedRanges.Range();
@@ -265,16 +265,15 @@
   /**
    * Update the status of the task.
    * 
-   * @param runstate
+   * This update is done by ping thread before sending the status. 
+   * 
    * @param progress
    * @param state
    * @param counters
    */
-  synchronized void statusUpdate(State runState, 
-                                 float progress,
+  synchronized void statusUpdate(float progress,
                                  String state, 
                                  Counters counters) {
-    setRunState(runState);
     setProgress(progress);
     setStateString(state);
     setCounters(counters);

Modified: hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskTracker.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskTracker.java?rev=748040&r1=748039&r2=748040&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskTracker.java (original)
+++ hadoop/core/branches/branch-0.19/src/mapred/org/apache/hadoop/mapred/TaskTracker.java Thu Feb 26 06:53:08 2009
@@ -1947,10 +1947,16 @@
       if (this.done || 
           (this.taskStatus.getRunState() != TaskStatus.State.RUNNING &&
           this.taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING &&
-          !isCleaningup())) {
+          !isCleaningup()) ||
+          ((this.taskStatus.getRunState() == TaskStatus.State.COMMIT_PENDING ||
+           this.taskStatus.getRunState() == TaskStatus.State.FAILED_UNCLEAN ||
+           this.taskStatus.getRunState() == TaskStatus.State.KILLED_UNCLEAN) &&
+           taskStatus.getRunState() == TaskStatus.State.RUNNING)) {
         //make sure we ignore progress messages after a task has 
         //invoked TaskUmbilicalProtocol.done() or if the task has been
-        //KILLED/FAILED
+        //KILLED/FAILED/FAILED_UNCLEAN/KILLED_UNCLEAN
+        //Also ignore progress update if the state change is from 
+        //COMMIT_PENDING/FAILED_UNCLEAN/KILLED_UNCLEA to RUNNING
         LOG.info(task.getTaskID() + " Ignoring status-update since " +
                  ((this.done) ? "task is 'done'" : 
                                 ("runState: " + this.taskStatus.getRunState()))
@@ -2303,7 +2309,10 @@
         if (wasFailure) {
           failures += 1;
         }
-        runner.kill();
+        // runner could be null if task-cleanup attempt is not localized yet
+        if (runner != null) {
+          runner.kill();
+        }
         setTaskFailState(wasFailure);
       } else if (taskStatus.getRunState() == TaskStatus.State.UNASSIGNED) {
         if (wasFailure) {
@@ -2382,6 +2391,11 @@
       }
       synchronized (this) {
         try {
+          // localJobConf could be null if localization has not happened
+          // then no cleanup will be required.
+          if (localJobConf == null) {
+            return;
+          }
           String taskDir = getLocalTaskDir(task.getJobID().toString(),
                              taskId.toString(), task.isTaskCleanupTask());
           if (needCleanup) {
@@ -2518,7 +2532,8 @@
   public synchronized void commitPending(TaskAttemptID taskid,
                                          TaskStatus taskStatus) 
   throws IOException {
-    LOG.info("Task " + taskid + " is in COMMIT_PENDING");
+    LOG.info("Task " + taskid + " is in commit-pending," +"" +
+             " task state:" +taskStatus.getRunState());
     statusUpdate(taskid, taskStatus);
     reportTaskFinished();
   }