浏览代码

MAPREDUCE-5466. Changed MR AM to not promote history files of intermediate AMs in case they are exiting because of errors and thus help history-server pick up the right history file for the last successful AM. Contributed by Jian He.
svn merge --ignore-ancestry -c 1516238 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1516241 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 11 年之前
父节点
当前提交
213fd741bd
共有 13 个文件被更改,包括 190 次插入6 次删除
  1. 5 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 29 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  3. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java
  4. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  5. 13 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  6. 114 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
  7. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java
  8. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
  9. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr
  10. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java
  11. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java
  12. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
  13. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

+ 5 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -53,6 +53,11 @@ Release 2.1.1-beta - UNRELEASED
     MAPREDUCE-5001. LocalJobRunner has race condition resulting in job
     MAPREDUCE-5001. LocalJobRunner has race condition resulting in job
     failures (Sandy Ryza via jlowe)
     failures (Sandy Ryza via jlowe)
 
 
+    MAPREDUCE-5466. Changed MR AM to not promote history files of intermediate
+    AMs in case they are exiting because of errors and thus help history-server
+    pick up the right history file for the last successful AM. (Jian He via
+    vinodkv)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

@@ -520,7 +520,7 @@ public class JobHistoryEventHandler extends AbstractService
         mi.getJobIndexInfo().setSubmitTime(jobSubmittedEvent.getSubmitTime());
         mi.getJobIndexInfo().setSubmitTime(jobSubmittedEvent.getSubmitTime());
         mi.getJobIndexInfo().setQueueName(jobSubmittedEvent.getJobQueueName());
         mi.getJobIndexInfo().setQueueName(jobSubmittedEvent.getJobQueueName());
       }
       }
-     
+
       // If this is JobFinishedEvent, close the writer and setup the job-index
       // If this is JobFinishedEvent, close the writer and setup the job-index
       if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {
       if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {
         try {
         try {
@@ -532,6 +532,24 @@ public class JobHistoryEventHandler extends AbstractService
               jFinishedEvent.getFinishedReduces());
               jFinishedEvent.getFinishedReduces());
           mi.getJobIndexInfo().setJobStatus(JobState.SUCCEEDED.toString());
           mi.getJobIndexInfo().setJobStatus(JobState.SUCCEEDED.toString());
           closeEventWriter(event.getJobID());
           closeEventWriter(event.getJobID());
+          processDoneFiles(event.getJobID());
+        } catch (IOException e) {
+          throw new YarnRuntimeException(e);
+        }
+      }
+      // In case of JOB_ERROR, only process all the Done files(e.g. job
+      // summary, job history file etc.) if it is last AM retry.
+      if (event.getHistoryEvent().getEventType() == EventType.JOB_ERROR) {
+        try {
+          JobUnsuccessfulCompletionEvent jucEvent =
+              (JobUnsuccessfulCompletionEvent) event.getHistoryEvent();
+          mi.getJobIndexInfo().setFinishTime(jucEvent.getFinishTime());
+          mi.getJobIndexInfo().setNumMaps(jucEvent.getFinishedMaps());
+          mi.getJobIndexInfo().setNumReduces(jucEvent.getFinishedReduces());
+          mi.getJobIndexInfo().setJobStatus(jucEvent.getStatus());
+          closeEventWriter(event.getJobID());
+          if(context.isLastAMRetry())
+            processDoneFiles(event.getJobID());
         } catch (IOException e) {
         } catch (IOException e) {
           throw new YarnRuntimeException(e);
           throw new YarnRuntimeException(e);
         }
         }
@@ -548,6 +566,7 @@ public class JobHistoryEventHandler extends AbstractService
           mi.getJobIndexInfo().setNumReduces(jucEvent.getFinishedReduces());
           mi.getJobIndexInfo().setNumReduces(jucEvent.getFinishedReduces());
           mi.getJobIndexInfo().setJobStatus(jucEvent.getStatus());
           mi.getJobIndexInfo().setJobStatus(jucEvent.getStatus());
           closeEventWriter(event.getJobID());
           closeEventWriter(event.getJobID());
+          processDoneFiles(event.getJobID());
         } catch (IOException e) {
         } catch (IOException e) {
           throw new YarnRuntimeException(e);
           throw new YarnRuntimeException(e);
         }
         }
@@ -634,7 +653,6 @@ public class JobHistoryEventHandler extends AbstractService
   }
   }
 
 
   protected void closeEventWriter(JobId jobId) throws IOException {
   protected void closeEventWriter(JobId jobId) throws IOException {
-
     final MetaInfo mi = fileMap.get(jobId);
     final MetaInfo mi = fileMap.get(jobId);
     if (mi == null) {
     if (mi == null) {
       throw new IOException("No MetaInfo found for JobId: [" + jobId + "]");
       throw new IOException("No MetaInfo found for JobId: [" + jobId + "]");
@@ -654,7 +672,15 @@ public class JobHistoryEventHandler extends AbstractService
       LOG.error("Error closing writer for JobID: " + jobId);
       LOG.error("Error closing writer for JobID: " + jobId);
       throw e;
       throw e;
     }
     }
-     
+  }
+
+  protected void processDoneFiles(JobId jobId) throws IOException {
+
+    final MetaInfo mi = fileMap.get(jobId);
+    if (mi == null) {
+      throw new IOException("No MetaInfo found for JobId: [" + jobId + "]");
+    }
+
     if (mi.getHistoryFile() == null) {
     if (mi.getHistoryFile() == null) {
       LOG.warn("No file for job-history with " + jobId + " found in cache!");
       LOG.warn("No file for job-history with " + jobId + " found in cache!");
     }
     }

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java

@@ -61,4 +61,6 @@ public interface AppContext {
   Set<String> getBlacklistedNodes();
   Set<String> getBlacklistedNodes();
   
   
   ClientToAMTokenSecretManager getClientToAMTokenSecretManager();
   ClientToAMTokenSecretManager getClientToAMTokenSecretManager();
+
+  boolean isLastAMRetry();
 }
 }

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -952,6 +952,11 @@ public class MRAppMaster extends CompositeService {
     public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
     public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
       return clientToAMTokenSecretManager;
       return clientToAMTokenSecretManager;
     }
     }
+
+    @Override
+    public boolean isLastAMRetry(){
+      return isLastAMRetry;
+    }
   }
   }
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")

+ 13 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -36,9 +36,12 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@@ -223,6 +226,7 @@ public abstract class RMCommunicator extends AbstractService
 
 
   protected void startAllocatorThread() {
   protected void startAllocatorThread() {
     allocatorThread = new Thread(new Runnable() {
     allocatorThread = new Thread(new Runnable() {
+      @SuppressWarnings("unchecked")
       @Override
       @Override
       public void run() {
       public void run() {
         while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
         while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
@@ -233,6 +237,15 @@ public abstract class RMCommunicator extends AbstractService
             } catch (YarnRuntimeException e) {
             } catch (YarnRuntimeException e) {
               LOG.error("Error communicating with RM: " + e.getMessage() , e);
               LOG.error("Error communicating with RM: " + e.getMessage() , e);
               return;
               return;
+            } catch (InvalidToken e) {
+              // This can happen if the RM has been restarted, since currently
+              // when RM restarts AMRMToken is not populated back to
+              // AMRMTokenSecretManager yet. Once this is fixed, no need
+              // to send JOB_AM_REBOOT event in this method any more.
+              eventHandler.handle(new JobEvent(job.getID(),
+                JobEventType.JOB_AM_REBOOT));
+              LOG.error("Error in authencating with RM: " ,e);
+              return;
             } catch (Exception e) {
             } catch (Exception e) {
               LOG.error("ERROR IN CONTACTING RM. ", e);
               LOG.error("ERROR IN CONTACTING RM. ", e);
               continue;
               continue;

+ 114 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java

@@ -25,10 +25,13 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.never;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 
 
+import junit.framework.Assert;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -43,6 +46,7 @@ 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.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -229,6 +233,98 @@ public class TestJobHistoryEventHandler {
     }
     }
   }
   }
 
 
+  // In case of all types of events, process Done files if it's last AM retry
+  @Test (timeout=50000)
+  public void testProcessDoneFilesOnLastAMRetry() throws Exception {
+    TestParams t = new TestParams(true);
+    Configuration conf = new Configuration();
+
+    JHEvenHandlerForTest realJheh =
+        new JHEvenHandlerForTest(t.mockAppContext, 0);
+    JHEvenHandlerForTest jheh = spy(realJheh);
+    jheh.init(conf);
+
+    EventWriter mockWriter = null;
+    try {
+      jheh.start();
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
+        t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+      verify(jheh, times(0)).processDoneFiles(any(JobId.class));
+
+      handleEvent(jheh, new JobHistoryEvent(t.jobId,
+        new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+          0, 0, JobStateInternal.ERROR.toString())));
+      verify(jheh, times(1)).processDoneFiles(any(JobId.class));
+
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
+        TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
+        new Counters(), new Counters())));
+      verify(jheh, times(2)).processDoneFiles(any(JobId.class));
+
+      handleEvent(jheh, new JobHistoryEvent(t.jobId,
+        new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+          0, 0, JobStateInternal.FAILED.toString())));
+      verify(jheh, times(3)).processDoneFiles(any(JobId.class));
+
+      handleEvent(jheh, new JobHistoryEvent(t.jobId,
+        new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+          0, 0, JobStateInternal.KILLED.toString())));
+      verify(jheh, times(4)).processDoneFiles(any(JobId.class));
+
+      mockWriter = jheh.getEventWriter();
+      verify(mockWriter, times(5)).write(any(HistoryEvent.class));
+    } finally {
+      jheh.stop();
+      verify(mockWriter).close();
+    }
+  }
+
+  // Skip processing Done files in case of ERROR, if it's not last AM retry
+  @Test (timeout=50000)
+  public void testProcessDoneFilesNotLastAMRetry() throws Exception {
+    TestParams t = new TestParams(false);
+    Configuration conf = new Configuration();
+    JHEvenHandlerForTest realJheh =
+        new JHEvenHandlerForTest(t.mockAppContext, 0);
+    JHEvenHandlerForTest jheh = spy(realJheh);
+    jheh.init(conf);
+
+    EventWriter mockWriter = null;
+    try {
+      jheh.start();
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
+        t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+      verify(jheh, times(0)).processDoneFiles(t.jobId);
+
+      // skip processing done files
+      handleEvent(jheh, new JobHistoryEvent(t.jobId,
+        new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+          0, 0, JobStateInternal.ERROR.toString())));
+      verify(jheh, times(0)).processDoneFiles(t.jobId);
+
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
+          TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
+          new Counters(), new Counters())));
+      verify(jheh, times(1)).processDoneFiles(t.jobId);
+
+      handleEvent(jheh, new JobHistoryEvent(t.jobId,
+        new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+          0, 0, JobStateInternal.FAILED.toString())));
+      verify(jheh, times(2)).processDoneFiles(t.jobId);
+
+      handleEvent(jheh, new JobHistoryEvent(t.jobId,
+        new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+          0, 0, JobStateInternal.KILLED.toString())));
+      verify(jheh, times(3)).processDoneFiles(t.jobId);
+
+      mockWriter = jheh.getEventWriter();
+      verify(mockWriter, times(5)).write(any(HistoryEvent.class));
+    } finally {
+      jheh.stop();
+      verify(mockWriter).close();
+    }
+  }
+
   private void queueEvent(JHEvenHandlerForTest jheh, JobHistoryEvent event) {
   private void queueEvent(JHEvenHandlerForTest jheh, JobHistoryEvent event) {
     jheh.handle(event);
     jheh.handle(event);
   }
   }
@@ -258,20 +354,23 @@ public class TestJobHistoryEventHandler {
     }
     }
   }
   }
 
 
-  private AppContext mockAppContext(ApplicationId appId) {
+  private AppContext mockAppContext(ApplicationId appId, boolean isLastAMRetry) {
     JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId));
     JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId));
     AppContext mockContext = mock(AppContext.class);
     AppContext mockContext = mock(AppContext.class);
     Job mockJob = mock(Job.class);
     Job mockJob = mock(Job.class);
+    when(mockJob.getAllCounters()).thenReturn(new Counters());
     when(mockJob.getTotalMaps()).thenReturn(10);
     when(mockJob.getTotalMaps()).thenReturn(10);
     when(mockJob.getTotalReduces()).thenReturn(10);
     when(mockJob.getTotalReduces()).thenReturn(10);
     when(mockJob.getName()).thenReturn("mockjob");
     when(mockJob.getName()).thenReturn("mockjob");
     when(mockContext.getJob(jobId)).thenReturn(mockJob);
     when(mockContext.getJob(jobId)).thenReturn(mockJob);
     when(mockContext.getApplicationID()).thenReturn(appId);
     when(mockContext.getApplicationID()).thenReturn(appId);
+    when(mockContext.isLastAMRetry()).thenReturn(isLastAMRetry);
     return mockContext;
     return mockContext;
   }
   }
 
 
 
 
   private class TestParams {
   private class TestParams {
+    boolean isLastAMRetry;
     String workDir = setupTestWorkDir();
     String workDir = setupTestWorkDir();
     ApplicationId appId = ApplicationId.newInstance(200, 1);
     ApplicationId appId = ApplicationId.newInstance(200, 1);
     ApplicationAttemptId appAttemptId =
     ApplicationAttemptId appAttemptId =
@@ -279,7 +378,15 @@ public class TestJobHistoryEventHandler {
     ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
     ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
     TaskID taskID = TaskID.forName("task_200707121733_0003_m_000005");
     TaskID taskID = TaskID.forName("task_200707121733_0003_m_000005");
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);
-    AppContext mockAppContext = mockAppContext(appId);
+    AppContext mockAppContext;
+
+    public TestParams() {
+      this(false);
+    }
+    public TestParams(boolean isLastAMRetry) {
+      this.isLastAMRetry = isLastAMRetry;
+      mockAppContext = mockAppContext(appId, this.isLastAMRetry);
+    }
   }
   }
 
 
   private JobHistoryEvent getEventToEnqueue(JobId jobId) {
   private JobHistoryEvent getEventToEnqueue(JobId jobId) {
@@ -344,7 +451,6 @@ public class TestJobHistoryEventHandler {
 class JHEvenHandlerForTest extends JobHistoryEventHandler {
 class JHEvenHandlerForTest extends JobHistoryEventHandler {
 
 
   private EventWriter eventWriter;
   private EventWriter eventWriter;
-
   public JHEvenHandlerForTest(AppContext context, int startCount) {
   public JHEvenHandlerForTest(AppContext context, int startCount) {
     super(context, startCount);
     super(context, startCount);
   }
   }
@@ -367,6 +473,11 @@ class JHEvenHandlerForTest extends JobHistoryEventHandler {
   public EventWriter getEventWriter() {
   public EventWriter getEventWriter() {
     return this.eventWriter;
     return this.eventWriter;
   }
   }
+
+  @Override
+  protected void processDoneFiles(JobId jobId){
+    // do nothing
+  }
 }
 }
 
 
 /**
 /**

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java

@@ -130,4 +130,9 @@ public class MockAppContext implements AppContext {
     // Not implemented
     // Not implemented
     return null;
     return null;
   }
   }
+
+  @Override
+  public boolean isLastAMRetry() {
+    return false;
+  }
 }
 }

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java

@@ -862,5 +862,10 @@ public class TestRuntimeEstimators {
     public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
     public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
       return null;
       return null;
     }
     }
+
+    @Override
+    public boolean isLastAMRetry() {
+      return false;
+    }
   }
   }
 }
 }

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr

@@ -269,6 +269,7 @@
           "JOB_STATUS_CHANGED",
           "JOB_STATUS_CHANGED",
           "JOB_FAILED",
           "JOB_FAILED",
           "JOB_KILLED",
           "JOB_KILLED",
+          "JOB_ERROR",
           "JOB_INFO_CHANGED",
           "JOB_INFO_CHANGED",
           "TASK_STARTED",
           "TASK_STARTED",
           "TASK_FINISHED",
           "TASK_FINISHED",

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java

@@ -104,6 +104,8 @@ public class EventReader implements Closeable {
       result = new JobUnsuccessfulCompletionEvent(); break;
       result = new JobUnsuccessfulCompletionEvent(); break;
     case JOB_KILLED:
     case JOB_KILLED:
       result = new JobUnsuccessfulCompletionEvent(); break;
       result = new JobUnsuccessfulCompletionEvent(); break;
+    case JOB_ERROR:
+      result = new JobUnsuccessfulCompletionEvent(); break;
     case JOB_INFO_CHANGED:
     case JOB_INFO_CHANGED:
       result = new JobInfoChangeEvent(); break;
       result = new JobInfoChangeEvent(); break;
     case TASK_STARTED:
     case TASK_STARTED:

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java

@@ -185,6 +185,7 @@ public class JobHistoryParser implements HistoryEventHandler {
       break;
       break;
     case JOB_FAILED:
     case JOB_FAILED:
     case JOB_KILLED:
     case JOB_KILLED:
+    case JOB_ERROR:
       handleJobFailedEvent((JobUnsuccessfulCompletionEvent) event);
       handleJobFailedEvent((JobUnsuccessfulCompletionEvent) event);
       break;
       break;
     case JOB_FINISHED:
     case JOB_FINISHED:

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java

@@ -72,6 +72,8 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
   public EventType getEventType() {
   public EventType getEventType() {
     if ("FAILED".equals(getStatus())) {
     if ("FAILED".equals(getStatus())) {
       return EventType.JOB_FAILED;
       return EventType.JOB_FAILED;
+    } else if ("ERROR".equals(getStatus())) {
+      return EventType.JOB_ERROR;
     } else
     } else
       return EventType.JOB_KILLED;
       return EventType.JOB_KILLED;
   }
   }

+ 6 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

@@ -315,4 +315,10 @@ public class JobHistory extends AbstractService implements HistoryContext {
     // Not implemented.
     // Not implemented.
     return null;
     return null;
   }
   }
+
+  @Override
+  public boolean isLastAMRetry() {
+    // bogus - Not Required
+    return false;
+  }
 }
 }