Преглед изворни кода

MAPREDUCE-2598. Fix NPE and UI for JobHistory. (Siddharth Seth via llu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/MR-279@1136225 13f79535-47bb-0310-9956-ffa450edef68
Luke Lu пре 14 година
родитељ
комит
c4b867395a
14 измењених фајлова са 267 додато и 65 уклоњено
  1. 2 0
      mapreduce/CHANGES.txt
  2. 6 3
      mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  3. 35 14
      mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  4. 41 10
      mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
  5. 9 0
      mapreduce/mr-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
  6. 8 1
      mapreduce/mr-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java
  7. 3 0
      mapreduce/mr-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java
  8. 11 2
      mapreduce/mr-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobIndexInfo.java
  9. 92 8
      mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
  10. 21 13
      mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTask.java
  11. 22 8
      mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTaskAttempt.java
  12. 0 2
      mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
  13. 13 1
      mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
  14. 4 3
      mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

+ 2 - 0
mapreduce/CHANGES.txt

@@ -5,6 +5,8 @@ Trunk (unreleased changes)
 
 
     MAPREDUCE-279
     MAPREDUCE-279
 
 
+    MAPREDUCE-2598. Fix NPE and UI for JobHistory. (Siddharth Seth via llu)
+
     Fix NPE when killing/failing already killed/failed tasks. (llu)
     Fix NPE when killing/failing already killed/failed tasks. (llu)
    
    
     Update install instructions with svn unsplit (Thomas Graves via mahadev)
     Update install instructions with svn unsplit (Thomas Graves via mahadev)

+ 6 - 3
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java

@@ -36,8 +36,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
@@ -295,7 +295,8 @@ public class JobHistoryEventHandler extends AbstractService
       throw new IOException(
       throw new IOException(
           "User is null while setting up jobhistory eventwriter");
           "User is null while setting up jobhistory eventwriter");
     }
     }
-    String jobName = TypeConverter.fromYarn(jobId).toString();
+
+    String jobName = context.getJob(jobId).getName();
     EventWriter writer = (oldFi == null) ? null : oldFi.writer;
     EventWriter writer = (oldFi == null) ? null : oldFi.writer;
  
  
     if (writer == null) {
     if (writer == null) {
@@ -402,6 +403,7 @@ public class JobHistoryEventHandler extends AbstractService
           mi.getJobIndexInfo().setNumMaps(jFinishedEvent.getFinishedMaps());
           mi.getJobIndexInfo().setNumMaps(jFinishedEvent.getFinishedMaps());
           mi.getJobIndexInfo().setNumReduces(
           mi.getJobIndexInfo().setNumReduces(
               jFinishedEvent.getFinishedReduces());
               jFinishedEvent.getFinishedReduces());
+          mi.getJobIndexInfo().setJobStatus(JobState.SUCCEEDED.toString());
           closeEventWriter(event.getJobID());
           closeEventWriter(event.getJobID());
         } catch (IOException e) {
         } catch (IOException e) {
           throw new YarnException(e);
           throw new YarnException(e);
@@ -416,6 +418,7 @@ public class JobHistoryEventHandler extends AbstractService
           mi.getJobIndexInfo().setFinishTime(jucEvent.getFinishTime());
           mi.getJobIndexInfo().setFinishTime(jucEvent.getFinishTime());
           mi.getJobIndexInfo().setNumMaps(jucEvent.getFinishedMaps());
           mi.getJobIndexInfo().setNumMaps(jucEvent.getFinishedMaps());
           mi.getJobIndexInfo().setNumReduces(jucEvent.getFinishedReduces());
           mi.getJobIndexInfo().setNumReduces(jucEvent.getFinishedReduces());
+          mi.getJobIndexInfo().setJobStatus(jucEvent.getStatus());
           closeEventWriter(event.getJobID());
           closeEventWriter(event.getJobID());
         } catch (IOException e) {
         } catch (IOException e) {
           throw new YarnException(e);
           throw new YarnException(e);
@@ -570,7 +573,7 @@ public class JobHistoryEventHandler extends AbstractService
       this.historyFile = historyFile;
       this.historyFile = historyFile;
       this.confFile = conf;
       this.confFile = conf;
       this.writer = writer;
       this.writer = writer;
-      this.jobIndexInfo = new JobIndexInfo(submitTime, -1, user, jobName, jobId, -1, -1);
+      this.jobIndexInfo = new JobIndexInfo(submitTime, -1, user, jobName, jobId, -1, -1, null);
       this.jobSummary = new JobSummary();
       this.jobSummary = new JobSummary();
     }
     }
 
 

+ 35 - 14
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -984,10 +984,15 @@ public abstract class TaskAttemptImpl implements
               TaskEventType.T_ATTEMPT_KILLED));
               TaskEventType.T_ATTEMPT_KILLED));
           break;
           break;
       }
       }
+      if (taskAttempt.getLaunchTime() != 0) {
       TaskAttemptUnsuccessfulCompletionEvent tauce = createTaskAttemptUnsuccessfulCompletionEvent(
       TaskAttemptUnsuccessfulCompletionEvent tauce = createTaskAttemptUnsuccessfulCompletionEvent(
           taskAttempt, finalState);
           taskAttempt, finalState);
-      taskAttempt.eventHandler.handle(new JobHistoryEvent(taskAttempt.attemptId
-          .getTaskId().getJobId(), tauce));
+        taskAttempt.eventHandler.handle(new JobHistoryEvent(
+            taskAttempt.attemptId.getTaskId().getJobId(), tauce));
+      } else {
+        LOG.debug("Not generating HistoryFinish event since start event not generated for taskAttempt: "
+            + taskAttempt.getID());
+      }
     }
     }
   }
   }
 
 
@@ -1087,22 +1092,28 @@ public abstract class TaskAttemptImpl implements
   private static class FailedTransition implements
   private static class FailedTransition implements
       SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
       SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
     @Override
     @Override
-    public void transition(TaskAttemptImpl taskAttempt, 
-        TaskAttemptEvent event) {
-      //set the finish time
+    public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
+      // set the finish time
       taskAttempt.setFinishTime();
       taskAttempt.setFinishTime();
+      if (taskAttempt.getLaunchTime() != 0) {
       TaskAttemptUnsuccessfulCompletionEvent tauce = createTaskAttemptUnsuccessfulCompletionEvent(
       TaskAttemptUnsuccessfulCompletionEvent tauce = createTaskAttemptUnsuccessfulCompletionEvent(
           taskAttempt, TaskAttemptState.FAILED);
           taskAttempt, TaskAttemptState.FAILED);
-      taskAttempt.eventHandler.handle(new JobHistoryEvent(taskAttempt.attemptId
-          .getTaskId().getJobId(), tauce));
-//      taskAttempt.logAttemptFinishedEvent(TaskAttemptState.FAILED); Not handling failed map/reduce events.
+        taskAttempt.eventHandler.handle(new JobHistoryEvent(
+            taskAttempt.attemptId.getTaskId().getJobId(), tauce));
+        // taskAttempt.logAttemptFinishedEvent(TaskAttemptState.FAILED); Not
+        // handling failed map/reduce events.
+      }else {
+        LOG.debug("Not generating HistoryFinish event since start event not generated for taskAttempt: "
+            + taskAttempt.getID());
+      }
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
-          taskAttempt.attemptId,
-          TaskEventType.T_ATTEMPT_FAILED));
+          taskAttempt.attemptId, TaskEventType.T_ATTEMPT_FAILED));
     }
     }
   }
   }
 
 
   private void logAttemptFinishedEvent(TaskAttemptState state) {
   private void logAttemptFinishedEvent(TaskAttemptState state) {
+    //Log finished events only if an attempt started.
+    if (getLaunchTime() == 0) return; 
     if (attemptId.getTaskId().getTaskType() == TaskType.MAP) {
     if (attemptId.getTaskId().getTaskType() == TaskType.MAP) {
       MapAttemptFinishedEvent mfe =
       MapAttemptFinishedEvent mfe =
          new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
          new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
@@ -1137,10 +1148,15 @@ public abstract class TaskAttemptImpl implements
       taskAttempt.addDiagnosticInfo("Too Many fetch failures.Failing the attempt");
       taskAttempt.addDiagnosticInfo("Too Many fetch failures.Failing the attempt");
       //set the finish time
       //set the finish time
       taskAttempt.setFinishTime();
       taskAttempt.setFinishTime();
+      if (taskAttempt.getLaunchTime() != 0) {
       TaskAttemptUnsuccessfulCompletionEvent tauce = createTaskAttemptUnsuccessfulCompletionEvent(
       TaskAttemptUnsuccessfulCompletionEvent tauce = createTaskAttemptUnsuccessfulCompletionEvent(
           taskAttempt, TaskAttemptState.FAILED);
           taskAttempt, TaskAttemptState.FAILED);
-      taskAttempt.eventHandler.handle(new JobHistoryEvent(taskAttempt.attemptId
-          .getTaskId().getJobId(), tauce));
+        taskAttempt.eventHandler.handle(new JobHistoryEvent(
+            taskAttempt.attemptId.getTaskId().getJobId(), tauce));
+      }else {
+        LOG.debug("Not generating HistoryFinish event since start event not generated for taskAttempt: "
+            + taskAttempt.getID());
+      }
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
           taskAttempt.attemptId, TaskEventType.T_ATTEMPT_FAILED));
           taskAttempt.attemptId, TaskEventType.T_ATTEMPT_FAILED));
     }
     }
@@ -1154,10 +1170,15 @@ public abstract class TaskAttemptImpl implements
         TaskAttemptEvent event) {
         TaskAttemptEvent event) {
       //set the finish time
       //set the finish time
       taskAttempt.setFinishTime();
       taskAttempt.setFinishTime();
+      if (taskAttempt.getLaunchTime() != 0) {
       TaskAttemptUnsuccessfulCompletionEvent tauce = createTaskAttemptUnsuccessfulCompletionEvent(
       TaskAttemptUnsuccessfulCompletionEvent tauce = createTaskAttemptUnsuccessfulCompletionEvent(
           taskAttempt, TaskAttemptState.KILLED);
           taskAttempt, TaskAttemptState.KILLED);
-      taskAttempt.eventHandler.handle(new JobHistoryEvent(taskAttempt.attemptId
-          .getTaskId().getJobId(), tauce));
+        taskAttempt.eventHandler.handle(new JobHistoryEvent(
+            taskAttempt.attemptId.getTaskId().getJobId(), tauce));
+      }else {
+        LOG.debug("Not generating HistoryFinish event since start event not generated for taskAttempt: "
+            + taskAttempt.getID());
+      }
 //      taskAttempt.logAttemptFinishedEvent(TaskAttemptState.KILLED); Not logging Map/Reduce attempts in case of failure.
 //      taskAttempt.logAttemptFinishedEvent(TaskAttemptState.KILLED); Not logging Map/Reduce attempts in case of failure.
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
       taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
           taskAttempt.attemptId,
           taskAttempt.attemptId,

+ 41 - 10
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Set;
 import java.util.Set;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.Lock;
@@ -108,6 +109,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
   //  they will come to be running when they get a Container
   //  they will come to be running when they get a Container
   private int numberUncompletedAttempts = 0;
   private int numberUncompletedAttempts = 0;
 
 
+  private boolean historyTaskStartGenerated = false;
+  
   private static final SingleArcTransition<TaskImpl, TaskEvent> 
   private static final SingleArcTransition<TaskImpl, TaskEvent> 
      ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
      ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
   private static final SingleArcTransition<TaskImpl, TaskEvent> 
   private static final SingleArcTransition<TaskImpl, TaskEvent> 
@@ -571,7 +574,10 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
       tce.setMapOutputServerAddress("http://" + attempt.getNodeHttpAddress().split(":")[0] + ":8080");
       tce.setMapOutputServerAddress("http://" + attempt.getNodeHttpAddress().split(":")[0] + ":8080");
       tce.setStatus(status);
       tce.setStatus(status);
       tce.setAttemptId(attempt.getID());
       tce.setAttemptId(attempt.getID());
-      tce.setAttemptRunTime(0); // TODO: set the exact run time of the task.
+      int runTime = 0;
+      if (attempt.getFinishTime() != 0 && attempt.getLaunchTime() !=0)
+        runTime = (int)(attempt.getFinishTime() - attempt.getLaunchTime());
+      tce.setAttemptRunTime(runTime);
       
       
       //raise the event to job so that it adds the completion event to its
       //raise the event to job so that it adds the completion event to its
       //data structures
       //data structures
@@ -589,13 +595,19 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
     return tfe;
     return tfe;
   }
   }
   
   
-  private static TaskFailedEvent createTaskFailedEvent(TaskImpl task, String error, TaskState taskState, TaskAttemptId taId) {
+  private static TaskFailedEvent createTaskFailedEvent(TaskImpl task, List<String> diag, TaskState taskState, TaskAttemptId taId) {
+    StringBuilder errorSb = new StringBuilder();
+    if (diag != null) {
+      for (String d : diag) {
+        errorSb.append(", ").append(d);
+      }
+    }
     TaskFailedEvent taskFailedEvent = new TaskFailedEvent(
     TaskFailedEvent taskFailedEvent = new TaskFailedEvent(
         TypeConverter.fromYarn(task.taskId),
         TypeConverter.fromYarn(task.taskId),
      // Hack since getFinishTime needs isFinished to be true and that doesn't happen till after the transition.
      // Hack since getFinishTime needs isFinished to be true and that doesn't happen till after the transition.
         task.getFinishTime(taId),
         task.getFinishTime(taId),
         TypeConverter.fromYarn(task.getType()),
         TypeConverter.fromYarn(task.getType()),
-        error == null ? "" : error,
+        errorSb.toString(),
         taskState.toString(),
         taskState.toString(),
         taId == null ? null : TypeConverter.fromYarn(taId));
         taId == null ? null : TypeConverter.fromYarn(taId));
     return taskFailedEvent;
     return taskFailedEvent;
@@ -615,6 +627,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
               .getTaskSplitMetaInfo().getLocations()) : "");
               .getTaskSplitMetaInfo().getLocations()) : "");
       task.eventHandler
       task.eventHandler
           .handle(new JobHistoryEvent(task.taskId.getJobId(), tse));
           .handle(new JobHistoryEvent(task.taskId.getJobId(), tse));
+      task.historyTaskStartGenerated = true;
       task.metrics.launchedTask(task);
       task.metrics.launchedTask(task);
     }
     }
     
     
@@ -681,8 +694,12 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
           task.taskId, TaskState.SUCCEEDED));
           task.taskId, TaskState.SUCCEEDED));
       LOG.info("Task succeeded with attempt " + task.successfulAttempt);
       LOG.info("Task succeeded with attempt " + task.successfulAttempt);
       // issue kill to all other attempts
       // issue kill to all other attempts
-      TaskFinishedEvent tfe = createTaskFinishedEvent(task, TaskState.SUCCEEDED);
-      task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(), tfe));
+      if (task.historyTaskStartGenerated) {
+        TaskFinishedEvent tfe = createTaskFinishedEvent(task,
+            TaskState.SUCCEEDED);
+        task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
+            tfe));
+      }
       for (TaskAttempt attempt : task.attempts.values()) {
       for (TaskAttempt attempt : task.attempts.values()) {
         if (attempt.getID() != task.successfulAttempt &&
         if (attempt.getID() != task.successfulAttempt &&
             // This is okay because it can only talk us out of sending a
             // This is okay because it can only talk us out of sending a
@@ -726,10 +743,15 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
           TaskAttemptCompletionEventStatus.KILLED);
           TaskAttemptCompletionEventStatus.KILLED);
       // check whether all attempts are finished
       // check whether all attempts are finished
       if (task.finishedAttempts == task.attempts.size()) {
       if (task.finishedAttempts == task.attempts.size()) {
-        TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, "",
-            finalState, null); //TODO JH verify failedAttempt null
+        if (task.historyTaskStartGenerated) {
+        TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, null,
+              finalState, null); // TODO JH verify failedAttempt null
         task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
         task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
             taskFailedEvent)); 
             taskFailedEvent)); 
+        } else {
+          LOG.debug("Not generating HistoryFinish event since start event not" +
+          		" generated for task: " + task.getID());
+        }
 
 
         task.eventHandler.handle(
         task.eventHandler.handle(
             new JobTaskEvent(task.taskId, finalState));
             new JobTaskEvent(task.taskId, finalState));
@@ -769,11 +791,15 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
         TaskTAttemptEvent ev = (TaskTAttemptEvent) event;
         TaskTAttemptEvent ev = (TaskTAttemptEvent) event;
         TaskAttemptId taId = ev.getTaskAttemptID();
         TaskAttemptId taId = ev.getTaskAttemptID();
         
         
-        //TODO JH Populate the error string. FailReason from TaskAttempt(taId)
-        TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, "",
+        if (task.historyTaskStartGenerated) {
+        TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, attempt.getDiagnostics(),
             TaskState.FAILED, taId);
             TaskState.FAILED, taId);
         task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
         task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
             taskFailedEvent));
             taskFailedEvent));
+        } else {
+          LOG.debug("Not generating HistoryFinish event since start event not" +
+          		" generated for task: " + task.getID());
+        }
         task.eventHandler.handle(
         task.eventHandler.handle(
             new JobTaskEvent(task.taskId, TaskState.FAILED));
             new JobTaskEvent(task.taskId, TaskState.FAILED));
         return task.finished(TaskState.FAILED);
         return task.finished(TaskState.FAILED);
@@ -825,10 +851,15 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
     @Override
     @Override
     public void transition(TaskImpl task, TaskEvent event) {
     public void transition(TaskImpl task, TaskEvent event) {
       
       
+      if (task.historyTaskStartGenerated) {
       TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, null,
       TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, null,
-          TaskState.KILLED, null); //TODO Verify failedAttemptId is null
+            TaskState.KILLED, null); // TODO Verify failedAttemptId is null
       task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
       task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
           taskFailedEvent));
           taskFailedEvent));
+      }else {
+        LOG.debug("Not generating HistoryFinish event since start event not" +
+        		" generated for task: " + task.getID());
+      }
 
 
       task.eventHandler.handle(
       task.eventHandler.handle(
           new JobTaskEvent(task.taskId, TaskState.KILLED));
           new JobTaskEvent(task.taskId, TaskState.KILLED));

+ 9 - 0
mapreduce/mr-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java

@@ -205,6 +205,9 @@ public class TypeConverter {
   
   
   public static org.apache.hadoop.mapreduce.Counters fromYarn(
   public static org.apache.hadoop.mapreduce.Counters fromYarn(
       Counters yCntrs) {
       Counters yCntrs) {
+    if (yCntrs == null) {
+      return null;
+    }
     org.apache.hadoop.mapreduce.Counters counters = 
     org.apache.hadoop.mapreduce.Counters counters = 
       new org.apache.hadoop.mapreduce.Counters();
       new org.apache.hadoop.mapreduce.Counters();
     for (CounterGroup yGrp : yCntrs.getAllCounterGroups().values()) {
     for (CounterGroup yGrp : yCntrs.getAllCounterGroups().values()) {
@@ -219,6 +222,9 @@ public class TypeConverter {
   }
   }
 
 
   public static Counters toYarn(org.apache.hadoop.mapred.Counters counters) {
   public static Counters toYarn(org.apache.hadoop.mapred.Counters counters) {
+    if (counters == null) {
+      return null;
+    }
     Counters yCntrs = recordFactory.newRecordInstance(Counters.class);
     Counters yCntrs = recordFactory.newRecordInstance(Counters.class);
     yCntrs.addAllCounterGroups(new HashMap<String, CounterGroup>());
     yCntrs.addAllCounterGroups(new HashMap<String, CounterGroup>());
     for (org.apache.hadoop.mapred.Counters.Group grp : counters) {
     for (org.apache.hadoop.mapred.Counters.Group grp : counters) {
@@ -239,6 +245,9 @@ public class TypeConverter {
   }
   }
 
 
   public static Counters toYarn(org.apache.hadoop.mapreduce.Counters counters) {
   public static Counters toYarn(org.apache.hadoop.mapreduce.Counters counters) {
+    if (counters == null) {
+      return null;
+    }
     Counters yCntrs = recordFactory.newRecordInstance(Counters.class);
     Counters yCntrs = recordFactory.newRecordInstance(Counters.class);
     yCntrs.addAllCounterGroups(new HashMap<String, CounterGroup>());
     yCntrs.addAllCounterGroups(new HashMap<String, CounterGroup>());
     for (org.apache.hadoop.mapreduce.CounterGroup grp : counters) {
     for (org.apache.hadoop.mapreduce.CounterGroup grp : counters) {

+ 8 - 1
mapreduce/mr-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/FileNameIndexUtils.java

@@ -42,7 +42,8 @@ public class FileNameIndexUtils {
   private static final int FINISH_TIME_INDEX = 4;
   private static final int FINISH_TIME_INDEX = 4;
   private static final int NUM_MAPS_INDEX = 5;
   private static final int NUM_MAPS_INDEX = 5;
   private static final int NUM_REDUCES_INDEX = 6;
   private static final int NUM_REDUCES_INDEX = 6;
-  private static final int MAX_INDEX = NUM_REDUCES_INDEX;
+  private static final int JOB_STATUS_INDEX = 7;
+  private static final int MAX_INDEX = JOB_STATUS_INDEX;
 
 
   /**
   /**
    * Constructs the job history file name from the JobIndexInfo.
    * Constructs the job history file name from the JobIndexInfo.
@@ -78,6 +79,10 @@ public class FileNameIndexUtils {
     
     
     //NumReduces
     //NumReduces
     sb.append(indexInfo.getNumReduces());
     sb.append(indexInfo.getNumReduces());
+    sb.append(DELIMITER);
+    
+    //JobStatus
+    sb.append(indexInfo.getJobStatus());
     
     
     sb.append(JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION);
     sb.append(JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION);
     return encodeJobHistoryFileName(sb.toString());
     return encodeJobHistoryFileName(sb.toString());
@@ -115,6 +120,8 @@ public class FileNameIndexUtils {
     
     
     indexInfo.setNumReduces(Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_REDUCES_INDEX])));
     indexInfo.setNumReduces(Integer.parseInt(decodeJobHistoryFileName(jobDetails[NUM_REDUCES_INDEX])));
     
     
+    indexInfo.setJobStatus(decodeJobHistoryFileName(jobDetails[JOB_STATUS_INDEX]));
+    
     return indexInfo;
     return indexInfo;
   }
   }
 
 

+ 3 - 0
mapreduce/mr-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java

@@ -123,6 +123,9 @@ public class JobHistoryUtils {
     }
     }
   };
   };
 
 
+  public static boolean isValidJobHistoryFileName(String pathString) {
+    return pathString.endsWith(JOB_HISTORY_FILE_EXTENSION);
+  }
 
 
   /**
   /**
    * Gets a PathFilter which would match configuration files.
    * Gets a PathFilter which would match configuration files.

+ 11 - 2
mapreduce/mr-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobIndexInfo.java

@@ -32,12 +32,13 @@ public class JobIndexInfo {
   private JobId jobId;
   private JobId jobId;
   private int numMaps;
   private int numMaps;
   private int numReduces;
   private int numReduces;
+  private String jobStatus;
   
   
   public JobIndexInfo() {
   public JobIndexInfo() {
   }
   }
   
   
   public JobIndexInfo(long submitTime, long finishTime, String user,
   public JobIndexInfo(long submitTime, long finishTime, String user,
-      String jobName, JobId jobId, int numMaps, int numReduces) {
+      String jobName, JobId jobId, int numMaps, int numReduces, String jobStatus) {
     this.submitTime = submitTime;
     this.submitTime = submitTime;
     this.finishTime = finishTime;
     this.finishTime = finishTime;
     this.user = user;
     this.user = user;
@@ -45,6 +46,7 @@ public class JobIndexInfo {
     this.jobId = jobId;
     this.jobId = jobId;
     this.numMaps = numMaps;
     this.numMaps = numMaps;
     this.numReduces = numReduces;
     this.numReduces = numReduces;
+    this.jobStatus = jobStatus;
   }
   }
   
   
   public long getSubmitTime() {
   public long getSubmitTime() {
@@ -89,12 +91,19 @@ public class JobIndexInfo {
   public void setNumReduces(int numReduces) {
   public void setNumReduces(int numReduces) {
     this.numReduces = numReduces;
     this.numReduces = numReduces;
   }
   }
+  public String getJobStatus() {
+    return jobStatus;
+  }
+  public void setJobStatus(String jobStatus) {
+    this.jobStatus = jobStatus;
+  }
 
 
   @Override
   @Override
   public String toString() {
   public String toString() {
     return "JobIndexInfo [submitTime=" + submitTime + ", finishTime="
     return "JobIndexInfo [submitTime=" + submitTime + ", finishTime="
         + finishTime + ", user=" + user + ", jobName=" + jobName + ", jobId="
         + finishTime + ", user=" + user + ", jobName=" + jobName + ", jobId="
-        + jobId + ", numMaps=" + numMaps + ", numReduces=" + numReduces + "]";
+        + jobId + ", numMaps=" + numMaps + ", numReduces=" + numReduces
+        + ", jobStatus=" + jobStatus + "]";
   }
   }
   
   
   
   

+ 92 - 8
mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java

@@ -20,7 +20,10 @@ package org.apache.hadoop.mapreduce.v2.hs;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
@@ -39,9 +42,12 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEventStatus;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
@@ -64,14 +70,14 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
   private final Map<TaskId, Task> mapTasks = new HashMap<TaskId, Task>();
   private final Map<TaskId, Task> mapTasks = new HashMap<TaskId, Task>();
   private final Map<TaskId, Task> reduceTasks = new HashMap<TaskId, Task>();
   private final Map<TaskId, Task> reduceTasks = new HashMap<TaskId, Task>();
   
   
-  private TaskAttemptCompletionEvent[] completionEvents;
+  private List<TaskAttemptCompletionEvent> completionEvents = null;
   private JobInfo jobInfo;
   private JobInfo jobInfo;
 
 
   public CompletedJob(Configuration conf, JobId jobId, Path historyFile, boolean loadTasks) throws IOException {
   public CompletedJob(Configuration conf, JobId jobId, Path historyFile, boolean loadTasks) throws IOException {
+    LOG.info("Loading job: " + jobId + " from file: " + historyFile);
     this.conf = conf;
     this.conf = conf;
     this.jobId = jobId;
     this.jobId = jobId;
     
     
-    //TODO: load the data lazily. for now load the full data upfront
     loadFullHistoryData(loadTasks, historyFile);
     loadFullHistoryData(loadTasks, historyFile);
 
 
     counters = TypeConverter.toYarn(jobInfo.getTotalCounters());
     counters = TypeConverter.toYarn(jobInfo.getTotalCounters());
@@ -81,6 +87,9 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
     report.setJobState(JobState.valueOf(jobInfo.getJobStatus()));
     report.setJobState(JobState.valueOf(jobInfo.getJobStatus()));
     report.setStartTime(jobInfo.getLaunchTime());
     report.setStartTime(jobInfo.getLaunchTime());
     report.setFinishTime(jobInfo.getFinishTime());
     report.setFinishTime(jobInfo.getFinishTime());
+    //TOODO Possibly populate job progress. Never used.
+    //report.setMapProgress(progress) 
+    //report.setReduceProgress(progress)
   }
   }
 
 
   @Override
   @Override
@@ -121,7 +130,85 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
   @Override
   @Override
   public TaskAttemptCompletionEvent[] getTaskAttemptCompletionEvents(
   public TaskAttemptCompletionEvent[] getTaskAttemptCompletionEvents(
       int fromEventId, int maxEvents) {
       int fromEventId, int maxEvents) {
-    return completionEvents;
+    if (completionEvents == null) {
+      constructTaskAttemptCompletionEvents();
+    }
+    TaskAttemptCompletionEvent[] events = new TaskAttemptCompletionEvent[0];
+    if (completionEvents.size() > fromEventId) {
+      int actualMax = Math.min(maxEvents,
+          (completionEvents.size() - fromEventId));
+      events = completionEvents.subList(fromEventId, actualMax + fromEventId)
+          .toArray(events);
+    }
+    return events;
+  }
+
+  private void constructTaskAttemptCompletionEvents() {
+    completionEvents = new LinkedList<TaskAttemptCompletionEvent>();
+    List<TaskAttempt> allTaskAttempts = new LinkedList<TaskAttempt>();
+    for (TaskId taskId : tasks.keySet()) {
+      Task task = tasks.get(taskId);
+      for (TaskAttemptId taskAttemptId : task.getAttempts().keySet()) {
+        TaskAttempt taskAttempt = task.getAttempts().get(taskAttemptId);
+        allTaskAttempts.add(taskAttempt);
+      }
+    }
+    Collections.sort(allTaskAttempts, new Comparator<TaskAttempt>() {
+
+      @Override
+      public int compare(TaskAttempt o1, TaskAttempt o2) {
+        if (o1.getFinishTime() == 0 || o2.getFinishTime() == 0) {
+          if (o1.getFinishTime() == 0 && o2.getFinishTime() == 0) {
+            if (o1.getLaunchTime() == 0 || o2.getLaunchTime() == 0) {
+              if (o1.getLaunchTime() == 0 && o2.getLaunchTime() == 0) {
+                return 0;
+              } else {
+                long res = o1.getLaunchTime() - o2.getLaunchTime();
+                return res > 0 ? -1 : 1;
+              }
+            } else {
+              return (int) (o1.getLaunchTime() - o2.getLaunchTime());
+            }
+          } else {
+            long res = o1.getFinishTime() - o2.getFinishTime();
+            return res > 0 ? -1 : 1;
+          }
+        } else {
+          return (int) (o1.getFinishTime() - o2.getFinishTime());
+        }
+      }
+    });
+
+    int eventId = 0;
+    for (TaskAttempt taskAttempt : allTaskAttempts) {
+
+      TaskAttemptCompletionEvent tace = RecordFactoryProvider.getRecordFactory(
+          null).newRecordInstance(TaskAttemptCompletionEvent.class);
+
+      int attemptRunTime = -1;
+      if (taskAttempt.getLaunchTime() != 0 && taskAttempt.getFinishTime() != 0) {
+        attemptRunTime = (int) (taskAttempt.getFinishTime() - taskAttempt
+            .getLaunchTime());
+      }
+      // Default to KILLED
+      TaskAttemptCompletionEventStatus taceStatus = TaskAttemptCompletionEventStatus.KILLED;
+      String taStateString = taskAttempt.getState().toString();
+      try {
+        taceStatus = TaskAttemptCompletionEventStatus.valueOf(taStateString);
+      } catch (Exception e) {
+        LOG.warn("Cannot constuct TACEStatus from TaskAtemptState: ["
+            + taStateString + "] for taskAttemptId: [" + taskAttempt.getID()
+            + "]. Defaulting to KILLED");
+      }
+
+      tace.setAttemptId(taskAttempt.getID());
+      tace.setAttemptRunTime(attemptRunTime);
+      tace.setEventId(eventId++);
+      tace.setMapOutputServerAddress(taskAttempt
+          .getAssignedContainerMgrAddress());
+      tace.setStatus(taceStatus);
+      completionEvents.add(tace);
+    }
   }
   }
 
 
   @Override
   @Override
@@ -131,6 +218,7 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
 
 
   //History data is leisurely loaded when task level data is requested
   //History data is leisurely loaded when task level data is requested
   private synchronized void loadFullHistoryData(boolean loadTasks, Path historyFileAbsolute) throws IOException {
   private synchronized void loadFullHistoryData(boolean loadTasks, Path historyFileAbsolute) throws IOException {
+    LOG.info("Loading history file: [" + historyFileAbsolute + "]");
     if (jobInfo != null) {
     if (jobInfo != null) {
       return; //data already loaded
       return; //data already loaded
     }
     }
@@ -148,7 +236,6 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
     }
     }
     
     
     if (loadTasks) {
     if (loadTasks) {
-    // populate the tasks
     for (Map.Entry<org.apache.hadoop.mapreduce.TaskID, TaskInfo> entry : jobInfo
     for (Map.Entry<org.apache.hadoop.mapreduce.TaskID, TaskInfo> entry : jobInfo
         .getAllTasks().entrySet()) {
         .getAllTasks().entrySet()) {
       TaskId yarnTaskID = TypeConverter.toYarn(entry.getKey());
       TaskId yarnTaskID = TypeConverter.toYarn(entry.getKey());
@@ -162,9 +249,6 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
       }
       }
     }
     }
     }
     }
-    
-    // TODO: populate the TaskAttemptCompletionEvent
-    completionEvents = new TaskAttemptCompletionEvent[0];
     LOG.info("TaskInfo loaded");
     LOG.info("TaskInfo loaded");
   }
   }
 
 
@@ -190,7 +274,7 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
 
 
   @Override
   @Override
   public boolean isUber() {
   public boolean isUber() {
-    return false;
+    return jobInfo.getIsUber();
   }
   }
 
 
   @Override
   @Override

+ 21 - 13
mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTask.java

@@ -41,10 +41,10 @@ public class CompletedTask implements Task {
 
 
 
 
   private final TaskType type;
   private final TaskType type;
-  private final Counters counters;
+  private Counters counters;
   private final long startTime;
   private final long startTime;
   private final long finishTime;
   private final long finishTime;
-  private final TaskState state;
+  private TaskState state;
   private final TaskId taskId;
   private final TaskId taskId;
   private final TaskReport report;
   private final TaskReport report;
   private final Map<TaskAttemptId, TaskAttempt> attempts =
   private final Map<TaskAttemptId, TaskAttempt> attempts =
@@ -52,27 +52,35 @@ public class CompletedTask implements Task {
   
   
   private static final Log LOG = LogFactory.getLog(CompletedTask.class);
   private static final Log LOG = LogFactory.getLog(CompletedTask.class);
 
 
-  CompletedTask(TaskId taskId, TaskInfo taskinfo) {
+  CompletedTask(TaskId taskId, TaskInfo taskInfo) {
+    //TODO JobHistoryParser.handleTaskFailedAttempt should use state from the event.
+    LOG.debug("HandlingTaskId: [" + taskId + "]");
     this.taskId = taskId;
     this.taskId = taskId;
-    this.startTime = taskinfo.getStartTime();
-    this.finishTime = taskinfo.getFinishTime();
-    this.type = TypeConverter.toYarn(taskinfo.getTaskType());
-    this.counters = TypeConverter.toYarn(
-        new org.apache.hadoop.mapred.Counters(taskinfo.getCounters()));
-    this.state = TaskState.valueOf(taskinfo.getTaskStatus());
+    this.startTime = taskInfo.getStartTime();
+    this.finishTime = taskInfo.getFinishTime();
+    this.type = TypeConverter.toYarn(taskInfo.getTaskType());
+    if (taskInfo.getCounters() != null)
+      this.counters = TypeConverter.toYarn(taskInfo.getCounters());
+    if (taskInfo.getTaskStatus() != null) {
+      this.state = TaskState.valueOf(taskInfo.getTaskStatus());
+    } else {
+      this.state = TaskState.KILLED;
+    }
     report = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(TaskReport.class);
     report = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(TaskReport.class);
-    for (TaskAttemptInfo attemptHistory : 
-                taskinfo.getAllTaskAttempts().values()) {
+    for (TaskAttemptInfo attemptHistory : taskInfo.getAllTaskAttempts()
+        .values()) {
       CompletedTaskAttempt attempt = new CompletedTaskAttempt(taskId, 
       CompletedTaskAttempt attempt = new CompletedTaskAttempt(taskId, 
           attemptHistory);
           attemptHistory);
+      report.addAllDiagnostics(attempt.getDiagnostics()); //TODO TMI?
       attempts.put(attempt.getID(), attempt);
       attempts.put(attempt.getID(), attempt);
-      if (attemptHistory.getState().equals(TaskState.SUCCEEDED.toString())
+      if (attemptHistory.getTaskStatus() != null
+          && attemptHistory.getTaskStatus().equals(
+              TaskState.SUCCEEDED.toString())
           && report.getSuccessfulAttempt() == null) {
           && report.getSuccessfulAttempt() == null) {
         report.setSuccessfulAttempt(TypeConverter.toYarn(attemptHistory
         report.setSuccessfulAttempt(TypeConverter.toYarn(attemptHistory
             .getAttemptId()));
             .getAttemptId()));
       }
       }
     }
     }
-    
     report.setTaskId(taskId);
     report.setTaskId(taskId);
     report.setStartTime(startTime);
     report.setStartTime(startTime);
     report.setFinishTime(finishTime);
     report.setFinishTime(finishTime);

+ 22 - 8
mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTaskAttempt.java

@@ -37,32 +37,46 @@ public class CompletedTaskAttempt implements TaskAttempt {
 
 
   private final TaskAttemptInfo attemptInfo;
   private final TaskAttemptInfo attemptInfo;
   private final TaskAttemptId attemptId;
   private final TaskAttemptId attemptId;
-  private final Counters counters;
+  private Counters counters;
   private final TaskAttemptState state;
   private final TaskAttemptState state;
   private final TaskAttemptReport report;
   private final TaskAttemptReport report;
   private final List<String> diagnostics = new ArrayList<String>();
   private final List<String> diagnostics = new ArrayList<String>();
 
 
+  private String localDiagMessage;
+
   CompletedTaskAttempt(TaskId taskId, TaskAttemptInfo attemptInfo) {
   CompletedTaskAttempt(TaskId taskId, TaskAttemptInfo attemptInfo) {
     this.attemptInfo = attemptInfo;
     this.attemptInfo = attemptInfo;
     this.attemptId = TypeConverter.toYarn(attemptInfo.getAttemptId());
     this.attemptId = TypeConverter.toYarn(attemptInfo.getAttemptId());
-    this.counters = TypeConverter.toYarn(
-        new org.apache.hadoop.mapred.Counters(attemptInfo.getCounters()));
-    this.state = TaskAttemptState.valueOf(attemptInfo.getState());
+    if (attemptInfo.getCounters() != null)
+      this.counters = TypeConverter.toYarn(attemptInfo.getCounters());
+    if (attemptInfo.getTaskStatus() != null) {
+      this.state = TaskAttemptState.valueOf(attemptInfo.getTaskStatus());
+    } else {
+      this.state = TaskAttemptState.KILLED;
+      localDiagMessage = "Attmpt state missing from History : marked as KILLED";
+      diagnostics.add(localDiagMessage);
+    }
     
     
     if (attemptInfo.getError() != null) {
     if (attemptInfo.getError() != null) {
       diagnostics.add(attemptInfo.getError());
       diagnostics.add(attemptInfo.getError());
     }
     }
     
     
     report = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(TaskAttemptReport.class);
     report = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(TaskAttemptReport.class);
+    report.setCounters(counters);
+    
     report.setTaskAttemptId(attemptId);
     report.setTaskAttemptId(attemptId);
     report.setTaskAttemptState(state);
     report.setTaskAttemptState(state);
     report.setProgress(getProgress());
     report.setProgress(getProgress());
     report.setStartTime(attemptInfo.getStartTime());
     report.setStartTime(attemptInfo.getStartTime());
     
     
     report.setFinishTime(attemptInfo.getFinishTime());
     report.setFinishTime(attemptInfo.getFinishTime());
+    if (localDiagMessage != null) {
+      report.setDiagnosticInfo(attemptInfo.getError() + ", " + localDiagMessage);
+    } else {
     report.setDiagnosticInfo(attemptInfo.getError());
     report.setDiagnosticInfo(attemptInfo.getError());
-    //result.phase = attemptInfo.get;//TODO
-    report.setStateString(state.toString());
+    }
+//    report.setPhase(attemptInfo.get); //TODO
+    report.setStateString(attemptInfo.getState());
     report.setCounters(getCounters());
     report.setCounters(getCounters());
   }
   }
 
 
@@ -79,8 +93,8 @@ public class CompletedTaskAttempt implements TaskAttempt {
 
 
   @Override
   @Override
   public String getAssignedContainerMgrAddress() {
   public String getAssignedContainerMgrAddress() {
-    // TODO Container details needs to be part of some historyEvent to be able to render the log directory.
-    return null;
+    // TODO Verify this is correct.
+    return attemptInfo.getTrackerName();
   }
   }
 
 
   @Override
   @Override

+ 0 - 2
mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java

@@ -116,7 +116,6 @@ public class HistoryClientService extends AbstractService {
       throw new YarnException(e);
       throw new YarnException(e);
     }
     }
 
 
-    //TODO: security
     server =
     server =
         rpc.getServer(MRClientProtocol.class, protocolHandler, address,
         rpc.getServer(MRClientProtocol.class, protocolHandler, address,
             conf, null,
             conf, null,
@@ -128,7 +127,6 @@ public class HistoryClientService extends AbstractService {
             + ":" + server.getPort());
             + ":" + server.getPort());
     LOG.info("Instantiated MRClientService at " + this.bindAddress);
     LOG.info("Instantiated MRClientService at " + this.bindAddress);
     
     
-    //TODO: start webApp on fixed port ??
     super.start();
     super.start();
   }
   }
 
 

+ 13 - 1
mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java

@@ -34,6 +34,8 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 
 
+import clover.org.apache.log4j.Logger;
+
 public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job {
 public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job {
 
 
   private JobIndexInfo jobIndexInfo = null;
   private JobIndexInfo jobIndexInfo = null;
@@ -59,7 +61,17 @@ public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job {
 
 
   @Override
   @Override
   public JobState getState() {
   public JobState getState() {
-    return JobState.SUCCEEDED;
+    JobState js = null;
+    try {
+      js = JobState.valueOf(jobIndexInfo.getJobStatus());
+    } catch (Exception e) {
+      // Meant for use by the display UI. Exception would prevent it from being
+      // rendered.e Defaulting to KILLED
+      Logger.getLogger(this.getClass().getName()).warn(
+          "Exception while parsing job state. Defaulting to KILLED", e);
+      js = JobState.KILLED;
+    }
+    return js;
   }
   }
 
 
   @Override
   @Override

+ 4 - 3
mapreduce/mr-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -148,13 +148,14 @@ public class ClientServiceDelegate {
      * on a allocating Application.
      * on a allocating Application.
      */
      */
     
     
-    if (currentAppState == ApplicationState.ALLOCATING || currentAppState == ApplicationState.KILLED
-        || currentAppState == ApplicationState.FAILED) {
+    if (currentAppState == ApplicationState.ALLOCATING) {
       realProxy = null;
       realProxy = null;
       return;
       return;
     }
     }
     
     
-    if (ApplicationState.COMPLETED.equals(appMaster.getState())) {
+    if (currentAppState == ApplicationState.COMPLETED
+        || currentAppState == ApplicationState.FAILED
+        || currentAppState == ApplicationState.KILLED) {
       serviceAddr = conf.get(JHConfig.HS_BIND_ADDRESS,
       serviceAddr = conf.get(JHConfig.HS_BIND_ADDRESS,
           JHConfig.DEFAULT_HS_BIND_ADDRESS);
           JHConfig.DEFAULT_HS_BIND_ADDRESS);
       LOG.info("Application state is completed. " +
       LOG.info("Application state is completed. " +