Selaa lähdekoodia

HADOOP-3245. Adds the feature for supporting JobTracker restart. Running jobs can be recovered from the history file. The history file format has been modified to support recovery. The task attempt ID now has the JobTracker start time to disinguish attempts of the same TIP across restarts. Contributed by Amar Ramesh Kamat.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@695823 13f79535-47bb-0310-9956-ffa450edef68
Devaraj Das 17 vuotta sitten
vanhempi
commit
1d8a1d7d2a
30 muutettua tiedostoa jossa 3002 lisäystä ja 342 poistoa
  1. 6 0
      CHANGES.txt
  2. 26 0
      conf/hadoop-default.xml
  3. 12 2
      src/core/org/apache/hadoop/util/StringUtils.java
  4. 45 5
      src/mapred/org/apache/hadoop/mapred/Counters.java
  5. 31 0
      src/mapred/org/apache/hadoop/mapred/HeartbeatResponse.java
  6. 4 3
      src/mapred/org/apache/hadoop/mapred/IsolationRunner.java
  7. 28 0
      src/mapred/org/apache/hadoop/mapred/JobClient.java
  8. 389 72
      src/mapred/org/apache/hadoop/mapred/JobHistory.java
  9. 17 0
      src/mapred/org/apache/hadoop/mapred/JobID.java
  10. 175 70
      src/mapred/org/apache/hadoop/mapred/JobInProgress.java
  11. 672 50
      src/mapred/org/apache/hadoop/mapred/JobTracker.java
  12. 9 5
      src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
  13. 1 1
      src/mapred/org/apache/hadoop/mapred/MapTask.java
  14. 67 0
      src/mapred/org/apache/hadoop/mapred/MapTaskCompletionEventsUpdate.java
  15. 13 2
      src/mapred/org/apache/hadoop/mapred/ReduceTask.java
  16. 100 19
      src/mapred/org/apache/hadoop/mapred/TaskAttemptID.java
  17. 24 0
      src/mapred/org/apache/hadoop/mapred/TaskCompletionEvent.java
  18. 70 15
      src/mapred/org/apache/hadoop/mapred/TaskInProgress.java
  19. 26 0
      src/mapred/org/apache/hadoop/mapred/TaskReport.java
  20. 78 17
      src/mapred/org/apache/hadoop/mapred/TaskTracker.java
  21. 14 3
      src/mapred/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
  22. 134 24
      src/test/org/apache/hadoop/mapred/MiniMRCluster.java
  23. 1 48
      src/test/org/apache/hadoop/mapred/TestCounters.java
  24. 832 0
      src/test/org/apache/hadoop/mapred/TestJobTrackerRestart.java
  25. 176 0
      src/test/org/apache/hadoop/mapred/TestJobTrackerRestartWithLostTracker.java
  26. 23 2
      src/test/org/apache/hadoop/mapred/TestMRServerPorts.java
  27. 1 1
      src/test/org/apache/hadoop/mapred/TestMiniMRMapRedDebugScript.java
  28. 6 3
      src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
  29. 10 0
      src/webapps/job/jobhistory.jsp
  30. 12 0
      src/webapps/job/jobtracker.jsp

+ 6 - 0
CHANGES.txt

@@ -56,6 +56,12 @@ Trunk (unreleased changes)
     output retention during the shuffle and reduce relative to maximum heap
     usage. (cdouglas)
 
+    HADOOP-3245. Adds the feature for supporting JobTracker restart. Running
+    jobs can be recovered from the history file. The history file format has
+    been modified to support recovery. The task attempt ID now has the 
+    JobTracker start time to disinguish attempts of the same TIP across 
+    restarts. (Amar Ramesh Kamat via ddas)
+
   NEW FEATURES
 
     HADOOP-3341. Allow streaming jobs to specify the field separator for map

+ 26 - 0
conf/hadoop-default.xml

@@ -749,6 +749,32 @@ creations/deletions), or "all".</description>
   </description>
 </property>
 
+<property>
+  <name>mapred.jobtracker.restart.recover</name>
+  <value>false</value>
+  <description>"true" to enable (job) recovery upon restart,
+               "false" to start afresh
+  </description>
+</property>
+
+<property>
+  <name>mapred.jobtracker.job.history.block.size</name>
+  <value>0</value>
+  <description>The block size of the job history file. Since the job recovery
+               uses job history, its important to dump job history to disk as 
+               soon as possible.
+  </description>
+</property>
+
+<property>
+  <name>mapred.jobtracker.job.history.buffer.size</name>
+  <value>4096</value>
+  <description>The buffer size for the job history file. Since the job 
+               recovery uses job history, its important to frequently flush the 
+               job history to disk. This will minimize the loss in recovery.
+  </description>
+</property>
+
 <property>
   <name>mapred.jobtracker.taskScheduler</name>
   <value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value>

+ 12 - 2
src/core/org/apache/hadoop/util/StringUtils.java

@@ -226,9 +226,19 @@ public class StringUtils {
    * @param startTime start time
    */
   public static String formatTimeDiff(long finishTime, long startTime){
-    StringBuffer buf = new StringBuffer();
-    
     long timeDiff = finishTime - startTime; 
+    return formatTime(timeDiff); 
+  }
+  
+  /**
+   * 
+   * Given the time in long milliseconds, returns a 
+   * String in the format Xhrs, Ymins, Z sec. 
+   * 
+   * @param timeDiff The time difference to format
+   */
+  public static String formatTime(long timeDiff){
+    StringBuffer buf = new StringBuffer();
     long hours = timeDiff / (60*60*1000);
     long rem = (timeDiff % (60*60*1000));
     long minutes =  rem / (60*1000);

+ 45 - 5
src/mapred/org/apache/hadoop/mapred/Counters.java

@@ -30,7 +30,6 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.MissingResourceException;
 import java.util.ResourceBundle;
-import java.util.Set;
 
 import org.apache.commons.logging.*;
 import org.apache.hadoop.io.IntWritable;
@@ -159,6 +158,13 @@ public class Counters implements Writable, Iterable<Counters.Group> {
       return buf.toString();
     }
     
+    // Checks for (content) equality of two (basic) counters
+    synchronized boolean contentEquals(Counter c) {
+      return name.equals(c.getName())
+             && displayName.equals(c.getDisplayName())
+             && value == c.getCounter();
+    }
+    
     /**
      * What is the current value of this counter?
      * @return the current value
@@ -263,11 +269,26 @@ public class Counters implements Writable, Iterable<Counters.Group> {
       return buf.toString();
     }
         
-    /**
-     * Returns the names of the counters within
+    /** 
+     * Checks for (content) equality of Groups
      */
-    public synchronized Set<String> getCounterNames() {
-      return subcounters.keySet();
+    synchronized boolean contentEquals(Group g) {
+      boolean isEqual = false;
+      if (g != null) {
+        if (size() == g.size()) {
+          isEqual = true;
+          for (Map.Entry<String, Counter> entry : subcounters.entrySet()) {
+            String key = entry.getKey();
+            Counter c1 = entry.getValue();
+            Counter c2 = g.getCounterForName(key);
+            if (!c1.contentEquals(c2)) {
+              isEqual = false;
+              break;
+            }
+          }
+        }
+      }
+      return isEqual;
     }
     
     /**
@@ -708,6 +729,25 @@ public class Counters implements Writable, Iterable<Counters.Group> {
                                       charsToEscape);
   }
   
+  synchronized boolean contentEquals(Counters counters) {
+    boolean isEqual = false;
+    if (counters != null) {
+      if (size() == counters.size()) {
+        isEqual = true;
+        for (Map.Entry<String, Group> entry : this.counters.entrySet()) {
+          String key = entry.getKey();
+          Group sourceGroup = entry.getValue();
+          Group targetGroup = counters.getGroup(key);
+          if (!sourceGroup.contentEquals(targetGroup)) {
+            isEqual = false;
+            break;
+          }
+        }
+      }
+    }
+    return isEqual;
+  }
+  
   public static class Application {
     //special counters which are written by the application and are 
     //used by the framework.

+ 31 - 0
src/mapred/org/apache/hadoop/mapred/HeartbeatResponse.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.mapred;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Map;
+import java.util.HashMap;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
@@ -37,6 +39,7 @@ class HeartbeatResponse implements Writable, Configurable {
   short responseId;
   int heartbeatInterval;
   TaskTrackerAction[] actions;
+  Map<JobID, Integer> lastKnownIndexMap = null;
 
   HeartbeatResponse() {}
   
@@ -54,6 +57,14 @@ class HeartbeatResponse implements Writable, Configurable {
     return responseId;
   }
   
+  public void setLastKnownIndices(Map<JobID, Integer> lastKnownIndexMap) {
+    this.lastKnownIndexMap = lastKnownIndexMap; 
+  }
+  
+  public Map<JobID, Integer> getLastKnownIndex() {
+    return lastKnownIndexMap;
+  }
+  
   public void setActions(TaskTrackerAction[] actions) {
     this.actions = actions;
   }
@@ -90,6 +101,16 @@ class HeartbeatResponse implements Writable, Configurable {
         action.write(out);
       }
     }
+    // Write the last map event index for the jobs
+    if (lastKnownIndexMap != null) {
+      out.writeInt(lastKnownIndexMap.size());
+      for (Map.Entry<JobID, Integer> entry : lastKnownIndexMap.entrySet()) {
+        entry.getKey().write(out);
+        out.writeInt(entry.getValue());
+      }
+    } else {
+      out.writeInt(0);
+    }
     //ObjectWritable.writeObject(out, actions, actions.getClass(), conf);
   }
   
@@ -108,6 +129,16 @@ class HeartbeatResponse implements Writable, Configurable {
     } else {
       actions = null;
     }
+    // Read the last map events index of the jobs
+    int size = in.readInt();
+    if (size != 0) {
+      lastKnownIndexMap = new HashMap<JobID, Integer>(size);
+      for (int i = 0; i < size; ++i) {
+        JobID id = JobID.read(in);
+        int count = in.readInt();
+        lastKnownIndexMap.put(id, count);
+      }
+    }
     //actions = (TaskTrackerAction[]) ObjectWritable.readObject(in, conf);
   }
 }

+ 4 - 3
src/mapred/org/apache/hadoop/mapred/IsolationRunner.java

@@ -96,9 +96,10 @@ public class IsolationRunner {
       LOG.info("Task " + taskid + " has problem " + trace);
     }
     
-    public TaskCompletionEvent[] getMapCompletionEvents(JobID jobId, 
-                                                        int fromEventId, int maxLocs) throws IOException {
-      return TaskCompletionEvent.EMPTY_ARRAY;
+    public MapTaskCompletionEventsUpdate getMapCompletionEvents(JobID jobId, 
+        int fromEventId, int maxLocs, TaskAttemptID id) throws IOException {
+      return new MapTaskCompletionEventsUpdate(TaskCompletionEvent.EMPTY_ARRAY, 
+                                               false);
     }
 
     public void reportNextRecordRange(TaskAttemptID taskid, 

+ 28 - 0
src/mapred/org/apache/hadoop/mapred/JobClient.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -813,6 +814,33 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     }
   }
 
+  /** 
+   * Checks if the job directory is clean and has all the required components 
+   * for (re) starting the job
+   */
+  public static boolean isJobDirValid(Path jobDirPath, FileSystem fs) 
+  throws IOException {
+    FileStatus[] contents = fs.listStatus(jobDirPath);
+    int matchCount = 0;
+    if (contents != null && contents.length >=3) {
+      for (FileStatus status : contents) {
+        if ("job.xml".equals(status.getPath().getName())) {
+          ++matchCount;
+        }
+        if ("job.jar".equals(status.getPath().getName())) {
+          ++matchCount;
+        }
+        if ("job.split".equals(status.getPath().getName())) {
+          ++matchCount;
+        }
+      }
+      if (matchCount == 3) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   static class RawSplit implements Writable {
     private String splitClass;
     private BytesWritable bytes = new BytesWritable();

+ 389 - 72
src/mapred/org/apache/hadoop/mapred/JobHistory.java

@@ -39,8 +39,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -64,6 +67,7 @@ public class JobHistory {
   
   public static final Log LOG = LogFactory.getLog(JobHistory.class);
   private static final String DELIMITER = " ";
+  private static final String LINE_DELIMITER = ".";
   private static final String KEY = "(\\w+)";
   private static final String VALUE = "[[^\"]?]+"; // anything but a " in ""
   
@@ -75,6 +79,10 @@ public class JobHistory {
   private static Map<String, ArrayList<PrintWriter>> openJobs = 
                      new HashMap<String, ArrayList<PrintWriter>>();
   private static boolean disableHistory = false; 
+  private static final String SECONDARY_FILE_SUFFIX = ".recover";
+  private static long jobHistoryBlockSize = 0;
+  private static int jobHistoryBufferSize = 0;
+  private static String jobtrackerHostname;
   /**
    * Record types are identifiers for each line of log in history files. 
    * A record type appears as the first token in a single line of log. 
@@ -93,7 +101,8 @@ public class JobHistory {
     LAUNCH_TIME, TOTAL_MAPS, TOTAL_REDUCES, FAILED_MAPS, FAILED_REDUCES, 
     FINISHED_MAPS, FINISHED_REDUCES, JOB_STATUS, TASKID, HOSTNAME, TASK_TYPE, 
     ERROR, TASK_ATTEMPT_ID, TASK_STATUS, COPY_PHASE, SORT_PHASE, REDUCE_PHASE, 
-    SHUFFLE_FINISHED, SORT_FINISHED, COUNTERS, SPLITS
+    SHUFFLE_FINISHED, SORT_FINISHED, COUNTERS, SPLITS, JOB_PRIORITY, HTTP_PORT, 
+    TRACKER_NAME, STATE_STRING
   }
 
   /**
@@ -102,7 +111,7 @@ public class JobHistory {
    * most places in history file. 
    */
   public static enum Values {
-    SUCCESS, FAILED, KILLED, MAP, REDUCE, CLEANUP
+    SUCCESS, FAILED, KILLED, MAP, REDUCE, CLEANUP, RUNNING
   }
 
   // temp buffer for parsed dataa
@@ -125,6 +134,7 @@ public class JobHistory {
         + File.separator + "history");
       JOBTRACKER_UNIQUE_STRING = hostname + "_" + 
                                     String.valueOf(jobTrackerStartTime) + "_";
+      jobtrackerHostname = hostname;
       Path logDir = new Path(LOG_DIR);
       FileSystem fs = logDir.getFileSystem(conf);
       if (!fs.exists(logDir)){
@@ -134,6 +144,14 @@ public class JobHistory {
       }
       conf.set("hadoop.job.history.location", LOG_DIR);
       disableHistory = false;
+      // set the job history block size
+      jobHistoryBlockSize = 
+        conf.getLong("mapred.jobtracker.job.history.block.size", 
+                     fs.getDefaultBlockSize());
+      // set the job history buffer size
+      jobHistoryBufferSize = 
+        Integer.parseInt(conf.get("mapred.jobtracker.job.history.buffer.size", 
+                                  "4096"));
     } catch(IOException e) {
         LOG.error("Failed to initialize JobHistory log file", e); 
         disableHistory = true;
@@ -159,7 +177,7 @@ public class JobHistory {
       StringBuffer buf = new StringBuffer(); 
       while ((line = reader.readLine())!= null){
         buf.append(line); 
-        if (!line.trim().endsWith("\"")){
+        if (!line.trim().endsWith("\"" + DELIMITER + LINE_DELIMITER)){
           continue; 
         }
         parseLine(buf.toString(), l);
@@ -206,7 +224,8 @@ public class JobHistory {
   
   static void log(PrintWriter out, RecordTypes recordType, Keys key, 
                   String value){
-    out.println(recordType.name() + DELIMITER + key + "=\"" + value + "\""); 
+    out.println(recordType.name() + DELIMITER + key + "=\"" + value + "\""
+                + DELIMITER + LINE_DELIMITER); 
   }
   
   /**
@@ -228,6 +247,7 @@ public class JobHistory {
       buf.append("\"");
       buf.append(DELIMITER); 
     }
+    buf.append(LINE_DELIMITER);
     
     for (PrintWriter out : writers) {
       out.println(buf.toString());
@@ -419,6 +439,237 @@ public class JobHistory {
       return decodedFileName;
     }
     
+    /**
+     * Get the job name from the job conf
+     */
+    static String getJobName(JobConf jobConf) {
+      String jobName = jobConf.getJobName();
+      if (jobName == null || jobName.length() == 0) {
+        jobName = "NA";
+      }
+      return jobName;
+    }
+    
+    /**
+     * Get the user name from the job conf
+     */
+    public static String getUserName(JobConf jobConf) {
+      String user = jobConf.getUser();
+      if (user == null || user.length() == 0) {
+        user = "NA";
+      }
+      return user;
+    }
+    
+    /**
+     * Get the job history file path given the history filename
+     */
+    public static Path getJobHistoryLogLocation(String logFileName)
+    {
+      return LOG_DIR == null ? null : new Path(LOG_DIR, logFileName);
+    }
+
+    /**
+     * Get the user job history file path
+     */
+    public static Path getJobHistoryLogLocationForUser(String logFileName, 
+                                                       JobConf jobConf) {
+      // find user log directory 
+      Path userLogFile = null;
+      Path outputPath = FileOutputFormat.getOutputPath(jobConf);
+      String userLogDir = jobConf.get("hadoop.job.history.user.location",
+                                      outputPath == null 
+                                      ? null 
+                                      : outputPath.toString());
+      if ("none".equals(userLogDir)) {
+        userLogDir = null;
+      }
+      if (userLogDir != null) {
+        userLogDir = userLogDir + Path.SEPARATOR + "_logs" + Path.SEPARATOR 
+                     + "history";
+        userLogFile = new Path(userLogDir, logFileName);
+      }
+      return userLogFile;
+    }
+
+    /**
+     * Generates the job history filename for a new job
+     */
+    private static String getNewJobHistoryFileName(JobConf jobConf, JobID id) {
+      return JOBTRACKER_UNIQUE_STRING
+             + id.toString() + "_" + getUserName(jobConf) + "_" 
+             + trimJobName(getJobName(jobConf));
+    }
+    
+    /**
+     * Trims the job-name if required
+     */
+    private static String trimJobName(String jobName) {
+      if (jobName.length() > JOB_NAME_TRIM_LENGTH) {
+        jobName = jobName.substring(0, JOB_NAME_TRIM_LENGTH);
+      }
+      return jobName;
+    }
+    
+    /**
+     * Recover the job history filename from the history folder. 
+     * Uses the following pattern
+     *    $jt-hostname_[0-9]*_$job-id_$user-$job-name*
+     * @param jobConf the job conf
+     * @param id job id
+     */
+    public static synchronized String getJobHistoryFileName(JobConf jobConf, 
+                                                            JobID id) 
+    throws IOException {
+      String user = getUserName(jobConf);
+      String jobName = trimJobName(getJobName(jobConf));
+      
+      FileSystem fs = new Path(LOG_DIR).getFileSystem(jobConf);
+      if (LOG_DIR == null) {
+        return null;
+      }
+      
+      // Make the pattern matching the job's history file
+      final Pattern historyFilePattern = 
+        Pattern.compile(jobtrackerHostname + "_" + "[0-9]+" + "_" 
+                        + id.toString() + "_" + user + "_" + jobName + "+");
+      // a path filter that matches 4 parts of the filenames namely
+      //  - jt-hostname
+      //  - job-id
+      //  - username
+      //  - jobname
+      PathFilter filter = new PathFilter() {
+        public boolean accept(Path path) {
+          String fileName = path.getName();
+          try {
+            fileName = decodeJobHistoryFileName(fileName);
+          } catch (IOException ioe) {
+            LOG.info("Error while decoding history file " + fileName + "."
+                     + " Ignoring file.", ioe);
+            return false;
+          }
+          return historyFilePattern.matcher(fileName).find();
+        }
+      };
+      
+      FileStatus[] statuses = fs.listStatus(new Path(LOG_DIR), filter);
+      String filename;
+      if (statuses.length == 0) {
+        filename = 
+          encodeJobHistoryFileName(getNewJobHistoryFileName(jobConf, id));
+      } else {
+        // return filename considering that fact the name can be a 
+        // secondary filename like filename.recover
+        filename = decodeJobHistoryFileName(statuses[0].getPath().getName());
+        // Remove the '.recover' suffix if it exists
+        if (filename.endsWith(jobName + SECONDARY_FILE_SUFFIX)) {
+          int newLength = filename.length() - SECONDARY_FILE_SUFFIX.length();
+          filename = filename.substring(0, newLength);
+        }
+        filename = encodeJobHistoryFileName(filename);
+      }
+      return filename;
+    }
+    
+    /** Since there was a restart, there should be a master file and 
+     * a recovery file. Once the recovery is complete, the master should be 
+     * deleted as an indication that the recovery file should be treated as the 
+     * master upon completion or next restart.
+     * @param fileName the history filename that needs checkpointing
+     * @param conf Job conf
+     * @throws IOException
+     */
+    static synchronized void checkpointRecovery(String fileName, JobConf conf) 
+    throws IOException {
+      Path logPath = JobHistory.JobInfo.getJobHistoryLogLocation(fileName);
+      if (logPath != null) {
+        FileSystem fs = logPath.getFileSystem(conf);
+        fs.delete(logPath, false);
+      }
+      // do the same for the user file too
+      logPath = JobHistory.JobInfo.getJobHistoryLogLocationForUser(fileName, 
+                                                                   conf);
+      if (logPath != null) {
+        FileSystem fs = logPath.getFileSystem(conf);
+        fs.delete(logPath, false);
+      }
+    }
+    
+    static String getSecondaryJobHistoryFile(String filename) 
+    throws IOException {
+      return encodeJobHistoryFileName(
+          decodeJobHistoryFileName(filename) + SECONDARY_FILE_SUFFIX);
+    }
+    
+    /** Selects one of the two files generated as a part of recovery. 
+     * The thumb rule is that always select the oldest file. 
+     * This call makes sure that only one file is left in the end. 
+     * @param conf job conf
+     * @param logFilePath Path of the log file
+     * @throws IOException 
+     */
+    public synchronized static Path recoverJobHistoryFile(JobConf conf, 
+                                                          Path logFilePath) 
+    throws IOException {
+      FileSystem fs = logFilePath.getFileSystem(conf);
+      String tmpFilename = getSecondaryJobHistoryFile(logFilePath.getName());
+      Path logDir = logFilePath.getParent();
+      Path tmpFilePath = new Path(logDir, tmpFilename);
+      if (fs.exists(logFilePath)) {
+        if (fs.exists(tmpFilePath)) {
+          fs.delete(tmpFilePath, false);
+        }
+        return tmpFilePath;
+      } else {
+        if (fs.exists(tmpFilePath)) {
+          fs.rename(tmpFilePath, logFilePath);
+          return tmpFilePath;
+        } else {
+          return logFilePath;
+        }
+      }
+    }
+
+    /** Finalize the recovery and make one file in the end. 
+     * This invloves renaming the recover file to the master file.
+     * @param id Job id  
+     * @param conf the job conf
+     * @throws IOException
+     */
+    static synchronized void finalizeRecovery(JobID id, JobConf conf) 
+    throws IOException {
+      String masterLogFileName = 
+        JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+      Path masterLogPath = 
+        JobHistory.JobInfo.getJobHistoryLogLocation(masterLogFileName);
+      String tmpLogFileName = getSecondaryJobHistoryFile(masterLogFileName);
+      Path tmpLogPath = 
+        JobHistory.JobInfo.getJobHistoryLogLocation(tmpLogFileName);
+      if (masterLogPath != null) {
+        FileSystem fs = masterLogPath.getFileSystem(conf);
+
+        // rename the tmp file to the master file. Note that this should be 
+        // done only when the file is closed and handles are released.
+        if(fs.exists(tmpLogPath)) {
+          fs.rename(tmpLogPath, masterLogPath);
+        }
+      }
+      
+      // do the same for the user file too
+      masterLogPath = 
+        JobHistory.JobInfo.getJobHistoryLogLocationForUser(masterLogFileName,
+                                                           conf);
+      tmpLogPath = 
+        JobHistory.JobInfo.getJobHistoryLogLocationForUser(tmpLogFileName, 
+                                                           conf);
+      if (masterLogPath != null) {
+        FileSystem fs = masterLogPath.getFileSystem(conf);
+        if (fs.exists(tmpLogPath)) {
+          fs.rename(tmpLogPath, masterLogPath);
+        }
+      }
+    }
+
     /**
      * Log job submitted event to history. Creates a new file in history 
      * for the job. if history file creation fails, it disables history 
@@ -438,46 +689,21 @@ public class JobHistory {
 
       if (!disableHistory){
         // Get the username and job name to be used in the actual log filename;
-        // sanity check them too
-        String jobName = jobConf.getJobName();
-        if (jobName == null || jobName.length() == 0) {
-          jobName = "NA";
-        }
+        // sanity check them too        
+        String jobName = getJobName(jobConf);
 
-        String user = jobConf.getUser();
-        if (user == null || user.length() == 0) {
-          user = "NA";
-        }
+        String user = getUserName(jobConf);
         
-        // setup the history log file for this job
-        String trimmedJobName = jobName;
-        if (jobName.length() > JOB_NAME_TRIM_LENGTH) {
-          trimmedJobName = jobName.substring(0, JOB_NAME_TRIM_LENGTH);
-        }
+        // get the history filename
         String logFileName = 
-            encodeJobHistoryFileName(jobUniqueString +  "_" + user + "_" + 
-                                     trimmedJobName);
-
-        // find user log directory 
-        Path outputPath = FileOutputFormat.getOutputPath(jobConf);
-        userLogDir = jobConf.get("hadoop.job.history.user.location",
-        		outputPath == null ? null : outputPath.toString());
-        if ("none".equals(userLogDir)) {
-          userLogDir = null;
-        }
-        if (userLogDir != null) {
-          userLogDir = userLogDir + Path.SEPARATOR + "_logs" + 
-                       Path.SEPARATOR + "history";
-        }
+          getJobHistoryFileName(jobConf, jobId);
 
-        Path logFile = null;
-        Path userLogFile = null;
-        if (LOG_DIR != null ) {
-          logFile = new Path(LOG_DIR, logFileName);
-        }
-        if (userLogDir != null ) {
-          userLogFile = new Path(userLogDir, logFileName);
-        }
+        // setup the history log file for this job
+        Path logFile = getJobHistoryLogLocation(logFileName);
+        
+        // find user log directory
+        Path userLogFile = 
+          getJobHistoryLogLocationForUser(logFileName, jobConf);
 
         try{
           ArrayList<PrintWriter> writers = new ArrayList<PrintWriter>();
@@ -487,14 +713,24 @@ public class JobHistory {
           if (LOG_DIR != null) {
             // create output stream for logging in hadoop.job.history.location
             fs = new Path(LOG_DIR).getFileSystem(jobConf);
-            out = fs.create(logFile, true, 4096);
+            
+            logFile = recoverJobHistoryFile(jobConf, logFile);
+            
+            out = fs.create(logFile, FsPermission.getDefault(), true, 
+                            jobHistoryBufferSize, 
+                            fs.getDefaultReplication(), 
+                            jobHistoryBlockSize, null);
             writer = new PrintWriter(out);
             writers.add(writer);
           }
-          if (userLogDir != null) {
+          if (userLogFile != null) {
+            userLogDir = userLogFile.getParent().toString();
             // create output stream for logging 
             // in hadoop.job.history.user.location
-            fs = new Path(userLogDir).getFileSystem(jobConf);
+            fs = userLogFile.getFileSystem(jobConf);
+ 
+            userLogFile = recoverJobHistoryFile(jobConf, userLogFile);
+            
             out = fs.create(userLogFile, true, 4096);
             writer = new PrintWriter(out);
             writers.add(writer);
@@ -595,8 +831,12 @@ public class JobHistory {
 
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Job, 
-                         new Keys[] {Keys.JOBID, Keys.LAUNCH_TIME, Keys.TOTAL_MAPS, Keys.TOTAL_REDUCES },
-                         new String[] {jobId.toString(),  String.valueOf(startTime), String.valueOf(totalMaps), String.valueOf(totalReduces)}); 
+              new Keys[] {Keys.JOBID, Keys.LAUNCH_TIME, Keys.TOTAL_MAPS, 
+                          Keys.TOTAL_REDUCES, Keys.JOB_STATUS},
+              new String[] {jobId.toString(), String.valueOf(startTime), 
+                            String.valueOf(totalMaps), 
+                            String.valueOf(totalReduces), 
+                            Values.RUNNING.name()}); 
         }
       }
     }
@@ -632,7 +872,7 @@ public class JobHistory {
                                        String.valueOf(finishedReduces),
                                        String.valueOf(failedMaps), 
                                        String.valueOf(failedReduces),
-                                       counters.makeCompactString()});
+                                       counters.makeEscapedCompactString()});
           for (PrintWriter out : writer) {
             out.close();
           }
@@ -666,6 +906,46 @@ public class JobHistory {
         }
       }
     }
+    /**
+     * Log job's priority. 
+     * @param jobid job id
+     * @param priority Jobs priority 
+     */
+    public static void logJobPriority(JobID jobid, JobPriority priority){
+      if (!disableHistory){
+        String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobid; 
+        ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
+
+        if (null != writer){
+          JobHistory.log(writer, RecordTypes.Job,
+                         new Keys[] {Keys.JOBID, Keys.JOB_PRIORITY},
+                         new String[] {jobid.toString(), priority.toString()});
+        }
+      }
+    }
+
+    /**
+     * Log job's submit-time/launch-time 
+     * @param jobid job id
+     * @param submitTime job's submit time
+     * @param launchTime job's launch time
+     */
+    public static void logJobSubmitTime(JobID jobid, long submitTime, 
+                                        long launchTime){
+      if (!disableHistory){
+        String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobid; 
+        ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
+
+        if (null != writer){
+          JobHistory.log(writer, RecordTypes.Job,
+                         new Keys[] {Keys.JOBID, Keys.SUBMIT_TIME, 
+                                     Keys.LAUNCH_TIME},
+                         new String[] {jobid.toString(), 
+                                       String.valueOf(submitTime), 
+                                       String.valueOf(launchTime)});
+        }
+      }
+    }
   }
   /**
    * Helper class for logging or reading back events related to Task's start, finish or failure. 
@@ -716,7 +996,7 @@ public class JobHistory {
                                     Keys.COUNTERS}, 
                          new String[]{ taskId.toString(), taskType, Values.SUCCESS.name(), 
                                        String.valueOf(finishTime),
-                                       counters.makeCompactString()});
+                                       counters.makeEscapedCompactString()});
         }
       }
     }
@@ -728,15 +1008,31 @@ public class JobHistory {
      * @param error error message for failure. 
      */
     public static void logFailed(TaskID taskId, String taskType, long time, String error){
+      logFailed(taskId, taskType, time, error, null);
+    }
+    
+    /**
+     * @param failedDueToAttempt The attempt that caused the failure, if any
+     */
+    public static void logFailed(TaskID taskId, String taskType, long time,
+                                 String error, 
+                                 TaskAttemptID failedDueToAttempt){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
                                                      + taskId.getJobID()); 
 
         if (null != writer){
+          String failedAttempt = failedDueToAttempt == null
+                                 ? ""
+                                 : failedDueToAttempt.toString();
           JobHistory.log(writer, RecordTypes.Task, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE, 
-                                    Keys.TASK_STATUS, Keys.FINISH_TIME, Keys.ERROR}, 
-                         new String[]{ taskId.toString(),  taskType, Values.FAILED.name(), String.valueOf(time) , error});
+                                    Keys.TASK_STATUS, Keys.FINISH_TIME, 
+                                    Keys.ERROR, Keys.TASK_ATTEMPT_ID}, 
+                         new String[]{ taskId.toString(),  taskType, 
+                                      Values.FAILED.name(), 
+                                      String.valueOf(time) , error, 
+                                      failedAttempt});
         }
       }
     }
@@ -764,11 +1060,12 @@ public class JobHistory {
      * @param startTime start time of task attempt as reported by task tracker. 
      * @param hostName host name of the task attempt. 
      * @deprecated Use 
-     *             {@link #logStarted(TaskAttemptID, long, String, boolean)}
+     *             {@link #logStarted(TaskAttemptID, long, String, int, 
+     *                                boolean)}
      */
     @Deprecated
     public static void logStarted(TaskAttemptID taskAttemptId, long startTime, String hostName){
-      logStarted(taskAttemptId, startTime, hostName, false);
+      logStarted(taskAttemptId, startTime, hostName, -1, false);
     }
     
     /**
@@ -776,11 +1073,13 @@ public class JobHistory {
      *  
      * @param taskAttemptId task attempt id
      * @param startTime start time of task attempt as reported by task tracker. 
-     * @param hostName host name of the task attempt.
+     * @param trackerName name of the tracker executing the task attempt.
+     * @param httpPort http port of the task tracker executing the task attempt
      * @param isCleanup Whether the attempt is cleanup or not 
      */
     public static void logStarted(TaskAttemptID taskAttemptId, long startTime,
-                                  String hostName, boolean isCleanup){
+                                  String trackerName, int httpPort, 
+                                  boolean isCleanup){
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
                                                    + taskAttemptId.getJobID()); 
@@ -789,12 +1088,13 @@ public class JobHistory {
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
                                      Keys.TASK_ATTEMPT_ID, Keys.START_TIME, 
-                                     Keys.HOSTNAME},
+                                     Keys.TRACKER_NAME, Keys.HTTP_PORT},
                          new String[]{isCleanup ? Values.CLEANUP.name() : 
                                                   Values.MAP.name(),
                                       taskAttemptId.getTaskID().toString(), 
                                       taskAttemptId.toString(), 
-                                      String.valueOf(startTime), hostName}); 
+                                      String.valueOf(startTime), trackerName,
+                                      String.valueOf(httpPort)}); 
         }
       }
     }
@@ -805,12 +1105,14 @@ public class JobHistory {
      * @param finishTime finish time
      * @param hostName host name 
      * @deprecated Use 
-     * {@link #logFinished(TaskAttemptID, long, String, boolean)}
+     * {@link #logFinished(TaskAttemptID, long, String, boolean, String, 
+     *                     Counters)}
      */
     @Deprecated
     public static void logFinished(TaskAttemptID taskAttemptId, long finishTime, 
                                    String hostName){
-      logFinished(taskAttemptId, finishTime, hostName, false);
+      logFinished(taskAttemptId, finishTime, hostName, false, "", 
+                  new Counters());
     }
 
     /**
@@ -820,11 +1122,14 @@ public class JobHistory {
      * @param finishTime finish time
      * @param hostName host name 
      * @param isCleanup Whether the attempt is cleanup or not 
+     * @param stateString state string of the task attempt
+     * @param counter counters of the task attempt
      */
     public static void logFinished(TaskAttemptID taskAttemptId, 
                                    long finishTime, 
                                    String hostName,
-                                   boolean isCleanup) {
+                                   boolean isCleanup, String stateString, 
+                                   Counters counter) {
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
                                                    + taskAttemptId.getJobID()); 
@@ -833,13 +1138,16 @@ public class JobHistory {
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
                                      Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
-                                     Keys.FINISH_TIME, Keys.HOSTNAME},
+                                     Keys.FINISH_TIME, Keys.HOSTNAME, 
+                                     Keys.STATE_STRING, Keys.COUNTERS},
                          new String[]{isCleanup ? Values.CLEANUP.name() : 
                                                   Values.MAP.name(), 
                                       taskAttemptId.getTaskID().toString(),
                                       taskAttemptId.toString(), 
                                       Values.SUCCESS.name(),  
-                                      String.valueOf(finishTime), hostName}); 
+                                      String.valueOf(finishTime), hostName, 
+                                      stateString, 
+                                      counter.makeEscapedCompactString()}); 
         }
       }
     }
@@ -951,12 +1259,12 @@ public class JobHistory {
      * @param startTime start time
      * @param hostName host name 
      * @deprecated Use 
-     * {@link #logStarted(TaskAttemptID, long, String, boolean)}
+     * {@link #logStarted(TaskAttemptID, long, String, int, boolean)}
      */
     @Deprecated
     public static void logStarted(TaskAttemptID taskAttemptId, 
                                   long startTime, String hostName){
-      logStarted(taskAttemptId, startTime, hostName, false);
+      logStarted(taskAttemptId, startTime, hostName, -1, false);
     }
     
     /**
@@ -964,12 +1272,13 @@ public class JobHistory {
      * 
      * @param taskAttemptId task attempt id
      * @param startTime start time
-     * @param hostName host name 
+     * @param trackerName tracker name 
+     * @param httpPort the http port of the tracker executing the task attempt
      * @param isCleanup Whether the attempt is cleanup or not 
      */
     public static void logStarted(TaskAttemptID taskAttemptId, 
-                                  long startTime, String hostName, 
-                                  boolean isCleanup) {
+                                  long startTime, String trackerName, 
+                                  int httpPort, boolean isCleanup) {
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
                                                    + taskAttemptId.getJobID()); 
@@ -978,12 +1287,13 @@ public class JobHistory {
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
                          new Keys[]{  Keys.TASK_TYPE, Keys.TASKID, 
                                       Keys.TASK_ATTEMPT_ID, Keys.START_TIME,
-                                      Keys.HOSTNAME},
+                                      Keys.TRACKER_NAME, Keys.HTTP_PORT},
                          new String[]{isCleanup ? Values.CLEANUP.name() : 
                                                   Values.REDUCE.name(),
                                       taskAttemptId.getTaskID().toString(), 
                                       taskAttemptId.toString(), 
-                                      String.valueOf(startTime), hostName}); 
+                                      String.valueOf(startTime), trackerName,
+                                      String.valueOf(httpPort)}); 
         }
       }
     }
@@ -996,14 +1306,15 @@ public class JobHistory {
      * @param finishTime finish time of task
      * @param hostName host name where task attempt executed
      * @deprecated Use 
-     * {@link #logFinished(TaskAttemptID, long, long, long, String, boolean)}
+     * {@link #logFinished(TaskAttemptID, long, long, long, String, boolean, 
+     *                     String, Counters)}
      */
     @Deprecated
     public static void logFinished(TaskAttemptID taskAttemptId, long shuffleFinished, 
                                    long sortFinished, long finishTime, 
                                    String hostName){
       logFinished(taskAttemptId, shuffleFinished, sortFinished, 
-                  finishTime, hostName, false);
+                  finishTime, hostName, false, "", new Counters());
     }
     
     /**
@@ -1015,11 +1326,14 @@ public class JobHistory {
      * @param finishTime finish time of task
      * @param hostName host name where task attempt executed
      * @param isCleanup Whether the attempt is cleanup or not 
+     * @param stateString the state string of the attempt
+     * @param counter counters of the attempt
      */
     public static void logFinished(TaskAttemptID taskAttemptId, 
                                    long shuffleFinished, 
                                    long sortFinished, long finishTime, 
-                                   String hostName, boolean isCleanup) {
+                                   String hostName, boolean isCleanup,
+                                   String stateString, Counters counter) {
       if (!disableHistory){
         ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
                                                    + taskAttemptId.getJobID()); 
@@ -1029,7 +1343,8 @@ public class JobHistory {
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
                                      Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
                                      Keys.SHUFFLE_FINISHED, Keys.SORT_FINISHED,
-                                     Keys.FINISH_TIME, Keys.HOSTNAME},
+                                     Keys.FINISH_TIME, Keys.HOSTNAME, 
+                                     Keys.STATE_STRING, Keys.COUNTERS},
                          new String[]{isCleanup ? Values.CLEANUP.name() : 
                                                   Values.REDUCE.name(),
                                       taskAttemptId.getTaskID().toString(), 
@@ -1037,7 +1352,9 @@ public class JobHistory {
                                       Values.SUCCESS.name(), 
                                       String.valueOf(shuffleFinished), 
                                       String.valueOf(sortFinished),
-                                      String.valueOf(finishTime), hostName}); 
+                                      String.valueOf(finishTime), hostName,
+                                      stateString, 
+                                      counter.makeEscapedCompactString()}); 
         }
       }
     }

+ 17 - 0
src/mapred/org/apache/hadoop/mapred/JobID.java

@@ -152,6 +152,23 @@ public class JobID extends ID {
         + " is not properly formed");
   }
   
+  /** Check if the given string represents a job-id or not 
+   */
+  public static boolean isJobNameValid(String str) {
+    if(str == null) {
+      return false;
+    }
+    String[] parts = str.split("_");
+    if(parts.length == 3) {
+      if(parts[0].equals(JOB)) {
+          // other 2 parts should be parseable
+          return JobTracker.validateIdentifier(parts[1])
+                 && JobTracker.validateJobNumber(parts[2]);
+      }
+    }
+    return false;
+  }
+  
   /** 
    * Returns a regex pattern which matches task IDs. Arguments can 
    * be given null, in which case that part of the regex will be generic.  

+ 175 - 70
src/mapred/org/apache/hadoop/mapred/JobInProgress.java

@@ -56,6 +56,7 @@ class JobInProgress {
     
   JobProfile profile;
   JobStatus status;
+  Path jobFile = null;
   Path localJobFile = null;
   Path localJarFile = null;
 
@@ -130,6 +131,7 @@ class JobInProgress {
   private ResourceEstimator resourceEstimator; 
   
   long startTime;
+  long launchTime;
   long finishTime;
 
   private JobConf conf;
@@ -195,7 +197,7 @@ class JobInProgress {
                                                       +"/"+ jobid + ".jar");
     Path sysDir = new Path(this.jobtracker.getSystemDir());
     FileSystem fs = sysDir.getFileSystem(default_conf);
-    Path jobFile = new Path(sysDir, jobid + "/job.xml");
+    jobFile = new Path(sysDir, jobid + "/job.xml");
     fs.copyToLocalFile(jobFile, localJobFile);
     conf = new JobConf(localJobFile);
     this.priority = conf.getJobPriority();
@@ -216,9 +218,6 @@ class JobInProgress {
     this.mapFailuresPercent = conf.getMaxMapTaskFailuresPercent();
     this.reduceFailuresPercent = conf.getMaxReduceTaskFailuresPercent();
         
-    JobHistory.JobInfo.logSubmitted(jobid, conf, jobFile.toString(), 
-                                    System.currentTimeMillis()); 
-        
     MetricsContext metricsContext = MetricsUtil.getContext("mapred");
     this.jobMetrics = MetricsUtil.createRecord(metricsContext, "job");
     this.jobMetrics.setTag("user", conf.getUser());
@@ -332,6 +331,12 @@ class JobInProgress {
       return;
     }
 
+    // log job info
+    JobHistory.JobInfo.logSubmitted(getJobID(), conf, jobFile.toString(), 
+                                    this.startTime);
+    // log the job priority
+    setPriority(this.priority);
+    
     //
     // read input splits and create a map per a split
     //
@@ -361,20 +366,23 @@ class JobInProgress {
       nonRunningMapCache = createCache(splits, maxLevel);
     }
         
+    // set the launch time
+    this.launchTime = System.currentTimeMillis();
+
     // if no split is returned, job is considered completed and successful
     if (numMapTasks == 0) {
       // Finished time need to be setted here to prevent this job to be retired
       // from the job tracker jobs at the next retire iteration.
-      this.finishTime = System.currentTimeMillis();
+      this.finishTime = this.launchTime;
       status.setMapProgress(1.0f);
       status.setReduceProgress(1.0f);
       status.setCleanupProgress(1.0f);
       status.setRunState(JobStatus.SUCCEEDED);
       tasksInited.set(true);
       JobHistory.JobInfo.logStarted(profile.getJobID(), 
-                                    System.currentTimeMillis(), 0, 0);
+                                    this.launchTime, 0, 0);
       JobHistory.JobInfo.logFinished(profile.getJobID(), 
-                                     System.currentTimeMillis(), 0, 0, 0, 0,
+                                     this.finishTime, 0, 0, 0, 0,
                                      getCounters());
       // Special case because the Job is not queued
       JobEndNotifier.registerNotification(this.getJobConf(), this.getStatus());
@@ -409,7 +417,8 @@ class JobInProgress {
     this.status = new JobStatus(status.getJobID(), 0.0f, 0.0f, JobStatus.RUNNING);
     tasksInited.set(true);
         
-    JobHistory.JobInfo.logStarted(profile.getJobID(), System.currentTimeMillis(), numMapTasks, numReduceTasks);
+    JobHistory.JobInfo.logStarted(profile.getJobID(), this.launchTime, 
+                                  numMapTasks, numReduceTasks);
   }
 
   /////////////////////////////////////////////////////
@@ -421,6 +430,9 @@ class JobInProgress {
   public JobStatus getStatus() {
     return status;
   }
+  public synchronized long getLaunchTime() {
+    return launchTime;
+  }
   public long getStartTime() {
     return startTime;
   }
@@ -462,6 +474,8 @@ class JobInProgress {
     } else {
       this.priority = priority;
     }
+    // log and change to the job's priority
+    JobHistory.JobInfo.logJobPriority(jobId, priority);
   }
 
   // Accessors for resources.
@@ -469,6 +483,14 @@ class JobInProgress {
     return maxVirtualMemoryForTask;
   }
   
+  // Update the job start/launch time (upon restart) and log to history
+  synchronized void updateJobTime(long startTime, long launchTime) {
+    // log and change to the job's start/launch time
+    this.startTime = startTime;
+    this.launchTime = launchTime;
+    JobHistory.JobInfo.logJobSubmitTime(jobId, startTime, launchTime);
+  }
+
   long getInputLength() {
     return inputLength;
   }
@@ -776,16 +798,7 @@ class JobInProgress {
     
     Task result = maps[target].getTaskToRun(tts.getTrackerName());
     if (result != null) {
-      runningMapTasks += 1;
-      if (maps[target].getActiveTasks().size() > 1)
-        speculativeMapTasks++;
-      if (maps[target].isFirstAttempt(result.getTaskID())) {
-        JobHistory.Task.logStarted(maps[target].getTIPId(), Values.MAP.name(),
-                                   System.currentTimeMillis(),
-                                   maps[target].getSplitNodes());
-      }
-
-      jobCounters.incrCounter(Counter.TOTAL_LAUNCHED_MAPS, 1);
+      addRunningTaskToTIP(maps[target], result.getTaskID(), tts, true);
     }
 
     return result;
@@ -888,21 +901,118 @@ class JobInProgress {
     
     Task result = reduces[target].getTaskToRun(tts.getTrackerName());
     if (result != null) {
-      runningReduceTasks += 1;
-      if (reduces[target].getActiveTasks().size() > 1)
-        speculativeReduceTasks++;
-      if (reduces[target].isFirstAttempt(result.getTaskID())) {
-        JobHistory.Task.logStarted(reduces[target].getTIPId(), Values.REDUCE.name(),
-                                   System.currentTimeMillis(), "");
-      }
-
-      jobCounters.incrCounter(Counter.TOTAL_LAUNCHED_REDUCES, 1);
+      addRunningTaskToTIP(reduces[target], result.getTaskID(), tts, true);
     }
 
     return result;
   }
+  
+  // returns the (cache)level at which the nodes matches
+  private int getMatchingLevelForNodes(Node n1, Node n2) {
+    int count = 0;
+    do {
+      if (n1.equals(n2)) {
+        return count;
+      }
+      ++count;
+      n1 = n1.getParent();
+      n2 = n2.getParent();
+    } while (n1 != null);
+    return this.maxLevel;
+  }
+
+  /**
+   * Populate the data structures as a task is scheduled.
+   * @param tip The tip for which the task is added
+   * @param id The attempt-id for the task
+   * @param tts task-tracker status
+   * @param isScheduled Whether this task is scheduled from the JT or has 
+   *        joined back upon restart
+   */
+  synchronized void addRunningTaskToTIP(TaskInProgress tip, TaskAttemptID id, 
+                                        TaskTrackerStatus tts, 
+                                        boolean isScheduled) {
+    // keeping the earlier ordering intact
+    String name;
+    Enum counter;
+    if (tip.isMapTask()) {
+      ++runningMapTasks;
+      name = Values.MAP.name();
+      counter = Counter.TOTAL_LAUNCHED_MAPS;
+      if (tip.getActiveTasks().size() > 1)
+        speculativeMapTasks++;
+    } else {
+      ++runningReduceTasks;
+      name = Values.REDUCE.name();
+      counter = Counter.TOTAL_LAUNCHED_REDUCES;
+      if (tip.getActiveTasks().size() > 1)
+        speculativeReduceTasks++;
+    }
+    // Note that the logs are for the scheduled tasks only. Tasks that join on 
+    // restart has already their logs in place.
+    if (tip.isFirstAttempt(id)) {
+      JobHistory.Task.logStarted(tip.getTIPId(), name,
+                                 tip.getExecStartTime(), "");
+    }
+    jobCounters.incrCounter(counter, 1);
+    
+    // Make an entry in the tip if the attempt is not scheduled i.e externally
+    // added
+    if (!isScheduled) {
+      tip.addRunningTask(id, tts.getTrackerName());
+    }
+
+    //TODO The only problem with these counters would be on restart.
+    // The jobtracker updates the counter only when the task that is scheduled
+    // if from a non-running tip and is local (data, rack ...). But upon restart
+    // as the reports come from the task tracker, there is no good way to infer
+    // when exactly to increment the locality counters. The only solution is to 
+    // increment the counters for all the tasks irrespective of 
+    //    - whether the tip is running or not
+    //    - whether its a speculative task or not
+    //
+    // So to simplify, increment the data locality counter whenever there is 
+    // data locality.
+    if (tip.isMapTask()) {
+      // increment the data locality counter for maps
+      Node tracker = jobtracker.getNode(tts.getHost());
+      int level = this.maxLevel;
+      // find the right level across split locations
+      for (String local : maps[tip.getIdWithinJob()].getSplitLocations()) {
+        Node datanode = jobtracker.getNode(local);
+        int newLevel = this.maxLevel;
+        if (tracker != null && datanode != null) {
+          newLevel = getMatchingLevelForNodes(tracker, datanode);
+        }
+        if (newLevel < level) {
+          level = newLevel;
+          // an optimization
+          if (level == 0) {
+            break;
+          }
+        }
+      }
+      switch (level) {
+      case 0 :
+        LOG.info("Choosing data-local task " + tip.getTIPId());
+        jobCounters.incrCounter(Counter.DATA_LOCAL_MAPS, 1);
+        break;
+      case 1:
+        LOG.info("Choosing rack-local task " + tip.getTIPId());
+        jobCounters.incrCounter(Counter.RACK_LOCAL_MAPS, 1);
+        break;
+      default :
+        // check if there is any locality
+        if (level != this.maxLevel) {
+          LOG.info("Choosing cached task at level " + level + tip.getTIPId());
+          jobCounters.incrCounter(Counter.OTHER_LOCAL_MAPS, 1);
+        }
+        break;
+      }
+    }
+  }
     
-  private String convertTrackerNameToHostName(String trackerName) {
+  static String convertTrackerNameToHostName(String trackerName) {
     // Ugly!
     // Convert the trackerName to it's host name
     int indexOfColon = trackerName.indexOf(":");
@@ -1294,18 +1404,6 @@ class JobInProgress {
               nonRunningMapCache.remove(key);
             }
 
-            if (level == 0) {
-              LOG.info("Choosing data-local task " + tip.getTIPId());
-              jobCounters.incrCounter(Counter.DATA_LOCAL_MAPS, 1);
-            } else if (level == 1){
-              LOG.info("Choosing rack-local task " + tip.getTIPId());
-              jobCounters.incrCounter(Counter.RACK_LOCAL_MAPS, 1);
-            } else {
-              LOG.info("Choosing cached task at level " + level 
-                       + tip.getTIPId());
-              jobCounters.incrCounter(Counter.OTHER_LOCAL_MAPS, 1);
-            }
-
             return tip.getIdWithinJob();
           }
         }
@@ -1377,16 +1475,6 @@ class JobInProgress {
               if (cacheForLevel.size() == 0) {
                 runningMapCache.remove(key);
               }
-              if (level == 0) {
-                LOG.info("Choosing a data-local task " + tip.getTIPId() 
-                         + " for speculation");
-              } else if (level == 1){
-                LOG.info("Choosing a rack-local task " + tip.getTIPId() 
-                         + " for speculation");
-              } else {
-                LOG.info("Choosing a cached task at level " + level
-                         + tip.getTIPId() + " for speculation");
-              }
               return tip.getIdWithinJob();
             }
           }
@@ -1536,29 +1624,36 @@ class JobInProgress {
     resourceEstimator.updateWithCompletedTask(status, tip);
 
     // Update jobhistory 
-    String taskTrackerName = jobtracker.getNode(jobtracker.getTaskTracker(
-                               status.getTaskTracker()).getHost()).toString();
+    TaskTrackerStatus ttStatus = 
+      this.jobtracker.getTaskTracker(status.getTaskTracker());
+    String trackerHostname = jobtracker.getNode(ttStatus.getHost()).toString();
     if (status.getIsMap()){
       JobHistory.MapAttempt.logStarted(status.getTaskID(), status.getStartTime(), 
-                                       taskTrackerName, tip.isCleanupTask()); 
+                                       status.getTaskTracker(), 
+                                       ttStatus.getHttpPort(), 
+                                       tip.isCleanupTask()); 
       JobHistory.MapAttempt.logFinished(status.getTaskID(), status.getFinishTime(), 
-                                        taskTrackerName, tip.isCleanupTask()); 
+                                        trackerHostname, tip.isCleanupTask(),
+                                        status.getStateString(), 
+                                        status.getCounters()); 
       JobHistory.Task.logFinished(tip.getTIPId(), 
                                   tip.isCleanupTask() ? Values.CLEANUP.name() :
                                   Values.MAP.name(), 
-                                  status.getFinishTime(),
+                                  tip.getExecFinishTime(),
                                   status.getCounters()); 
     }else{
       JobHistory.ReduceAttempt.logStarted( status.getTaskID(), status.getStartTime(), 
-                                          taskTrackerName, 
+                                          status.getTaskTracker(),
+                                          ttStatus.getHttpPort(), 
                                           tip.isCleanupTask()); 
       JobHistory.ReduceAttempt.logFinished(status.getTaskID(), status.getShuffleFinishTime(),
                                            status.getSortFinishTime(), status.getFinishTime(), 
-                                           taskTrackerName,
-                                           tip.isCleanupTask()); 
+                                           trackerHostname, tip.isCleanupTask(), 
+                                           status.getStateString(), 
+                                           status.getCounters()); 
       JobHistory.Task.logFinished(tip.getTIPId(), 
                                   tip.isCleanupTask() ? Values.CLEANUP.name() :
-                                  Values.REDUCE.name(), status.getFinishTime(),
+                                  Values.REDUCE.name(), tip.getExecFinishTime(),
                                   status.getCounters()); 
     }
         
@@ -1631,13 +1726,16 @@ class JobInProgress {
       this.status.setRunState(JobStatus.SUCCEEDED);
       this.status.setCleanupProgress(1.0f);
       this.finishTime = System.currentTimeMillis();
-      garbageCollect();
       LOG.info("Job " + this.status.getJobID() + 
                " has completed successfully.");
       JobHistory.JobInfo.logFinished(this.status.getJobID(), finishTime, 
                                      this.finishedMapTasks, 
                                      this.finishedReduceTasks, failedMapTasks, 
                                      failedReduceTasks, getCounters());
+      // Note that finalize will close the job history handles which garbage collect
+      // might try to finalize
+      garbageCollect();
+      
       metrics.completeJob();
     }
   }
@@ -1741,28 +1839,30 @@ class JobInProgress {
     }
         
     // update job history
-    String taskTrackerName = jobtracker.getNode(
-                               taskTrackerStatus.getHost()).toString();
+    String taskTrackerName = taskTrackerStatus.getHost();
+    long finishTime = status.getFinishTime();
     if (status.getIsMap()) {
       JobHistory.MapAttempt.logStarted(status.getTaskID(), status.getStartTime(), 
-                taskTrackerName, tip.isCleanupTask());
+          status.getTaskTracker(), taskTrackerStatus.getHttpPort(), 
+          tip.isCleanupTask());
       if (status.getRunState() == TaskStatus.State.FAILED) {
-        JobHistory.MapAttempt.logFailed(status.getTaskID(), System.currentTimeMillis(),
+        JobHistory.MapAttempt.logFailed(status.getTaskID(), finishTime,
                 taskTrackerName, status.getDiagnosticInfo(), tip.isCleanupTask());
       } else {
-        JobHistory.MapAttempt.logKilled(status.getTaskID(), System.currentTimeMillis(),
+        JobHistory.MapAttempt.logKilled(status.getTaskID(), finishTime,
                 taskTrackerName, status.getDiagnosticInfo(),
                 tip.isCleanupTask());
       }
     } else {
       JobHistory.ReduceAttempt.logStarted(status.getTaskID(), status.getStartTime(), 
-                taskTrackerName, tip.isCleanupTask());
+          status.getTaskTracker(), taskTrackerStatus.getHttpPort(), 
+          tip.isCleanupTask());
       if (status.getRunState() == TaskStatus.State.FAILED) {
-        JobHistory.ReduceAttempt.logFailed(status.getTaskID(), System.currentTimeMillis(),
+        JobHistory.ReduceAttempt.logFailed(status.getTaskID(), finishTime,
                 taskTrackerName, status.getDiagnosticInfo(), 
                 tip.isCleanupTask());
       } else {
-        JobHistory.ReduceAttempt.logKilled(status.getTaskID(), System.currentTimeMillis(),
+        JobHistory.ReduceAttempt.logKilled(status.getTaskID(), finishTime,
                 taskTrackerName, status.getDiagnosticInfo(), 
                 tip.isCleanupTask());
       }
@@ -1813,7 +1913,7 @@ class JobInProgress {
                                   tip.isMapTask() ? 
                                           Values.MAP.name() : 
                                           Values.REDUCE.name(),  
-                                  System.currentTimeMillis(), 
+                                          status.getFinishTime(), 
                                   status.getDiagnosticInfo());
         if (tip.isCleanupTask()) {
           // kill the other tip
@@ -1859,11 +1959,12 @@ class JobInProgress {
                                                     reason,
                                                     trackerName, phase,
                                                     null);
+    status.setFinishTime(System.currentTimeMillis());
     updateTaskStatus(tip, status, metrics);
     JobHistory.Task.logFailed(tip.getTIPId(), 
                               tip.isCleanupTask() ? Values.CLEANUP.name() : 
                               tip.isMapTask() ? Values.MAP.name() : Values.REDUCE.name(), 
-                              System.currentTimeMillis(), reason); 
+                              tip.getExecFinishTime(), reason, taskid); 
   }
        
                            
@@ -1953,6 +2054,10 @@ class JobInProgress {
     }
     return null;
   }
+  
+  synchronized int getNumTaskCompletionEvents() {
+    return taskCompletionEvents.size();
+  }
     
   synchronized public TaskCompletionEvent[] getTaskCompletionEvents(
                                                                     int fromEventId, int maxEvents) {

+ 672 - 50
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
+import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -45,6 +46,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.AccessControlException;
@@ -53,6 +55,9 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
+import org.apache.hadoop.mapred.JobHistory.Keys;
+import org.apache.hadoop.mapred.JobHistory.Listener;
+import org.apache.hadoop.mapred.JobHistory.Values;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
@@ -389,6 +394,501 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   }
 
  
+  ///////////////////////////////////////////////////////
+  // Used to recover the jobs upon restart
+  ///////////////////////////////////////////////////////
+  class RecoveryManager {
+    Set<JobID> jobsToRecover; // set of jobs to be recovered
+    
+    private int totalEventsRecovered = 0;
+    
+    /** A custom listener that replays the events in the order in which the 
+     * events (task attempts) occurred. 
+     */
+    class JobRecoveryListener implements Listener {
+      // The owner job
+      private JobInProgress jip;
+      
+      private JobHistory.JobInfo job; // current job's info object
+      
+      // Maintain the count of the (attempt) events recovered
+      private int numEventsRecovered = 0;
+      
+      // Maintains open transactions
+      private Map<String, String> hangingAttempts = 
+        new HashMap<String, String>();
+      
+      // Whether there are any updates for this job
+      private boolean hasUpdates = false;
+      
+      public JobRecoveryListener(JobInProgress jip) {
+        this.jip = jip;
+        this.job = new JobHistory.JobInfo(jip.getJobID().toString());
+      }
+
+      /**
+       * Process a task. Note that a task might commit a previously pending 
+       * transaction.
+       */
+      private void processTask(String taskId, JobHistory.Task task) {
+        // Any TASK info commits the previous transaction
+        boolean hasHanging = hangingAttempts.remove(taskId) != null;
+        if (hasHanging) {
+          numEventsRecovered += 2;
+        }
+        
+        TaskID id = TaskID.forName(taskId);
+        TaskInProgress tip = getTip(id);
+        
+        updateTip(tip, task);
+      }
+
+      /**
+       * Adds a task-attempt in the listener
+       */
+      private void processTaskAttempt(String taskAttemptId, 
+                                      JobHistory.TaskAttempt attempt) {
+        TaskAttemptID id = TaskAttemptID.forName(taskAttemptId);
+        
+        // Check if the transaction for this attempt can be committed
+        String taskStatus = attempt.get(Keys.TASK_STATUS);
+        
+        if (taskStatus.length() > 0) {
+          // This means this is an update event
+          if (taskStatus.equals(Values.SUCCESS.name())) {
+            // Mark this attempt as hanging
+            hangingAttempts.put(id.getTaskID().toString(), taskAttemptId);
+            addSuccessfulAttempt(jip, id, attempt);
+          } else {
+            addUnsuccessfulAttempt(jip, id, attempt);
+            numEventsRecovered += 2;
+          }
+        } else {
+          createTaskAttempt(jip, id, attempt);
+        }
+      }
+
+      public void handle(JobHistory.RecordTypes recType, Map<Keys, 
+                         String> values) throws IOException {
+        if (recType == JobHistory.RecordTypes.Job) {
+          // Update the meta-level job information
+          job.handle(values);
+          
+          // Forcefully init the job as we have some updates for it
+          checkAndInit();
+        } else if (recType.equals(JobHistory.RecordTypes.Task)) {
+          String taskId = values.get(Keys.TASKID);
+          
+          // Create a task
+          JobHistory.Task task = new JobHistory.Task();
+          task.handle(values);
+          
+          // Ignore if its a cleanup task
+          if (isCleanup(task)) {
+            return;
+          }
+            
+          // Process the task i.e update the tip state
+          processTask(taskId, task);
+        } else if (recType.equals(JobHistory.RecordTypes.MapAttempt)) {
+          String attemptId = values.get(Keys.TASK_ATTEMPT_ID);
+          
+          // Create a task attempt
+          JobHistory.MapAttempt attempt = new JobHistory.MapAttempt();
+          attempt.handle(values);
+          
+          // Ignore if its a cleanup task
+          if (isCleanup(attempt)) {
+            return;
+          }
+          
+          // Process the attempt i.e update the attempt state via job
+          processTaskAttempt(attemptId, attempt);
+        } else if (recType.equals(JobHistory.RecordTypes.ReduceAttempt)) {
+          String attemptId = values.get(Keys.TASK_ATTEMPT_ID);
+          
+          // Create a task attempt
+          JobHistory.ReduceAttempt attempt = new JobHistory.ReduceAttempt();
+          attempt.handle(values);
+          
+          // Ignore if its a cleanup task
+          if (isCleanup(attempt)) {
+            return;
+          }
+          
+          // Process the attempt i.e update the job state via job
+          processTaskAttempt(attemptId, attempt);
+        }
+      }
+
+      // Check if the task is of type CLEANUP
+      private boolean isCleanup(JobHistory.Task task) {
+        String taskType = task.get(Keys.TASK_TYPE);
+        return Values.CLEANUP.name().equals(taskType);
+      }
+      
+      // Init the job if its ready for init. Also make sure that the scheduler
+      // is updated
+      private void checkAndInit() throws IOException {
+        String jobStatus = this.job.get(Keys.JOB_STATUS);
+        if (Values.RUNNING.name().equals(jobStatus)) {
+          hasUpdates = true;
+          LOG.info("Calling init from RM for job " + jip.getJobID().toString());
+          jip.initTasks();
+          updateJobListeners();
+        }
+      }
+      
+      private void updateJobListeners() {
+        // The scheduler needs to be informed as the recovery-manager
+        // has inited the jobs
+        for (JobInProgressListener listener : jobInProgressListeners) {
+          listener.jobUpdated(jip);
+        }
+      }
+      
+      void close() {
+        if (hasUpdates) {
+          // Apply the final (job-level) updates
+          updateJob(jip, job);
+          // Update the job listeners as the start/submit time and the job 
+          // priority has changed
+          updateJobListeners();
+        }
+      }
+      
+      public int getNumEventsRecovered() {
+        return numEventsRecovered;
+      }
+
+    }
+    
+    public RecoveryManager() {
+      jobsToRecover = new TreeSet<JobID>();
+    }
+
+    public boolean contains(JobID id) {
+      return jobsToRecover.contains(id);
+    }
+
+    void addJobForRecovery(JobID id) {
+      jobsToRecover.add(id);
+    }
+
+    public boolean shouldRecover() {
+      return jobsToRecover.size() != 0;
+    }
+
+    // checks if the job dir has the required files
+    public void checkAndAddJob(FileStatus status) throws IOException {
+      String jobName = status.getPath().getName();
+      if (JobID.isJobNameValid(jobName)) {
+        if (JobClient.isJobDirValid(status.getPath(), fs)) {
+          recoveryManager.addJobForRecovery(JobID.forName(jobName));
+        } else {
+          LOG.info("Found an incomplete job directory " + jobName + "." 
+                   + " Deleting it!!");
+          fs.delete(status.getPath(), true);
+        }
+      } else {
+        LOG.info("Deleting " + status.getPath());
+        fs.delete(status.getPath(), true);
+      }
+    }
+    
+    private void updateJob(JobInProgress jip, JobHistory.JobInfo job) {
+      // Set the start/launch time only if there are recovered tasks
+      jip.updateJobTime(job.getLong(JobHistory.Keys.SUBMIT_TIME), 
+                        job.getLong(JobHistory.Keys.LAUNCH_TIME));
+      
+      // Change the job priority
+      String jobpriority = job.get(Keys.JOB_PRIORITY);
+      if (jobpriority.length() > 0) {
+        JobPriority priority = JobPriority.valueOf(jobpriority);
+        // Its important to update this via the jobtracker's api
+        setJobPriority(jip.getJobID(), priority);
+      }
+    }
+    
+    private void updateTip(TaskInProgress tip, JobHistory.Task task) {
+      long startTime = task.getLong(Keys.START_TIME);
+      if (startTime != 0) {
+        tip.setExecStartTime(startTime);
+      }
+      
+      long finishTime = task.getLong(Keys.FINISH_TIME);
+      // For failed tasks finish-time will be missing
+      if (finishTime != 0) {
+        tip.setExecFinishTime(finishTime);
+      }
+      
+      String cause = task.get(Keys.TASK_ATTEMPT_ID);
+      if (cause.length() > 0) {
+        // This means that the this is a FAILED events
+        TaskAttemptID id = TaskAttemptID.forName(cause);
+        TaskStatus status = tip.getTaskStatus(id);
+        // This will add the tip failed event in the new log
+        tip.getJob().failedTask(tip, id, status.getDiagnosticInfo(), 
+                                status.getPhase(), status.getRunState(), 
+                                status.getTaskTracker(), myInstrumentation);
+      }
+    }
+    
+    private void createTaskAttempt(JobInProgress job, 
+                                   TaskAttemptID attemptId, 
+                                   JobHistory.TaskAttempt attempt) {
+      TaskID id = attemptId.getTaskID();
+      String type = attempt.get(Keys.TASK_TYPE);
+      TaskInProgress tip = job.getTaskInProgress(id);
+      
+      //    I. Get the required info
+      TaskStatus taskStatus = null;
+      String trackerName = attempt.get(Keys.TRACKER_NAME);
+      String trackerHostName = 
+        JobInProgress.convertTrackerNameToHostName(trackerName);
+      int index = trackerHostName.indexOf("_");
+      trackerHostName = 
+        trackerHostName.substring(index + 1, trackerHostName.length());
+      int port = attempt.getInt(Keys.HTTP_PORT);
+      
+      long attemptStartTime = attempt.getLong(Keys.START_TIME);
+
+      // II. Create the (appropriate) task status
+      if (type.equals(Values.MAP.name())) {
+        taskStatus = 
+          new MapTaskStatus(attemptId, 0.0f, TaskStatus.State.RUNNING, 
+                            "", "", trackerName, TaskStatus.Phase.MAP, 
+                            new Counters());
+      } else {
+        taskStatus = 
+          new ReduceTaskStatus(attemptId, 0.0f, TaskStatus.State.RUNNING, 
+                               "", "", trackerName, TaskStatus.Phase.REDUCE, 
+                               new Counters());
+      }
+
+      // Set the start time
+      taskStatus.setStartTime(attemptStartTime);
+
+      List<TaskStatus> ttStatusList = new ArrayList<TaskStatus>();
+      ttStatusList.add(taskStatus);
+      
+      // III. Create the dummy tasktracker status
+      TaskTrackerStatus ttStatus = 
+        new TaskTrackerStatus(trackerName, trackerHostName, port, ttStatusList, 
+                              0 , 0, 0);
+      ttStatus.setLastSeen(System.currentTimeMillis());
+
+      // IV. Register a new tracker
+      boolean isTrackerRegistered = getTaskTracker(trackerName) != null;
+      if (!isTrackerRegistered) {
+        addNewTracker(ttStatus);
+      }
+      
+      // V. Update the tracker status
+      //    This will update the meta info of the jobtracker and also add the
+      //    tracker status if missing i.e register it
+      updateTaskTrackerStatus(trackerName, ttStatus);
+      
+      // VI. Register the attempt
+      //   a) In the job
+      job.addRunningTaskToTIP(tip, attemptId, ttStatus, false);
+      //   b) In the tip
+      tip.updateStatus(taskStatus);
+      
+      // VII. Make an entry in the launched tasks
+      expireLaunchingTasks.addNewTask(attemptId);
+    }
+    
+    private void addSuccessfulAttempt(JobInProgress job, 
+                                      TaskAttemptID attemptId, 
+                                      JobHistory.TaskAttempt attempt) {
+      // I. Get the required info
+      TaskID taskId = attemptId.getTaskID();
+      String type = attempt.get(Keys.TASK_TYPE);
+
+      TaskInProgress tip = job.getTaskInProgress(taskId);
+      long attemptFinishTime = attempt.getLong(Keys.FINISH_TIME);
+
+      // Get the task status and the tracker name and make a copy of it
+      TaskStatus taskStatus = (TaskStatus)tip.getTaskStatus(attemptId).clone();
+      taskStatus.setFinishTime(attemptFinishTime);
+
+      String stateString = attempt.get(Keys.STATE_STRING);
+
+      // Update the basic values
+      taskStatus.setStateString(stateString);
+      taskStatus.setProgress(1.0f);
+      taskStatus.setRunState(TaskStatus.State.SUCCEEDED);
+
+      // Set the shuffle/sort finished times
+      if (type.equals(Values.REDUCE.name())) {
+        long shuffleTime = 
+          Long.parseLong(attempt.get(Keys.SHUFFLE_FINISHED));
+        long sortTime = 
+          Long.parseLong(attempt.get(Keys.SORT_FINISHED));
+        taskStatus.setShuffleFinishTime(shuffleTime);
+        taskStatus.setSortFinishTime(sortTime);
+      }
+
+      // Add the counters
+      String counterString = attempt.get(Keys.COUNTERS);
+      Counters counter = null;
+      //TODO Check if an exception should be thrown
+      try {
+        counter = Counters.fromEscapedCompactString(counterString);
+      } catch (ParseException pe) { 
+        counter = new Counters(); // Set it to empty counter
+      }
+      taskStatus.setCounters(counter);
+      
+      // II. Replay the status
+      job.updateTaskStatus(tip, taskStatus, myInstrumentation);
+      
+      // III. Prevent the task from expiry
+      expireLaunchingTasks.removeTask(attemptId);
+    }
+    
+    private void addUnsuccessfulAttempt(JobInProgress job,
+                                        TaskAttemptID attemptId,
+                                        JobHistory.TaskAttempt attempt) {
+      // I. Get the required info
+      TaskID taskId = attemptId.getTaskID();
+      TaskInProgress tip = job.getTaskInProgress(taskId);
+      long attemptFinishTime = attempt.getLong(Keys.FINISH_TIME);
+
+      TaskStatus taskStatus = (TaskStatus)tip.getTaskStatus(attemptId).clone();
+      taskStatus.setFinishTime(attemptFinishTime);
+
+      // Reset the progress
+      taskStatus.setProgress(0.0f);
+      
+      String stateString = attempt.get(Keys.STATE_STRING);
+      taskStatus.setStateString(stateString);
+
+      boolean hasFailed = 
+        attempt.get(Keys.TASK_STATUS).equals(Values.FAILED.name());
+      // Set the state failed/killed
+      if (hasFailed) {
+        taskStatus.setRunState(TaskStatus.State.FAILED);
+      } else {
+        taskStatus.setRunState(TaskStatus.State.KILLED);
+      }
+
+      // Get/Set the error msg
+      String diagInfo = attempt.get(Keys.ERROR);
+      taskStatus.setDiagnosticInfo(diagInfo); // diag info
+
+      // II. Update the task status
+     job.updateTaskStatus(tip, taskStatus, myInstrumentation);
+
+     // III. Prevent the task from expiry
+     expireLaunchingTasks.removeTask(attemptId);
+    }
+  
+    public void recover() throws IOException {
+      // I. Init the jobs and cache the recovered job history filenames
+      Map<JobID, Path> jobHistoryFilenameMap = new HashMap<JobID, Path>();
+      for (JobID id : jobsToRecover) {
+        // 1. Create the job object
+        JobInProgress job = new JobInProgress(id, JobTracker.this, conf);
+        
+        // 2. Get the log file and the file path
+        String logFileName = 
+          JobHistory.JobInfo.getJobHistoryFileName(job.getJobConf(), id);
+        Path jobHistoryFilePath = 
+          JobHistory.JobInfo.getJobHistoryLogLocation(logFileName);
+        
+        // 3. Recover the history file. This involved
+        //     - deleting file.recover if file exists
+        //     - renaming file.recover to file if file doesnt exist
+        // This makes sure that the (master) file exists
+        JobHistory.JobInfo.recoverJobHistoryFile(job.getJobConf(), 
+                                                 jobHistoryFilePath);
+
+        // 4. Cache the history file name as it costs one dfs access
+        jobHistoryFilenameMap.put(job.getJobID(), jobHistoryFilePath);
+
+        // 5. Sumbit the job to the jobtracker
+        addJob(id, job);
+      }
+
+      long recoveryStartTime = System.currentTimeMillis();
+
+      // II. Recover each job
+      for (JobID id : jobsToRecover) {
+        JobInProgress pJob = getJob(id);
+
+        // 1. Get the required info
+        // Get the recovered history file
+        Path jobHistoryFilePath = jobHistoryFilenameMap.get(pJob.getJobID());
+        String logFileName = jobHistoryFilePath.getName();
+
+        FileSystem fs = jobHistoryFilePath.getFileSystem(conf);
+
+        // 2. Parse the history file
+        // Note that this also involves job update
+        JobRecoveryListener listener = new JobRecoveryListener(pJob);
+        try {
+          JobHistory.parseHistoryFromFS(jobHistoryFilePath.toString(), 
+                                        listener, fs);
+        } catch (IOException e) {
+          LOG.info("JobTracker failed to recover job " + pJob + "."
+                   + " Ignoring it.", e);
+          continue;
+        }
+
+        // 3. Close the listener
+        listener.close();
+
+        // 4. Update the recovery metric
+        totalEventsRecovered += listener.getNumEventsRecovered();
+
+        // 5. Cleanup history
+        // Delete the master log file as an indication that the new file
+        // should be used in future
+        synchronized (pJob) {
+          JobHistory.JobInfo.checkpointRecovery(logFileName, 
+              pJob.getJobConf());
+        }
+
+        // 6. Inform the jobtracker as to how much of the data is recovered.
+        // This is done so that TT should rollback to account for lost
+        // updates
+        lastSeenEventMapOnRestart.put(pJob.getStatus().getJobID(), 
+                                      pJob.getNumTaskCompletionEvents());
+      }
+
+      recoveryDuration = System.currentTimeMillis() - recoveryStartTime;
+      hasRecovered = true;
+
+      // III. Finalize the recovery
+      // Make sure that the tracker statuses in the expiry-tracker queue
+      // are updated
+      long now = System.currentTimeMillis();
+      int size = trackerExpiryQueue.size();
+      for (int i = 0; i < size ; ++i) {
+        // Get the first status
+        TaskTrackerStatus status = trackerExpiryQueue.first();
+
+        // Remove it
+        trackerExpiryQueue.remove(status);
+
+        // Set the new time
+        status.setLastSeen(now);
+
+        // Add back to get the sorted list
+        trackerExpiryQueue.add(status);
+      }
+
+      // IV. Cleanup
+      jobsToRecover.clear();
+      LOG.info("Restoration complete");
+    }
+    
+    int totalEventsRecovered() {
+      return totalEventsRecovered;
+    }
+  }
 
   private JobTrackerInstrumentation myInstrumentation = null;
     
@@ -403,6 +903,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   private int totalMapTaskCapacity;
   private int totalReduceTaskCapacity;
   private HostsFileReader hostsReader;
+  
+  // JobTracker recovery variables
+  private volatile boolean hasRestarted = false;
+  private volatile boolean hasRecovered = false;
+  private volatile long recoveryDuration;
 
   //
   // Properties to maintain while running Jobs and Tasks:
@@ -450,6 +955,10 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   Map<String, Node> hostnameToNodeMap = 
     Collections.synchronizedMap(new TreeMap<String, Node>());
   
+  // A map from JobID to the last known task-completion-event-index on restart
+  Map<JobID, Integer> lastSeenEventMapOnRestart = 
+    new HashMap<JobID, Integer>();
+  
   // Number of resolved entries
   int numResolved;
     
@@ -472,6 +981,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
 
   CompletedJobStatusStore completedJobStatusStore = null;
   Thread completedJobsStoreThread = null;
+  RecoveryManager recoveryManager;
 
   /**
    * It might seem like a bug to maintain a TreeSet of status objects,
@@ -548,7 +1058,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     this.port = addr.getPort();
     int handlerCount = conf.getInt("mapred.job.tracker.handler.count", 10);
     this.interTrackerServer = RPC.getServer(this, addr.getHostName(), addr.getPort(), handlerCount, false, conf);
-    this.interTrackerServer.start();
     if (LOG.isDebugEnabled()) {
       Properties p = System.getProperties();
       for (Iterator it = p.keySet().iterator(); it.hasNext();) {
@@ -582,8 +1091,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     }
     infoServer.start();
     
-    SimpleDateFormat dateFormat = new SimpleDateFormat("yyyyMMddHHmm");
-    trackerIdentifier = dateFormat.format(new Date());
+    trackerIdentifier = getDateFormat().format(new Date());
 
     Class<? extends JobTrackerInstrumentation> metricsInst = getInstrumentationClass(jobConf);
     try {
@@ -608,6 +1116,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
         infoBindAddress + ":" + this.infoPort); 
     LOG.info("JobTracker webserver: " + this.infoServer.getPort());
     
+    // start the recovery manager
+    recoveryManager = new RecoveryManager();
+    
     while (true) {
       try {
         // if we haven't contacted the namenode go ahead and do it
@@ -619,6 +1130,24 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
         if(systemDir == null) {
           systemDir = new Path(getSystemDir());    
         }
+        // Make sure that the backup data is preserved
+        FileStatus[] systemDirData = fs.listStatus(this.systemDir);
+        LOG.info("Cleaning up the system directory");
+        // Check if the history is enabled .. as we cant have persistence with 
+        // history disabled
+        if (conf.getBoolean("mapred.jobtracker.restart.recover", false) 
+            && !JobHistory.isDisableHistory()
+            && systemDirData != null) {
+          for (FileStatus status : systemDirData) {
+            recoveryManager.checkAndAddJob(status);
+          }
+          
+          // Check if there are jobs to be recovered
+          hasRestarted = recoveryManager.shouldRecover();
+          if (hasRestarted) {
+            break; // if there is something to recover else clean the sys dir
+          }
+        }
         fs.delete(systemDir, true);
         if (FileSystem.mkdirs(fs, systemDir, 
             new FsPermission(SYSTEM_DIR_PERMISSION))) {
@@ -653,14 +1182,54 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
             DNSToSwitchMapping.class), conf);
     this.numTaskCacheLevels = conf.getInt("mapred.task.cache.levels", 
         NetworkTopology.DEFAULT_HOST_LEVEL);
-    synchronized (this) {
-      state = State.RUNNING;
-    }
 
     //initializes the job status store
     completedJobStatusStore = new CompletedJobStatusStore(conf,fs);
+  }
 
-    LOG.info("Starting RUNNING");
+  private static SimpleDateFormat getDateFormat() {
+    return new SimpleDateFormat("yyyyMMddHHmm");
+  }
+
+  static boolean validateIdentifier(String id) {
+    try {
+      // the jobtracker id should be 'date' parseable
+      getDateFormat().parse(id);
+      return true;
+    } catch (ParseException pe) {}
+    return false;
+  }
+
+  static boolean validateJobNumber(String id) {
+    try {
+      // the job number should be integer parseable
+      Integer.parseInt(id);
+      return true;
+    } catch (IllegalArgumentException pe) {}
+    return false;
+  }
+
+  /**
+   * Whether the JT has restarted
+   */
+  public boolean hasRestarted() {
+    return hasRestarted;
+  }
+
+  /**
+   * Whether the JT has recovered upon restart
+   */
+  public boolean hasRecovered() {
+    return hasRecovered;
+  }
+
+  /**
+   * How long the jobtracker took to recover from restart.
+   */
+  public long getRecoveryDuration() {
+    return hasRestarted() 
+           ? recoveryDuration
+           : 0;
   }
 
   public static Class<? extends JobTrackerInstrumentation> getInstrumentationClass(Configuration conf) {
@@ -683,12 +1252,16 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
    * Run forever
    */
   public void offerService() throws InterruptedException, IOException {
+    taskScheduler.start();
+    
+    //  Start the recovery after starting the scheduler
+    recoveryManager.recover();
+    
     this.expireTrackersThread = new Thread(this.expireTrackers,
                                           "expireTrackers");
     this.expireTrackersThread.start();
     this.retireJobsThread = new Thread(this.retireJobs, "retireJobs");
     this.retireJobsThread.start();
-    taskScheduler.start();
     expireLaunchingTaskThread.start();
 
     if (completedJobStatusStore.isActive()) {
@@ -697,6 +1270,14 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       completedJobsStoreThread.start();
     }
 
+    // start the inter-tracker server once the jt is ready
+    this.interTrackerServer.start();
+    
+    synchronized (this) {
+      state = State.RUNNING;
+    }
+    LOG.info("Starting RUNNING");
+    
     this.interTrackerServer.join();
     LOG.info("Stopped interTrackerServer");
   }
@@ -778,6 +1359,13 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
 
     // taskid --> TIP
     taskidToTIPMap.put(taskid, tip);
+    
+    // Note this launch
+    if (taskid.isMap()) {
+      myInstrumentation.launchMap(taskid);
+    } else {
+      myInstrumentation.launchReduce(taskid);
+    }
   }
     
   void removeTaskEntry(TaskAttemptID taskid) {
@@ -908,6 +1496,15 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     
     JobEndNotifier.registerNotification(job.getJobConf(), job.getStatus());
 
+    // start the merge of log files
+    JobID id = job.getStatus().getJobID();
+    try {
+      JobHistory.JobInfo.finalizeRecovery(id, job.getJobConf());
+    } catch (IOException ioe) {
+      LOG.info("Failed to finalize the log file recovery for job " + id, ioe);
+    }
+    
+    
     // Purge oldest jobs and keep at-most MAX_COMPLETE_USER_JOBS_IN_MEMORY jobs of a given user
     // in memory; information about the purged jobs is available via
     // JobHistory.
@@ -1055,6 +1652,21 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     }
   }
 
+  /**
+   * Adds a new node to the jobtracker. It involves adding it to the expiry
+   * thread and adding it for resolution
+   * @param status Task Tracker's status
+   * @param resolveInline Should the resolution happen inline?
+   */
+  private void addNewTracker(TaskTrackerStatus status) {
+    trackerExpiryQueue.add(status);
+    //  Register the tracker if its not registered
+    if (getNode(status.getTrackerName()) == null) {
+      // Making the network location resolution inline .. 
+      resolveAndAddToTopology(status.getHost());
+    }
+  }
+
   public Node resolveAndAddToTopology(String name) {
     List <String> tmpList = new ArrayList<String>(1);
     tmpList.add(name);
@@ -1168,6 +1780,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     
     HeartbeatResponse prevHeartbeatResponse =
       trackerToHeartbeatResponseMap.get(trackerName);
+    boolean addRestartInfo = false;
 
     if (initialContact != true) {
       // If this isn't the 'initial contact' from the tasktracker,
@@ -1175,32 +1788,34 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       // no record of the 'previous heartbeat'; if so, ask the 
       // tasktracker to re-initialize itself.
       if (prevHeartbeatResponse == null) {
-        LOG.warn("Serious problem, cannot find record of 'previous' " +
-                 "heartbeat for '" + trackerName + 
-                 "'; reinitializing the tasktracker");
-        return new HeartbeatResponse(responseId, 
-                                     new TaskTrackerAction[] {new ReinitTrackerAction()});
+        // This is the first heartbeat from the old tracker to the newly 
+        // started JobTracker
+        if (hasRestarted()) {
+          addRestartInfo = true;
+        } else {
+          // Jobtracker might have restarted but no recovery is needed
+          LOG.warn("Serious problem, cannot find record of 'previous' " +
+                   "heartbeat for '" + trackerName + 
+                   "'; reinitializing the tasktracker");
+          return new HeartbeatResponse(responseId, 
+              new TaskTrackerAction[] {new ReinitTrackerAction()});
+        }
 
-      }
+      } else {
                 
-      // It is completely safe to not process a 'duplicate' heartbeat from a 
-      // {@link TaskTracker} since it resends the heartbeat when rpcs are lost - 
-      // @see {@link TaskTracker.transmitHeartbeat()};
-      // acknowledge it by re-sending the previous response to let the 
-      // {@link TaskTracker} go forward. 
-      if (prevHeartbeatResponse.getResponseId() != responseId) {
-        LOG.info("Ignoring 'duplicate' heartbeat from '" + 
-                 trackerName + "'; resending the previous 'lost' response");
-        return prevHeartbeatResponse;
+        // It is completely safe to not process a 'duplicate' heartbeat from a 
+        // {@link TaskTracker} since it resends the heartbeat when rpcs are 
+        // lost see {@link TaskTracker.transmitHeartbeat()};
+        // acknowledge it by re-sending the previous response to let the 
+        // {@link TaskTracker} go forward. 
+        if (prevHeartbeatResponse.getResponseId() != responseId) {
+          LOG.info("Ignoring 'duplicate' heartbeat from '" + 
+              trackerName + "'; resending the previous 'lost' response");
+          return prevHeartbeatResponse;
+        }
       }
     }
       
-    // Register the tracker if its not registered
-    if (getNode(trackerName) == null) {
-      // Making the network location resolution inline .. 
-      resolveAndAddToTopology(status.getHost());
-    }
-    
     // Process this heartbeat 
     short newResponseId = (short)(responseId + 1);
     if (!processHeartbeat(status, initialContact)) {
@@ -1229,11 +1844,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
         if (tasks != null) {
           for (Task task : tasks) {
             expireLaunchingTasks.addNewTask(task.getTaskID());
-            if (task.isMapTask()) {
-              myInstrumentation.launchMap(task.getTaskID());
-            } else {
-              myInstrumentation.launchReduce(task.getTaskID());
-            }
             LOG.debug(trackerName + " -> LaunchTask: " + task.getTaskID());
             actions.add(new LaunchTaskAction(task));
           }
@@ -1258,6 +1868,11 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     response.setHeartbeatInterval(nextInterval);
     response.setActions(
                         actions.toArray(new TaskTrackerAction[actions.size()]));
+    
+    // check if the restart info is req
+    if (addRestartInfo) {
+      response.setLastKnownIndices(lastSeenEventMapOnRestart);
+    }
         
     // Update the trackerToHeartbeatResponseMap
     trackerToHeartbeatResponseMap.put(trackerName, response);
@@ -1388,7 +2003,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
         }
 
         if (initialContact) {
-          trackerExpiryQueue.add(trackerStatus);
+          addNewTracker(trackerStatus);
         }
       }
     }
@@ -1526,20 +2141,10 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   // JobSubmissionProtocol
   ////////////////////////////////////////////////////
 
-  /**
-   * Make sure the JobTracker is done initializing.
-   */
-  private synchronized void ensureRunning() throws IllegalStateException {
-    if (state != State.RUNNING) {
-      throw new IllegalStateException("Job tracker still initializing");
-    }
-  }
-
   /**
    * Allocates a new JobId string.
    */
   public synchronized JobID getNewJobId() throws IOException {
-    ensureRunning();
     return new JobID(getTrackerIdentifier(), nextJobId++);
   }
 
@@ -1552,14 +2157,23 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
    * the JobTracker alone.
    */
   public synchronized JobStatus submitJob(JobID jobId) throws IOException {
-    ensureRunning();
     if(jobs.containsKey(jobId)) {
       //job already running, don't start twice
       return jobs.get(jobId).getStatus();
     }
     
-    totalSubmissions++;
     JobInProgress job = new JobInProgress(jobId, this, this.conf);
+   return addJob(jobId, job); 
+  }
+
+  /**
+   * Adds a job to the jobtracker. Make sure that the checks are inplace before
+   * adding a job. This is the core job submission logic
+   * @param jobId The id for the job submitted which needs to be added
+   */
+  private synchronized JobStatus addJob(JobID jobId, JobInProgress job) 
+  throws IOException {
+    totalSubmissions++;
     checkAccess(job, QueueManager.QueueOperation.SUBMIT_JOB);
 
     synchronized (jobs) {
@@ -1885,11 +2499,19 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
       report.setTaskTracker(trackerName);
       TaskAttemptID taskId = report.getTaskID();
       TaskInProgress tip = taskidToTIPMap.get(taskId);
-      if (tip == null) {
-        LOG.info("Serious problem.  While updating status, cannot find taskid " + report.getTaskID());
-      } else {
+      // Check if the tip is known to the jobtracker. In case of a restarted
+      // jt, some tasks might join in later
+      if (tip != null || hasRestarted()) {
+        if (tip == null) {
+          JobInProgress job = getJob(taskId.getJobID());
+          tip = job.getTaskInProgress(taskId.getTaskID());
+          job.addRunningTaskToTIP(tip, taskId, status, false);
+        }
         expireLaunchingTasks.removeTask(taskId);
         tip.getJob().updateTaskStatus(tip, report, myInstrumentation);
+      } else {
+        LOG.info("Serious problem.  While updating status, cannot find taskid " 
+                 + report.getTaskID());
       }
       
       // Process 'failed fetch' notifications 

+ 9 - 5
src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -104,6 +104,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
         InputSplit[] splits;
         splits = job.getInputFormat().getSplits(job, 1);
         JobID jobId = profile.getJobID();
+        long timstamp = System.currentTimeMillis();
         
         int numReduceTasks = job.getNumReduceTasks();
         if (numReduceTasks > 1 || numReduceTasks < 0) {
@@ -116,7 +117,8 @@ class LocalJobRunner implements JobSubmissionProtocol {
         
         DataOutputBuffer buffer = new DataOutputBuffer();
         for (int i = 0; i < splits.length; i++) {
-          TaskAttemptID mapId = new TaskAttemptID(new TaskID(jobId, true, i), 0);  
+          TaskAttemptID mapId = new TaskAttemptID(new TaskID(jobId, true, i), 
+                                                  0, timstamp);  
           mapIds.add(mapId);
           buffer.reset();
           splits[i].write(buffer);
@@ -137,7 +139,8 @@ class LocalJobRunner implements JobSubmissionProtocol {
           map_tasks -= 1;
           updateCounters(map);
         }
-        TaskAttemptID reduceId = new TaskAttemptID(new TaskID(jobId, false, 0), 0);
+        TaskAttemptID reduceId = new TaskAttemptID(new TaskID(jobId, false, 0),
+                                                   0, timstamp);
         try {
           if (numReduceTasks > 0) {
             // move map output to reduce input  
@@ -277,9 +280,10 @@ class LocalJobRunner implements JobSubmissionProtocol {
       LOG.fatal("shuffleError: "+ message + "from task: " + taskId);
     }
     
-    public TaskCompletionEvent[] getMapCompletionEvents(JobID jobId
-        , int fromEventId, int maxLocs) throws IOException {
-      return TaskCompletionEvent.EMPTY_ARRAY;
+    public MapTaskCompletionEventsUpdate getMapCompletionEvents(JobID jobId, 
+        int fromEventId, int maxLocs, TaskAttemptID id) throws IOException {
+      return new MapTaskCompletionEventsUpdate(TaskCompletionEvent.EMPTY_ARRAY,
+                                               false);
     }
     
   }

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/MapTask.java

@@ -87,7 +87,7 @@ class MapTask extends Task {
 
   public MapTask(String jobFile, TaskAttemptID taskId, 
                  int partition, String splitClass, BytesWritable split
-                 ) throws IOException {
+                 ) {
     super(jobFile, taskId, partition);
     this.splitClass = splitClass;
     this.split = split;

+ 67 - 0
src/mapred/org/apache/hadoop/mapred/MapTaskCompletionEventsUpdate.java

@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.hadoop.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * A class that represents the communication between the tasktracker and child
+ * tasks w.r.t the map task completion events. It also indicates whether the 
+ * child task should reset its events index.
+ */
+class MapTaskCompletionEventsUpdate implements Writable {
+  TaskCompletionEvent[] events;
+  boolean reset;
+
+  public MapTaskCompletionEventsUpdate() { }
+
+  public MapTaskCompletionEventsUpdate(TaskCompletionEvent[] events,
+      boolean reset) {
+    this.events = events;
+    this.reset = reset;
+  }
+
+  public boolean shouldReset() {
+    return reset;
+  }
+
+  public TaskCompletionEvent[] getMapTaskCompletionEvents() {
+    return events;
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeBoolean(reset);
+    out.writeInt(events.length);
+    for (TaskCompletionEvent event : events) {
+      event.write(out);
+    }
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    reset = in.readBoolean();
+    events = new TaskCompletionEvent[in.readInt()];
+    for (int i = 0; i < events.length; ++i) {
+      events[i] = new TaskCompletionEvent();
+      events[i].readFields(in);
+    }
+  }
+}

+ 13 - 2
src/mapred/org/apache/hadoop/mapred/ReduceTask.java

@@ -2236,9 +2236,20 @@ class ReduceTask extends Task {
         currentTime = System.currentTimeMillis();
       }
       
-      TaskCompletionEvent events[] = 
+      MapTaskCompletionEventsUpdate update = 
         umbilical.getMapCompletionEvents(reduceTask.getJobID(), 
-                                       fromEventId.get(), MAX_EVENTS_TO_FETCH);
+                                         fromEventId.get(), MAX_EVENTS_TO_FETCH,
+                                         reduceTask.getTaskID());
+      TaskCompletionEvent events[] = update.getMapTaskCompletionEvents();
+        
+      // Check if the reset is required.
+      // Since there is no ordering of the task completion events at the 
+      // reducer, the only option to sync with the new jobtracker is to reset 
+      // the events index
+      if (update.shouldReset()) {
+        fromEventId.set(0);
+        obsoleteMapIds.clear(); // clear the obsolete map
+      }
       
       // Note the last successful poll time-stamp
       lastPollTime = currentTime;

+ 100 - 19
src/mapred/org/apache/hadoop/mapred/TaskAttemptID.java

@@ -22,18 +22,28 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.hadoop.io.WritableUtils;
+
 /**
  * TaskAttemptID represents the immutable and unique identifier for 
  * a task attempt. Each task attempt is one particular instance of a Map or
  * Reduce Task identified by its TaskID. 
  * 
- * TaskAttemptID consists of 2 parts. First part is the 
+ * TaskAttemptID consists of 3 parts. First part is the 
  * {@link TaskID}, that this TaskAttemptID belongs to.
- * Second part is the task attempt number. <br> 
+ * Second part is the task attempt number. Third part is the unique identifier
+ * for distinguishing tasks-attempts across jobtracker restarts.<br> 
  * An example TaskAttemptID is : 
- * <code>attempt_200707121733_0003_m_000005_0</code> , which represents the
- * zeroth task attempt for the fifth map task in the third job 
- * running at the jobtracker started at <code>200707121733</code>. 
+ * <code>attempt_200707121733_0003_m_000005_0_1234567890123</code> , which 
+ * represents the zeroth task attempt for the fifth map task in the third job 
+ * running at the jobtracker started at <code>200707121733</code> with 
+ * timestamp <code>1234567890123</code>. There could be another attempt with id
+ * <code>attempt_200707121733_0003_m_000005_0_1234567890124</code> which 
+ * indicates that the task was scheduled by the jobtracker started at timestamp
+ * <code>1234567890124</code>. <code>200707121733</code> here indicates that 
+ * the job was started by the jobtracker that was started at 
+ * <code>200707121733</code>, although this task-attempt was scheduled by the 
+ * new jobtracker. 
  * <p>
  * Applications should never construct or parse TaskAttemptID strings
  * , but rather use appropriate constructors or {@link #forName(String)} 
@@ -45,19 +55,39 @@ import java.io.IOException;
 public class TaskAttemptID extends ID {
   private static final String ATTEMPT = "attempt";
   private TaskID taskId;
+  private long jtTimestamp = 0;
   private static final char UNDERSCORE = '_';
   
+  /**
+   * @deprecated Use {@link #TaskAttemptID(TaskID, int, long)} instead.
+   */
+  public TaskAttemptID(TaskID taskId, int id) {
+    this(taskId, id, 0);
+  }
+  
   /**
    * Constructs a TaskAttemptID object from given {@link TaskID}.  
    * @param taskId TaskID that this task belongs to  
    * @param id the task attempt number
+   * @param jtTimestamp timestamp that uniquely identifies the task 
+   *        attempt across restarts
    */
-  public TaskAttemptID(TaskID taskId, int id) {
+  public TaskAttemptID(TaskID taskId, int id, long jtTimestamp) {
     super(id);
     if(taskId == null) {
       throw new IllegalArgumentException("taskId cannot be null");
     }
     this.taskId = taskId;
+    this.jtTimestamp = jtTimestamp;
+  }
+  
+  /**
+   * @deprecated 
+   *   Use {@link #TaskAttemptID(String, int, boolean, int, int, long)} instead
+   */
+  public TaskAttemptID(String jtIdentifier, int jobId, boolean isMap, 
+                       int taskId, int id) {
+    this(new TaskID(jtIdentifier, jobId, isMap, taskId), id, 0);
   }
   
   /**
@@ -67,10 +97,13 @@ public class TaskAttemptID extends ID {
    * @param isMap whether the tip is a map 
    * @param taskId taskId number
    * @param id the task attempt number
+   * @param jtTimestamp timestamp that uniquely identifies the task attempt 
+   *        across restarts
    */
-  public TaskAttemptID(String jtIdentifier, int jobId, boolean isMap
-      , int taskId, int id) {
-    this(new TaskID(jtIdentifier, jobId, isMap, taskId), id);
+  public TaskAttemptID(String jtIdentifier, int jobId, boolean isMap, 
+                       int taskId, int id, long jtTimestamp) {
+    this(new TaskID(jtIdentifier, jobId, isMap, taskId), id, 
+                    jtTimestamp);
   }
   
   private TaskAttemptID() { }
@@ -97,7 +130,8 @@ public class TaskAttemptID extends ID {
     if(o.getClass().equals(TaskAttemptID.class)) {
       TaskAttemptID that = (TaskAttemptID)o;
       return this.id==that.id
-        && this.taskId.equals(that.taskId);
+             && this.taskId.equals(that.taskId) 
+             && this.jtTimestamp == that.jtTimestamp;
     }
     else return false;
   }
@@ -108,9 +142,12 @@ public class TaskAttemptID extends ID {
     TaskAttemptID that = (TaskAttemptID)o;
     int tipComp = this.taskId.compareTo(that.taskId);
     if(tipComp == 0) {
-      return this.id - that.id;
+      tipComp = this.id - that.id;
+    }
+    if (tipComp == 0) {
+      tipComp = Long.valueOf(this.jtTimestamp).compareTo(that.jtTimestamp);
     }
-    else return tipComp;
+    return tipComp;
   }
   @Override
   public String toString() { 
@@ -120,9 +157,13 @@ public class TaskAttemptID extends ID {
   }
 
   StringBuilder toStringWOPrefix() {
+    // This is just for backward compability.
+    String appendForTimestamp = (jtTimestamp == 0) 
+                                ? "" 
+                                : UNDERSCORE + String.valueOf(jtTimestamp);
     StringBuilder builder = new StringBuilder();
     return builder.append(taskId.toStringWOPrefix())
-      .append(UNDERSCORE).append(id);
+                  .append(UNDERSCORE).append(id).append(appendForTimestamp);
   }
   
   @Override
@@ -134,12 +175,14 @@ public class TaskAttemptID extends ID {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     this.taskId = TaskID.read(in);
+    this.jtTimestamp = WritableUtils.readVLong(in);
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
     taskId.write(out);
+    WritableUtils.writeVLong(out, jtTimestamp);
   }
   
   public static TaskAttemptID read(DataInput in) throws IOException {
@@ -157,14 +200,20 @@ public class TaskAttemptID extends ID {
       return null;
     try {
       String[] parts = str.split("_");
-      if(parts.length == 6) {
+      long jtTimestamp = 0;
+      // This is for backward compability
+      if(parts.length == 6 || parts.length == 7) {
         if(parts[0].equals(ATTEMPT)) {
           boolean isMap = false;
           if(parts[3].equals("m")) isMap = true;
           else if(parts[3].equals("r")) isMap = false;
           else throw new Exception();
+          if (parts.length == 7) {
+            jtTimestamp = Long.parseLong(parts[6]);
+          }
           return new TaskAttemptID(parts[1], Integer.parseInt(parts[2]),
-              isMap, Integer.parseInt(parts[4]), Integer.parseInt(parts[5]));
+                                   isMap, Integer.parseInt(parts[4]), 
+                                   Integer.parseInt(parts[5]), jtTimestamp);
         }
       }
     }catch (Exception ex) {//fall below
@@ -174,6 +223,20 @@ public class TaskAttemptID extends ID {
   }
   
   /** 
+   * @return a regex pattern matching TaskAttemptIDs
+   * @deprecated Use {@link #getTaskAttemptIDsPattern(String, Integer, Boolean,
+   *                                                  Integer, Integer, Long)} 
+   *             instead.
+   */
+  public static String getTaskAttemptIDsPattern(String jtIdentifier,
+      Integer jobId, Boolean isMap, Integer taskId, Integer attemptId) {
+    StringBuilder builder = new StringBuilder(ATTEMPT).append(UNDERSCORE);
+    builder.append(getTaskAttemptIDsPatternWOPrefix(jtIdentifier, jobId,
+                   isMap, taskId, attemptId, null));
+    return builder.toString();
+  }
+  
+  /**
    * Returns a regex pattern which matches task attempt IDs. Arguments can 
    * be given null, in which case that part of the regex will be generic.  
    * For example to obtain a regex matching <i>all task attempt IDs</i> 
@@ -189,16 +252,23 @@ public class TaskAttemptID extends ID {
    * @param isMap whether the tip is a map, or null 
    * @param taskId taskId number, or null
    * @param attemptId the task attempt number, or null
-   * @return a regex pattern matching TaskAttemptIDs
+   * @param jtTimestamp Timestamp that is used to identify task attempts across
+   *        jobtracker restarts. Make sure that timestamp has some valid value.
    */
-  public static String getTaskAttemptIDsPattern(String jtIdentifier,
-      Integer jobId, Boolean isMap, Integer taskId, Integer attemptId) {
+  public static String getTaskAttemptIDsPattern(String jtIdentifier, 
+      Integer jobId, Boolean isMap, Integer taskId, Integer attemptId, Long jtTimestamp) {
     StringBuilder builder = new StringBuilder(ATTEMPT).append(UNDERSCORE);
     builder.append(getTaskAttemptIDsPatternWOPrefix(jtIdentifier, jobId,
-        isMap, taskId, attemptId));
+                   isMap, taskId, attemptId, jtTimestamp));
     return builder.toString();
   }
   
+  /**
+   * @deprecated 
+   * Use {@link #getTaskAttemptIDsPatternWOPrefix(String, Integer, Boolean, 
+   *                                              Integer, Integer, Long)} 
+   * instead.
+   */
   static StringBuilder getTaskAttemptIDsPatternWOPrefix(String jtIdentifier
       , Integer jobId, Boolean isMap, Integer taskId, Integer attemptId) {
     StringBuilder builder = new StringBuilder();
@@ -209,4 +279,15 @@ public class TaskAttemptID extends ID {
     return builder;
   }
   
+  static StringBuilder getTaskAttemptIDsPatternWOPrefix(String jtIdentifier, 
+      Integer jobId, Boolean isMap, Integer taskId, Integer attemptId, 
+      Long jtTimestamp) {
+    StringBuilder builder = new StringBuilder();
+    builder.append(TaskID.getTaskIDsPatternWOPrefix(jtIdentifier, jobId, isMap, taskId))
+           .append(UNDERSCORE)
+           .append(attemptId != null ? attemptId : "[0-9]*")
+           .append(UNDERSCORE)
+           .append(jtTimestamp != null ? jtTimestamp : "[0-9]*");
+    return builder;
+  }
 }

+ 24 - 0
src/mapred/org/apache/hadoop/mapred/TaskCompletionEvent.java

@@ -177,6 +177,28 @@ public class TaskCompletionEvent implements Writable{
     return buf.toString();
   }
     
+  @Override
+  public boolean equals(Object o) {
+    if(o == null)
+      return false;
+    if(o.getClass().equals(TaskCompletionEvent.class)) {
+      TaskCompletionEvent event = (TaskCompletionEvent) o;
+      return this.isMap == event.isMapTask() 
+             && this.eventId == event.getEventId()
+             && this.idWithinJob == event.idWithinJob() 
+             && this.status.equals(event.getTaskStatus())
+             && this.taskId.equals(event.getTaskAttemptId()) 
+             && this.taskRunTime == event.getTaskRunTime()
+             && this.taskTrackerHttp.equals(event.getTaskTrackerHttp());
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode(); 
+  }
+
   public boolean isMapTask() {
     return isMap;
   }
@@ -194,6 +216,7 @@ public class TaskCompletionEvent implements Writable{
     WritableUtils.writeEnum(out, status); 
     WritableUtils.writeString(out, taskTrackerHttp);
     WritableUtils.writeVInt(out, taskRunTime);
+    WritableUtils.writeVInt(out, eventId);
   }
   
   public void readFields(DataInput in) throws IOException {
@@ -203,5 +226,6 @@ public class TaskCompletionEvent implements Writable{
     this.status = WritableUtils.readEnum(in, Status.class);
     this.taskTrackerHttp = WritableUtils.readString(in);
     this.taskRunTime = WritableUtils.readVInt(in);
+    this.eventId = WritableUtils.readVInt(in);
   }
 }

+ 70 - 15
src/mapred/org/apache/hadoop/mapred/TaskInProgress.java

@@ -206,6 +206,41 @@ class TaskInProgress {
   // Accessors, info, profiles, etc.
   ////////////////////////////////////
 
+  /**
+   * Return the start time
+   */
+  public long getStartTime() {
+    return startTime;
+  }
+  
+  /**
+   * Return the exec start time
+   */
+  public long getExecStartTime() {
+    return execStartTime;
+  }
+  
+  /**
+   * Set the exec start time
+   */
+  public void setExecStartTime(long startTime) {
+    execStartTime = startTime;
+  }
+  
+  /**
+   * Return the exec finish time
+   */
+  public long getExecFinishTime() {
+    return execFinishTime;
+  }
+
+  /**
+   * Set the exec finish time
+   */
+  public void setExecFinishTime(long finishTime) {
+    execFinishTime = finishTime;
+  }
+  
   /**
    * Return the parent job
    */
@@ -366,7 +401,7 @@ class TaskInProgress {
    * Returns whether the task attempt should be committed or not 
    */
   public boolean shouldCommit(TaskAttemptID taskid) {
-    return taskToCommit.equals(taskid);
+    return !isComplete() && taskToCommit.equals(taskid);
   }
 
   /**
@@ -510,7 +545,9 @@ class TaskInProgress {
       }
     }
 
-    this.activeTasks.remove(taskid);
+    // Note that there can be failures of tasks that are hosted on a machine 
+    // that has not yet registered with restarted jobtracker
+    boolean isPresent = this.activeTasks.remove(taskid) != null;
     
     // Since we do not fail completed reduces (whose outputs go to hdfs), we 
     // should note this failure only for completed maps, only if this taskid;
@@ -524,15 +561,17 @@ class TaskInProgress {
       resetSuccessfulTaskid();
     }
 
-
-    if (taskState == TaskStatus.State.FAILED) {
-      numTaskFailures++;
-      machinesWhereFailed.add(trackerHostName);
-      LOG.debug("TaskInProgress adding" + status.getNextRecordRange());
-      failedRanges.add(status.getNextRecordRange());
-      skipping = startSkipping();
-    } else {
-      numKilledTasks++;
+    // recalculate the counts only if its a genuine failure
+    if (isPresent) {
+      if (taskState == TaskStatus.State.FAILED) {
+        numTaskFailures++;
+        machinesWhereFailed.add(trackerHostName);
+        LOG.debug("TaskInProgress adding" + status.getNextRecordRange());
+        failedRanges.add(status.getNextRecordRange());
+        skipping = startSkipping();
+      } else {
+        numKilledTasks++;
+      }
     }
 
     if (numTaskFailures >= maxTaskAttempts) {
@@ -602,6 +641,7 @@ class TaskInProgress {
     //
 
     this.completes++;
+    this.execFinishTime = System.currentTimeMillis();
     recomputeProgress();
     
   }
@@ -637,6 +677,7 @@ class TaskInProgress {
     }
     this.failed = true;
     killed = true;
+    this.execFinishTime = System.currentTimeMillis();
     recomputeProgress();
   }
 
@@ -674,10 +715,15 @@ class TaskInProgress {
   void recomputeProgress() {
     if (isComplete()) {
       this.progress = 1;
-      this.execFinishTime = System.currentTimeMillis();
+      // update the counters and the state
+      TaskStatus completedStatus = taskStatuses.get(getSuccessfulTaskid());
+      this.counters = completedStatus.getCounters();
+      this.state = completedStatus.getStateString();
     } else if (failed) {
       this.progress = 0;
-      this.execFinishTime = System.currentTimeMillis();
+      // reset the counters and the state
+      this.state = "";
+      this.counters = new Counters();
     } else {
       double bestProgress = 0;
       String bestState = "";
@@ -747,7 +793,6 @@ class TaskInProgress {
    * Return a Task that can be sent to a TaskTracker for execution.
    */
   public Task getTaskToRun(String taskTracker) throws IOException {
-    Task t = null;
     if (0 == execStartTime){
       // assume task starts running now
       execStartTime = System.currentTimeMillis();
@@ -756,7 +801,7 @@ class TaskInProgress {
     // Create the 'taskid'; do not count the 'killed' tasks against the job!
     TaskAttemptID taskid = null;
     if (nextTaskId < (MAX_TASK_EXECS + maxTaskAttempts + numKilledTasks)) {
-      taskid = new TaskAttemptID( id, nextTaskId);
+      taskid = new TaskAttemptID( id, nextTaskId, jobtracker.getStartTime());
       ++nextTaskId;
     } else {
       LOG.warn("Exceeded limit of " + (MAX_TASK_EXECS + maxTaskAttempts) +
@@ -765,6 +810,16 @@ class TaskInProgress {
       return null;
     }
 
+    return addRunningTask(taskid, taskTracker);
+  }
+  
+  /**
+   * Adds a previously running task to this tip. This is used in case of 
+   * jobtracker restarts.
+   */
+  public Task addRunningTask(TaskAttemptID taskid, String taskTracker) {
+    // create the task
+    Task t = null;
     if (isMapTask()) {
       LOG.debug("attemdpt "+  numTaskFailures   +
           " sending skippedRecords "+failedRanges.getIndicesCount());

+ 26 - 0
src/mapred/org/apache/hadoop/mapred/TaskReport.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.mapred;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -93,6 +94,31 @@ public class TaskReport implements Writable {
   void setStartTime(long startTime) {
     this.startTime = startTime;
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if(o == null)
+      return false;
+    if(o.getClass().equals(TaskReport.class)) {
+      TaskReport report = (TaskReport) o;
+      return counters.contentEquals(report.getCounters())
+             && Arrays.toString(this.diagnostics)
+                      .equals(Arrays.toString(report.getDiagnostics()))
+             && this.finishTime == report.getFinishTime()
+             && this.progress == report.getProgress()
+             && this.startTime == report.getStartTime()
+             && this.state.equals(report.getState())
+             && this.taskid.equals(report.getTaskID());
+    }
+    return false; 
+  }
+
+  @Override
+  public int hashCode() {
+    return (counters.toString() + Arrays.toString(this.diagnostics) 
+            + this.finishTime + this.progress + this.startTime + this.state 
+            + this.taskid.toString()).hashCode();
+  }
   //////////////////////////////////////////////
   // Writable
   //////////////////////////////////////////////

+ 78 - 17
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.mapred.TaskStatus.Phase;
 import org.apache.hadoop.mapred.pipes.Submitter;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsException;
@@ -154,6 +155,8 @@ public class TaskTracker
   volatile int mapTotal = 0;
   volatile int reduceTotal = 0;
   boolean justStarted = true;
+  // Mark reduce tasks that are shuffling to rollback their events index
+  Set<TaskAttemptID> shouldReset = new HashSet<TaskAttemptID>();
     
   //dir -> DF
   Map<String, DF> localDirsDf = new HashMap<String, DF>();
@@ -598,6 +601,23 @@ public class TaskTracker
       this.allMapEvents = new ArrayList<TaskCompletionEvent>(numMaps);
     }
       
+    /**
+     * Check if the number of events that are obtained are more than required.
+     * If yes then purge the extra ones.
+     */
+    public void purgeMapEvents(int lastKnownIndex) {
+      // Note that the sync is first on fromEventId and then on allMapEvents
+      synchronized (fromEventId) {
+        synchronized (allMapEvents) {
+          int index = 0;
+          if (allMapEvents.size() > lastKnownIndex) {
+            fromEventId.set(lastKnownIndex);
+            allMapEvents = allMapEvents.subList(0, lastKnownIndex);
+          }
+        }
+      }
+    }
+    
     public TaskCompletionEvent[] getMapEvents(int fromId, int max) {
         
       TaskCompletionEvent[] mapEvents = 
@@ -626,19 +646,22 @@ public class TaskTracker
       if (!fetchAgain && (currTime - lastFetchTime) < heartbeatInterval) {
         return false;
       }
-      int currFromEventId = fromEventId.get();
-      List <TaskCompletionEvent> recentMapEvents = 
-        queryJobTracker(fromEventId, jobId, jobClient);
-      synchronized (allMapEvents) {
-        allMapEvents.addAll(recentMapEvents);
-      }
-      lastFetchTime = currTime;
-      if (fromEventId.get() - currFromEventId >= probe_sample_size) {
-        //return true when we have fetched the full payload, indicating
-        //that we should fetch again immediately (there might be more to
-        //fetch
-        fetchAgain = true;
-        return true;
+      int currFromEventId = 0;
+      synchronized (fromEventId) {
+        currFromEventId = fromEventId.get();
+        List <TaskCompletionEvent> recentMapEvents = 
+          queryJobTracker(fromEventId, jobId, jobClient);
+        synchronized (allMapEvents) {
+          allMapEvents.addAll(recentMapEvents);
+        }
+        lastFetchTime = currTime;
+        if (fromEventId.get() - currFromEventId >= probe_sample_size) {
+          //return true when we have fetched the full payload, indicating
+          //that we should fetch again immediately (there might be more to
+          //fetch
+          fetchAgain = true;
+          return true;
+        }
       }
       fetchAgain = false;
       return false;
@@ -965,6 +988,39 @@ public class TaskTracker
         // next heartbeat   
         lastHeartbeat = System.currentTimeMillis();
         
+        
+        // Check if the map-event list needs purging
+        if (heartbeatResponse.getLastKnownIndex() != null) {
+          synchronized (this) {
+            // purge the local map events list
+            for (Map.Entry<JobID, Integer> entry 
+                 : heartbeatResponse.getLastKnownIndex().entrySet()) {
+              RunningJob rjob;
+              synchronized (runningJobs) {
+                rjob = runningJobs.get(entry.getKey());          
+                if (rjob != null) {
+                  synchronized (rjob) {
+                    FetchStatus f = rjob.getFetchStatus();
+                    if (f != null) {
+                      f.purgeMapEvents(entry.getValue());
+                    }
+                  }
+                }
+              }
+            }
+
+            // Mark the reducers in shuffle for rollback
+            synchronized (shouldReset) {
+              for (Map.Entry<TaskAttemptID, TaskInProgress> entry 
+                   : runningTasks.entrySet()) {
+                if (entry.getValue().getStatus().getPhase() == Phase.SHUFFLE) {
+                  this.shouldReset.add(entry.getKey());
+                }
+              }
+            }
+          }
+        }
+        
         TaskTrackerAction[] actions = heartbeatResponse.getActions();
         if(LOG.isDebugEnabled()) {
           LOG.debug("Got heartbeatResponse from JobTracker with responseId: " + 
@@ -2227,10 +2283,15 @@ public class TaskTracker
     purgeTask(tip, true);
   }
 
-  public TaskCompletionEvent[] getMapCompletionEvents(JobID jobId
-      , int fromEventId, int maxLocs) throws IOException {
-      
+  public synchronized MapTaskCompletionEventsUpdate getMapCompletionEvents(
+      JobID jobId, int fromEventId, int maxLocs, TaskAttemptID id) 
+  throws IOException {
     TaskCompletionEvent[]mapEvents = TaskCompletionEvent.EMPTY_ARRAY;
+    synchronized (shouldReset) {
+      if (shouldReset.remove(id)) {
+        return new MapTaskCompletionEventsUpdate(mapEvents, true);
+      }
+    }
     RunningJob rjob;
     synchronized (runningJobs) {
       rjob = runningJobs.get(jobId);          
@@ -2243,7 +2304,7 @@ public class TaskTracker
         }
       }
     }
-    return mapEvents;
+    return new MapTaskCompletionEventsUpdate(mapEvents, false);
   }
     
   /////////////////////////////////////////////////////

+ 14 - 3
src/mapred/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java

@@ -46,6 +46,9 @@ interface TaskUmbilicalProtocol extends VersionedProtocol {
    * Version 10 changed the TaskStatus format and added reportNextRecordRange
    *            for HADOOP-153
    * Version 11 Adds RPCs for task commit as part of HADOOP-3150
+   * Version 12 getMapCompletionEvents() now also indicates if the events are 
+   *            stale or not. Hence the return type is a class that 
+   *            encapsulates the events and whether to reset events index.
    * */
 
   public static final long versionID = 11L;
@@ -117,14 +120,22 @@ interface TaskUmbilicalProtocol extends VersionedProtocol {
   void fsError(TaskAttemptID taskId, String message) throws IOException;
 
   /** Called by a reduce task to get the map output locations for finished maps.
+   * Returns an update centered around the map-task-completion-events. 
+   * The update also piggybacks the information whether the events copy at the 
+   * task-tracker has changed or not. This will trigger some action at the 
+   * child-process.
    *
    * @param taskId the reduce task id
    * @param fromIndex the index starting from which the locations should be 
    * fetched
    * @param maxLocs the max number of locations to fetch
-   * @return an array of TaskCompletionEvent
+   * @param id The attempt id of the task that is trying to communicate
+   * @return A {@link MapTaskCompletionEventsUpdate} 
    */
-  TaskCompletionEvent[] getMapCompletionEvents(JobID jobId, 
-                                               int fromIndex, int maxLocs) throws IOException;
+  MapTaskCompletionEventsUpdate getMapCompletionEvents(JobID jobId, 
+                                                       int fromIndex, 
+                                                       int maxLocs,
+                                                       TaskAttemptID id) 
+  throws IOException;
 
 }

+ 134 - 24
src/test/org/apache/hadoop/mapred/MiniMRCluster.java

@@ -52,6 +52,7 @@ public class MiniMRCluster {
     
   private String namenode;
   private UnixUserGroupInformation ugi = null;
+  private JobConf conf;
     
   private JobConf job;
   
@@ -83,6 +84,10 @@ public class MiniMRCluster {
       return tracker;
     }
     
+    public TaskAttemptID getTaskAttemptId(TaskID taskId, int attemptId) {
+      return new TaskAttemptID(taskId, attemptId, tracker.getStartTime());
+    }
+
     /**
      * Create the job tracker and run it.
      */
@@ -296,6 +301,10 @@ public class MiniMRCluster {
     return createJobConf(new JobConf());
   }
 
+  public TaskAttemptID getTaskAttemptId(TaskID taskId, int attemptId) {
+    return this.jobTracker.getTaskAttemptId(taskId, attemptId);
+  }
+
   public JobConf createJobConf(JobConf conf) {
     if(conf == null) {
       conf = new JobConf();
@@ -430,23 +439,10 @@ public class MiniMRCluster {
     this.numTaskTrackers = numTaskTrackers;
     this.namenode = namenode;
     this.ugi = ugi;
-    
-    // Create the JobTracker
-    jobTracker = new JobTrackerRunner(conf);
-    jobTrackerThread = new Thread(jobTracker);
-        
-    jobTrackerThread.start();
-    while (!jobTracker.isUp()) {
-      try {                                     // let daemons get started
-        LOG.info("Waiting for JobTracker to start...");
-        Thread.sleep(1000);
-      } catch(InterruptedException e) {
-      }
-    }
-        
-    // Set the configuration for the task-trackers
-    this.jobTrackerPort = jobTracker.getJobTrackerPort();
-    this.jobTrackerInfoPort = jobTracker.getJobTrackerInfoPort();
+    this.conf = conf; // this is the conf the mr starts with
+
+    // start the jobtracker
+    startJobTracker();
 
     // Create the TaskTrackers
     for (int idx = 0; idx < numTaskTrackers; idx++) {
@@ -475,6 +471,126 @@ public class MiniMRCluster {
     waitUntilIdle();
   }
     
+  /**
+   * Get the map task completion events
+   */
+  public TaskCompletionEvent[] getMapTaskCompletionEvents(JobID id, int from, 
+                                                          int max) 
+  throws IOException {
+    return jobTracker.getJobTracker().getTaskCompletionEvents(id, from, max);
+  }
+
+  /**
+   * Change the job's priority
+   */
+  public void setJobPriority(JobID jobId, JobPriority priority) {
+    jobTracker.getJobTracker().setJobPriority(jobId, priority);
+  }
+
+  /**
+   * Get the job's priority
+   */
+  public JobPriority getJobPriority(JobID jobId) {
+    return jobTracker.getJobTracker().getJob(jobId).getPriority();
+  }
+
+  /**
+   * Get the job finish time
+   */
+  public long getJobFinishTime(JobID jobId) {
+    return jobTracker.getJobTracker().getJob(jobId).getFinishTime();
+  }
+
+  /**
+   * Init the job
+   */
+  public void initializeJob(JobID jobId) throws IOException {
+    JobInProgress job = jobTracker.getJobTracker().getJob(jobId);
+    job.initTasks();
+  }
+  
+  /**
+   * Get the events list at the tasktracker
+   */
+  public MapTaskCompletionEventsUpdate 
+         getMapTaskCompletionEventsUpdates(int index, JobID jobId, int max) 
+  throws IOException {
+    String jtId = jobTracker.getJobTracker().getTrackerIdentifier();
+    long jtStart = jobTracker.getJobTracker().getStartTime();
+    TaskAttemptID dummy = 
+      new TaskAttemptID(jtId, jobId.getId(), false, 0, 0, jtStart);
+    return taskTrackerList.get(index).getTaskTracker()
+                                     .getMapCompletionEvents(jobId, 0, max, 
+                                                             dummy);
+  }
+  
+  /**
+   * Get jobtracker conf
+   */
+  public JobConf getJobTrackerConf() {
+    return this.conf;
+  }
+  
+  /**
+   * Get num events recovered
+   */
+  public int getNumEventsRecovered() {
+    return jobTracker.getJobTracker().recoveryManager.totalEventsRecovered();
+  }
+  
+  /**
+   * Start the jobtracker.
+   */
+  public void startJobTracker() {
+    //  Create the JobTracker
+    jobTracker = new JobTrackerRunner(conf);
+    jobTrackerThread = new Thread(jobTracker);
+        
+    jobTrackerThread.start();
+    while (!jobTracker.isUp()) {
+      try {                                     // let daemons get started
+        Thread.sleep(1000);
+      } catch(InterruptedException e) {
+      }
+    }
+        
+    // Set the configuration for the task-trackers
+    this.jobTrackerPort = jobTracker.getJobTrackerPort();
+    this.jobTrackerInfoPort = jobTracker.getJobTrackerInfoPort();
+  }
+
+  /**
+   * Kill the jobtracker.
+   */
+  public void stopJobTracker() {
+    //jobTracker.exit(-1);
+    jobTracker.shutdown();
+
+    jobTrackerThread.interrupt();
+    try {
+      jobTrackerThread.join();
+    } catch (InterruptedException ex) {
+      LOG.error("Problem waiting for job tracker to finish", ex);
+    }
+  }
+
+  /**
+   * Kill the tasktracker.
+   */
+  public void stopTaskTracker(int id) {
+    taskTrackerList.get(id).shutdown();
+
+    taskTrackerThreadList.get(id).interrupt();
+    
+    try {
+      taskTrackerThreadList.get(id).join();
+      // This will break the wait until idle loop
+      taskTrackerList.get(id).isDead = true;
+    } catch (InterruptedException ex) {
+      LOG.error("Problem waiting for task tracker to finish", ex);
+    }
+  }
+  
   /**
    * Shut down the servers.
    */
@@ -492,13 +608,7 @@ public class MiniMRCluster {
           LOG.error("Problem shutting down task tracker", ex);
         }
       }
-      jobTracker.shutdown();
-      jobTrackerThread.interrupt();
-      try {
-        jobTrackerThread.join();
-      } catch (InterruptedException ex) {
-        LOG.error("Problem waiting for job tracker to finish", ex);
-      }
+      stopJobTracker();
     } finally {
       File configDir = new File("build", "minimr");
       File siteFile = new File(configDir, "hadoop-site.xml");

+ 1 - 48
src/test/org/apache/hadoop/mapred/TestCounters.java

@@ -21,9 +21,6 @@ import junit.framework.TestCase;
 import java.io.IOException;
 import java.text.ParseException;
 
-import org.apache.hadoop.mapred.Counters.Counter;
-import org.apache.hadoop.mapred.Counters.Group;
-
 /**
  * TestCounters checks the sanity and recoverability of {@code Counters}
  */
@@ -65,54 +62,10 @@ public class TestCounters extends TestCase {
       Counters.fromEscapedCompactString(compactEscapedString);
     // Check for recovery from string
     assertTrue("Recovered counter does not match on content", 
-               compareCounters(counter, recoveredCounter));
+               counter.contentEquals(recoveredCounter));
     
   }
   
-  // Checks for (content) equality of two Counter
-  private boolean compareCounter(Counter c1, Counter c2) {
-    return c1.getName().equals(c2.getName())
-           && c1.getDisplayName().equals(c2.getDisplayName())
-           && c1.getCounter() == c2.getCounter();
-  }
-  
-  // Checks for (content) equality of Groups
-  private boolean compareGroup(Group g1, Group g2) {
-    boolean isEqual = false;
-    if (g1 != null && g2 != null) {
-      if (g1.size() == g2.size()) {
-        isEqual = true;
-        for (String cname : g1.getCounterNames()) {
-          Counter c1 = g1.getCounterForName(cname);
-          Counter c2 = g2.getCounterForName(cname);
-          if (!compareCounter(c1, c2)) {
-            isEqual = false;
-            break;
-          }
-        }
-      }
-    }
-    return isEqual;
-  }
-  
-  // Checks for (content) equality of Counters
-  private boolean compareCounters(Counters c1, Counters c2) {
-    boolean isEqual = false;
-    if (c1 != null && c2 != null) {
-      if (c1.size() == c2.size()) {
-        isEqual = true;
-        for (Group g1 : c1) {
-          Group g2 = c2.getGroup(g1.getName());
-          if (!compareGroup(g1, g2)) {
-            isEqual = false;
-            break;
-          }
-        }
-      }
-    }
-    return isEqual;
-  }
-  
   public void testCounters() throws IOException {
     Enum[] keysWithResource = {Task.FileSystemCounter.HDFS_READ, 
                                Task.Counter.MAP_INPUT_BYTES, 

+ 832 - 0
src/test/org/apache/hadoop/mapred/TestJobTrackerRestart.java

@@ -0,0 +1,832 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat;
+import org.apache.hadoop.mapred.ThreadedMapBenchmark.RandomInputFormat;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
+
+import junit.framework.TestCase;
+import java.io.*;
+import java.util.Iterator;
+
+/** 
+ * TestJobTrackerRestart checks if the jobtracker can restart. JobTracker 
+ * should be able to continue running the previously running jobs and also 
+ * recover previosuly submitted jobs.
+ */
+public class TestJobTrackerRestart extends TestCase {
+  final static Object waitLock = new Object();
+  final Path testDir = new Path("/jt-restart-testing");
+  final Path inDir = new Path(testDir, "input");
+  final Path shareDir = new Path(testDir, "share");
+  final Path outputDir = new Path(testDir, "output");
+  private static int numJobsSubmitted = 0;
+  
+  /**
+   * Gets job status from the jobtracker given the jobclient and the job id
+   */
+  static JobStatus getJobStatus(JobClient jc, JobID id) throws IOException {
+    JobStatus[] statuses = jc.getAllJobs();
+    for (JobStatus jobStatus : statuses) {
+      if (jobStatus.getJobID().equals(id)) {
+        return jobStatus;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Return the job conf configured with the priorities and mappers as passed.
+   * @param conf The default conf
+   * @param priorities priorities for the jobs
+   * @param numMaps number of maps for the jobs
+   * @param numReds number of reducers for the jobs
+   * @param outputDir output dir
+   * @param inDir input dir
+   * @param mapSignalFile filename thats acts as a signal for maps
+   * @param reduceSignalFile filename thats acts as a signal for reducers
+   * @return a array of jobconfs configured as needed
+   * @throws IOException
+   */
+  static JobConf[] getJobs(JobConf conf, JobPriority[] priorities, 
+                           int[] numMaps, int[] numReds,
+                           Path outputDir, Path inDir,
+                           String mapSignalFile, String reduceSignalFile) 
+  throws IOException {
+    JobConf[] jobs = new JobConf[priorities.length];
+    for (int i = 0; i < jobs.length; ++i) {
+      jobs[i] = new JobConf(conf);
+      Path newOutputDir = outputDir.suffix(String.valueOf(numJobsSubmitted++));
+      configureWaitingJobConf(jobs[i], inDir, newOutputDir, 
+                              numMaps[i], numReds[i], "jt-restart-test-job", 
+                              mapSignalFile, reduceSignalFile);
+      jobs[i].setJobPriority(priorities[i]);
+    }
+    return jobs;
+  }
+
+  /**
+   * A utility that waits for specified amount of time
+   */
+  static void waitFor(long duration) {
+    try {
+      synchronized (waitLock) {
+        waitLock.wait(duration);
+      }
+    } catch (InterruptedException ie) {}
+  }
+  
+  /**
+   * Wait for the jobtracker to be RUNNING.
+   */
+  static void waitForJobTracker(JobClient jobClient) {
+    while (true) {
+      try {
+        ClusterStatus status = jobClient.getClusterStatus();
+        while (status.getJobTrackerState() != JobTracker.State.RUNNING) {
+          waitFor(100);
+          status = jobClient.getClusterStatus();
+        }
+        break; // means that the jt is ready
+      } catch (IOException ioe) {}
+    }
+  }
+  
+  /**
+   * Signal the maps/reduces to start.
+   */
+  static void signalTasks(MiniDFSCluster dfs, FileSystem fileSys, 
+                          boolean isMap, String mapSignalFile, 
+                          String reduceSignalFile)
+  throws IOException {
+    //  signal the maps to complete
+    TestRackAwareTaskPlacement.writeFile(dfs.getNameNode(), fileSys.getConf(),
+                                         isMap 
+                                         ? new Path(mapSignalFile)
+                                         : new Path(reduceSignalFile), 
+                                         (short)1);
+  }
+  
+  /**
+   * Waits until all the jobs at the jobtracker complete.
+   */
+  static void waitTillDone(JobClient jobClient) throws IOException {
+    // Wait for the last job to complete
+    while (true) {
+      boolean shouldWait = false;
+      for (JobStatus jobStatuses : jobClient.getAllJobs()) {
+        if (jobStatuses.getRunState() == JobStatus.RUNNING) {
+          shouldWait = true;
+          break;
+        }
+      }
+      if (shouldWait) {
+        waitFor(1000);
+      } else {
+        break;
+      }
+    }
+  }
+  
+  /**
+   * Clean up the signals.
+   */
+  static void cleanUp(FileSystem fileSys, Path dir) throws IOException {
+    // Delete the map signal file
+    fileSys.delete(new Path(getMapSignalFile(dir)), false);
+    // Delete the reduce signal file
+    fileSys.delete(new Path(getReduceSignalFile(dir)), false);
+  }
+  
+ /**
+   * Tests multiple jobs on jobtracker with restart-recovery turned on.
+   * Preparation :
+   *    - Configure 3 jobs as follows [format {prio, maps, reducers}]
+   *       - job1 : {normal, 50, 1}
+   *       - job2 : {low, 1, 1}
+   *       - job3 : {high, 1, 1}
+   *    - Configure the cluster to run 1 reducer
+   *    - Lower the history file block size and buffer
+   *    
+   * Submit these 3 jobs but make sure that job1's priority is changed and job1
+   * is RUNNING before submitting other jobs
+   * The order in which the jobs will be executed will be job1, job3 and job2.
+   * 
+   * Above ordering makes sure that job1 runs before everyone else.
+   * Wait for job1 to complete 50%. Note that all the jobs are configured to 
+   * use {@link HalfWaitingMapper} and {@link WaitingReducer}. So job1 will 
+   * eventually wait on 50%
+   * 
+   * Make a note of the following things
+   *    - Job start times
+   *    
+   * Restart the jobtracker
+   * 
+   * Wait for job1 to finish all the maps and note the TaskCompletion events at
+   * the tracker.
+   * 
+   * Wait for all the jobs to finish
+   * 
+   * Also make sure that the order in which the jobs were sorted before restart
+   * remains same. For this check the follwoing
+   *   job1.start-time < job2.start-time < job3.start-time and 
+   *   job1.finish-time < job3.finish-time < job2.finish-time
+   * This ordering makes sure that the change of priority is logged and 
+   * recovered back
+   */
+  public void testRecoveryWithMultipleJobs(MiniDFSCluster dfs,
+                                           MiniMRCluster mr) 
+  throws IOException {
+    FileSystem fileSys = dfs.getFileSystem();
+    JobConf jobConf = mr.createJobConf();
+    JobPriority[] priorities = {JobPriority.NORMAL, JobPriority.LOW, 
+                                JobPriority.HIGH};
+    // Note that there is only 1 tracker
+    int[] numMaps = {50, 1, 1};
+    int[] numReds = {1, 1, 1};
+
+    cleanUp(fileSys, shareDir);
+    
+    // Configure the jobs
+    JobConf[] jobs = getJobs(jobConf, priorities, numMaps, numReds,
+                             outputDir, inDir, 
+                             getMapSignalFile(shareDir), 
+                             getReduceSignalFile(shareDir));
+
+    // Master job parameters
+    int masterJob = 0;
+    JobPriority masterJobNewPriority = JobPriority.HIGH;
+
+    // Submit a master job   
+    JobClient jobClient = new JobClient(jobs[masterJob]);
+    RunningJob job = jobClient.submitJob(jobs[masterJob]);
+    JobID id = job.getID();
+
+    // Wait for the job to be inited
+    mr.initializeJob(id);
+
+    // Change the master job's priority so that priority logging is tested
+    mr.setJobPriority(id, masterJobNewPriority);
+
+    // Submit the remaining jobs and find the last job id
+    for (int i = 1; i < jobs.length; ++i) {
+      RunningJob rJob = (new JobClient(jobs[i])).submitJob(jobs[i]);
+      mr.initializeJob(rJob.getID());
+    }
+
+    // Make sure that the master job is 50% completed
+    while (getJobStatus(jobClient, id).mapProgress() < 0.5f) {
+      waitFor(100);
+    }
+
+    // Note the data that needs to be tested upon restart
+    long jobStartTime = getJobStatus(jobClient, id).getStartTime();
+
+    // Kill the jobtracker
+    mr.stopJobTracker();
+
+    // Signal the maps to complete
+    signalTasks(dfs, fileSys, true, getMapSignalFile(shareDir), 
+                getReduceSignalFile(shareDir));
+
+    // Signal the reducers to complete
+    signalTasks(dfs, fileSys, false, getMapSignalFile(shareDir), 
+                getReduceSignalFile(shareDir));
+    
+    // Enable recovery on restart
+    mr.getJobTrackerConf().setBoolean("mapred.jobtracker.restart.recover", 
+                                      true);
+
+    //  Wait for a minute before submitting a job
+    waitFor(60 * 1000);
+    
+    // Restart the jobtracker
+    mr.startJobTracker();
+
+    // Check if the jobs are still running
+
+    // Wait for the JT to be ready
+    waitForJobTracker(jobClient);
+
+    // Check if the job recovered
+    assertEquals("Restart failed as previously submitted job was missing", 
+                 true, getJobStatus(jobClient, id) != null);
+
+    // check if the job's priority got changed
+    assertEquals("Restart failed as job's priority did not match", 
+                 true, mr.getJobPriority(id).equals(masterJobNewPriority));
+
+    
+
+    waitTillDone(jobClient);
+
+    // Check if the jobs are in order .. the order is 1->3->2
+    JobStatus[] newStatuses = jobClient.getAllJobs();
+    // Check if the jobs are in the order of submission
+    //   This is important for the following checks
+    boolean jobOrder = newStatuses[0].getJobID().getId() == 1
+                       && newStatuses[1].getJobID().getId() == 2
+                       && newStatuses[2].getJobID().getId() == 3;
+    assertTrue("Job submission order changed", jobOrder);
+    
+    // Start times are in order and non zero
+    boolean startTimeOrder = newStatuses[0].getStartTime() > 0
+                             && newStatuses[0].getStartTime() 
+                                < newStatuses[1].getStartTime()
+                             && newStatuses[1].getStartTime() 
+                                < newStatuses[2].getStartTime();
+    assertTrue("Job start-times are out of order", startTimeOrder);
+    
+    boolean finishTimeOrder = 
+      mr.getJobFinishTime(newStatuses[0].getJobID()) > 0
+      && mr.getJobFinishTime(newStatuses[0].getJobID()) 
+         < mr.getJobFinishTime(newStatuses[2].getJobID())
+      && mr.getJobFinishTime(newStatuses[2].getJobID()) 
+         < mr.getJobFinishTime(newStatuses[1].getJobID());
+    assertTrue("Jobs finish-times are out of order", finishTimeOrder);
+            
+    
+    // This should be used for testing job counters
+    job.getCounters();
+
+    // check if the job was successful
+    assertTrue("Previously submitted job was not successful", 
+               job.isSuccessful());
+
+    // Check if the start time was recovered
+    assertTrue("Previously submitted job's start time has changed", 
+               getJobStatus(jobClient, id).getStartTime() == jobStartTime);
+
+    // Test history files
+    testJobHistoryFiles(id, jobs[masterJob]);
+  }
+  
+  /**
+   * Tests the jobtracker with restart-recovery turned off.
+   * Submit a job with normal priority, maps = 2, reducers = 0}
+   * 
+   * Wait for the job to complete 50%
+   * 
+   * Restart the jobtracker with recovery turned off
+   * 
+   * Check if the job is missing
+   */
+  public void testRestartWithoutRecovery(MiniDFSCluster dfs, 
+                                         MiniMRCluster mr) 
+  throws IOException {
+    // III. Test a job with waiting mapper and recovery turned off
+    
+    FileSystem fileSys = dfs.getFileSystem();
+    
+    cleanUp(fileSys, shareDir);
+    
+    JobConf newConf = getJobs(mr.createJobConf(), 
+                              new JobPriority[] {JobPriority.NORMAL}, 
+                              new int[] {2}, new int[] {0},
+                              outputDir, inDir, 
+                              getMapSignalFile(shareDir), 
+                              getReduceSignalFile(shareDir))[0];
+    
+    JobClient jobClient = new JobClient(newConf);
+    RunningJob job = jobClient.submitJob(newConf);
+    JobID id = job.getID();
+    
+    //  make sure that the job is 50% completed
+    while (getJobStatus(jobClient, id).mapProgress() < 0.5f) {
+      waitFor(100);
+    }
+    
+    mr.stopJobTracker();
+    
+    // Turn off the recovery
+    mr.getJobTrackerConf().setBoolean("mapred.jobtracker.restart.recover", 
+                                      false);
+    
+    // Wait for a minute before submitting a job
+    waitFor(60 * 1000);
+    
+    mr.startJobTracker();
+    
+    // Signal the tasks
+    signalTasks(dfs, fileSys, true, getMapSignalFile(shareDir), 
+                getReduceSignalFile(shareDir));
+    
+    // Wait for the JT to be ready
+    waitForJobTracker(jobClient);
+    
+    waitTillDone(jobClient);
+    
+    // The submitted job should not exist
+    assertTrue("Submitted job was detected with recovery disabled", 
+               getJobStatus(jobClient, id) == null);
+  }
+
+  /** Tests a job on jobtracker with restart-recovery turned on.
+   * Preparation :
+   *    - Configure a job with
+   *       - num-maps : 50
+   *       - num-reducers : 1
+   *    - Configure the cluster to run 1 reducer
+   *    - Lower the history file block size and buffer
+   *    
+   * Wait for the job to complete 50%. Note that all the job is configured to 
+   * use {@link HalfWaitingMapper} and {@link WaitingReducer}. So job will 
+   * eventually wait on 50%
+   * 
+   * Make a note of the following things
+   *    - Task completion events
+   *    - Cluster status
+   *    - Task Reports
+   *    - Job start time
+   *    
+   * Restart the jobtracker
+   * 
+   * Wait for job to finish all the maps and note the TaskCompletion events at
+   * the tracker.
+   * 
+   * Wait for all the jobs to finish and note the following
+   *    - New task completion events at the jobtracker
+   *    - Task reports
+   *    - Cluster status
+   * 
+   * Check for the following
+   *    - Task completion events for recovered tasks should match 
+   *    - Task completion events at the tasktracker and the restarted 
+   *      jobtracker should be same
+   *    - Cluster status should be fine.
+   *    - Task Reports for recovered tasks should match
+   *      Checks
+   *        - start time
+   *        - finish time
+   *        - counters
+   *        - http-location
+   *        - task-id
+   *    - Job start time should match
+   *    - Check if the counters can be accessed
+   *    - Check if the history files are (re)named properly
+   */
+  public void testTaskEventsAndReportsWithRecovery(MiniDFSCluster dfs, 
+                                                   MiniMRCluster mr) 
+  throws IOException {
+    // II. Test a tasktracker with waiting mapper and recovery turned on.
+    //     Ideally the tracker should SYNC with the new/restarted jobtracker
+    
+    FileSystem fileSys = dfs.getFileSystem();
+    final int numMaps = 50;
+    final int numReducers = 1;
+    
+    
+    cleanUp(fileSys, shareDir);
+    
+    JobConf newConf = getJobs(mr.createJobConf(), 
+                              new JobPriority[] {JobPriority.NORMAL}, 
+                              new int[] {numMaps}, new int[] {numReducers},
+                              outputDir, inDir, 
+                              getMapSignalFile(shareDir), 
+                              getReduceSignalFile(shareDir))[0];
+    
+    JobClient jobClient = new JobClient(newConf);
+    RunningJob job = jobClient.submitJob(newConf);
+    JobID id = job.getID();
+    
+    mr.initializeJob(id);
+    
+    //  make sure that atleast on reducer is spawned
+    while (jobClient.getClusterStatus().getReduceTasks() == 0) {
+      waitFor(100);
+    }
+    
+    while(true) {
+      // Since we are using a half waiting mapper, maps should be stuck at 50%
+      TaskCompletionEvent[] trackerEvents = 
+        mr.getMapTaskCompletionEventsUpdates(0, id, numMaps)
+          .getMapTaskCompletionEvents();
+      if (trackerEvents.length < numMaps / 2) {
+        waitFor(1000);
+      } else {
+        break;
+      }
+    }
+    
+    TaskCompletionEvent[] prevEvents = 
+      mr.getMapTaskCompletionEvents(id, 0, numMaps);
+    TaskReport[] prevReports = jobClient.getMapTaskReports(id);
+    ClusterStatus prevStatus = jobClient.getClusterStatus();
+    
+    mr.stopJobTracker();
+    
+    // Turn off the recovery
+    mr.getJobTrackerConf().setBoolean("mapred.jobtracker.restart.recover", 
+                                      true);
+    
+    //  Wait for a minute before submitting a job
+    waitFor(60 * 1000);
+    
+    mr.startJobTracker();
+    
+    // Signal the map tasks
+    signalTasks(dfs, fileSys, true, getMapSignalFile(shareDir), 
+                getReduceSignalFile(shareDir));
+    
+    // Wait for the JT to be ready
+    waitForJobTracker(jobClient);
+    
+    int numToMatch = mr.getNumEventsRecovered() / 2;
+    
+    //  make sure that the maps are completed
+    while (getJobStatus(jobClient, id).mapProgress() < 1.0f) {
+      waitFor(100);
+    }
+    
+    // Get the new jobtrackers events
+    TaskCompletionEvent[] jtEvents =  
+      mr.getMapTaskCompletionEvents(id, 0, 2 * numMaps);
+    
+    // Test if all the events that were recovered match exactly
+    testTaskCompletionEvents(prevEvents, jtEvents, false, numToMatch);
+    
+    TaskCompletionEvent[] trackerEvents;
+    while(true) {
+      trackerEvents = 
+        mr.getMapTaskCompletionEventsUpdates(0, id, 2 * numMaps)
+          .getMapTaskCompletionEvents();
+      if (trackerEvents.length < jtEvents.length) {
+        waitFor(1000);
+      } else {
+        break;
+      }
+    }
+    
+    // Check the task reports
+    // The reports should match exactly if the attempts are same
+    TaskReport[] afterReports = jobClient.getMapTaskReports(id);
+    testTaskReports(prevReports, afterReports, numToMatch);
+    
+    //  Signal the reduce tasks
+    signalTasks(dfs, fileSys, false, getMapSignalFile(shareDir), 
+                getReduceSignalFile(shareDir));
+    
+    waitTillDone(jobClient);
+    
+    testTaskCompletionEvents(jtEvents, trackerEvents, true, 2 * numMaps);
+    
+    // check if the cluster status is insane
+    ClusterStatus status = jobClient.getClusterStatus();
+    assertTrue("Cluster status is insane", 
+               checkClusterStatusOnCompletion(status, prevStatus));
+  }
+  
+  /**
+   * Checks if the history files are as expected
+   * @param id job id
+   * @param conf job conf
+   */
+  private void testJobHistoryFiles(JobID id, JobConf conf) 
+  throws IOException  {
+    // Get the history files for users
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+    String tempLogFileName = 
+      JobHistory.JobInfo.getSecondaryJobHistoryFile(logFileName);
+    
+    // I. User files
+    Path logFile = 
+      JobHistory.JobInfo.getJobHistoryLogLocationForUser(logFileName, conf);
+    FileSystem fileSys = logFile.getFileSystem(conf);
+    
+    // Check if the history file exists
+    assertTrue("User log file does not exist", fileSys.exists(logFile));
+    
+    // Check if the temporary file is deleted
+    Path tempLogFile = 
+      JobHistory.JobInfo.getJobHistoryLogLocationForUser(tempLogFileName, 
+                                                         conf);
+    assertFalse("User temporary log file exists", fileSys.exists(tempLogFile));
+    
+    // II. Framework files
+    // Get the history file
+    logFile = JobHistory.JobInfo.getJobHistoryLogLocation(logFileName);
+    fileSys = logFile.getFileSystem(conf);
+    
+    // Check if the history file exists
+    assertTrue("Log file does not exist", fileSys.exists(logFile));
+    
+    // Check if the temporary file is deleted
+    tempLogFile = JobHistory.JobInfo.getJobHistoryLogLocation(tempLogFileName);
+    assertFalse("Temporary log file exists", fileSys.exists(tempLogFile));
+  }
+  
+  /**
+   * Matches specified number of task reports.
+   * @param source the reports to be matched
+   * @param target reports to match with
+   * @param numToMatch num reports to match
+   * @param mismatchSet reports that should not match
+   */
+  private void testTaskReports(TaskReport[] source, TaskReport[] target, 
+                               int numToMatch) {
+    for (int i = 0; i < numToMatch; ++i) {
+      // Check if the task reports was recovered correctly
+      assertTrue("Task reports for same attempt has changed", 
+                 source[i].equals(target[i]));
+    }
+  }
+  
+  /**
+   * Matches the task completion events.
+   * @param source the events to be matched
+   * @param target events to match with
+   * @param fullMatch whether to match the events completely or partially
+   * @param numToMatch number of events to match in case full match is not 
+   *        desired
+   * @param ignoreSet a set of taskids to ignore
+   */
+  private void testTaskCompletionEvents(TaskCompletionEvent[] source, 
+                                       TaskCompletionEvent[] target, 
+                                       boolean fullMatch,
+                                       int numToMatch) {
+    //  Check if the event list size matches
+    // The lengths should match only incase of full match
+    if (fullMatch) {
+      assertEquals("Map task completion events mismatch", 
+                   source.length, target.length);
+      numToMatch = source.length;
+    }
+    // Check if the events match
+    for (int i = 0; i < numToMatch; ++i) {
+      if (source[i].getTaskAttemptId().equals(target[i].getTaskAttemptId())){
+        assertTrue("Map task completion events ordering mismatch", 
+                   source[i].equals(target[i]));
+      }
+    }
+  }
+  
+  private boolean checkClusterStatusOnCompletion(ClusterStatus status, 
+                                                 ClusterStatus prevStatus) {
+    return status.getJobTrackerState() == prevStatus.getJobTrackerState()
+           && status.getMapTasks() == 0
+           && status.getReduceTasks() == 0;
+  }
+  
+  public void testJobTrackerRestart() throws IOException {
+    String namenode = null;
+    MiniDFSCluster dfs = null;
+    MiniMRCluster mr = null;
+    FileSystem fileSys = null;
+
+    try {
+      Configuration conf = new Configuration();
+      conf.setBoolean("dfs.replication.considerLoad", false);
+      dfs = new MiniDFSCluster(conf, 1, true, null, null);
+      dfs.waitActive();
+      fileSys = dfs.getFileSystem();
+      
+      // clean up
+      fileSys.delete(testDir, true);
+      
+      if (!fileSys.mkdirs(inDir)) {
+        throw new IOException("Mkdirs failed to create " + inDir.toString());
+      }
+
+      // Write the input file
+      TestRackAwareTaskPlacement.writeFile(dfs.getNameNode(), conf, 
+                                           new Path(inDir + "/file"), 
+                                           (short)1);
+
+      dfs.startDataNodes(conf, 1, true, null, null, null, null);
+      dfs.waitActive();
+
+      namenode = (dfs.getFileSystem()).getUri().getHost() + ":" 
+                 + (dfs.getFileSystem()).getUri().getPort();
+
+      // Make sure that jobhistory leads to a proper job restart
+      // So keep the blocksize and the buffer size small
+      JobConf jtConf = new JobConf();
+      jtConf.set("mapred.jobtracker.job.history.block.size", "1024");
+      jtConf.set("mapred.jobtracker.job.history.buffer.size", "1024");
+      jtConf.setInt("mapred.tasktracker.reduce.tasks.maximum", 1);
+      jtConf.setLong("mapred.tasktracker.expiry.interval", 25 * 1000);
+      
+      mr = new MiniMRCluster(1, namenode, 1, null, null, jtConf);
+      
+      // Test multiple jobs on jobtracker with restart-recovery turned on
+      testRecoveryWithMultipleJobs(dfs, mr);
+      
+      // Test the tasktracker SYNC
+      testTaskEventsAndReportsWithRecovery(dfs, mr);
+      
+      // Test jobtracker with restart-recovery turned off
+      testRestartWithoutRecovery(dfs, mr);
+    } finally {
+      if (mr != null) {
+        try {
+          mr.shutdown();
+        } catch (Exception e) {}
+      }
+      if (dfs != null) {
+        try {
+          dfs.shutdown();
+        } catch (Exception e) {}
+      }
+    }
+  }
+
+  static String getMapSignalFile(Path dir) {
+    return dir.suffix("/jt-restart-map-signal").toString();
+  }
+
+  static String getReduceSignalFile(Path dir) {
+    return dir.suffix("/jt-restart-reduce-signal").toString();
+  }
+  
+  /** 
+   * Map is a Mapper that just waits for a file to be created on the dfs. The 
+   * file creation is a signal to the mappers and hence acts as a waiting job. 
+   * Only the later half of the maps wait for the signal while the rest 
+   * complete immediately.
+   */
+
+  static class HalfWaitingMapper 
+  extends MapReduceBase 
+  implements Mapper<WritableComparable, Writable, 
+                    WritableComparable, Writable> {
+
+    FileSystem fs = null;
+    Path signal;
+    int id = 0;
+    int totalMaps = 0;
+
+    /** The waiting function.  The map exits once it gets a signal. Here the 
+     * signal is the file existence. 
+     */
+    public void map(WritableComparable key, Writable val, 
+                    OutputCollector<WritableComparable, Writable> output,
+                    Reporter reporter)
+    throws IOException {
+      if (id > totalMaps / 2) {
+        if (fs != null) {
+          while (!fs.exists(signal)) {
+            try {
+              reporter.progress();
+              synchronized (this) {
+                this.wait(1000); // wait for 1 sec
+              }
+            } catch (InterruptedException ie) {
+              System.out.println("Interrupted while the map was waiting for "
+                                 + " the signal.");
+              break;
+            }
+          }
+        } else {
+          throw new IOException("Could not get the DFS!!");
+        }
+      }
+    }
+
+    public void configure(JobConf conf) {
+      try {
+        String taskId = conf.get("mapred.task.id");
+        id = Integer.parseInt(taskId.split("_")[4]);
+        totalMaps = Integer.parseInt(conf.get("mapred.map.tasks"));
+        fs = FileSystem.get(conf);
+        signal = new Path(conf.get("test.mapred.map.waiting.target"));
+      } catch (IOException ioe) {
+        System.out.println("Got an exception while obtaining the filesystem");
+      }
+    }
+  }
+  
+  /** 
+   * Reduce that just waits for a file to be created on the dfs. The 
+   * file creation is a signal to the reduce.
+   */
+
+  static class WaitingReducer extends MapReduceBase 
+  implements Reducer<WritableComparable, Writable, 
+                     WritableComparable, Writable> {
+
+    FileSystem fs = null;
+    Path signal;
+    
+    /** The waiting function.  The reduce exits once it gets a signal. Here the
+     * signal is the file existence. 
+     */
+    public void reduce(WritableComparable key, Iterator<Writable> val, 
+                       OutputCollector<WritableComparable, Writable> output,
+                       Reporter reporter)
+    throws IOException {
+      if (fs != null) {
+        while (!fs.exists(signal)) {
+          try {
+            reporter.progress();
+            synchronized (this) {
+              this.wait(1000); // wait for 1 sec
+            }
+          } catch (InterruptedException ie) {
+            System.out.println("Interrupted while the map was waiting for the"
+                               + " signal.");
+            break;
+          }
+        }
+      } else {
+        throw new IOException("Could not get the DFS!!");
+      }
+    }
+
+    public void configure(JobConf conf) {
+      try {
+        fs = FileSystem.get(conf);
+        signal = new Path(conf.get("test.mapred.reduce.waiting.target"));
+      } catch (IOException ioe) {
+        System.out.println("Got an exception while obtaining the filesystem");
+      }
+    }
+  }
+  
+  static void configureWaitingJobConf(JobConf jobConf, Path inDir,
+                                      Path outputPath, int numMaps, int numRed,
+                                      String jobName, String mapSignalFilename,
+                                      String redSignalFilename)
+  throws IOException {
+    jobConf.setJobName(jobName);
+    jobConf.setInputFormat(NonSplitableSequenceFileInputFormat.class);
+    jobConf.setOutputFormat(SequenceFileOutputFormat.class);
+    FileInputFormat.setInputPaths(jobConf, inDir);
+    FileOutputFormat.setOutputPath(jobConf, outputPath);
+    jobConf.setMapperClass(HalfWaitingMapper.class);
+    jobConf.setReducerClass(IdentityReducer.class);
+    jobConf.setOutputKeyClass(BytesWritable.class);
+    jobConf.setOutputValueClass(BytesWritable.class);
+    jobConf.setInputFormat(RandomInputFormat.class);
+    jobConf.setNumMapTasks(numMaps);
+    jobConf.setNumReduceTasks(numRed);
+    jobConf.setJar("build/test/testjar/testjob.jar");
+    jobConf.set("test.mapred.map.waiting.target", mapSignalFilename);
+    jobConf.set("test.mapred.reduce.waiting.target", redSignalFilename);
+  }
+
+  public static void main(String[] args) throws IOException {
+    new TestJobTrackerRestart().testJobTrackerRestart();
+  }
+}

+ 176 - 0
src/test/org/apache/hadoop/mapred/TestJobTrackerRestartWithLostTracker.java

@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.TestJobTrackerRestart;
+
+import junit.framework.TestCase;
+import java.io.*;
+
+/** 
+ * This test checks if the jobtracker can detect and recover a tracker that was
+ * lost while the jobtracker was down.
+ */
+public class TestJobTrackerRestartWithLostTracker extends TestCase {
+  final Path testDir = new Path("/jt-restart-lost-tt-testing");
+  final Path inDir = new Path(testDir, "input");
+  final Path shareDir = new Path(testDir, "share");
+  final Path outputDir = new Path(testDir, "output");
+  
+  private JobConf configureJob(JobConf conf, int[] maps, int[] reduces,
+                               String mapSignal, String redSignal) 
+  throws IOException {
+    JobPriority[] priority = new JobPriority[] {JobPriority.NORMAL};
+    return TestJobTrackerRestart.getJobs(conf, priority, 
+                                         maps, reduces, outputDir, inDir, 
+                                         mapSignal, redSignal)[0];
+  }
+  
+  public void testRecoveryWithLostTracker(MiniDFSCluster dfs,
+                                          MiniMRCluster mr) 
+  throws IOException {
+    FileSystem fileSys = dfs.getFileSystem();
+    JobConf jobConf = mr.createJobConf();
+    int numMaps = 50;
+    int numReds = 1;
+    String mapSignalFile = TestJobTrackerRestart.getMapSignalFile(shareDir);
+    String redSignalFile = TestJobTrackerRestart.getReduceSignalFile(shareDir);
+    
+    // Configure the jobs
+    JobConf job = configureJob(jobConf, new int[] {numMaps}, 
+                               new int[] {numReds}, 
+                               mapSignalFile, redSignalFile);
+      
+    TestJobTrackerRestart.cleanUp(fileSys, shareDir);
+    
+    // Submit a master job   
+    JobClient jobClient = new JobClient(job);
+    RunningJob rJob = jobClient.submitJob(job);
+    JobID id = rJob.getID();
+    
+    // wait for the job to be inited
+    mr.initializeJob(id);
+    
+    // Make sure that the master job is 50% completed
+    while (TestJobTrackerRestart.getJobStatus(jobClient, id).mapProgress() 
+           < 0.5f) {
+      TestJobTrackerRestart.waitFor(100);
+    }
+
+    // Kill the jobtracker
+    mr.stopJobTracker();
+
+    // Signal the maps to complete
+    TestJobTrackerRestart.signalTasks(dfs, fileSys, true, 
+                                      mapSignalFile, redSignalFile);
+    
+    // Enable recovery on restart
+    mr.getJobTrackerConf().setBoolean("mapred.jobtracker.restart.recover", 
+                                      true);
+    
+    // Kill the 2nd tasktracker
+    mr.stopTaskTracker(1);
+    
+    // Wait for a minute before submitting a job
+    TestJobTrackerRestart.waitFor(60 * 1000);
+    
+    // Restart the jobtracker
+    mr.startJobTracker();
+
+    // Check if the jobs are still running
+    
+    // Wait for the JT to be ready
+    TestJobTrackerRestart.waitForJobTracker(jobClient);
+
+    // Signal the reducers to complete
+    TestJobTrackerRestart.signalTasks(dfs, fileSys, false, 
+                                      mapSignalFile, redSignalFile);
+    
+    TestJobTrackerRestart.waitTillDone(jobClient);
+
+    // Check if the tasks on the lost tracker got re-executed
+    assertTrue("Tracker killed while the jobtracker was down did not get lost "
+                + "upon restart", 
+                jobClient.getClusterStatus().getTaskTrackers() 
+                < mr.getNumTaskTrackers());
+  }
+  
+  public void testRestartWithLostTracker() throws IOException {
+    String namenode = null;
+    MiniDFSCluster dfs = null;
+    MiniMRCluster mr = null;
+    FileSystem fileSys = null;
+
+    try {
+      Configuration conf = new Configuration();
+      conf.setBoolean("dfs.replication.considerLoad", false);
+      dfs = new MiniDFSCluster(conf, 1, true, null, null);
+      dfs.waitActive();
+      fileSys = dfs.getFileSystem();
+      
+      // clean up
+      fileSys.delete(testDir, true);
+      
+      if (!fileSys.mkdirs(inDir)) {
+        throw new IOException("Mkdirs failed to create " + inDir.toString());
+      }
+
+      // Write the input file
+      TestRackAwareTaskPlacement.writeFile(dfs.getNameNode(), conf, 
+                                           new Path(inDir + "/file"), 
+                                           (short)1);
+
+      dfs.startDataNodes(conf, 1, true, null, null, null, null);
+      dfs.waitActive();
+
+      namenode = (dfs.getFileSystem()).getUri().getHost() + ":" 
+                 + (dfs.getFileSystem()).getUri().getPort();
+
+      // Make sure that jobhistory leads to a proper job restart
+      // So keep the blocksize and the buffer size small
+      JobConf jtConf = new JobConf();
+      jtConf.set("mapred.jobtracker.job.history.block.size", "1024");
+      jtConf.set("mapred.jobtracker.job.history.buffer.size", "1024");
+      jtConf.setInt("mapred.tasktracker.reduce.tasks.maximum", 1);
+      jtConf.setLong("mapred.tasktracker.expiry.interval", 25 * 1000);
+      
+      mr = new MiniMRCluster(2, namenode, 1, null, null, jtConf);
+      
+      // Test Lost tracker case
+      testRecoveryWithLostTracker(dfs, mr);
+    } finally {
+      if (mr != null) {
+        try {
+          mr.shutdown();
+        } catch (Exception e) {}
+      }
+      if (dfs != null) {
+        try {
+          dfs.shutdown();
+        } catch (Exception e) {}
+      }
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    new TestJobTrackerRestartWithLostTracker().testRestartWithLostTracker();
+  }
+}

+ 23 - 2
src/test/org/apache/hadoop/mapred/TestMRServerPorts.java

@@ -36,16 +36,33 @@ import org.apache.hadoop.fs.FileSystem;
 public class TestMRServerPorts extends TestCase {
   TestHDFSServerPorts hdfs = new TestHDFSServerPorts();
 
+  // Runs the JT in a separate thread
+  private static class JTRunner extends Thread {
+    JobTracker jt;
+    void setJobTracker(JobTracker jt) {
+      this.jt = jt;
+    }
+
+    public void run() {
+      if (jt != null) {
+        try {
+          jt.offerService();
+        } catch (Exception ioe) {}
+      }
+    }
+  }
   /**
    * Check whether the JobTracker can be started.
    */
-  private JobTracker startJobTracker(JobConf conf) 
+  private JobTracker startJobTracker(JobConf conf, JTRunner runner) 
   throws IOException {
     conf.set("mapred.job.tracker", "localhost:0");
     conf.set("mapred.job.tracker.http.address", "0.0.0.0:0");
     JobTracker jt = null;
     try {
       jt = JobTracker.startTracker(conf);
+      runner.setJobTracker(jt);
+      runner.start();
       conf.set("mapred.job.tracker", "localhost:" + jt.getTrackerPort());
       conf.set("mapred.job.tracker.http.address", 
                             "0.0.0.0:" + jt.getInfoPort());
@@ -132,11 +149,13 @@ public class TestMRServerPorts extends TestCase {
   public void testTaskTrackerPorts() throws Exception {
     NameNode nn = null;
     JobTracker jt = null;
+    JTRunner runner = null;
     try {
       nn = hdfs.startNameNode();
 
       JobConf conf2 = new JobConf(hdfs.getConfig());
-      jt = startJobTracker(conf2);
+      runner = new JTRunner();
+      jt = startJobTracker(conf2, runner);
 
       // start job tracker on the same port as name-node
       conf2.set("mapred.task.tracker.report.address",
@@ -165,6 +184,8 @@ public class TestMRServerPorts extends TestCase {
       if (jt != null) {
         jt.fs.close();
         jt.stopTracker();
+        runner.interrupt();
+        runner.join();
       }
       hdfs.stopNameNode(nn);
     }

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMiniMRMapRedDebugScript.java

@@ -163,7 +163,7 @@ public class TestMiniMRMapRedDebugScript extends TestCase {
 
     JobID jobId = job.getID();
     // construct the task id of first map task of failmap
-    TaskAttemptID taskId = new TaskAttemptID(new TaskID(jobId,true, 0), 0);
+    TaskAttemptID taskId = mr.getTaskAttemptId(new TaskID(jobId,true, 0), 0);
     // wait for the job to finish.
     while (!job.isComplete()) ;
     

+ 6 - 3
src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java

@@ -181,7 +181,9 @@ public class TestMiniMRWithDFS extends TestCase {
     LOG.info("runWordCount");
     // Run a word count example
     // Keeping tasks that match this pattern
-    jobConf.setKeepTaskFilesPattern(TaskAttemptID.getTaskAttemptIDsPattern(null, null, true, 1, null));
+    String pattern = 
+      TaskAttemptID.getTaskAttemptIDsPattern(null, null, true, 1, null, null);
+    jobConf.setKeepTaskFilesPattern(pattern);
     TestResult result;
     final Path inDir = new Path("./wc/input");
     final Path outDir = new Path("./wc/output");
@@ -190,8 +192,9 @@ public class TestMiniMRWithDFS extends TestCase {
     assertEquals("The\t1\nbrown\t1\nfox\t2\nhas\t1\nmany\t1\n" +
                  "quick\t1\nred\t1\nsilly\t1\nsox\t1\n", result.output);
     JobID jobid = result.job.getID();
-    TaskAttemptID taskid = new TaskAttemptID(new TaskID(jobid, true, 1),0);
-    checkTaskDirectories(mr, new String[]{jobid.toString()}, new String[]{taskid.toString()});
+    String taskId = mr.getTaskAttemptId(new TaskID(jobid, true, 1), 0).toString();
+    checkTaskDirectories(mr, new String[]{jobid.toString()}, 
+                         new String[]{taskId});
     // test with maps=0
     jobConf = mr.createJobConf();
     input = "owen is oom";

+ 10 - 0
src/webapps/job/jobhistory.jsp

@@ -81,6 +81,8 @@
               "<td>Job tracker Start time</td>" +
               "<td>Job Id</td><td>Name</td><td>User</td>") ; 
     out.print("</tr>"); 
+    
+    Set<String> displayedJobs = new HashSet<String>();
     for (Path jobFile: jobFiles) {
       String decodedJobFileName = 
           JobHistory.JobInfo.decodeJobHistoryFileName(jobFile.getName());
@@ -92,6 +94,14 @@
       String user = jobDetails[5];
       String jobName = jobDetails[6];
       
+      // Check if the job is already displayed. There can be multiple job 
+      // history files for jobs that have restarted
+      if (displayedJobs.contains(jobId)) {
+        continue;
+      } else {
+        displayedJobs.add(jobId);
+      }
+      
       // Encode the logfile name again to cancel the decoding done by the browser
       String encodedJobFileName = 
           JobHistory.JobInfo.encodeJobHistoryFileName(jobFile.getName());

+ 12 - 0
src/webapps/job/jobtracker.jsp

@@ -152,6 +152,18 @@
               "</td><td>" + status.getMaxReduceTasks() +
               "</td><td>" + tasksPerNode +
               "</td></tr></table>\n");
+
+    out.print("<br>");
+    if (tracker.hasRestarted()) {
+      out.print("<span class=\"small\"><i>");
+      if (tracker.hasRecovered()) {
+        out.print("The JobTracker got restarted and recovered back in " );
+        out.print(StringUtils.formatTime(tracker.getRecoveryDuration()));
+      } else {
+        out.print("The JobTracker got restarted and is still recovering");
+      }
+      out.print("</i></span>");
+    }
   }%>
 
 <%@page import="org.apache.hadoop.hdfs.server.namenode.JspHelper"%>