소스 검색

HADOOP-4191. Adds a testcase for JobHistory. Contributed by Ravi Gummadi.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@747297 13f79535-47bb-0310-9956-ffa450edef68
Devaraj Das 16 년 전
부모
커밋
29701d37b7

+ 2 - 0
CHANGES.txt

@@ -129,6 +129,8 @@ Trunk (unreleased changes)
     HADOOP-5240. Skip re-building javadoc when it is already
     HADOOP-5240. Skip re-building javadoc when it is already
     up-to-date. (Aaron Kimball via cutting)
     up-to-date. (Aaron Kimball via cutting)
 
 
+    HADOOP-4191. Adds a testcase for JobHistory. (Ravi Gummadi via ddas)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 8 - 6
src/mapred/org/apache/hadoop/mapred/JobHistory.java

@@ -77,14 +77,16 @@ public class JobHistory {
   static final long VERSION = 1L;
   static final long VERSION = 1L;
   public static final Log LOG = LogFactory.getLog(JobHistory.class);
   public static final Log LOG = LogFactory.getLog(JobHistory.class);
   private static final String DELIMITER = " ";
   private static final String DELIMITER = " ";
-  private static final char LINE_DELIMITER_CHAR = '.';
-  private static final char[] charsToEscape = new char[] {'"', '=', 
+  static final char LINE_DELIMITER_CHAR = '.';
+  static final char[] charsToEscape = new char[] {'"', '=', 
                                                 LINE_DELIMITER_CHAR};
                                                 LINE_DELIMITER_CHAR};
-  private static final String KEY = "(\\w+)";
+  static final String DIGITS = "[0-9]+";
+
+  static final String KEY = "(\\w+)";
   // value is any character other than quote, but escaped quotes can be there
   // value is any character other than quote, but escaped quotes can be there
-  private static final String VALUE = "[^\"\\\\]*(?:\\\\.[^\"\\\\]*)*"; 
+  static final String VALUE = "[^\"\\\\]*(?:\\\\.[^\"\\\\]*)*"; 
   
   
-  private static final Pattern pattern = Pattern.compile(KEY + "=" + "\"" + VALUE + "\"");
+  static final Pattern pattern = Pattern.compile(KEY + "=" + "\"" + VALUE + "\"");
   
   
   public static final int JOB_NAME_TRIM_LENGTH = 50;
   public static final int JOB_NAME_TRIM_LENGTH = 50;
   private static String JOBTRACKER_UNIQUE_STRING = null;
   private static String JOBTRACKER_UNIQUE_STRING = null;
@@ -634,7 +636,7 @@ public class JobHistory {
 
 
       // Make the pattern matching the job's history file
       // Make the pattern matching the job's history file
       final Pattern historyFilePattern = 
       final Pattern historyFilePattern = 
-        Pattern.compile(jobtrackerHostname + "_" + "[0-9]+" + "_" 
+        Pattern.compile(jobtrackerHostname + "_" + DIGITS + "_" 
                         + id.toString() + "_" + user + "_" + jobName + "+");
                         + id.toString() + "_" + user + "_" + jobName + "+");
       // a path filter that matches 4 parts of the filenames namely
       // a path filter that matches 4 parts of the filenames namely
       //  - jt-hostname
       //  - jt-hostname

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

@@ -299,7 +299,7 @@ class TaskInProgress {
     return !activeTasks.isEmpty();
     return !activeTasks.isEmpty();
   }
   }
     
     
-  private TaskAttemptID getSuccessfulTaskid() {
+  TaskAttemptID getSuccessfulTaskid() {
     return successfulTaskId;
     return successfulTaskId;
   }
   }
   
   

+ 15 - 2
src/test/org/apache/hadoop/mapred/NotificationTestCase.java

@@ -165,8 +165,20 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     }
     }
     assertEquals(2, NotificationServlet.counter);
     assertEquals(2, NotificationServlet.counter);
     
     
+    Path inDir = new Path("notificationjob/input");
+    Path outDir = new Path("notificationjob/output");
+
+    // Hack for local FS that does not have the concept of a 'mounting point'
+    if (isLocalFS()) {
+      String localPathRoot = System.getProperty("test.build.data","/tmp")
+        .toString().replace(' ', '+');;
+      inDir = new Path(localPathRoot, inDir);
+      outDir = new Path(localPathRoot, outDir);
+    }
+
     // run a job with KILLED status
     // run a job with KILLED status
-    System.out.println(TestJobKillAndFail.runJobKill(this.createJobConf()));
+    System.out.println(UtilsForTests.runJobKill(this.createJobConf(), inDir,
+                                                outDir).getID());
     synchronized(Thread.currentThread()) {
     synchronized(Thread.currentThread()) {
       stdPrintln("Sleeping for 2 seconds to give time for retry");
       stdPrintln("Sleeping for 2 seconds to give time for retry");
       Thread.currentThread().sleep(2000);
       Thread.currentThread().sleep(2000);
@@ -174,7 +186,8 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     assertEquals(4, NotificationServlet.counter);
     assertEquals(4, NotificationServlet.counter);
     
     
     // run a job with FAILED status
     // run a job with FAILED status
-    System.out.println(TestJobKillAndFail.runJobFail(this.createJobConf()));
+    System.out.println(UtilsForTests.runJobFail(this.createJobConf(), inDir,
+                                                outDir).getID());
     synchronized(Thread.currentThread()) {
     synchronized(Thread.currentThread()) {
       stdPrintln("Sleeping for 2 seconds to give time for retry");
       stdPrintln("Sleeping for 2 seconds to give time for retry");
       Thread.currentThread().sleep(2000);
       Thread.currentThread().sleep(2000);

+ 1980 - 0
src/test/org/apache/hadoop/mapred/TestJobHistory.java

@@ -0,0 +1,1980 @@
+/**
+ * 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.File;
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Iterator;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobHistory.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Tests the JobHistory files - to catch any changes to JobHistory that can
+ * cause issues for the execution of JobTracker.RecoveryManager, HistoryViewer.
+ *
+ * testJobHistoryFile
+ * Run a job that will be succeeded and validate its history file format and
+ * content.
+ *
+ * testJobHistoryUserLogLocation
+ * Run jobs with the given values of hadoop.job.history.user.location as
+ *   (1)null(default case), (2)"none", and (3)some dir like "/tmp".
+ *   Validate user history file location in each case.
+ *
+ * testJobHistoryJobStatus
+ * Run jobs that will be (1) succeeded (2) failed (3) killed.
+ *   Validate job status read from history file in each case.
+ *
+ * Future changes to job history are to be reflected here in this file.
+ */
+public class TestJobHistory extends TestCase {
+   private static final Log LOG = LogFactory.getLog(TestJobHistory.class);
+ 
+  private static String TEST_ROOT_DIR = new File(System.getProperty(
+      "test.build.data", "/tmp")).toURI().toString().replace(' ', '+');
+
+  private static final Pattern digitsPattern =
+                                     Pattern.compile(JobHistory.DIGITS);
+
+  // hostname like   /default-rack/host1.foo.com OR host1.foo.com
+  private static final Pattern hostNamePattern = Pattern.compile(
+                                       "(/(([\\w\\-\\.]+)/)+)?([\\w\\-\\.]+)");
+
+  private static final String IP_ADDR =
+                       "\\d\\d?\\d?\\.\\d\\d?\\d?\\.\\d\\d?\\d?\\.\\d\\d?\\d?";
+
+  // hostname like   /default-rack/host1.foo.com OR host1.foo.com
+  private static final Pattern trackerNamePattern = Pattern.compile(
+                         "tracker_" + hostNamePattern + ":([\\w\\-\\.]+)/" +
+                         IP_ADDR + ":" + JobHistory.DIGITS);
+
+  private static final Pattern splitsPattern = Pattern.compile(
+                              hostNamePattern + "(," + hostNamePattern + ")*");
+
+  private static Map<String, List<String>> taskIDsToAttemptIDs =
+                                     new HashMap<String, List<String>>();
+
+  //Each Task End seen from history file is added here
+  private static List<String> taskEnds = new ArrayList<String>();
+
+  // List of tasks that appear in history file after JT reatart. This is to
+  // allow START_TIME=0 for these tasks.
+  private static List<String> ignoreStartTimeOfTasks = new ArrayList<String>();
+
+  // List of potential tasks whose start time can be 0 because of JT restart
+  private static List<String> tempIgnoreStartTimeOfTasks = new ArrayList<String>();
+
+  /**
+   * Listener for history log file, it populates JobHistory.JobInfo
+   * object with data from log file and validates the data.
+   */
+  static class TestListener
+                    extends DefaultJobHistoryParser.JobTasksParseListener {
+    int lineNum;//line number of history log file
+    boolean isJobLaunched;
+    boolean isJTRestarted;
+
+    TestListener(JobInfo job) {
+      super(job);
+      lineNum = 0;
+      isJobLaunched = false;
+      isJTRestarted = false;
+    }
+
+    // TestListener implementation
+    public void handle(RecordTypes recType, Map<Keys, String> values)
+    throws IOException {
+
+      lineNum++;
+
+      // Check if the record is of type Meta
+      if (recType == JobHistory.RecordTypes.Meta) {
+        long version = Long.parseLong(values.get(Keys.VERSION));
+        assertTrue("Unexpected job history version ",
+                   (version >= 0 && version <= JobHistory.VERSION));
+      }
+      else if (recType.equals(RecordTypes.Job)) {
+        String jobid = values.get(Keys.JOBID);
+        assertTrue("record type 'Job' is seen without JOBID key" +
+        		" in history file at line " + lineNum, jobid != null);
+        JobID id = JobID.forName(jobid);
+        assertTrue("JobID in history file is in unexpected format " +
+                  "at line " + lineNum, id != null);
+        String time = values.get(Keys.LAUNCH_TIME);
+        if (time != null) {
+          if (values.get(Keys.RESTART_COUNT) == null) {// restart is fine
+            assertFalse ("Duplicate Job LAUNCH_TIME is seen in history " +
+                         "file at line " + lineNum, isJobLaunched);
+          }
+          else {// JT restart
+            isJTRestarted = true;
+          }
+          isJobLaunched = true;
+        }
+        time = values.get(Keys.FINISH_TIME);
+        if (time != null) {
+          assertTrue ("Job FINISH_TIME is seen in history file at line " +
+                      lineNum + " before LAUNCH_TIME is seen", isJobLaunched);
+        }
+      }
+      else if (recType.equals(RecordTypes.Task)) {
+        String taskid = values.get(Keys.TASKID);
+        assertTrue("record type 'Task' is seen without TASKID key" +
+        		" in history file at line " + lineNum, taskid != null);
+        TaskID id = TaskID.forName(taskid);
+        assertTrue("TaskID in history file is in unexpected format " +
+                  "at line " + lineNum, id != null);
+        
+        String time = values.get(Keys.START_TIME);
+        if (time != null) {
+          List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+          assertTrue("Duplicate START_TIME seen for task " + taskid +
+                     " in history file at line " + lineNum, attemptIDs == null);
+          attemptIDs = new ArrayList<String>();
+          taskIDsToAttemptIDs.put(taskid, attemptIDs);
+
+          if (isJTRestarted) {
+            // This maintains a potential ignoreStartTimeTasks list
+            tempIgnoreStartTimeOfTasks.add(taskid);
+          }
+        }
+
+        time = values.get(Keys.FINISH_TIME);
+        if (time != null) {
+          String s = values.get(Keys.TASK_STATUS);
+          if (s != null) {
+            List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+            assertTrue ("Task FINISH_TIME is seen in history file at line " +
+                    lineNum + " before START_TIME is seen", attemptIDs != null);
+
+            // Check if all the attemptIDs of this task are finished
+            assertTrue("TaskId " + taskid + " is finished at line " +
+                       lineNum + " but its attemptID is not finished.",
+                       (attemptIDs.size() <= 1));
+
+            // Check if at least 1 attempt of this task is seen
+            assertTrue("TaskId " + taskid + " is finished at line " +
+                       lineNum + " but no attemptID is seen before this.",
+                       attemptIDs.size() == 1);
+
+            if (s.equals("KILLED") || s.equals("FAILED")) {
+              // Task End with KILLED/FAILED status in history file is
+              // considered as TaskEnd, TaskStart. This is useful in checking
+              // the order of history lines.
+              attemptIDs = new ArrayList<String>();
+              taskIDsToAttemptIDs.put(taskid, attemptIDs);
+            }
+            else {
+              taskEnds.add(taskid);
+            }
+          }
+          else {
+            // This line of history file could be just an update to finish time
+          }
+        }
+      }
+      else if (recType.equals(RecordTypes.MapAttempt) ||
+                 recType.equals(RecordTypes.ReduceAttempt)) {
+        String taskid =  values.get(Keys.TASKID);
+        assertTrue("record type " + recType + " is seen without TASKID key" +
+        		" in history file at line " + lineNum, taskid != null);
+        
+        String attemptId = values.get(Keys.TASK_ATTEMPT_ID);
+        TaskAttemptID id = TaskAttemptID.forName(attemptId);
+        assertTrue("AttemptID in history file is in unexpected format " +
+                   "at line " + lineNum, id != null);
+        
+        String time = values.get(Keys.START_TIME);
+        if (time != null) {
+          List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+          assertTrue ("TaskAttempt is seen in history file at line " + lineNum +
+                      " before Task is seen", attemptIDs != null);
+          assertFalse ("Duplicate TaskAttempt START_TIME is seen in history " +
+                      "file at line " + lineNum, attemptIDs.remove(attemptId));
+
+          if (attemptIDs.isEmpty()) {
+            //just a boolean whether any attempt is seen or not
+            attemptIDs.add("firstAttemptIsSeen");
+          }
+          attemptIDs.add(attemptId);
+
+          if (tempIgnoreStartTimeOfTasks.contains(taskid) &&
+              (id.getId() < 1000)) {
+            // If Task line of this attempt is seen in history file after
+            // JT restart and if this attempt is < 1000(i.e. attempt is noti
+            // started after JT restart) - assuming single JT restart happened
+            ignoreStartTimeOfTasks.add(taskid);
+          }
+        }
+
+        time = values.get(Keys.FINISH_TIME);
+        if (time != null) {
+          List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+          assertTrue ("TaskAttempt FINISH_TIME is seen in history file at line "
+                      + lineNum + " before Task is seen", attemptIDs != null);
+
+          assertTrue ("TaskAttempt FINISH_TIME is seen in history file at line "
+                      + lineNum + " before TaskAttempt START_TIME is seen",
+                      attemptIDs.remove(attemptId));
+        }
+      }
+      super.handle(recType, values);
+    }
+  }
+
+  // Check if the time is in the expected format
+  private static boolean isTimeValid(String time) {
+    Matcher m = digitsPattern.matcher(time);
+    return m.matches() && (Long.parseLong(time) > 0);
+  }
+
+  private static boolean areTimesInOrder(String time1, String time2) {
+    return (Long.parseLong(time1) <= Long.parseLong(time2));
+  }
+
+  // Validate Format of Job Level Keys, Values read from history file
+  private static void validateJobLevelKeyValuesFormat(Map<Keys, String> values,
+                                                      String status) {
+    String time = values.get(Keys.SUBMIT_TIME);
+    assertTrue("Job SUBMIT_TIME is in unexpected format:" + time +
+               " in history file", isTimeValid(time));
+
+    time = values.get(Keys.LAUNCH_TIME);
+    assertTrue("Job LAUNCH_TIME is in unexpected format:" + time +
+               " in history file", isTimeValid(time));
+
+    String time1 = values.get(Keys.FINISH_TIME);
+    assertTrue("Job FINISH_TIME is in unexpected format:" + time1 +
+               " in history file", isTimeValid(time1));
+    assertTrue("Job FINISH_TIME is < LAUNCH_TIME in history file",
+               areTimesInOrder(time, time1));
+
+    String stat = values.get(Keys.JOB_STATUS);
+    assertTrue("Unexpected JOB_STATUS \"" + stat + "\" is seen in" +
+               " history file", (status.equals(stat)));
+
+    String priority = values.get(Keys.JOB_PRIORITY);
+    assertTrue("Unknown priority for the job in history file",
+               (priority.equals("HIGH") ||
+                priority.equals("LOW")  || priority.equals("NORMAL") ||
+                priority.equals("VERY_HIGH") || priority.equals("VERY_LOW")));
+  }
+
+  // Validate Format of Task Level Keys, Values read from history file
+  private static void validateTaskLevelKeyValuesFormat(JobInfo job,
+                                  boolean splitsCanBeEmpty) {
+    Map<String, JobHistory.Task> tasks = job.getAllTasks();
+
+    // validate info of each task
+    for (JobHistory.Task task : tasks.values()) {
+
+      String tid = task.get(Keys.TASKID);
+      String time = task.get(Keys.START_TIME);
+      // We allow START_TIME=0 for tasks seen in history after JT restart
+      if (!ignoreStartTimeOfTasks.contains(tid) || (Long.parseLong(time) != 0)) {
+        assertTrue("Task START_TIME of " + tid + " is in unexpected format:" +
+                 time + " in history file", isTimeValid(time));
+      }
+
+      String time1 = task.get(Keys.FINISH_TIME);
+      assertTrue("Task FINISH_TIME of " + tid + " is in unexpected format:" +
+                 time1 + " in history file", isTimeValid(time1));
+      assertTrue("Task FINISH_TIME is < START_TIME in history file",
+                 areTimesInOrder(time, time1));
+
+      // Make sure that the Task type exists and it is valid
+      String type = task.get(Keys.TASK_TYPE);
+      assertTrue("Unknown Task type \"" + type + "\" is seen in " +
+                 "history file for task " + tid,
+                 (type.equals("MAP") || type.equals("REDUCE") ||
+                  type.equals("SETUP") || type.equals("CLEANUP")));
+
+      if (type.equals("MAP")) {
+        String splits = task.get(Keys.SPLITS);
+        //order in the condition OR check is important here
+        if (!splitsCanBeEmpty || splits.length() != 0) {
+          Matcher m = splitsPattern.matcher(splits);
+          assertTrue("Unexpected format of SPLITS \"" + splits + "\" is seen" +
+                     " in history file for task " + tid, m.matches());
+        }
+      }
+
+      // Validate task status
+      String status = task.get(Keys.TASK_STATUS);
+      assertTrue("Unexpected TASK_STATUS \"" + status + "\" is seen in" +
+                 " history file for task " + tid, (status.equals("SUCCESS") ||
+                 status.equals("FAILED") || status.equals("KILLED")));
+    }
+  }
+
+  // Validate foramt of Task Attempt Level Keys, Values read from history file
+  private static void validateTaskAttemptLevelKeyValuesFormat(JobInfo job) {
+    Map<String, JobHistory.Task> tasks = job.getAllTasks();
+
+    // For each task
+    for (JobHistory.Task task : tasks.values()) {
+      // validate info of each attempt
+      for (JobHistory.TaskAttempt attempt : task.getTaskAttempts().values()) {
+
+        String id = attempt.get(Keys.TASK_ATTEMPT_ID);
+        String time = attempt.get(Keys.START_TIME);
+        assertTrue("START_TIME of task attempt " + id +
+                   " is in unexpected format:" + time +
+                   " in history file", isTimeValid(time));
+
+        String time1 = attempt.get(Keys.FINISH_TIME);
+        assertTrue("FINISH_TIME of task attempt " + id +
+                   " is in unexpected format:" + time1 +
+                   " in history file", isTimeValid(time1));
+        assertTrue("Task FINISH_TIME is < START_TIME in history file",
+                   areTimesInOrder(time, time1));
+
+        // Make sure that the Task type exists and it is valid
+        String type = attempt.get(Keys.TASK_TYPE);
+        assertTrue("Unknown Task type \"" + type + "\" is seen in " +
+                   "history file for task attempt " + id,
+                   (type.equals("MAP") || type.equals("REDUCE") ||
+                    type.equals("SETUP") || type.equals("CLEANUP")));
+
+        // Validate task status
+        String status = attempt.get(Keys.TASK_STATUS);
+        assertTrue("Unexpected TASK_STATUS \"" + status + "\" is seen in" +
+                   " history file for task attempt " + id,
+                   (status.equals("SUCCESS") || status.equals("FAILED") ||
+                    status.equals("KILLED")));
+
+        // Reduce Task Attempts should have valid SHUFFLE_FINISHED time and
+        // SORT_FINISHED time
+        if (type.equals("REDUCE") && status.equals("SUCCESS")) {
+          time1 = attempt.get(Keys.SHUFFLE_FINISHED);
+          assertTrue("SHUFFLE_FINISHED time of task attempt " + id +
+                     " is in unexpected format:" + time1 +
+                     " in history file", isTimeValid(time1));
+          assertTrue("Reduce Task SHUFFLE_FINISHED time is < START_TIME " +
+                     "in history file", areTimesInOrder(time, time1));
+          time = attempt.get(Keys.SORT_FINISHED);
+          assertTrue("SORT_FINISHED of task attempt " + id +
+                     " is in unexpected format:" + time +
+                     " in history file", isTimeValid(time));
+          assertTrue("Reduce Task SORT_FINISHED time is < SORT_FINISHED time" +
+                     " in history file", areTimesInOrder(time1, time));
+        }
+
+        // check if hostname is valid
+        String hostname = attempt.get(Keys.HOSTNAME);
+        Matcher m = hostNamePattern.matcher(hostname);
+        assertTrue("Unexpected Host name of task attempt " + id, m.matches());
+
+        // check if trackername is valid
+        String trackerName = attempt.get(Keys.TRACKER_NAME);
+        m = trackerNamePattern.matcher(trackerName);
+        assertTrue("Unexpected tracker name of task attempt " + id,
+                   m.matches());
+
+        if (!status.equals("KILLED")) {
+          // check if http port is valid
+          String httpPort = attempt.get(Keys.HTTP_PORT);
+          m = digitsPattern.matcher(httpPort);
+          assertTrue("Unexpected http port of task attempt " + id, m.matches());
+        }
+        
+        // check if counters are parsable
+        String counters = attempt.get(Keys.COUNTERS);
+        try {
+          Counters readCounters = Counters.fromEscapedCompactString(counters);
+          assertTrue("Counters of task attempt " + id + " are not parsable",
+                     readCounters != null);
+        } catch (ParseException pe) {
+          LOG.warn("While trying to parse counters of task attempt " + id +
+                   ", " + pe);
+        }
+      }
+    }
+  }
+
+  /**
+   *  Validates the format of contents of history file
+   *  (1) history file exists and in correct location
+   *  (2) Verify if the history file is parsable
+   *  (3) Validate the contents of history file
+   *     (a) Format of all TIMEs are checked against a regex
+   *     (b) validate legality/format of job level key, values
+   *     (c) validate legality/format of task level key, values
+   *     (d) validate legality/format of attempt level key, values
+   *     (e) check if all the TaskAttempts, Tasks started are finished.
+   *         Check finish of each TaskAttemptID against its start to make sure
+   *         that all TaskAttempts, Tasks started are indeed finished and the
+   *         history log lines are in the proper order.
+   *         We want to catch ordering of history lines like
+   *            Task START
+   *            Attempt START
+   *            Task FINISH
+   *            Attempt FINISH
+   *         (speculative execution is turned off for this).
+   * @param id job id
+   * @param conf job conf
+   */
+  static void validateJobHistoryFileFormat(JobID id, JobConf conf,
+                 String status, boolean splitsCanBeEmpty) throws IOException  {
+
+    // Get the history file name
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+
+    // Framework history log file location
+    Path logFile = JobHistory.JobInfo.getJobHistoryLogLocation(logFileName);
+    FileSystem fileSys = logFile.getFileSystem(conf);
+ 
+    // Check if the history file exists
+    assertTrue("History file does not exist", fileSys.exists(logFile));
+
+
+    // check if the history file is parsable
+    String[] jobDetails = JobHistory.JobInfo.decodeJobHistoryFileName(
+    		                                   logFileName).split("_");
+
+    String jobId = jobDetails[2] + "_" + jobDetails[3] + "_" + jobDetails[4];
+    JobHistory.JobInfo jobInfo = new JobHistory.JobInfo(jobId);
+
+    TestListener l = new TestListener(jobInfo);
+    JobHistory.parseHistoryFromFS(logFile.toString().substring(5), l, fileSys);
+
+
+    // validate format of job level key, values
+    validateJobLevelKeyValuesFormat(jobInfo.getValues(), status);
+
+    // validate format of task level key, values
+    validateTaskLevelKeyValuesFormat(jobInfo, splitsCanBeEmpty);
+
+    // validate format of attempt level key, values
+    validateTaskAttemptLevelKeyValuesFormat(jobInfo);
+
+    // check if all the TaskAttempts, Tasks started are finished for
+    // successful jobs
+    if (status.equals("SUCCESS")) {
+      // Make sure that the lists in taskIDsToAttemptIDs are empty.
+      for(Iterator<String> it = taskIDsToAttemptIDs.keySet().iterator();it.hasNext();) {
+        String taskid = it.next();
+        assertTrue("There are some Tasks which are not finished in history " +
+                   "file.", taskEnds.contains(taskid));
+        List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+        if(attemptIDs != null) {
+          assertTrue("Unexpected. TaskID " + taskid + " has task attempt(s)" +
+                     " that are not finished.", (attemptIDs.size() == 1));
+        }
+      }
+    }
+  }
+
+  // Validate Job Level Keys, Values read from history file by
+  // comparing them with the actual values from JT.
+  private static void validateJobLevelKeyValues(MiniMRCluster mr,
+          RunningJob job, JobInfo jobInfo, JobConf conf) throws IOException  {
+
+    JobTracker jt = mr.getJobTrackerRunner().getJobTracker();
+    JobInProgress jip = jt.getJob(job.getID());
+
+    Map<Keys, String> values = jobInfo.getValues();
+
+    assertTrue("SUBMIT_TIME of job obtained from history file did not " +
+               "match the expected value", jip.getStartTime() ==
+               Long.parseLong(values.get(Keys.SUBMIT_TIME)));
+
+    assertTrue("LAUNCH_TIME of job obtained from history file did not " +
+               "match the expected value", jip.getLaunchTime() ==
+               Long.parseLong(values.get(Keys.LAUNCH_TIME)));
+
+    assertTrue("FINISH_TIME of job obtained from history file did not " +
+               "match the expected value", jip.getFinishTime() ==
+               Long.parseLong(values.get(Keys.FINISH_TIME)));
+
+    assertTrue("Job Status of job obtained from history file did not " +
+               "match the expected value",
+               values.get(Keys.JOB_STATUS).equals("SUCCESS"));
+
+    assertTrue("Job Priority of job obtained from history file did not " +
+               "match the expected value", jip.getPriority().toString().equals(
+               values.get(Keys.JOB_PRIORITY)));
+
+    assertTrue("Job Name of job obtained from history file did not " +
+               "match the expected value", JobInfo.getJobName(conf).equals(
+               values.get(Keys.JOBNAME)));
+
+    assertTrue("User Name of job obtained from history file did not " +
+               "match the expected value", JobInfo.getUserName(conf).equals(
+               values.get(Keys.USER)));
+
+    // Validate job counters
+    Counters c = jip.getCounters();
+    assertTrue("Counters of job obtained from history file did not " +
+               "match the expected value",
+               c.makeEscapedCompactString().equals(values.get(Keys.COUNTERS)));
+
+    // Validate number of total maps, total reduces, finished maps,
+    // finished reduces, failed maps, failed recudes
+    String totalMaps = values.get(Keys.TOTAL_MAPS);
+    assertTrue("Unexpected number of total maps in history file",
+               Integer.parseInt(totalMaps) == jip.desiredMaps());
+
+    String totalReduces = values.get(Keys.TOTAL_REDUCES);
+    assertTrue("Unexpected number of total reduces in history file",
+               Integer.parseInt(totalReduces) == jip.desiredReduces());
+
+    String finMaps = values.get(Keys.FINISHED_MAPS);
+    assertTrue("Unexpected number of finished maps in history file",
+               Integer.parseInt(finMaps) == jip.finishedMaps());
+
+    String finReduces = values.get(Keys.FINISHED_REDUCES);
+    assertTrue("Unexpected number of finished reduces in history file",
+               Integer.parseInt(finReduces) == jip.finishedReduces());
+
+    String failedMaps = values.get(Keys.FAILED_MAPS);
+    assertTrue("Unexpected number of failed maps in history file",
+               Integer.parseInt(failedMaps) == jip.failedMapTasks);
+
+    String failedReduces = values.get(Keys.FAILED_REDUCES);
+    assertTrue("Unexpected number of failed reduces in history file",
+               Integer.parseInt(failedReduces) == jip.failedReduceTasks);
+  }
+
+  // Validate Task Level Keys, Values read from history file by
+  // comparing them with the actual values from JT.
+  private static void validateTaskLevelKeyValues(MiniMRCluster mr,
+                      RunningJob job, JobInfo jobInfo) throws IOException  {
+
+    JobTracker jt = mr.getJobTrackerRunner().getJobTracker();
+    JobInProgress jip = jt.getJob(job.getID());
+
+    // Get the 1st map, 1st reduce, cleanup & setup taskIDs and
+    // validate their history info
+    TaskID mapTaskId = new TaskID(job.getID(), true, 0);
+    TaskID reduceTaskId = new TaskID(job.getID(), false, 0);
+
+    TaskInProgress cleanups[] = jip.getCleanupTasks();
+    TaskID cleanupTaskId;
+    if (cleanups[0].isComplete()) {
+      cleanupTaskId = cleanups[0].getTIPId();
+    }
+    else {
+      cleanupTaskId = cleanups[1].getTIPId();
+    }
+
+    TaskInProgress setups[] = jip.getSetupTasks();
+    TaskID setupTaskId;
+    if (setups[0].isComplete()) {
+      setupTaskId = setups[0].getTIPId();
+    }
+    else {
+      setupTaskId = setups[1].getTIPId();
+    }
+
+    Map<String, JobHistory.Task> tasks = jobInfo.getAllTasks();
+
+    // validate info of the 4 tasks(cleanup, setup, 1st map, 1st reduce)
+    for (JobHistory.Task task : tasks.values()) {
+
+      String tid = task.get(Keys.TASKID);
+      if (tid.equals(mapTaskId.toString()) ||
+          tid.equals(reduceTaskId.toString()) ||
+          tid.equals(cleanupTaskId.toString()) ||
+          tid.equals(setupTaskId.toString())) {
+
+        TaskID taskId = null;
+        if (tid.equals(mapTaskId.toString())) {
+          taskId = mapTaskId;
+        }
+        else if (tid.equals(reduceTaskId.toString())) {
+          taskId = reduceTaskId;
+        }
+        else if (tid.equals(cleanupTaskId.toString())) {
+          taskId = cleanupTaskId;
+        }
+        else if (tid.equals(setupTaskId.toString())) {
+          taskId = setupTaskId;
+        }
+        TaskInProgress tip = jip.getTaskInProgress(taskId);
+        assertTrue("START_TIME of Task " + tid + " obtained from history " +
+             "file did not match the expected value", tip.getExecStartTime() ==
+             Long.parseLong(task.get(Keys.START_TIME)));
+
+        assertTrue("FINISH_TIME of Task " + tid + " obtained from history " +
+             "file did not match the expected value", tip.getExecFinishTime() ==
+             Long.parseLong(task.get(Keys.FINISH_TIME)));
+
+        if (taskId == mapTaskId) {//check splits only for map task
+          assertTrue("Splits of Task " + tid + " obtained from history file " +
+                     " did not match the expected value",
+                     tip.getSplitNodes().equals(task.get(Keys.SPLITS)));
+        }
+
+        TaskAttemptID attemptId = tip.getSuccessfulTaskid();
+        TaskStatus ts = tip.getTaskStatus(attemptId);
+
+        // Validate task counters
+        Counters c = ts.getCounters();
+        assertTrue("Counters of Task " + tid + " obtained from history file " +
+                   " did not match the expected value",
+                  c.makeEscapedCompactString().equals(task.get(Keys.COUNTERS)));
+      }
+    }
+  }
+
+  // Validate Task Attempt Level Keys, Values read from history file by
+  // comparing them with the actual values from JT.
+  private static void validateTaskAttemptLevelKeyValues(MiniMRCluster mr,
+                      RunningJob job, JobInfo jobInfo) throws IOException  {
+
+    JobTracker jt = mr.getJobTrackerRunner().getJobTracker();
+    JobInProgress jip = jt.getJob(job.getID());
+
+    Map<String, JobHistory.Task> tasks = jobInfo.getAllTasks();
+
+    // For each task
+    for (JobHistory.Task task : tasks.values()) {
+      // validate info of each attempt
+      for (JobHistory.TaskAttempt attempt : task.getTaskAttempts().values()) {
+
+        String idStr = attempt.get(Keys.TASK_ATTEMPT_ID);
+        TaskAttemptID attemptId = TaskAttemptID.forName(idStr);
+        TaskID tid = attemptId.getTaskID();
+
+        // Validate task id
+        assertTrue("Task id of Task Attempt " + idStr + " obtained from " +
+                   "history file did not match the expected value",
+                   tid.toString().equals(attempt.get(Keys.TASKID)));
+
+        TaskInProgress tip = jip.getTaskInProgress(tid);
+        TaskStatus ts = tip.getTaskStatus(attemptId);
+
+        // Validate task attempt start time
+        assertTrue("START_TIME of Task attempt " + idStr + " obtained from " +
+                   "history file did not match the expected value",
+            ts.getStartTime() == Long.parseLong(attempt.get(Keys.START_TIME)));
+
+        // Validate task attempt finish time
+        assertTrue("FINISH_TIME of Task attempt " + idStr + " obtained from " +
+                   "history file did not match the expected value",
+            ts.getFinishTime() == Long.parseLong(attempt.get(Keys.FINISH_TIME)));
+
+
+        TaskTrackerStatus ttStatus = jt.getTaskTracker(ts.getTaskTracker());
+
+        if (ttStatus != null) {
+          assertTrue("http port of task attempt " + idStr + " obtained from " +
+                     "history file did not match the expected value",
+                     ttStatus.getHttpPort() ==
+                     Integer.parseInt(attempt.get(Keys.HTTP_PORT)));
+
+          if (attempt.get(Keys.TASK_STATUS).equals("SUCCESS")) {
+            String ttHostname = jt.getNode(ttStatus.getHost()).toString();
+
+            // check if hostname is valid
+            assertTrue("Host name of task attempt " + idStr + " obtained from" +
+                       " history file did not match the expected value",
+                       ttHostname.equals(attempt.get(Keys.HOSTNAME)));
+          }
+        }
+        if (attempt.get(Keys.TASK_STATUS).equals("SUCCESS")) {
+          // Validate SHUFFLE_FINISHED time and SORT_FINISHED time of
+          // Reduce Task Attempts
+          if (attempt.get(Keys.TASK_TYPE).equals("REDUCE")) {
+            assertTrue("SHUFFLE_FINISHED time of task attempt " + idStr +
+                     " obtained from history file did not match the expected" +
+                     " value", ts.getShuffleFinishTime() ==
+                     Long.parseLong(attempt.get(Keys.SHUFFLE_FINISHED)));
+            assertTrue("SORT_FINISHED time of task attempt " + idStr +
+                     " obtained from history file did not match the expected" +
+                     " value", ts.getSortFinishTime() ==
+                     Long.parseLong(attempt.get(Keys.SORT_FINISHED)));
+          }
+
+          //Validate task counters
+          Counters c = ts.getCounters();
+          assertTrue("Counters of Task Attempt " + idStr + " obtained from " +
+                     "history file did not match the expected value",
+               c.makeEscapedCompactString().equals(attempt.get(Keys.COUNTERS)));
+        }
+        
+        // check if tracker name is valid
+        assertTrue("Tracker name of task attempt " + idStr + " obtained from " +
+                   "history file did not match the expected value",
+                   ts.getTaskTracker().equals(attempt.get(Keys.TRACKER_NAME)));
+      }
+    }
+  }
+
+  /**
+   * Checks if the history file content is as expected comparing with the
+   * actual values obtained from JT.
+   * Job Level, Task Level and Task Attempt Level Keys, Values are validated.
+   * @param job RunningJob object of the job whose history is to be validated
+   * @param conf job conf
+   */
+  static void validateJobHistoryFileContent(MiniMRCluster mr,
+                              RunningJob job, JobConf conf) throws IOException  {
+
+    JobID id = job.getID();
+    // Get the history file name
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+
+    // Framework history log file location
+    Path logFile = JobHistory.JobInfo.getJobHistoryLogLocation(logFileName);
+    FileSystem fileSys = logFile.getFileSystem(conf);
+ 
+    // Check if the history file exists
+    assertTrue("History file does not exist", fileSys.exists(logFile));
+
+
+    // check if the history file is parsable
+    String[] jobDetails = JobHistory.JobInfo.decodeJobHistoryFileName(
+    		                                   logFileName).split("_");
+
+    String jobId = jobDetails[2] + "_" + jobDetails[3] + "_" + jobDetails[4];
+    JobHistory.JobInfo jobInfo = new JobHistory.JobInfo(jobId);
+
+    DefaultJobHistoryParser.JobTasksParseListener l =
+                   new DefaultJobHistoryParser.JobTasksParseListener(jobInfo);
+    JobHistory.parseHistoryFromFS(logFile.toString().substring(5), l, fileSys);
+
+    // Now the history file contents are available in jobInfo. Let us compare
+    // them with the actual values from JT.
+    validateJobLevelKeyValues(mr, job, jobInfo, conf);
+    validateTaskLevelKeyValues(mr, job, jobInfo);
+    validateTaskAttemptLevelKeyValues(mr, job, jobInfo);
+  }
+
+  /** Run a job that will be succeeded and validate its history file format
+   *  and its content.
+   */
+  public void testJobHistoryFile() throws IOException {
+    MiniMRCluster mr = null;
+    try {
+      mr = new MiniMRCluster(2, "file:///", 3);
+
+      // run the TCs
+      JobConf conf = mr.createJobConf();
+
+      FileSystem fs = FileSystem.get(conf);
+      // clean up
+      fs.delete(new Path(TEST_ROOT_DIR + "/succeed"), true);
+
+      Path inDir = new Path(TEST_ROOT_DIR + "/succeed/input");
+      Path outDir = new Path(TEST_ROOT_DIR + "/succeed/output");
+
+      //Disable speculative execution
+      conf.setSpeculativeExecution(false);
+
+      // Make sure that the job is not removed from memory until we do finish
+      // the validation of history file content
+      conf.setInt("mapred.jobtracker.completeuserjobs.maximum", 10);
+
+      // Run a job that will be succeeded and validate its history file
+      RunningJob job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryFileFormat(job.getID(), conf, "SUCCESS", false);
+      validateJobHistoryFileContent(mr, job, conf);
+
+    } finally {
+      if (mr != null) {
+        mr.shutdown();
+      }
+    }
+  }
+
+  // Returns the output path where user history log file is written to with
+  // default configuration setting for hadoop.job.history.user.location
+  private static Path getLogLocationInOutputPath(String logFileName,
+                                                      JobConf conf) {
+    JobConf jobConf = new JobConf(true);//default JobConf
+    FileOutputFormat.setOutputPath(jobConf,
+                     FileOutputFormat.getOutputPath(conf));
+    return JobHistory.JobInfo.getJobHistoryLogLocationForUser(
+                                             logFileName, jobConf);
+  }
+
+  /**
+   * Checks if the user history file exists in the correct dir
+   * @param id job id
+   * @param conf job conf
+   */
+  private static void validateJobHistoryUserLogLocation(JobID id, JobConf conf) 
+          throws IOException  {
+    // Get the history file name
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+
+    // User history log file location
+    Path logFile = JobHistory.JobInfo.getJobHistoryLogLocationForUser(
+                                                     logFileName, conf);
+    if(logFile == null) {
+      // get the output path where history file is written to when
+      // hadoop.job.history.user.location is not set
+      logFile = getLogLocationInOutputPath(logFileName, conf);
+    }
+    FileSystem fileSys = null;
+    fileSys = logFile.getFileSystem(conf);
+
+    // Check if the user history file exists in the correct dir
+    if (conf.get("hadoop.job.history.user.location") == null) {
+      assertTrue("User log file " + logFile + " does not exist",
+                 fileSys.exists(logFile));
+    }
+    else if (conf.get("hadoop.job.history.user.location") == "none") {
+      // history file should not exist in the output path
+      assertFalse("Unexpected. User log file exists in output dir when " +
+                 "hadoop.job.history.user.location is set to \"none\"",
+                 fileSys.exists(logFile));
+    }
+    else {
+      //hadoop.job.history.user.location is set to a specific location.
+      // User log file should exist in that location
+      assertTrue("User log file " + logFile + " does not exist",
+                 fileSys.exists(logFile));
+
+      // User log file should not exist in output path.
+
+      // get the output path where history file is written to when
+      // hadoop.job.history.user.location is not set
+      Path logFile1 = getLogLocationInOutputPath(logFileName, conf);
+      
+      if (logFile != logFile1) {
+        fileSys = logFile1.getFileSystem(conf);
+        assertFalse("Unexpected. User log file exists in output dir when " +
+              "hadoop.job.history.user.location is set to a different location",
+              fileSys.exists(logFile1));
+      }
+    }
+  }
+
+  // Validate user history file location for the given values of
+  // hadoop.job.history.user.location as
+  // (1)null(default case), (2)"none", and (3)some dir "/tmp"
+  public void testJobHistoryUserLogLocation() throws IOException {
+    MiniMRCluster mr = null;
+    try {
+      mr = new MiniMRCluster(2, "file:///", 3);
+
+      // run the TCs
+      JobConf conf = mr.createJobConf();
+
+      FileSystem fs = FileSystem.get(conf);
+      // clean up
+      fs.delete(new Path(TEST_ROOT_DIR + "/succeed"), true);
+
+      Path inDir = new Path(TEST_ROOT_DIR + "/succeed/input1");
+      Path outDir = new Path(TEST_ROOT_DIR + "/succeed/output1");
+
+      // validate for the case of null(default)
+      RunningJob job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryUserLogLocation(job.getID(), conf);
+
+      inDir = new Path(TEST_ROOT_DIR + "/succeed/input2");
+      outDir = new Path(TEST_ROOT_DIR + "/succeed/output2");
+      // validate for the case of "none"
+      conf.set("hadoop.job.history.user.location", "none");
+      job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryUserLogLocation(job.getID(), conf);
+ 
+      inDir = new Path(TEST_ROOT_DIR + "/succeed/input3");
+      outDir = new Path(TEST_ROOT_DIR + "/succeed/output3");
+      // validate for the case of any dir
+      conf.set("hadoop.job.history.user.location", "/tmp");
+      job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryUserLogLocation(job.getID(), conf);
+
+    } finally {
+      if (mr != null) {
+        mr.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Checks if the history file has expected job status
+   * @param id job id
+   * @param conf job conf
+   */
+  private static void validateJobHistoryJobStatus(JobID id, JobConf conf,
+          String status) throws IOException  {
+
+    // Get the history file name
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+
+    // Framework history log file location
+    Path logFile = JobHistory.JobInfo.getJobHistoryLogLocation(logFileName);
+    FileSystem fileSys = logFile.getFileSystem(conf);
+ 
+    // Check if the history file exists
+    assertTrue("History file does not exist", fileSys.exists(logFile));
+
+
+    // check if the history file is parsable
+    String[] jobDetails = JobHistory.JobInfo.decodeJobHistoryFileName(
+    		                                   logFileName).split("_");
+
+    String jobId = jobDetails[2] + "_" + jobDetails[3] + "_" + jobDetails[4];
+    JobHistory.JobInfo jobInfo = new JobHistory.JobInfo(jobId);
+
+    DefaultJobHistoryParser.JobTasksParseListener l =
+                  new DefaultJobHistoryParser.JobTasksParseListener(jobInfo);
+    JobHistory.parseHistoryFromFS(logFile.toString().substring(5), l, fileSys);
+
+    assertTrue("Job Status read from job history file is not the expected" +
+         " status", status.equals(jobInfo.getValues().get(Keys.JOB_STATUS)));
+  }
+
+  // run jobs that will be (1) succeeded (2) failed (3) killed
+  // and validate job status read from history file in each case
+  public void testJobHistoryJobStatus() throws IOException {
+    MiniMRCluster mr = null;
+    try {
+      mr = new MiniMRCluster(2, "file:///", 3);
+
+      // run the TCs
+      JobConf conf = mr.createJobConf();
+
+      FileSystem fs = FileSystem.get(conf);
+      // clean up
+      fs.delete(new Path(TEST_ROOT_DIR + "/succeedfailkilljob"), true);
+
+      Path inDir = new Path(TEST_ROOT_DIR + "/succeedfailkilljob/input");
+      Path outDir = new Path(TEST_ROOT_DIR + "/succeedfailkilljob/output");
+
+      // Run a job that will be succeeded and validate its job status
+      // existing in history file
+      RunningJob job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryJobStatus(job.getID(), conf, "SUCCESS");
+      
+      // Run a job that will be failed and validate its job status
+      // existing in history file
+      job = UtilsForTests.runJobFail(conf, inDir, outDir);
+      validateJobHistoryJobStatus(job.getID(), conf, "FAILED");
+      
+      // Run a job that will be killed and validate its job status
+      // existing in history file
+      job = UtilsForTests.runJobKill(conf, inDir, outDir);
+      validateJobHistoryJobStatus(job.getID(), conf, "KILLED");
+      
+    } finally {
+      if (mr != null) {
+        mr.shutdown();
+      }
+    }
+  }
+}
+/**
+ * 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.File;
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Iterator;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobHistory.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Tests the JobHistory files - to catch any changes to JobHistory that can
+ * cause issues for the execution of JobTracker.RecoveryManager, HistoryViewer.
+ *
+ * testJobHistoryFile
+ * Run a job that will be succeeded and validate its history file format and
+ * content.
+ *
+ * testJobHistoryUserLogLocation
+ * Run jobs with the given values of hadoop.job.history.user.location as
+ *   (1)null(default case), (2)"none", and (3)some dir like "/tmp".
+ *   Validate user history file location in each case.
+ *
+ * testJobHistoryJobStatus
+ * Run jobs that will be (1) succeeded (2) failed (3) killed.
+ *   Validate job status read from history file in each case.
+ *
+ * Future changes to job history are to be reflected here in this file.
+ */
+public class TestJobHistory extends TestCase {
+   private static final Log LOG = LogFactory.getLog(TestJobHistory.class);
+ 
+  private static String TEST_ROOT_DIR = new File(System.getProperty(
+      "test.build.data", "/tmp")).toURI().toString().replace(' ', '+');
+
+  private static final Pattern digitsPattern =
+                                     Pattern.compile(JobHistory.DIGITS);
+
+  // hostname like   /default-rack/host1.foo.com OR host1.foo.com
+  private static final Pattern hostNamePattern = Pattern.compile(
+                                       "(/(([\\w\\-\\.]+)/)+)?([\\w\\-\\.]+)");
+
+  private static final String IP_ADDR =
+                       "\\d\\d?\\d?\\.\\d\\d?\\d?\\.\\d\\d?\\d?\\.\\d\\d?\\d?";
+
+  // hostname like   /default-rack/host1.foo.com OR host1.foo.com
+  private static final Pattern trackerNamePattern = Pattern.compile(
+                         "tracker_" + hostNamePattern + ":([\\w\\-\\.]+)/" +
+                         IP_ADDR + ":" + JobHistory.DIGITS);
+
+  private static final Pattern splitsPattern = Pattern.compile(
+                              hostNamePattern + "(," + hostNamePattern + ")*");
+
+  private static Map<String, List<String>> taskIDsToAttemptIDs =
+                                     new HashMap<String, List<String>>();
+
+  //Each Task End seen from history file is added here
+  private static List<String> taskEnds = new ArrayList<String>();
+
+  // List of tasks that appear in history file after JT reatart. This is to
+  // allow START_TIME=0 for these tasks.
+  private static List<String> ignoreStartTimeOfTasks = new ArrayList<String>();
+
+  // List of potential tasks whose start time can be 0 because of JT restart
+  private static List<String> tempIgnoreStartTimeOfTasks = new ArrayList<String>();
+
+  /**
+   * Listener for history log file, it populates JobHistory.JobInfo
+   * object with data from log file and validates the data.
+   */
+  static class TestListener
+                    extends DefaultJobHistoryParser.JobTasksParseListener {
+    int lineNum;//line number of history log file
+    boolean isJobLaunched;
+    boolean isJTRestarted;
+
+    TestListener(JobInfo job) {
+      super(job);
+      lineNum = 0;
+      isJobLaunched = false;
+      isJTRestarted = false;
+    }
+
+    // TestListener implementation
+    public void handle(RecordTypes recType, Map<Keys, String> values)
+    throws IOException {
+
+      lineNum++;
+
+      // Check if the record is of type Meta
+      if (recType == JobHistory.RecordTypes.Meta) {
+        long version = Long.parseLong(values.get(Keys.VERSION));
+        assertTrue("Unexpected job history version ",
+                   (version >= 0 && version <= JobHistory.VERSION));
+      }
+      else if (recType.equals(RecordTypes.Job)) {
+        String jobid = values.get(Keys.JOBID);
+        assertTrue("record type 'Job' is seen without JOBID key" +
+        		" in history file at line " + lineNum, jobid != null);
+        JobID id = JobID.forName(jobid);
+        assertTrue("JobID in history file is in unexpected format " +
+                  "at line " + lineNum, id != null);
+        String time = values.get(Keys.LAUNCH_TIME);
+        if (time != null) {
+          if (values.get(Keys.RESTART_COUNT) == null) {// restart is fine
+            assertFalse ("Duplicate Job LAUNCH_TIME is seen in history " +
+                         "file at line " + lineNum, isJobLaunched);
+          }
+          else {// JT restart
+            isJTRestarted = true;
+          }
+          isJobLaunched = true;
+        }
+        time = values.get(Keys.FINISH_TIME);
+        if (time != null) {
+          assertTrue ("Job FINISH_TIME is seen in history file at line " +
+                      lineNum + " before LAUNCH_TIME is seen", isJobLaunched);
+        }
+      }
+      else if (recType.equals(RecordTypes.Task)) {
+        String taskid = values.get(Keys.TASKID);
+        assertTrue("record type 'Task' is seen without TASKID key" +
+        		" in history file at line " + lineNum, taskid != null);
+        TaskID id = TaskID.forName(taskid);
+        assertTrue("TaskID in history file is in unexpected format " +
+                  "at line " + lineNum, id != null);
+        
+        String time = values.get(Keys.START_TIME);
+        if (time != null) {
+          List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+          assertTrue("Duplicate START_TIME seen for task " + taskid +
+                     " in history file at line " + lineNum, attemptIDs == null);
+          attemptIDs = new ArrayList<String>();
+          taskIDsToAttemptIDs.put(taskid, attemptIDs);
+
+          if (isJTRestarted) {
+            // This maintains a potential ignoreStartTimeTasks list
+            tempIgnoreStartTimeOfTasks.add(taskid);
+          }
+        }
+
+        time = values.get(Keys.FINISH_TIME);
+        if (time != null) {
+          String s = values.get(Keys.TASK_STATUS);
+          if (s != null) {
+            List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+            assertTrue ("Task FINISH_TIME is seen in history file at line " +
+                    lineNum + " before START_TIME is seen", attemptIDs != null);
+
+            // Check if all the attemptIDs of this task are finished
+            assertTrue("TaskId " + taskid + " is finished at line " +
+                       lineNum + " but its attemptID is not finished.",
+                       (attemptIDs.size() <= 1));
+
+            // Check if at least 1 attempt of this task is seen
+            assertTrue("TaskId " + taskid + " is finished at line " +
+                       lineNum + " but no attemptID is seen before this.",
+                       attemptIDs.size() == 1);
+
+            if (s.equals("KILLED") || s.equals("FAILED")) {
+              // Task End with KILLED/FAILED status in history file is
+              // considered as TaskEnd, TaskStart. This is useful in checking
+              // the order of history lines.
+              attemptIDs = new ArrayList<String>();
+              taskIDsToAttemptIDs.put(taskid, attemptIDs);
+            }
+            else {
+              taskEnds.add(taskid);
+            }
+          }
+          else {
+            // This line of history file could be just an update to finish time
+          }
+        }
+      }
+      else if (recType.equals(RecordTypes.MapAttempt) ||
+                 recType.equals(RecordTypes.ReduceAttempt)) {
+        String taskid =  values.get(Keys.TASKID);
+        assertTrue("record type " + recType + " is seen without TASKID key" +
+        		" in history file at line " + lineNum, taskid != null);
+        
+        String attemptId = values.get(Keys.TASK_ATTEMPT_ID);
+        TaskAttemptID id = TaskAttemptID.forName(attemptId);
+        assertTrue("AttemptID in history file is in unexpected format " +
+                   "at line " + lineNum, id != null);
+        
+        String time = values.get(Keys.START_TIME);
+        if (time != null) {
+          List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+          assertTrue ("TaskAttempt is seen in history file at line " + lineNum +
+                      " before Task is seen", attemptIDs != null);
+          assertFalse ("Duplicate TaskAttempt START_TIME is seen in history " +
+                      "file at line " + lineNum, attemptIDs.remove(attemptId));
+
+          if (attemptIDs.isEmpty()) {
+            //just a boolean whether any attempt is seen or not
+            attemptIDs.add("firstAttemptIsSeen");
+          }
+          attemptIDs.add(attemptId);
+
+          if (tempIgnoreStartTimeOfTasks.contains(taskid) &&
+              (id.getId() < 1000)) {
+            // If Task line of this attempt is seen in history file after
+            // JT restart and if this attempt is < 1000(i.e. attempt is noti
+            // started after JT restart) - assuming single JT restart happened
+            ignoreStartTimeOfTasks.add(taskid);
+          }
+        }
+
+        time = values.get(Keys.FINISH_TIME);
+        if (time != null) {
+          List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+          assertTrue ("TaskAttempt FINISH_TIME is seen in history file at line "
+                      + lineNum + " before Task is seen", attemptIDs != null);
+
+          assertTrue ("TaskAttempt FINISH_TIME is seen in history file at line "
+                      + lineNum + " before TaskAttempt START_TIME is seen",
+                      attemptIDs.remove(attemptId));
+        }
+      }
+      super.handle(recType, values);
+    }
+  }
+
+  // Check if the time is in the expected format
+  private static boolean isTimeValid(String time) {
+    Matcher m = digitsPattern.matcher(time);
+    return m.matches() && (Long.parseLong(time) > 0);
+  }
+
+  private static boolean areTimesInOrder(String time1, String time2) {
+    return (Long.parseLong(time1) <= Long.parseLong(time2));
+  }
+
+  // Validate Format of Job Level Keys, Values read from history file
+  private static void validateJobLevelKeyValuesFormat(Map<Keys, String> values,
+                                                      String status) {
+    String time = values.get(Keys.SUBMIT_TIME);
+    assertTrue("Job SUBMIT_TIME is in unexpected format:" + time +
+               " in history file", isTimeValid(time));
+
+    time = values.get(Keys.LAUNCH_TIME);
+    assertTrue("Job LAUNCH_TIME is in unexpected format:" + time +
+               " in history file", isTimeValid(time));
+
+    String time1 = values.get(Keys.FINISH_TIME);
+    assertTrue("Job FINISH_TIME is in unexpected format:" + time1 +
+               " in history file", isTimeValid(time1));
+    assertTrue("Job FINISH_TIME is < LAUNCH_TIME in history file",
+               areTimesInOrder(time, time1));
+
+    String stat = values.get(Keys.JOB_STATUS);
+    assertTrue("Unexpected JOB_STATUS \"" + stat + "\" is seen in" +
+               " history file", (status.equals(stat)));
+
+    String priority = values.get(Keys.JOB_PRIORITY);
+    assertTrue("Unknown priority for the job in history file",
+               (priority.equals("HIGH") ||
+                priority.equals("LOW")  || priority.equals("NORMAL") ||
+                priority.equals("VERY_HIGH") || priority.equals("VERY_LOW")));
+  }
+
+  // Validate Format of Task Level Keys, Values read from history file
+  private static void validateTaskLevelKeyValuesFormat(JobInfo job,
+                                  boolean splitsCanBeEmpty) {
+    Map<String, JobHistory.Task> tasks = job.getAllTasks();
+
+    // validate info of each task
+    for (JobHistory.Task task : tasks.values()) {
+
+      String tid = task.get(Keys.TASKID);
+      String time = task.get(Keys.START_TIME);
+      // We allow START_TIME=0 for tasks seen in history after JT restart
+      if (!ignoreStartTimeOfTasks.contains(tid) || (Long.parseLong(time) != 0)) {
+        assertTrue("Task START_TIME of " + tid + " is in unexpected format:" +
+                 time + " in history file", isTimeValid(time));
+      }
+
+      String time1 = task.get(Keys.FINISH_TIME);
+      assertTrue("Task FINISH_TIME of " + tid + " is in unexpected format:" +
+                 time1 + " in history file", isTimeValid(time1));
+      assertTrue("Task FINISH_TIME is < START_TIME in history file",
+                 areTimesInOrder(time, time1));
+
+      // Make sure that the Task type exists and it is valid
+      String type = task.get(Keys.TASK_TYPE);
+      assertTrue("Unknown Task type \"" + type + "\" is seen in " +
+                 "history file for task " + tid,
+                 (type.equals("MAP") || type.equals("REDUCE") ||
+                  type.equals("SETUP") || type.equals("CLEANUP")));
+
+      if (type.equals("MAP")) {
+        String splits = task.get(Keys.SPLITS);
+        //order in the condition OR check is important here
+        if (!splitsCanBeEmpty || splits.length() != 0) {
+          Matcher m = splitsPattern.matcher(splits);
+          assertTrue("Unexpected format of SPLITS \"" + splits + "\" is seen" +
+                     " in history file for task " + tid, m.matches());
+        }
+      }
+
+      // Validate task status
+      String status = task.get(Keys.TASK_STATUS);
+      assertTrue("Unexpected TASK_STATUS \"" + status + "\" is seen in" +
+                 " history file for task " + tid, (status.equals("SUCCESS") ||
+                 status.equals("FAILED") || status.equals("KILLED")));
+    }
+  }
+
+  // Validate foramt of Task Attempt Level Keys, Values read from history file
+  private static void validateTaskAttemptLevelKeyValuesFormat(JobInfo job) {
+    Map<String, JobHistory.Task> tasks = job.getAllTasks();
+
+    // For each task
+    for (JobHistory.Task task : tasks.values()) {
+      // validate info of each attempt
+      for (JobHistory.TaskAttempt attempt : task.getTaskAttempts().values()) {
+
+        String id = attempt.get(Keys.TASK_ATTEMPT_ID);
+        String time = attempt.get(Keys.START_TIME);
+        assertTrue("START_TIME of task attempt " + id +
+                   " is in unexpected format:" + time +
+                   " in history file", isTimeValid(time));
+
+        String time1 = attempt.get(Keys.FINISH_TIME);
+        assertTrue("FINISH_TIME of task attempt " + id +
+                   " is in unexpected format:" + time1 +
+                   " in history file", isTimeValid(time1));
+        assertTrue("Task FINISH_TIME is < START_TIME in history file",
+                   areTimesInOrder(time, time1));
+
+        // Make sure that the Task type exists and it is valid
+        String type = attempt.get(Keys.TASK_TYPE);
+        assertTrue("Unknown Task type \"" + type + "\" is seen in " +
+                   "history file for task attempt " + id,
+                   (type.equals("MAP") || type.equals("REDUCE") ||
+                    type.equals("SETUP") || type.equals("CLEANUP")));
+
+        // Validate task status
+        String status = attempt.get(Keys.TASK_STATUS);
+        assertTrue("Unexpected TASK_STATUS \"" + status + "\" is seen in" +
+                   " history file for task attempt " + id,
+                   (status.equals("SUCCESS") || status.equals("FAILED") ||
+                    status.equals("KILLED")));
+
+        // Reduce Task Attempts should have valid SHUFFLE_FINISHED time and
+        // SORT_FINISHED time
+        if (type.equals("REDUCE") && status.equals("SUCCESS")) {
+          time1 = attempt.get(Keys.SHUFFLE_FINISHED);
+          assertTrue("SHUFFLE_FINISHED time of task attempt " + id +
+                     " is in unexpected format:" + time1 +
+                     " in history file", isTimeValid(time1));
+          assertTrue("Reduce Task SHUFFLE_FINISHED time is < START_TIME " +
+                     "in history file", areTimesInOrder(time, time1));
+          time = attempt.get(Keys.SORT_FINISHED);
+          assertTrue("SORT_FINISHED of task attempt " + id +
+                     " is in unexpected format:" + time +
+                     " in history file", isTimeValid(time));
+          assertTrue("Reduce Task SORT_FINISHED time is < SORT_FINISHED time" +
+                     " in history file", areTimesInOrder(time1, time));
+        }
+
+        // check if hostname is valid
+        String hostname = attempt.get(Keys.HOSTNAME);
+        Matcher m = hostNamePattern.matcher(hostname);
+        assertTrue("Unexpected Host name of task attempt " + id, m.matches());
+
+        // check if trackername is valid
+        String trackerName = attempt.get(Keys.TRACKER_NAME);
+        m = trackerNamePattern.matcher(trackerName);
+        assertTrue("Unexpected tracker name of task attempt " + id,
+                   m.matches());
+
+        if (!status.equals("KILLED")) {
+          // check if http port is valid
+          String httpPort = attempt.get(Keys.HTTP_PORT);
+          m = digitsPattern.matcher(httpPort);
+          assertTrue("Unexpected http port of task attempt " + id, m.matches());
+        }
+        
+        // check if counters are parsable
+        String counters = attempt.get(Keys.COUNTERS);
+        try {
+          Counters readCounters = Counters.fromEscapedCompactString(counters);
+          assertTrue("Counters of task attempt " + id + " are not parsable",
+                     readCounters != null);
+        } catch (ParseException pe) {
+          LOG.warn("While trying to parse counters of task attempt " + id +
+                   ", " + pe);
+        }
+      }
+    }
+  }
+
+  /**
+   *  Validates the format of contents of history file
+   *  (1) history file exists and in correct location
+   *  (2) Verify if the history file is parsable
+   *  (3) Validate the contents of history file
+   *     (a) Format of all TIMEs are checked against a regex
+   *     (b) validate legality/format of job level key, values
+   *     (c) validate legality/format of task level key, values
+   *     (d) validate legality/format of attempt level key, values
+   *     (e) check if all the TaskAttempts, Tasks started are finished.
+   *         Check finish of each TaskAttemptID against its start to make sure
+   *         that all TaskAttempts, Tasks started are indeed finished and the
+   *         history log lines are in the proper order.
+   *         We want to catch ordering of history lines like
+   *            Task START
+   *            Attempt START
+   *            Task FINISH
+   *            Attempt FINISH
+   *         (speculative execution is turned off for this).
+   * @param id job id
+   * @param conf job conf
+   */
+  static void validateJobHistoryFileFormat(JobID id, JobConf conf,
+                 String status, boolean splitsCanBeEmpty) throws IOException  {
+
+    // Get the history file name
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+
+    // Framework history log file location
+    Path logFile = JobHistory.JobInfo.getJobHistoryLogLocation(logFileName);
+    FileSystem fileSys = logFile.getFileSystem(conf);
+ 
+    // Check if the history file exists
+    assertTrue("History file does not exist", fileSys.exists(logFile));
+
+
+    // check if the history file is parsable
+    String[] jobDetails = JobHistory.JobInfo.decodeJobHistoryFileName(
+    		                                   logFileName).split("_");
+
+    String jobId = jobDetails[2] + "_" + jobDetails[3] + "_" + jobDetails[4];
+    JobHistory.JobInfo jobInfo = new JobHistory.JobInfo(jobId);
+
+    TestListener l = new TestListener(jobInfo);
+    JobHistory.parseHistoryFromFS(logFile.toString().substring(5), l, fileSys);
+
+
+    // validate format of job level key, values
+    validateJobLevelKeyValuesFormat(jobInfo.getValues(), status);
+
+    // validate format of task level key, values
+    validateTaskLevelKeyValuesFormat(jobInfo, splitsCanBeEmpty);
+
+    // validate format of attempt level key, values
+    validateTaskAttemptLevelKeyValuesFormat(jobInfo);
+
+    // check if all the TaskAttempts, Tasks started are finished for
+    // successful jobs
+    if (status.equals("SUCCESS")) {
+      // Make sure that the lists in taskIDsToAttemptIDs are empty.
+      for(Iterator<String> it = taskIDsToAttemptIDs.keySet().iterator();it.hasNext();) {
+        String taskid = it.next();
+        assertTrue("There are some Tasks which are not finished in history " +
+                   "file.", taskEnds.contains(taskid));
+        List<String> attemptIDs = taskIDsToAttemptIDs.get(taskid);
+        if(attemptIDs != null) {
+          assertTrue("Unexpected. TaskID " + taskid + " has task attempt(s)" +
+                     " that are not finished.", (attemptIDs.size() == 1));
+        }
+      }
+    }
+  }
+
+  // Validate Job Level Keys, Values read from history file by
+  // comparing them with the actual values from JT.
+  private static void validateJobLevelKeyValues(MiniMRCluster mr,
+          RunningJob job, JobInfo jobInfo, JobConf conf) throws IOException  {
+
+    JobTracker jt = mr.getJobTrackerRunner().getJobTracker();
+    JobInProgress jip = jt.getJob(job.getID());
+
+    Map<Keys, String> values = jobInfo.getValues();
+
+    assertTrue("SUBMIT_TIME of job obtained from history file did not " +
+               "match the expected value", jip.getStartTime() ==
+               Long.parseLong(values.get(Keys.SUBMIT_TIME)));
+
+    assertTrue("LAUNCH_TIME of job obtained from history file did not " +
+               "match the expected value", jip.getLaunchTime() ==
+               Long.parseLong(values.get(Keys.LAUNCH_TIME)));
+
+    assertTrue("FINISH_TIME of job obtained from history file did not " +
+               "match the expected value", jip.getFinishTime() ==
+               Long.parseLong(values.get(Keys.FINISH_TIME)));
+
+    assertTrue("Job Status of job obtained from history file did not " +
+               "match the expected value",
+               values.get(Keys.JOB_STATUS).equals("SUCCESS"));
+
+    assertTrue("Job Priority of job obtained from history file did not " +
+               "match the expected value", jip.getPriority().toString().equals(
+               values.get(Keys.JOB_PRIORITY)));
+
+    assertTrue("Job Name of job obtained from history file did not " +
+               "match the expected value", JobInfo.getJobName(conf).equals(
+               values.get(Keys.JOBNAME)));
+
+    assertTrue("User Name of job obtained from history file did not " +
+               "match the expected value", JobInfo.getUserName(conf).equals(
+               values.get(Keys.USER)));
+
+    // Validate job counters
+    Counters c = jip.getCounters();
+    assertTrue("Counters of job obtained from history file did not " +
+               "match the expected value",
+               c.makeEscapedCompactString().equals(values.get(Keys.COUNTERS)));
+
+    // Validate number of total maps, total reduces, finished maps,
+    // finished reduces, failed maps, failed recudes
+    String totalMaps = values.get(Keys.TOTAL_MAPS);
+    assertTrue("Unexpected number of total maps in history file",
+               Integer.parseInt(totalMaps) == jip.desiredMaps());
+
+    String totalReduces = values.get(Keys.TOTAL_REDUCES);
+    assertTrue("Unexpected number of total reduces in history file",
+               Integer.parseInt(totalReduces) == jip.desiredReduces());
+
+    String finMaps = values.get(Keys.FINISHED_MAPS);
+    assertTrue("Unexpected number of finished maps in history file",
+               Integer.parseInt(finMaps) == jip.finishedMaps());
+
+    String finReduces = values.get(Keys.FINISHED_REDUCES);
+    assertTrue("Unexpected number of finished reduces in history file",
+               Integer.parseInt(finReduces) == jip.finishedReduces());
+
+    String failedMaps = values.get(Keys.FAILED_MAPS);
+    assertTrue("Unexpected number of failed maps in history file",
+               Integer.parseInt(failedMaps) == jip.failedMapTasks);
+
+    String failedReduces = values.get(Keys.FAILED_REDUCES);
+    assertTrue("Unexpected number of failed reduces in history file",
+               Integer.parseInt(failedReduces) == jip.failedReduceTasks);
+  }
+
+  // Validate Task Level Keys, Values read from history file by
+  // comparing them with the actual values from JT.
+  private static void validateTaskLevelKeyValues(MiniMRCluster mr,
+                      RunningJob job, JobInfo jobInfo) throws IOException  {
+
+    JobTracker jt = mr.getJobTrackerRunner().getJobTracker();
+    JobInProgress jip = jt.getJob(job.getID());
+
+    // Get the 1st map, 1st reduce, cleanup & setup taskIDs and
+    // validate their history info
+    TaskID mapTaskId = new TaskID(job.getID(), true, 0);
+    TaskID reduceTaskId = new TaskID(job.getID(), false, 0);
+
+    TaskInProgress cleanups[] = jip.getCleanupTasks();
+    TaskID cleanupTaskId;
+    if (cleanups[0].isComplete()) {
+      cleanupTaskId = cleanups[0].getTIPId();
+    }
+    else {
+      cleanupTaskId = cleanups[1].getTIPId();
+    }
+
+    TaskInProgress setups[] = jip.getSetupTasks();
+    TaskID setupTaskId;
+    if (setups[0].isComplete()) {
+      setupTaskId = setups[0].getTIPId();
+    }
+    else {
+      setupTaskId = setups[1].getTIPId();
+    }
+
+    Map<String, JobHistory.Task> tasks = jobInfo.getAllTasks();
+
+    // validate info of the 4 tasks(cleanup, setup, 1st map, 1st reduce)
+    for (JobHistory.Task task : tasks.values()) {
+
+      String tid = task.get(Keys.TASKID);
+      if (tid.equals(mapTaskId.toString()) ||
+          tid.equals(reduceTaskId.toString()) ||
+          tid.equals(cleanupTaskId.toString()) ||
+          tid.equals(setupTaskId.toString())) {
+
+        TaskID taskId = null;
+        if (tid.equals(mapTaskId.toString())) {
+          taskId = mapTaskId;
+        }
+        else if (tid.equals(reduceTaskId.toString())) {
+          taskId = reduceTaskId;
+        }
+        else if (tid.equals(cleanupTaskId.toString())) {
+          taskId = cleanupTaskId;
+        }
+        else if (tid.equals(setupTaskId.toString())) {
+          taskId = setupTaskId;
+        }
+        TaskInProgress tip = jip.getTaskInProgress(taskId);
+        assertTrue("START_TIME of Task " + tid + " obtained from history " +
+             "file did not match the expected value", tip.getExecStartTime() ==
+             Long.parseLong(task.get(Keys.START_TIME)));
+
+        assertTrue("FINISH_TIME of Task " + tid + " obtained from history " +
+             "file did not match the expected value", tip.getExecFinishTime() ==
+             Long.parseLong(task.get(Keys.FINISH_TIME)));
+
+        if (taskId == mapTaskId) {//check splits only for map task
+          assertTrue("Splits of Task " + tid + " obtained from history file " +
+                     " did not match the expected value",
+                     tip.getSplitNodes().equals(task.get(Keys.SPLITS)));
+        }
+
+        TaskAttemptID attemptId = tip.getSuccessfulTaskid();
+        TaskStatus ts = tip.getTaskStatus(attemptId);
+
+        // Validate task counters
+        Counters c = ts.getCounters();
+        assertTrue("Counters of Task " + tid + " obtained from history file " +
+                   " did not match the expected value",
+                  c.makeEscapedCompactString().equals(task.get(Keys.COUNTERS)));
+      }
+    }
+  }
+
+  // Validate Task Attempt Level Keys, Values read from history file by
+  // comparing them with the actual values from JT.
+  private static void validateTaskAttemptLevelKeyValues(MiniMRCluster mr,
+                      RunningJob job, JobInfo jobInfo) throws IOException  {
+
+    JobTracker jt = mr.getJobTrackerRunner().getJobTracker();
+    JobInProgress jip = jt.getJob(job.getID());
+
+    Map<String, JobHistory.Task> tasks = jobInfo.getAllTasks();
+
+    // For each task
+    for (JobHistory.Task task : tasks.values()) {
+      // validate info of each attempt
+      for (JobHistory.TaskAttempt attempt : task.getTaskAttempts().values()) {
+
+        String idStr = attempt.get(Keys.TASK_ATTEMPT_ID);
+        TaskAttemptID attemptId = TaskAttemptID.forName(idStr);
+        TaskID tid = attemptId.getTaskID();
+
+        // Validate task id
+        assertTrue("Task id of Task Attempt " + idStr + " obtained from " +
+                   "history file did not match the expected value",
+                   tid.toString().equals(attempt.get(Keys.TASKID)));
+
+        TaskInProgress tip = jip.getTaskInProgress(tid);
+        TaskStatus ts = tip.getTaskStatus(attemptId);
+
+        // Validate task attempt start time
+        assertTrue("START_TIME of Task attempt " + idStr + " obtained from " +
+                   "history file did not match the expected value",
+            ts.getStartTime() == Long.parseLong(attempt.get(Keys.START_TIME)));
+
+        // Validate task attempt finish time
+        assertTrue("FINISH_TIME of Task attempt " + idStr + " obtained from " +
+                   "history file did not match the expected value",
+            ts.getFinishTime() == Long.parseLong(attempt.get(Keys.FINISH_TIME)));
+
+
+        TaskTrackerStatus ttStatus = jt.getTaskTracker(ts.getTaskTracker());
+
+        if (ttStatus != null) {
+          assertTrue("http port of task attempt " + idStr + " obtained from " +
+                     "history file did not match the expected value",
+                     ttStatus.getHttpPort() ==
+                     Integer.parseInt(attempt.get(Keys.HTTP_PORT)));
+
+          if (attempt.get(Keys.TASK_STATUS).equals("SUCCESS")) {
+            String ttHostname = jt.getNode(ttStatus.getHost()).toString();
+
+            // check if hostname is valid
+            assertTrue("Host name of task attempt " + idStr + " obtained from" +
+                       " history file did not match the expected value",
+                       ttHostname.equals(attempt.get(Keys.HOSTNAME)));
+          }
+        }
+        if (attempt.get(Keys.TASK_STATUS).equals("SUCCESS")) {
+          // Validate SHUFFLE_FINISHED time and SORT_FINISHED time of
+          // Reduce Task Attempts
+          if (attempt.get(Keys.TASK_TYPE).equals("REDUCE")) {
+            assertTrue("SHUFFLE_FINISHED time of task attempt " + idStr +
+                     " obtained from history file did not match the expected" +
+                     " value", ts.getShuffleFinishTime() ==
+                     Long.parseLong(attempt.get(Keys.SHUFFLE_FINISHED)));
+            assertTrue("SORT_FINISHED time of task attempt " + idStr +
+                     " obtained from history file did not match the expected" +
+                     " value", ts.getSortFinishTime() ==
+                     Long.parseLong(attempt.get(Keys.SORT_FINISHED)));
+          }
+
+          //Validate task counters
+          Counters c = ts.getCounters();
+          assertTrue("Counters of Task Attempt " + idStr + " obtained from " +
+                     "history file did not match the expected value",
+               c.makeEscapedCompactString().equals(attempt.get(Keys.COUNTERS)));
+        }
+        
+        // check if tracker name is valid
+        assertTrue("Tracker name of task attempt " + idStr + " obtained from " +
+                   "history file did not match the expected value",
+                   ts.getTaskTracker().equals(attempt.get(Keys.TRACKER_NAME)));
+      }
+    }
+  }
+
+  /**
+   * Checks if the history file content is as expected comparing with the
+   * actual values obtained from JT.
+   * Job Level, Task Level and Task Attempt Level Keys, Values are validated.
+   * @param job RunningJob object of the job whose history is to be validated
+   * @param conf job conf
+   */
+  static void validateJobHistoryFileContent(MiniMRCluster mr,
+                              RunningJob job, JobConf conf) throws IOException  {
+
+    JobID id = job.getID();
+    // Get the history file name
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+
+    // Framework history log file location
+    Path logFile = JobHistory.JobInfo.getJobHistoryLogLocation(logFileName);
+    FileSystem fileSys = logFile.getFileSystem(conf);
+ 
+    // Check if the history file exists
+    assertTrue("History file does not exist", fileSys.exists(logFile));
+
+
+    // check if the history file is parsable
+    String[] jobDetails = JobHistory.JobInfo.decodeJobHistoryFileName(
+    		                                   logFileName).split("_");
+
+    String jobId = jobDetails[2] + "_" + jobDetails[3] + "_" + jobDetails[4];
+    JobHistory.JobInfo jobInfo = new JobHistory.JobInfo(jobId);
+
+    DefaultJobHistoryParser.JobTasksParseListener l =
+                   new DefaultJobHistoryParser.JobTasksParseListener(jobInfo);
+    JobHistory.parseHistoryFromFS(logFile.toString().substring(5), l, fileSys);
+
+    // Now the history file contents are available in jobInfo. Let us compare
+    // them with the actual values from JT.
+    validateJobLevelKeyValues(mr, job, jobInfo, conf);
+    validateTaskLevelKeyValues(mr, job, jobInfo);
+    validateTaskAttemptLevelKeyValues(mr, job, jobInfo);
+  }
+
+  /** Run a job that will be succeeded and validate its history file format
+   *  and its content.
+   */
+  public void testJobHistoryFile() throws IOException {
+    MiniMRCluster mr = null;
+    try {
+      mr = new MiniMRCluster(2, "file:///", 3);
+
+      // run the TCs
+      JobConf conf = mr.createJobConf();
+
+      FileSystem fs = FileSystem.get(conf);
+      // clean up
+      fs.delete(new Path(TEST_ROOT_DIR + "/succeed"), true);
+
+      Path inDir = new Path(TEST_ROOT_DIR + "/succeed/input");
+      Path outDir = new Path(TEST_ROOT_DIR + "/succeed/output");
+
+      //Disable speculative execution
+      conf.setSpeculativeExecution(false);
+
+      // Make sure that the job is not removed from memory until we do finish
+      // the validation of history file content
+      conf.setInt("mapred.jobtracker.completeuserjobs.maximum", 10);
+
+      // Run a job that will be succeeded and validate its history file
+      RunningJob job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryFileFormat(job.getID(), conf, "SUCCESS", false);
+      validateJobHistoryFileContent(mr, job, conf);
+
+    } finally {
+      if (mr != null) {
+        mr.shutdown();
+      }
+    }
+  }
+
+  // Returns the output path where user history log file is written to with
+  // default configuration setting for hadoop.job.history.user.location
+  private static Path getLogLocationInOutputPath(String logFileName,
+                                                      JobConf conf) {
+    JobConf jobConf = new JobConf(true);//default JobConf
+    FileOutputFormat.setOutputPath(jobConf,
+                     FileOutputFormat.getOutputPath(conf));
+    return JobHistory.JobInfo.getJobHistoryLogLocationForUser(
+                                             logFileName, jobConf);
+  }
+
+  /**
+   * Checks if the user history file exists in the correct dir
+   * @param id job id
+   * @param conf job conf
+   */
+  private static void validateJobHistoryUserLogLocation(JobID id, JobConf conf) 
+          throws IOException  {
+    // Get the history file name
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+
+    // User history log file location
+    Path logFile = JobHistory.JobInfo.getJobHistoryLogLocationForUser(
+                                                     logFileName, conf);
+    if(logFile == null) {
+      // get the output path where history file is written to when
+      // hadoop.job.history.user.location is not set
+      logFile = getLogLocationInOutputPath(logFileName, conf);
+    }
+    FileSystem fileSys = null;
+    fileSys = logFile.getFileSystem(conf);
+
+    // Check if the user history file exists in the correct dir
+    if (conf.get("hadoop.job.history.user.location") == null) {
+      assertTrue("User log file " + logFile + " does not exist",
+                 fileSys.exists(logFile));
+    }
+    else if (conf.get("hadoop.job.history.user.location") == "none") {
+      // history file should not exist in the output path
+      assertFalse("Unexpected. User log file exists in output dir when " +
+                 "hadoop.job.history.user.location is set to \"none\"",
+                 fileSys.exists(logFile));
+    }
+    else {
+      //hadoop.job.history.user.location is set to a specific location.
+      // User log file should exist in that location
+      assertTrue("User log file " + logFile + " does not exist",
+                 fileSys.exists(logFile));
+
+      // User log file should not exist in output path.
+
+      // get the output path where history file is written to when
+      // hadoop.job.history.user.location is not set
+      Path logFile1 = getLogLocationInOutputPath(logFileName, conf);
+      
+      if (logFile != logFile1) {
+        fileSys = logFile1.getFileSystem(conf);
+        assertFalse("Unexpected. User log file exists in output dir when " +
+              "hadoop.job.history.user.location is set to a different location",
+              fileSys.exists(logFile1));
+      }
+    }
+  }
+
+  // Validate user history file location for the given values of
+  // hadoop.job.history.user.location as
+  // (1)null(default case), (2)"none", and (3)some dir "/tmp"
+  public void testJobHistoryUserLogLocation() throws IOException {
+    MiniMRCluster mr = null;
+    try {
+      mr = new MiniMRCluster(2, "file:///", 3);
+
+      // run the TCs
+      JobConf conf = mr.createJobConf();
+
+      FileSystem fs = FileSystem.get(conf);
+      // clean up
+      fs.delete(new Path(TEST_ROOT_DIR + "/succeed"), true);
+
+      Path inDir = new Path(TEST_ROOT_DIR + "/succeed/input1");
+      Path outDir = new Path(TEST_ROOT_DIR + "/succeed/output1");
+
+      // validate for the case of null(default)
+      RunningJob job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryUserLogLocation(job.getID(), conf);
+
+      inDir = new Path(TEST_ROOT_DIR + "/succeed/input2");
+      outDir = new Path(TEST_ROOT_DIR + "/succeed/output2");
+      // validate for the case of "none"
+      conf.set("hadoop.job.history.user.location", "none");
+      job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryUserLogLocation(job.getID(), conf);
+ 
+      inDir = new Path(TEST_ROOT_DIR + "/succeed/input3");
+      outDir = new Path(TEST_ROOT_DIR + "/succeed/output3");
+      // validate for the case of any dir
+      conf.set("hadoop.job.history.user.location", "/tmp");
+      job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryUserLogLocation(job.getID(), conf);
+
+    } finally {
+      if (mr != null) {
+        mr.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Checks if the history file has expected job status
+   * @param id job id
+   * @param conf job conf
+   */
+  private static void validateJobHistoryJobStatus(JobID id, JobConf conf,
+          String status) throws IOException  {
+
+    // Get the history file name
+    String logFileName = JobHistory.JobInfo.getJobHistoryFileName(conf, id);
+
+    // Framework history log file location
+    Path logFile = JobHistory.JobInfo.getJobHistoryLogLocation(logFileName);
+    FileSystem fileSys = logFile.getFileSystem(conf);
+ 
+    // Check if the history file exists
+    assertTrue("History file does not exist", fileSys.exists(logFile));
+
+
+    // check if the history file is parsable
+    String[] jobDetails = JobHistory.JobInfo.decodeJobHistoryFileName(
+    		                                   logFileName).split("_");
+
+    String jobId = jobDetails[2] + "_" + jobDetails[3] + "_" + jobDetails[4];
+    JobHistory.JobInfo jobInfo = new JobHistory.JobInfo(jobId);
+
+    DefaultJobHistoryParser.JobTasksParseListener l =
+                  new DefaultJobHistoryParser.JobTasksParseListener(jobInfo);
+    JobHistory.parseHistoryFromFS(logFile.toString().substring(5), l, fileSys);
+
+    assertTrue("Job Status read from job history file is not the expected" +
+         " status", status.equals(jobInfo.getValues().get(Keys.JOB_STATUS)));
+  }
+
+  // run jobs that will be (1) succeeded (2) failed (3) killed
+  // and validate job status read from history file in each case
+  public void testJobHistoryJobStatus() throws IOException {
+    MiniMRCluster mr = null;
+    try {
+      mr = new MiniMRCluster(2, "file:///", 3);
+
+      // run the TCs
+      JobConf conf = mr.createJobConf();
+
+      FileSystem fs = FileSystem.get(conf);
+      // clean up
+      fs.delete(new Path(TEST_ROOT_DIR + "/succeedfailkilljob"), true);
+
+      Path inDir = new Path(TEST_ROOT_DIR + "/succeedfailkilljob/input");
+      Path outDir = new Path(TEST_ROOT_DIR + "/succeedfailkilljob/output");
+
+      // Run a job that will be succeeded and validate its job status
+      // existing in history file
+      RunningJob job = UtilsForTests.runJobSucceed(conf, inDir, outDir);
+      validateJobHistoryJobStatus(job.getID(), conf, "SUCCESS");
+      
+      // Run a job that will be failed and validate its job status
+      // existing in history file
+      job = UtilsForTests.runJobFail(conf, inDir, outDir);
+      validateJobHistoryJobStatus(job.getID(), conf, "FAILED");
+      
+      // Run a job that will be killed and validate its job status
+      // existing in history file
+      job = UtilsForTests.runJobKill(conf, inDir, outDir);
+      validateJobHistoryJobStatus(job.getID(), conf, "KILLED");
+      
+    } finally {
+      if (mr != null) {
+        mr.shutdown();
+      }
+    }
+  }
+}

+ 33 - 49
src/test/org/apache/hadoop/mapred/TestJobHistoryVersion.java

@@ -27,6 +27,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobHistory.JobInfo;
 import org.apache.hadoop.mapred.JobHistory.JobInfo;
 import org.apache.hadoop.mapred.JobHistory.RecordTypes;
 import org.apache.hadoop.mapred.JobHistory.RecordTypes;
 
 
+/**
+ * Tests the JobHistory parser with different versions of job history files.
+ * This may have to change when new versions of job history files come up.
+ */
 public class TestJobHistoryVersion extends TestCase {
 public class TestJobHistoryVersion extends TestCase {
   private static final String HOSTNAME = "localhost";
   private static final String HOSTNAME = "localhost";
   private static final String TIME= "1234567890123";
   private static final String TIME= "1234567890123";
@@ -48,13 +52,16 @@ public class TestJobHistoryVersion extends TestCase {
              "test-history-version");
              "test-history-version");
   private static final String DELIM = ".";
   private static final String DELIM = ".";
   
   
-  
-  private void writeHistoryFile(FSDataOutputStream out, boolean old)
+  /**
+   * Creates a job history file of a given specific version. This method should
+   * change if format/content of future versions of job history file changes.
+   */
+  private void writeHistoryFile(FSDataOutputStream out, long version)
   throws IOException {
   throws IOException {
-    String delim = "\n";
+    String delim = "\n"; // '\n' for version 0
     String counters = COUNTERS;
     String counters = COUNTERS;
     String jobConf = "job.xml";
     String jobConf = "job.xml";
-    if (!old) {
+    if (version > 0) { // line delimeter should be '.' for later versions
       // Change the delimiter
       // Change the delimiter
       delim = DELIM + delim;
       delim = DELIM + delim;
       
       
@@ -111,58 +118,35 @@ public class TestJobHistoryVersion extends TestCase {
   }
   }
   
   
   /**
   /**
-   * Tests the JobHistory parser with old files
+   * Tests the JobHistory parser with different versions of job history files
    */
    */
-  public void testJobHistoryWithoutVersion() throws IOException {
-    JobConf conf = new JobConf();
-    FileSystem fs = FileSystem.getLocal(conf);
+  public void testJobHistoryVersion() throws IOException {
+    // If new job history version comes up, the modified parser may fail for
+    // the history file created by writeHistoryFile().
+    for (long version = 0; version <= JobHistory.VERSION; version++) {
+      JobConf conf = new JobConf();
+      FileSystem fs = FileSystem.getLocal(conf);
     
     
-    // cleanup
-    fs.delete(TEST_DIR, true);
+      // cleanup
+      fs.delete(TEST_DIR, true);
     
     
-    Path historyPath = new Path(TEST_DIR + "/_logs/history/" + FILENAME);
+      Path historyPath = new Path(TEST_DIR + "/_logs/history/" +
+                                  FILENAME + version);
     
     
-    fs.delete(historyPath, false);
+      fs.delete(historyPath, false);
     
     
-    FSDataOutputStream out = fs.create(historyPath);
-    writeHistoryFile(out, true);
-    out.close();
+      FSDataOutputStream out = fs.create(historyPath);
+      writeHistoryFile(out, version);
+      out.close();
     
     
-    JobInfo job = new JobHistory.JobInfo(JOB); 
-    DefaultJobHistoryParser.parseJobTasks(historyPath.toString(), job, fs);
+      JobInfo job = new JobHistory.JobInfo(JOB); 
+      DefaultJobHistoryParser.parseJobTasks(historyPath.toString(), job, fs);
     
     
-    assertTrue("Failed to parse old jobhistory files", 
-               job.getAllTasks().size() > 0);
+      assertTrue("Failed to parse jobhistory files of version " + version,
+                 job.getAllTasks().size() > 0);
     
     
-    // cleanup
-    fs.delete(TEST_DIR, true);
-  }
-  
-  /**
-   * Tests the JobHistory parser with new file
-   */
-  public void testJobHistoryWithVersion() throws IOException {
-    JobConf conf = new JobConf();
-    FileSystem fs = FileSystem.getLocal(conf);
-    
-    // cleanup
-    fs.delete(TEST_DIR, true);
-    
-    Path historyPath = new Path(TEST_DIR + "/_logs/history/" + FILENAME);
-    
-    fs.delete(historyPath, false);
-    
-    FSDataOutputStream out = fs.create(historyPath);
-    writeHistoryFile(out, false);
-    out.close();
-    
-    JobInfo job = new JobHistory.JobInfo(JOB); 
-    DefaultJobHistoryParser.parseJobTasks(historyPath.toString(), job, fs);
-    
-    assertTrue("Failed to parse old jobhistory files", 
-               job.getAllTasks().size() > 0);
-    
-    // cleanup
-    fs.delete(TEST_DIR, true);
+      // cleanup
+      fs.delete(TEST_DIR, true);
+    }
   }
   }
 }
 }

+ 24 - 6
src/test/org/apache/hadoop/mapred/TestJobInProgressListener.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.List;
 import java.util.List;
 
 
@@ -39,6 +40,9 @@ public class TestJobInProgressListener extends TestCase {
     LogFactory.getLog(TestJobInProgressListener.class);
     LogFactory.getLog(TestJobInProgressListener.class);
   private final Path testDir = new Path("test-jip-listener-update");
   private final Path testDir = new Path("test-jip-listener-update");
   
   
+  private static String TEST_ROOT_DIR = new File(System.getProperty(
+          "test.build.data", "/tmp")).toURI().toString().replace(' ', '+');
+
   private JobConf configureJob(JobConf conf, int m, int r, 
   private JobConf configureJob(JobConf conf, int m, int r, 
                                Path inDir, Path outputDir,
                                Path inDir, Path outputDir,
                                String mapSignalFile, String redSignalFile) 
                                String mapSignalFile, String redSignalFile) 
@@ -267,9 +271,13 @@ public class TestJobInProgressListener extends TestCase {
     
     
     mr.getJobTrackerRunner().getJobTracker()
     mr.getJobTrackerRunner().getJobTracker()
       .addJobInProgressListener(myListener);
       .addJobInProgressListener(myListener);
-    
-    // submit and kill the job   
-    JobID id = TestJobKillAndFail.runJobFail(job);
+
+    Path inDir = new Path(TEST_ROOT_DIR + "/jiplistenerfailjob/input");
+    Path outDir = new Path(TEST_ROOT_DIR + "/jiplistenerfailjob/output");
+
+    // submit a job that fails 
+    RunningJob rJob = UtilsForTests.runJobFail(job, inDir, outDir);
+    JobID id = rJob.getID();
 
 
     // check if the job failure was notified
     // check if the job failure was notified
     assertFalse("Missing event notification on failing a running job", 
     assertFalse("Missing event notification on failing a running job", 
@@ -288,8 +296,12 @@ public class TestJobInProgressListener extends TestCase {
     mr.getJobTrackerRunner().getJobTracker()
     mr.getJobTrackerRunner().getJobTracker()
       .addJobInProgressListener(myListener);
       .addJobInProgressListener(myListener);
     
     
+    Path inDir = new Path(TEST_ROOT_DIR + "/jiplistenerkilljob/input");
+    Path outDir = new Path(TEST_ROOT_DIR + "/jiplistenerkilljob/output");
+
     // submit and kill the job   
     // submit and kill the job   
-    JobID id = TestJobKillAndFail.runJobKill(job);
+    RunningJob rJob = UtilsForTests.runJobKill(job, inDir, outDir);
+    JobID id = rJob.getID();
 
 
     // check if the job failure was notified
     // check if the job failure was notified
     assertFalse("Missing event notification on killing a running job", 
     assertFalse("Missing event notification on killing a running job", 
@@ -308,8 +320,11 @@ public class TestJobInProgressListener extends TestCase {
     mr.getJobTrackerRunner().getJobTracker()
     mr.getJobTrackerRunner().getJobTracker()
       .addJobInProgressListener(myListener);
       .addJobInProgressListener(myListener);
     
     
+    Path inDir = new Path(TEST_ROOT_DIR + "/jiplistenerjob/input");
+    Path outDir = new Path(TEST_ROOT_DIR + "/jiplistenerjob/output");
+
     // submit the job   
     // submit the job   
-    RunningJob rJob = TestJobKillAndFail.runJob(job);
+    RunningJob rJob = UtilsForTests.runJob(job, inDir, outDir);
     
     
     // wait for the job to be running
     // wait for the job to be running
     while (rJob.getJobState() != JobStatus.RUNNING) {
     while (rJob.getJobState() != JobStatus.RUNNING) {
@@ -363,7 +378,10 @@ public class TestJobInProgressListener extends TestCase {
     mr.getJobTrackerRunner().getJobTracker()
     mr.getJobTrackerRunner().getJobTracker()
       .addJobInProgressListener(myListener);
       .addJobInProgressListener(myListener);
     
     
-    RunningJob rJob = TestJobKillAndFail.runJob(job);
+    Path inDir = new Path(TEST_ROOT_DIR + "/jiplistenerjob/input");
+    Path outDir = new Path(TEST_ROOT_DIR + "/jiplistenerjob/output");
+
+    RunningJob rJob = UtilsForTests.runJob(job, inDir, outDir);
     JobID id = rJob.getID();
     JobID id = rJob.getID();
     LOG.info("Job : " + id.toString() + " submitted");
     LOG.info("Job : " + id.toString() + " submitted");
     
     

+ 11 - 114
src/test/org/apache/hadoop/mapred/TestJobKillAndFail.java

@@ -18,18 +18,12 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import java.io.DataOutputStream;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
 
 
 /**
 /**
  * A JUnit test to test Kill Job & Fail Job functionality with local file
  * A JUnit test to test Kill Job & Fail Job functionality with local file
@@ -40,86 +34,6 @@ public class TestJobKillAndFail extends TestCase {
   private static String TEST_ROOT_DIR = new File(System.getProperty(
   private static String TEST_ROOT_DIR = new File(System.getProperty(
       "test.build.data", "/tmp")).toURI().toString().replace(' ', '+');
       "test.build.data", "/tmp")).toURI().toString().replace(' ', '+');
 
 
-  static JobID runJobFail(JobConf conf) throws IOException {
-
-    conf.setJobName("testjobfail");
-    conf.setMapperClass(FailMapper.class);
-
-    RunningJob job = runJob(conf);
-    while (!job.isComplete()) {
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        break;
-      }
-    }
-    // Checking that the Job got failed
-    assertEquals(job.getJobState(), JobStatus.FAILED);
-    
-    return job.getID();
-  }
-
-  static JobID runJobKill(JobConf conf) throws IOException {
-
-    conf.setJobName("testjobkill");
-    conf.setMapperClass(KillMapper.class);
-
-    RunningJob job = runJob(conf);
-    while (job.getJobState() != JobStatus.RUNNING) {
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        break;
-      }
-    }
-    job.killJob();
-    while (job.cleanupProgress() == 0.0f) {
-      try {
-        Thread.sleep(10);
-      } catch (InterruptedException ie) {
-        break;
-      }
-    }
-    // Checking that the Job got killed
-    assertTrue(job.isComplete());
-    assertEquals(job.getJobState(), JobStatus.KILLED);
-    
-    return job.getID();
-  }
-
-  static RunningJob runJob(JobConf conf) throws IOException {
-
-    final Path inDir = new Path(TEST_ROOT_DIR + "/failkilljob/input");
-    final Path outDir = new Path(TEST_ROOT_DIR + "/failkilljob/output");
-
-    // run the dummy sleep map
-    FileSystem fs = FileSystem.get(conf);
-    fs.delete(outDir, true);
-    if (!fs.exists(inDir)) {
-      fs.mkdirs(inDir);
-    }
-    String input = "The quick brown fox\n" + "has many silly\n"
-        + "red fox sox\n";
-    DataOutputStream file = fs.create(new Path(inDir, "part-0"));
-    file.writeBytes(input);
-    file.close();
-
-    conf.setInputFormat(TextInputFormat.class);
-    conf.setOutputKeyClass(Text.class);
-    conf.setOutputValueClass(IntWritable.class);
-
-    FileInputFormat.setInputPaths(conf, inDir);
-    FileOutputFormat.setOutputPath(conf, outDir);
-    conf.setNumMapTasks(1);
-    conf.setNumReduceTasks(0);
-
-    JobClient jobClient = new JobClient(conf);
-    RunningJob job = jobClient.submitJob(conf);
-
-    return job;
-
-  }
-
   public void testJobFailAndKill() throws IOException {
   public void testJobFailAndKill() throws IOException {
     MiniMRCluster mr = null;
     MiniMRCluster mr = null;
     try {
     try {
@@ -127,38 +41,21 @@ public class TestJobKillAndFail extends TestCase {
 
 
       // run the TCs
       // run the TCs
       JobConf conf = mr.createJobConf();
       JobConf conf = mr.createJobConf();
-      runJobFail(conf);
-      runJobKill(conf);
+
+      Path inDir = new Path(TEST_ROOT_DIR + "/failkilljob/input");
+      Path outDir = new Path(TEST_ROOT_DIR + "/failkilljob/output");
+      RunningJob job = UtilsForTests.runJobFail(conf, inDir, outDir);
+      // Checking that the Job got failed
+      assertEquals(job.getJobState(), JobStatus.FAILED);
+
+      job = UtilsForTests.runJobKill(conf, inDir, outDir);
+      // Checking that the Job got killed
+      assertTrue(job.isComplete());
+      assertEquals(job.getJobState(), JobStatus.KILLED);
     } finally {
     } finally {
       if (mr != null) {
       if (mr != null) {
         mr.shutdown();
         mr.shutdown();
       }
       }
     }
     }
   }
   }
-
-  static class FailMapper extends MapReduceBase implements
-      Mapper<WritableComparable, Writable, WritableComparable, Writable> {
-
-    public void map(WritableComparable key, Writable value,
-        OutputCollector<WritableComparable, Writable> out, Reporter reporter)
-        throws IOException {
-
-      throw new RuntimeException("failing map");
-    }
-  }
-
-  static class KillMapper extends MapReduceBase implements
-      Mapper<WritableComparable, Writable, WritableComparable, Writable> {
-
-    public void map(WritableComparable key, Writable value,
-        OutputCollector<WritableComparable, Writable> out, Reporter reporter)
-        throws IOException {
-
-      try {
-        Thread.sleep(100000);
-      } catch (InterruptedException e) {
-        // Do nothing
-      }
-    }
-  }
 }
 }

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

@@ -286,6 +286,10 @@ public class TestJobTrackerRestart extends TestCase {
     
     
     testTaskCompletionEvents(jtEvents, trackerEvents, true, 2 * numMaps);
     testTaskCompletionEvents(jtEvents, trackerEvents, true, 2 * numMaps);
     
     
+    // validate the history file
+    TestJobHistory.validateJobHistoryFileFormat(id, newConf, "SUCCESS", true);
+    TestJobHistory.validateJobHistoryFileContent(mr, job, newConf);
+    
     // check if the cluster status is insane
     // check if the cluster status is insane
     ClusterStatus status = jobClient.getClusterStatus();
     ClusterStatus status = jobClient.getClusterStatus();
     assertTrue("Cluster status is insane", 
     assertTrue("Cluster status is insane", 

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

@@ -112,6 +112,10 @@ public class TestJobTrackerRestartWithLostTracker extends TestCase {
       mr.getJobTrackerRunner().getJobTracker().getJob(id).failedMapTasks;
       mr.getJobTrackerRunner().getJobTracker().getJob(id).failedMapTasks;
     assertTrue("Tasks that were run on the lost tracker were not killed", 
     assertTrue("Tasks that were run on the lost tracker were not killed", 
                failedMaps > 0);
                failedMaps > 0);
+
+    // validate the history file
+    TestJobHistory.validateJobHistoryFileFormat(id, job, "SUCCESS", true);
+    TestJobHistory.validateJobHistoryFileContent(mr, rJob, job);
   }
   }
   
   
   public void testRestartWithLostTracker() throws IOException {
   public void testRestartWithLostTracker() throws IOException {

+ 3 - 0
src/test/org/apache/hadoop/mapred/TestLostTracker.java

@@ -100,6 +100,9 @@ public class TestLostTracker extends TestCase {
       testTaskStatuses(taskInProgress.getTaskStatuses());
       testTaskStatuses(taskInProgress.getTaskStatuses());
     }
     }
     
     
+    // validate the history file
+    TestJobHistory.validateJobHistoryFileFormat(id, job, "SUCCESS", true);
+    TestJobHistory.validateJobHistoryFileContent(mr, rJob, job);
   }
   }
   
   
   private void testTaskStatuses(TaskStatus[] tasks) {
   private void testTaskStatuses(TaskStatus[] tasks) {

+ 129 - 0
src/test/org/apache/hadoop/mapred/UtilsForTests.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.mapred;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.text.DecimalFormat;
 import java.text.DecimalFormat;
@@ -37,9 +38,12 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat;
 import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 
 
 /** 
 /** 
@@ -530,4 +534,129 @@ public class UtilsForTests {
       return new RandomRecordReader(((FileSplit) split).getPath());
       return new RandomRecordReader(((FileSplit) split).getPath());
     }
     }
   }
   }
+
+  // Start a job and return its RunningJob object
+  static RunningJob runJob(JobConf conf, Path inDir, Path outDir)
+                    throws IOException {
+
+    FileSystem fs = FileSystem.get(conf);
+    fs.delete(outDir, true);
+    if (!fs.exists(inDir)) {
+      fs.mkdirs(inDir);
+    }
+    String input = "The quick brown fox\n" + "has many silly\n"
+        + "red fox sox\n";
+    DataOutputStream file = fs.create(new Path(inDir, "part-0"));
+    file.writeBytes(input);
+    file.close();
+
+    conf.setInputFormat(TextInputFormat.class);
+    conf.setOutputKeyClass(LongWritable.class);
+    conf.setOutputValueClass(Text.class);
+
+    FileInputFormat.setInputPaths(conf, inDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
+    conf.setNumMapTasks(1);
+    conf.setNumReduceTasks(1);
+
+    JobClient jobClient = new JobClient(conf);
+    RunningJob job = jobClient.submitJob(conf);
+
+    return job;
+  }
+
+  // Run a job that will be succeeded and wait until it completes
+  static RunningJob runJobSucceed(JobConf conf, Path inDir, Path outDir)
+         throws IOException {
+    conf.setJobName("test-job-succeed");
+    conf.setMapperClass(IdentityMapper.class);
+    conf.setReducerClass(IdentityReducer.class);
+    
+    RunningJob job = UtilsForTests.runJob(conf, inDir, outDir);
+    while (!job.isComplete()) {
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+        break;
+      }
+    }
+
+    return job;
+  }
+
+  // Run a job that will be failed and wait until it completes
+  static RunningJob runJobFail(JobConf conf, Path inDir, Path outDir)
+         throws IOException {
+    conf.setJobName("test-job-fail");
+    conf.setMapperClass(FailMapper.class);
+    conf.setReducerClass(IdentityReducer.class);
+    
+    RunningJob job = UtilsForTests.runJob(conf, inDir, outDir);
+    while (!job.isComplete()) {
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+        break;
+      }
+    }
+
+    return job;
+  }
+
+  // Run a job that will be killed and wait until it completes
+  static RunningJob runJobKill(JobConf conf,  Path inDir, Path outDir)
+         throws IOException {
+
+    conf.setJobName("test-job-kill");
+    conf.setMapperClass(KillMapper.class);
+    conf.setReducerClass(IdentityReducer.class);
+    
+    RunningJob job = UtilsForTests.runJob(conf, inDir, outDir);
+    while (job.getJobState() != JobStatus.RUNNING) {
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+        break;
+      }
+    }
+    job.killJob();
+    while (job.cleanupProgress() == 0.0f) {
+      try {
+        Thread.sleep(10);
+      } catch (InterruptedException ie) {
+        break;
+      }
+    }
+
+    return job;
+  }
+
+  // Mapper that fails
+  static class FailMapper extends MapReduceBase implements
+      Mapper<WritableComparable, Writable, WritableComparable, Writable> {
+
+    public void map(WritableComparable key, Writable value,
+        OutputCollector<WritableComparable, Writable> out, Reporter reporter)
+        throws IOException {
+
+      throw new RuntimeException("failing map");
+    }
+  }
+
+  // Mapper that sleeps for a long time.
+  // Used for running a job that will be killed
+  static class KillMapper extends MapReduceBase implements
+      Mapper<WritableComparable, Writable, WritableComparable, Writable> {
+
+    public void map(WritableComparable key, Writable value,
+        OutputCollector<WritableComparable, Writable> out, Reporter reporter)
+        throws IOException {
+
+      try {
+        Thread.sleep(1000000);
+      } catch (InterruptedException e) {
+        // Do nothing
+      }
+    }
+  }
 }
 }