|
@@ -29,9 +29,6 @@ import java.util.LinkedHashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
-import java.util.concurrent.ScheduledFuture;
|
|
|
-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.locks.Lock;
|
|
|
import java.util.concurrent.locks.ReadWriteLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
@@ -283,8 +280,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
.addTransition
|
|
|
(JobStateInternal.RUNNING,
|
|
|
EnumSet.of(JobStateInternal.RUNNING,
|
|
|
- JobStateInternal.COMMITTING, JobStateInternal.FAIL_WAIT,
|
|
|
- JobStateInternal.FAIL_ABORT),
|
|
|
+ JobStateInternal.COMMITTING, JobStateInternal.FAIL_ABORT),
|
|
|
JobEventType.JOB_TASK_COMPLETED,
|
|
|
new TaskCompletedTransition())
|
|
|
.addTransition
|
|
@@ -379,35 +375,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
EnumSet.of(JobEventType.JOB_KILL,
|
|
|
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE))
|
|
|
|
|
|
- // Transitions from FAIL_WAIT state
|
|
|
- .addTransition(JobStateInternal.FAIL_WAIT,
|
|
|
- JobStateInternal.FAIL_WAIT,
|
|
|
- JobEventType.JOB_DIAGNOSTIC_UPDATE,
|
|
|
- DIAGNOSTIC_UPDATE_TRANSITION)
|
|
|
- .addTransition(JobStateInternal.FAIL_WAIT,
|
|
|
- JobStateInternal.FAIL_WAIT,
|
|
|
- JobEventType.JOB_COUNTER_UPDATE, COUNTER_UPDATE_TRANSITION)
|
|
|
- .addTransition(JobStateInternal.FAIL_WAIT,
|
|
|
- EnumSet.of(JobStateInternal.FAIL_WAIT, JobStateInternal.FAIL_ABORT),
|
|
|
- JobEventType.JOB_TASK_COMPLETED,
|
|
|
- new JobFailWaitTransition())
|
|
|
- .addTransition(JobStateInternal.FAIL_WAIT,
|
|
|
- JobStateInternal.FAIL_ABORT, JobEventType.JOB_FAIL_WAIT_TIMEDOUT,
|
|
|
- new JobFailWaitTimedOutTransition())
|
|
|
- .addTransition(JobStateInternal.FAIL_WAIT, JobStateInternal.KILLED,
|
|
|
- JobEventType.JOB_KILL,
|
|
|
- new KilledDuringAbortTransition())
|
|
|
- .addTransition(JobStateInternal.FAIL_WAIT,
|
|
|
- JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
|
|
|
- INTERNAL_ERROR_TRANSITION)
|
|
|
- // Ignore-able events
|
|
|
- .addTransition(JobStateInternal.FAIL_WAIT,
|
|
|
- JobStateInternal.FAIL_WAIT,
|
|
|
- EnumSet.of(JobEventType.JOB_TASK_ATTEMPT_COMPLETED,
|
|
|
- JobEventType.JOB_MAP_TASK_RESCHEDULED,
|
|
|
- JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE))
|
|
|
-
|
|
|
- //Transitions from FAIL_ABORT state
|
|
|
+ // Transitions from FAIL_ABORT state
|
|
|
.addTransition(JobStateInternal.FAIL_ABORT,
|
|
|
JobStateInternal.FAIL_ABORT,
|
|
|
JobEventType.JOB_DIAGNOSTIC_UPDATE,
|
|
@@ -553,10 +521,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
|
|
|
private JobStateInternal forcedState = null;
|
|
|
|
|
|
- //Executor used for running future tasks. Setting thread pool size to 1
|
|
|
- private ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
|
|
|
- private ScheduledFuture failWaitTriggerScheduledFuture;
|
|
|
-
|
|
|
public JobImpl(JobId jobId, ApplicationAttemptId applicationAttemptId,
|
|
|
Configuration conf, EventHandler eventHandler,
|
|
|
TaskAttemptListener taskAttemptListener,
|
|
@@ -917,7 +881,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
case SETUP:
|
|
|
case COMMITTING:
|
|
|
return JobState.RUNNING;
|
|
|
- case FAIL_WAIT:
|
|
|
case FAIL_ABORT:
|
|
|
return JobState.FAILED;
|
|
|
default:
|
|
@@ -1442,43 +1405,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
job.unsuccessfulFinish(finalState);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- //This transition happens when a job is to be failed. It waits for all the
|
|
|
- //tasks to finish / be killed.
|
|
|
- private static class JobFailWaitTransition
|
|
|
- implements MultipleArcTransition<JobImpl, JobEvent, JobStateInternal> {
|
|
|
- @Override
|
|
|
- public JobStateInternal transition(JobImpl job, JobEvent event) {
|
|
|
- if(!job.failWaitTriggerScheduledFuture.isCancelled()) {
|
|
|
- for(Task task: job.tasks.values()) {
|
|
|
- if(!task.isFinished()) {
|
|
|
- return JobStateInternal.FAIL_WAIT;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- //Finished waiting. All tasks finished / were killed
|
|
|
- job.failWaitTriggerScheduledFuture.cancel(false);
|
|
|
- job.eventHandler.handle(new CommitterJobAbortEvent(job.jobId,
|
|
|
- job.jobContext, org.apache.hadoop.mapreduce.JobStatus.State.FAILED));
|
|
|
- return JobStateInternal.FAIL_ABORT;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- //This transition happens when a job to be failed times out while waiting on
|
|
|
- //tasks that had been sent the KILL signal. It is triggered by a
|
|
|
- //ScheduledFuture task queued in the executor.
|
|
|
- private static class JobFailWaitTimedOutTransition
|
|
|
- implements SingleArcTransition<JobImpl, JobEvent> {
|
|
|
- @Override
|
|
|
- public void transition(JobImpl job, JobEvent event) {
|
|
|
- LOG.info("Timeout expired in FAIL_WAIT waiting for tasks to get killed."
|
|
|
- + " Going to fail job anyway");
|
|
|
- job.failWaitTriggerScheduledFuture.cancel(false);
|
|
|
- job.eventHandler.handle(new CommitterJobAbortEvent(job.jobId,
|
|
|
- job.jobContext, org.apache.hadoop.mapreduce.JobStatus.State.FAILED));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
+
|
|
|
// JobFinishedEvent triggers the move of the history file out of the staging
|
|
|
// area. May need to create a new event type for this if JobFinished should
|
|
|
// not be generated for KilledJobs, etc.
|
|
@@ -1689,23 +1616,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
return checkJobAfterTaskCompletion(job);
|
|
|
}
|
|
|
|
|
|
- //This class is used to queue a ScheduledFuture to send an event to a job
|
|
|
- //after some delay. This can be used to wait for maximum amount of time
|
|
|
- //before proceeding anyway. e.g. When a job is waiting in FAIL_WAIT for
|
|
|
- //all tasks to be killed.
|
|
|
- static class TriggerScheduledFuture implements Runnable {
|
|
|
- JobEvent toSend;
|
|
|
- JobImpl job;
|
|
|
- TriggerScheduledFuture(JobImpl job, JobEvent toSend) {
|
|
|
- this.toSend = toSend;
|
|
|
- this.job = job;
|
|
|
- }
|
|
|
- public void run() {
|
|
|
- LOG.info("Sending event " + toSend + " to " + job.getID());
|
|
|
- job.getEventHandler().handle(toSend);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
protected JobStateInternal checkJobAfterTaskCompletion(JobImpl job) {
|
|
|
//check for Job failure
|
|
|
if (job.failedMapTaskCount*100 >
|
|
@@ -1719,30 +1629,10 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|
|
" failedReduces:" + job.failedReduceTaskCount;
|
|
|
LOG.info(diagnosticMsg);
|
|
|
job.addDiagnostic(diagnosticMsg);
|
|
|
-
|
|
|
- //Send kill signal to all unfinished tasks here.
|
|
|
- boolean allDone = true;
|
|
|
- for (Task task : job.tasks.values()) {
|
|
|
- if(!task.isFinished()) {
|
|
|
- allDone = false;
|
|
|
- job.eventHandler.handle(
|
|
|
- new TaskEvent(task.getID(), TaskEventType.T_KILL));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- //If all tasks are already done, we should go directly to FAIL_ABORT
|
|
|
- if(allDone) {
|
|
|
- return JobStateInternal.FAIL_ABORT;
|
|
|
- }
|
|
|
-
|
|
|
- //Set max timeout to wait for the tasks to get killed
|
|
|
- job.failWaitTriggerScheduledFuture = job.executor.schedule(
|
|
|
- new TriggerScheduledFuture(job, new JobEvent(job.getID(),
|
|
|
- JobEventType.JOB_FAIL_WAIT_TIMEDOUT)), job.conf.getInt(
|
|
|
- MRJobConfig.MR_AM_COMMITTER_CANCEL_TIMEOUT_MS,
|
|
|
- MRJobConfig.DEFAULT_MR_AM_COMMITTER_CANCEL_TIMEOUT_MS),
|
|
|
- TimeUnit.MILLISECONDS);
|
|
|
- return JobStateInternal.FAIL_WAIT;
|
|
|
+ job.eventHandler.handle(new CommitterJobAbortEvent(job.jobId,
|
|
|
+ job.jobContext,
|
|
|
+ org.apache.hadoop.mapreduce.JobStatus.State.FAILED));
|
|
|
+ return JobStateInternal.FAIL_ABORT;
|
|
|
}
|
|
|
|
|
|
return job.checkReadyForCommit();
|