|
@@ -31,6 +31,7 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
@@ -49,6 +50,7 @@ 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.job.Job;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
|
|
@@ -348,7 +350,9 @@ public class JobHistoryEventHandler extends AbstractService
|
|
|
JobUnsuccessfulCompletionEvent jucEvent =
|
|
|
new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(toClose),
|
|
|
System.currentTimeMillis(), job.getCompletedMaps(),
|
|
|
- job.getCompletedReduces(), JobState.KILLED.toString(),
|
|
|
+ job.getCompletedReduces(),
|
|
|
+ createJobStateForJobUnsuccessfulCompletionEvent(
|
|
|
+ mi.getForcedJobStateOnShutDown()),
|
|
|
job.getDiagnostics());
|
|
|
JobHistoryEvent jfEvent = new JobHistoryEvent(toClose, jucEvent);
|
|
|
//Bypass the queue mechanism which might wait. Call the method directly
|
|
@@ -381,9 +385,10 @@ public class JobHistoryEventHandler extends AbstractService
|
|
|
* This should be the first call to history for a job
|
|
|
*
|
|
|
* @param jobId the jobId.
|
|
|
+ * @param forcedJobStateOnShutDown
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- protected void setupEventWriter(JobId jobId)
|
|
|
+ protected void setupEventWriter(JobId jobId, String forcedJobStateOnShutDown)
|
|
|
throws IOException {
|
|
|
if (stagingDirPath == null) {
|
|
|
LOG.error("Log Directory is null, returning");
|
|
@@ -438,7 +443,7 @@ public class JobHistoryEventHandler extends AbstractService
|
|
|
}
|
|
|
|
|
|
MetaInfo fi = new MetaInfo(historyFile, logDirConfPath, writer,
|
|
|
- user, jobName, jobId);
|
|
|
+ user, jobName, jobId, forcedJobStateOnShutDown);
|
|
|
fi.getJobSummary().setJobId(jobId);
|
|
|
fileMap.put(jobId, fi);
|
|
|
}
|
|
@@ -481,13 +486,17 @@ public class JobHistoryEventHandler extends AbstractService
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- protected void handleEvent(JobHistoryEvent event) {
|
|
|
+ @Private
|
|
|
+ public void handleEvent(JobHistoryEvent event) {
|
|
|
synchronized (lock) {
|
|
|
|
|
|
// If this is JobSubmitted Event, setup the writer
|
|
|
if (event.getHistoryEvent().getEventType() == EventType.AM_STARTED) {
|
|
|
try {
|
|
|
- setupEventWriter(event.getJobID());
|
|
|
+ AMStartedEvent amStartedEvent =
|
|
|
+ (AMStartedEvent) event.getHistoryEvent();
|
|
|
+ setupEventWriter(event.getJobID(),
|
|
|
+ amStartedEvent.getForcedJobStateOnShutDown());
|
|
|
} catch (IOException ioe) {
|
|
|
LOG.error("Error JobHistoryEventHandler in handleEvent: " + event,
|
|
|
ioe);
|
|
@@ -804,9 +813,10 @@ public class JobHistoryEventHandler extends AbstractService
|
|
|
Timer flushTimer;
|
|
|
FlushTimerTask flushTimerTask;
|
|
|
private boolean isTimerShutDown = false;
|
|
|
+ private String forcedJobStateOnShutDown;
|
|
|
|
|
|
MetaInfo(Path historyFile, Path conf, EventWriter writer, String user,
|
|
|
- String jobName, JobId jobId) {
|
|
|
+ String jobName, JobId jobId, String forcedJobStateOnShutDown) {
|
|
|
this.historyFile = historyFile;
|
|
|
this.confFile = conf;
|
|
|
this.writer = writer;
|
|
@@ -814,6 +824,7 @@ public class JobHistoryEventHandler extends AbstractService
|
|
|
new JobIndexInfo(-1, -1, user, jobName, jobId, -1, -1, null);
|
|
|
this.jobSummary = new JobSummary();
|
|
|
this.flushTimer = new Timer("FlushTimer", true);
|
|
|
+ this.forcedJobStateOnShutDown = forcedJobStateOnShutDown;
|
|
|
}
|
|
|
|
|
|
Path getHistoryFile() {
|
|
@@ -840,6 +851,10 @@ public class JobHistoryEventHandler extends AbstractService
|
|
|
return isTimerShutDown;
|
|
|
}
|
|
|
|
|
|
+ String getForcedJobStateOnShutDown() {
|
|
|
+ return forcedJobStateOnShutDown;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
return "Job MetaInfo for "+ jobSummary.getJobId()
|
|
@@ -983,4 +998,20 @@ public class JobHistoryEventHandler extends AbstractService
|
|
|
LOG.info("JobHistoryEventHandler notified that forceJobCompletion is "
|
|
|
+ forceJobCompletion);
|
|
|
}
|
|
|
+
|
|
|
+ private String createJobStateForJobUnsuccessfulCompletionEvent(
|
|
|
+ String forcedJobStateOnShutDown) {
|
|
|
+ if (forcedJobStateOnShutDown == null || forcedJobStateOnShutDown
|
|
|
+ .isEmpty()) {
|
|
|
+ return JobState.KILLED.toString();
|
|
|
+ } else if (forcedJobStateOnShutDown.equals(
|
|
|
+ JobStateInternal.ERROR.toString()) ||
|
|
|
+ forcedJobStateOnShutDown.equals(JobStateInternal.FAILED.toString())) {
|
|
|
+ return JobState.FAILED.toString();
|
|
|
+ } else if (forcedJobStateOnShutDown.equals(JobStateInternal.SUCCEEDED
|
|
|
+ .toString())) {
|
|
|
+ return JobState.SUCCEEDED.toString();
|
|
|
+ }
|
|
|
+ return JobState.KILLED.toString();
|
|
|
+ }
|
|
|
}
|