|
@@ -184,8 +184,20 @@ public abstract class TaskAttemptImpl implements
|
|
|
private Locality locality;
|
|
|
private Avataar avataar;
|
|
|
|
|
|
- private static final CleanupContainerTransition CLEANUP_CONTAINER_TRANSITION =
|
|
|
- new CleanupContainerTransition();
|
|
|
+ private static final CleanupContainerTransition
|
|
|
+ CLEANUP_CONTAINER_TRANSITION = new CleanupContainerTransition();
|
|
|
+ private static final MoveContainerToSucceededFinishingTransition
|
|
|
+ SUCCEEDED_FINISHING_TRANSITION =
|
|
|
+ new MoveContainerToSucceededFinishingTransition();
|
|
|
+ private static final MoveContainerToFailedFinishingTransition
|
|
|
+ FAILED_FINISHING_TRANSITION =
|
|
|
+ new MoveContainerToFailedFinishingTransition();
|
|
|
+ private static final ExitFinishingOnTimeoutTransition
|
|
|
+ FINISHING_ON_TIMEOUT_TRANSITION =
|
|
|
+ new ExitFinishingOnTimeoutTransition();
|
|
|
+
|
|
|
+ private static final FinalizeFailedTransition FINALIZE_FAILED_TRANSITION =
|
|
|
+ new FinalizeFailedTransition();
|
|
|
|
|
|
private static final DiagnosticInformationUpdater
|
|
|
DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION
|
|
@@ -204,6 +216,8 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEventType.TA_COMMIT_PENDING,
|
|
|
TaskAttemptEventType.TA_DONE,
|
|
|
TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
+ TaskAttemptEventType.TA_TIMED_OUT,
|
|
|
TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE);
|
|
|
|
|
|
private static final StateMachineFactory
|
|
@@ -221,16 +235,16 @@ public abstract class TaskAttemptImpl implements
|
|
|
.addTransition(TaskAttemptStateInternal.NEW, TaskAttemptStateInternal.KILLED,
|
|
|
TaskAttemptEventType.TA_KILL, new KilledTransition())
|
|
|
.addTransition(TaskAttemptStateInternal.NEW, TaskAttemptStateInternal.FAILED,
|
|
|
- TaskAttemptEventType.TA_FAILMSG, new FailedTransition())
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT, new FailedTransition())
|
|
|
.addTransition(TaskAttemptStateInternal.NEW,
|
|
|
EnumSet.of(TaskAttemptStateInternal.FAILED,
|
|
|
TaskAttemptStateInternal.KILLED,
|
|
|
TaskAttemptStateInternal.SUCCEEDED),
|
|
|
TaskAttemptEventType.TA_RECOVER, new RecoverTransition())
|
|
|
.addTransition(TaskAttemptStateInternal.NEW,
|
|
|
- TaskAttemptStateInternal.NEW,
|
|
|
- TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
|
|
- DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
|
|
|
+ TaskAttemptStateInternal.NEW,
|
|
|
+ TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
|
|
+ DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
|
|
|
|
|
|
// Transitions from the UNASSIGNED state.
|
|
|
.addTransition(TaskAttemptStateInternal.UNASSIGNED,
|
|
@@ -238,14 +252,14 @@ public abstract class TaskAttemptImpl implements
|
|
|
new ContainerAssignedTransition())
|
|
|
.addTransition(TaskAttemptStateInternal.UNASSIGNED, TaskAttemptStateInternal.KILLED,
|
|
|
TaskAttemptEventType.TA_KILL, new DeallocateContainerTransition(
|
|
|
- TaskAttemptStateInternal.KILLED, true))
|
|
|
+ TaskAttemptStateInternal.KILLED, true))
|
|
|
.addTransition(TaskAttemptStateInternal.UNASSIGNED, TaskAttemptStateInternal.FAILED,
|
|
|
- TaskAttemptEventType.TA_FAILMSG, new DeallocateContainerTransition(
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT, new DeallocateContainerTransition(
|
|
|
TaskAttemptStateInternal.FAILED, true))
|
|
|
.addTransition(TaskAttemptStateInternal.UNASSIGNED,
|
|
|
- TaskAttemptStateInternal.UNASSIGNED,
|
|
|
- TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
|
|
- DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
|
|
|
+ TaskAttemptStateInternal.UNASSIGNED,
|
|
|
+ TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
|
|
+ DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
|
|
|
|
|
|
// Transitions from the ASSIGNED state.
|
|
|
.addTransition(TaskAttemptStateInternal.ASSIGNED, TaskAttemptStateInternal.RUNNING,
|
|
@@ -258,15 +272,19 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
|
|
|
new DeallocateContainerTransition(TaskAttemptStateInternal.FAILED, false))
|
|
|
.addTransition(TaskAttemptStateInternal.ASSIGNED,
|
|
|
- TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
|
+ TaskAttemptStateInternal.FAILED,
|
|
|
TaskAttemptEventType.TA_CONTAINER_COMPLETED,
|
|
|
- CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ FINALIZE_FAILED_TRANSITION)
|
|
|
.addTransition(TaskAttemptStateInternal.ASSIGNED,
|
|
|
TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
TaskAttemptEventType.TA_KILL, CLEANUP_CONTAINER_TRANSITION)
|
|
|
- .addTransition(TaskAttemptStateInternal.ASSIGNED,
|
|
|
+ .addTransition(TaskAttemptStateInternal.ASSIGNED,
|
|
|
+ TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG, FAILED_FINISHING_TRANSITION)
|
|
|
+ .addTransition(TaskAttemptStateInternal.ASSIGNED,
|
|
|
TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
|
- TaskAttemptEventType.TA_FAILMSG, CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
+ CLEANUP_CONTAINER_TRANSITION)
|
|
|
|
|
|
// Transitions from RUNNING state.
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.RUNNING,
|
|
@@ -274,23 +292,27 @@ public abstract class TaskAttemptImpl implements
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING, TaskAttemptStateInternal.RUNNING,
|
|
|
TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
|
|
DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
|
|
|
- // If no commit is required, task directly goes to success
|
|
|
+ // If no commit is required, task goes to finishing state
|
|
|
+ // This will give a chance for the container to exit by itself
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING,
|
|
|
- TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
|
|
|
- TaskAttemptEventType.TA_DONE, CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptEventType.TA_DONE, SUCCEEDED_FINISHING_TRANSITION)
|
|
|
// If commit is required, task goes through commit pending state.
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING,
|
|
|
TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
TaskAttemptEventType.TA_COMMIT_PENDING, new CommitPendingTransition())
|
|
|
// Failure handling while RUNNING
|
|
|
+ .addTransition(TaskAttemptStateInternal.RUNNING,
|
|
|
+ TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG, FAILED_FINISHING_TRANSITION)
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING,
|
|
|
TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
|
- TaskAttemptEventType.TA_FAILMSG, CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT, CLEANUP_CONTAINER_TRANSITION)
|
|
|
//for handling container exit without sending the done or fail msg
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING,
|
|
|
- TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
|
+ TaskAttemptStateInternal.FAILED,
|
|
|
TaskAttemptEventType.TA_CONTAINER_COMPLETED,
|
|
|
- CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ FINALIZE_FAILED_TRANSITION)
|
|
|
// Timeout handling while RUNNING
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING,
|
|
|
TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
@@ -301,12 +323,97 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEventType.TA_CONTAINER_CLEANED, new KilledTransition())
|
|
|
// Kill handling
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING,
|
|
|
- TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP, TaskAttemptEventType.TA_KILL,
|
|
|
+ TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
+ TaskAttemptEventType.TA_KILL,
|
|
|
CLEANUP_CONTAINER_TRANSITION)
|
|
|
.addTransition(TaskAttemptStateInternal.RUNNING,
|
|
|
TaskAttemptStateInternal.KILLED,
|
|
|
TaskAttemptEventType.TA_PREEMPTED, new PreemptedTransition())
|
|
|
|
|
|
+ // Transitions from SUCCESS_FINISHING_CONTAINER state
|
|
|
+ // When the container exits by itself, the notification of container
|
|
|
+ // completed event will be routed via NM -> RM -> AM.
|
|
|
+ // After MRAppMaster gets notification from RM, it will generate
|
|
|
+ // TA_CONTAINER_COMPLETED event.
|
|
|
+ .addTransition(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.SUCCEEDED,
|
|
|
+ TaskAttemptEventType.TA_CONTAINER_COMPLETED,
|
|
|
+ new ExitFinishingOnContainerCompletedTransition())
|
|
|
+ // Given TA notifies task T_ATTEMPT_SUCCEEDED when it transitions to
|
|
|
+ // SUCCESS_FINISHING_CONTAINER, it is possible to receive the event
|
|
|
+ // TA_CONTAINER_CLEANED in the following scenario.
|
|
|
+ // 1. It is the last task for the job.
|
|
|
+ // 2. After the task receives T_ATTEMPT_SUCCEEDED, it will notify job.
|
|
|
+ // 3. Job will be marked completed.
|
|
|
+ // 4. As part of MRAppMaster's shutdown, all containers will be killed.
|
|
|
+ .addTransition(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.SUCCEEDED,
|
|
|
+ TaskAttemptEventType.TA_CONTAINER_CLEANED,
|
|
|
+ new ExitFinishingOnContainerCleanedupTransition())
|
|
|
+ // The client wants to kill the task. Given the task is in finishing
|
|
|
+ // state, it could go to succeeded state or killed state. If it is a
|
|
|
+ // reducer, it will go to succeeded state;
|
|
|
+ // otherwise, it goes to killed state.
|
|
|
+ .addTransition(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ EnumSet.of(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
|
|
|
+ TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP),
|
|
|
+ TaskAttemptEventType.TA_KILL,
|
|
|
+ new KilledAfterSucceededFinishingTransition())
|
|
|
+ // The attempt stays in finishing state for too long
|
|
|
+ // Let us clean up the container
|
|
|
+ .addTransition(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
|
|
|
+ TaskAttemptEventType.TA_TIMED_OUT, FINISHING_ON_TIMEOUT_TRANSITION)
|
|
|
+ .addTransition(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
|
|
+ DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
|
|
|
+ // ignore-able events
|
|
|
+ .addTransition(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ EnumSet.of(TaskAttemptEventType.TA_UPDATE,
|
|
|
+ TaskAttemptEventType.TA_DONE,
|
|
|
+ TaskAttemptEventType.TA_COMMIT_PENDING,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT))
|
|
|
+
|
|
|
+ // Transitions from FAIL_FINISHING_CONTAINER state
|
|
|
+ // When the container exits by itself, the notification of container
|
|
|
+ // completed event will be routed via NM -> RM -> AM.
|
|
|
+ // After MRAppMaster gets notification from RM, it will generate
|
|
|
+ // TA_CONTAINER_COMPLETED event.
|
|
|
+ .addTransition(TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.FAILED,
|
|
|
+ TaskAttemptEventType.TA_CONTAINER_COMPLETED,
|
|
|
+ new ExitFinishingOnContainerCompletedTransition())
|
|
|
+ // Given TA notifies task T_ATTEMPT_FAILED when it transitions to
|
|
|
+ // FAIL_FINISHING_CONTAINER, it is possible to receive the event
|
|
|
+ // TA_CONTAINER_CLEANED in the following scenario.
|
|
|
+ // 1. It is the last task attempt for the task.
|
|
|
+ // 2. After the task receives T_ATTEMPT_FAILED, it will notify job.
|
|
|
+ // 3. Job will be marked failed.
|
|
|
+ // 4. As part of MRAppMaster's shutdown, all containers will be killed.
|
|
|
+ .addTransition(TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.FAILED,
|
|
|
+ TaskAttemptEventType.TA_CONTAINER_CLEANED,
|
|
|
+ new ExitFinishingOnContainerCleanedupTransition())
|
|
|
+ .addTransition(TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
|
+ TaskAttemptEventType.TA_TIMED_OUT, FINISHING_ON_TIMEOUT_TRANSITION)
|
|
|
+ .addTransition(TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
|
|
+ DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
|
|
|
+ // ignore-able events
|
|
|
+ .addTransition(TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ EnumSet.of(TaskAttemptEventType.TA_KILL,
|
|
|
+ TaskAttemptEventType.TA_UPDATE,
|
|
|
+ TaskAttemptEventType.TA_DONE,
|
|
|
+ TaskAttemptEventType.TA_COMMIT_PENDING,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT))
|
|
|
+
|
|
|
// Transitions from COMMIT_PENDING state
|
|
|
.addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
TaskAttemptStateInternal.COMMIT_PENDING, TaskAttemptEventType.TA_UPDATE,
|
|
@@ -316,22 +423,27 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
|
|
DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
|
|
|
.addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
- TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
|
|
|
- TaskAttemptEventType.TA_DONE, CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptEventType.TA_DONE, SUCCEEDED_FINISHING_TRANSITION)
|
|
|
.addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
- TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP, TaskAttemptEventType.TA_KILL,
|
|
|
+ TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
|
|
|
+ TaskAttemptEventType.TA_KILL,
|
|
|
CLEANUP_CONTAINER_TRANSITION)
|
|
|
// if container killed by AM shutting down
|
|
|
.addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
TaskAttemptStateInternal.KILLED,
|
|
|
TaskAttemptEventType.TA_CONTAINER_CLEANED, new KilledTransition())
|
|
|
.addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
- TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
|
- TaskAttemptEventType.TA_FAILMSG, CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ TaskAttemptStateInternal.FAIL_FINISHING_CONTAINER,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG, FAILED_FINISHING_TRANSITION)
|
|
|
.addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
+ CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ .addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
+ TaskAttemptStateInternal.FAILED,
|
|
|
TaskAttemptEventType.TA_CONTAINER_COMPLETED,
|
|
|
- CLEANUP_CONTAINER_TRANSITION)
|
|
|
+ FINALIZE_FAILED_TRANSITION)
|
|
|
.addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
|
|
|
TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
|
|
|
TaskAttemptEventType.TA_TIMED_OUT, CLEANUP_CONTAINER_TRANSITION)
|
|
@@ -348,8 +460,8 @@ public abstract class TaskAttemptImpl implements
|
|
|
// Transitions from SUCCESS_CONTAINER_CLEANUP state
|
|
|
// kill and cleanup the container
|
|
|
.addTransition(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
|
|
|
- TaskAttemptStateInternal.SUCCEEDED, TaskAttemptEventType.TA_CONTAINER_CLEANED,
|
|
|
- new SucceededTransition())
|
|
|
+ TaskAttemptStateInternal.SUCCEEDED,
|
|
|
+ TaskAttemptEventType.TA_CONTAINER_CLEANED)
|
|
|
.addTransition(
|
|
|
TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
|
|
|
TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
|
|
@@ -360,6 +472,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
|
|
|
EnumSet.of(TaskAttemptEventType.TA_KILL,
|
|
|
TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
TaskAttemptEventType.TA_TIMED_OUT,
|
|
|
TaskAttemptEventType.TA_CONTAINER_COMPLETED))
|
|
|
|
|
@@ -383,6 +496,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
|
|
|
TaskAttemptEventType.TA_DONE,
|
|
|
TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
TaskAttemptEventType.TA_TIMED_OUT))
|
|
|
|
|
|
// Transitions from KILL_CONTAINER_CLEANUP
|
|
@@ -405,6 +519,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
|
|
|
TaskAttemptEventType.TA_DONE,
|
|
|
TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
TaskAttemptEventType.TA_TIMED_OUT))
|
|
|
|
|
|
// Transitions from FAIL_TASK_CLEANUP
|
|
@@ -425,6 +540,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEventType.TA_COMMIT_PENDING,
|
|
|
TaskAttemptEventType.TA_DONE,
|
|
|
TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
TaskAttemptEventType.TA_CONTAINER_CLEANED,
|
|
|
// Container launch events can arrive late
|
|
|
TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
|
|
@@ -447,6 +563,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEventType.TA_COMMIT_PENDING,
|
|
|
TaskAttemptEventType.TA_DONE,
|
|
|
TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
TaskAttemptEventType.TA_CONTAINER_CLEANED,
|
|
|
TaskAttemptEventType.TA_PREEMPTED,
|
|
|
// Container launch events can arrive late
|
|
@@ -460,7 +577,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
new TooManyFetchFailureTransition())
|
|
|
.addTransition(TaskAttemptStateInternal.SUCCEEDED,
|
|
|
EnumSet.of(TaskAttemptStateInternal.SUCCEEDED, TaskAttemptStateInternal.KILLED),
|
|
|
- TaskAttemptEventType.TA_KILL,
|
|
|
+ TaskAttemptEventType.TA_KILL,
|
|
|
new KilledAfterSuccessTransition())
|
|
|
.addTransition(
|
|
|
TaskAttemptStateInternal.SUCCEEDED, TaskAttemptStateInternal.SUCCEEDED,
|
|
@@ -470,6 +587,10 @@ public abstract class TaskAttemptImpl implements
|
|
|
.addTransition(TaskAttemptStateInternal.SUCCEEDED,
|
|
|
TaskAttemptStateInternal.SUCCEEDED,
|
|
|
EnumSet.of(TaskAttemptEventType.TA_FAILMSG,
|
|
|
+ TaskAttemptEventType.TA_FAILMSG_BY_CLIENT,
|
|
|
+ // TaskAttemptFinishingMonitor might time out the attempt right
|
|
|
+ // after the attempt receives TA_CONTAINER_COMPLETED.
|
|
|
+ TaskAttemptEventType.TA_TIMED_OUT,
|
|
|
TaskAttemptEventType.TA_CONTAINER_CLEANED,
|
|
|
TaskAttemptEventType.TA_CONTAINER_COMPLETED))
|
|
|
|
|
@@ -1213,21 +1334,21 @@ public abstract class TaskAttemptImpl implements
|
|
|
return TaskAttemptState.STARTING;
|
|
|
case COMMIT_PENDING:
|
|
|
return TaskAttemptState.COMMIT_PENDING;
|
|
|
- case FAILED:
|
|
|
- return TaskAttemptState.FAILED;
|
|
|
- case KILLED:
|
|
|
- return TaskAttemptState.KILLED;
|
|
|
- // All CLEANUP states considered as RUNNING since events have not gone out
|
|
|
- // to the Task yet. May be possible to consider them as a Finished state.
|
|
|
case FAIL_CONTAINER_CLEANUP:
|
|
|
case FAIL_TASK_CLEANUP:
|
|
|
+ case FAIL_FINISHING_CONTAINER:
|
|
|
+ case FAILED:
|
|
|
+ return TaskAttemptState.FAILED;
|
|
|
case KILL_CONTAINER_CLEANUP:
|
|
|
case KILL_TASK_CLEANUP:
|
|
|
- case SUCCESS_CONTAINER_CLEANUP:
|
|
|
+ case KILLED:
|
|
|
+ return TaskAttemptState.KILLED;
|
|
|
case RUNNING:
|
|
|
return TaskAttemptState.RUNNING;
|
|
|
case NEW:
|
|
|
return TaskAttemptState.NEW;
|
|
|
+ case SUCCESS_CONTAINER_CLEANUP:
|
|
|
+ case SUCCESS_FINISHING_CONTAINER:
|
|
|
case SUCCEEDED:
|
|
|
return TaskAttemptState.SUCCEEDED;
|
|
|
default:
|
|
@@ -1429,6 +1550,15 @@ public abstract class TaskAttemptImpl implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static void finalizeProgress(TaskAttemptImpl taskAttempt) {
|
|
|
+ // unregister it to TaskAttemptListener so that it stops listening
|
|
|
+ taskAttempt.taskAttemptListener.unregister(
|
|
|
+ taskAttempt.attemptId, taskAttempt.jvmID);
|
|
|
+ taskAttempt.reportedStatus.progress = 1.0f;
|
|
|
+ taskAttempt.updateProgressSplits();
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
static class RequestContainerTransition implements
|
|
|
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
private final boolean rescheduled;
|
|
@@ -1661,53 +1791,66 @@ public abstract class TaskAttemptImpl implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static class SucceededTransition implements
|
|
|
+ /**
|
|
|
+ * Transition from SUCCESS_FINISHING_CONTAINER or FAIL_FINISHING_CONTAINER
|
|
|
+ * state upon receiving TA_CONTAINER_COMPLETED event
|
|
|
+ */
|
|
|
+ private static class ExitFinishingOnContainerCompletedTransition implements
|
|
|
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
@SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
- public void transition(TaskAttemptImpl taskAttempt,
|
|
|
+ public void transition(TaskAttemptImpl taskAttempt,
|
|
|
+ TaskAttemptEvent event) {
|
|
|
+ taskAttempt.appContext.getTaskAttemptFinishingMonitor().unregister(
|
|
|
+ taskAttempt.attemptId);
|
|
|
+ sendContainerCompleted(taskAttempt);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class ExitFinishingOnContainerCleanedupTransition implements
|
|
|
+ SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Override
|
|
|
+ public void transition(TaskAttemptImpl taskAttempt,
|
|
|
TaskAttemptEvent event) {
|
|
|
- //set the finish time
|
|
|
- taskAttempt.setFinishTime();
|
|
|
- taskAttempt.eventHandler.handle(
|
|
|
- createJobCounterUpdateEventTASucceeded(taskAttempt));
|
|
|
- taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
|
|
|
- taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
|
|
|
- taskAttempt.attemptId,
|
|
|
- TaskEventType.T_ATTEMPT_SUCCEEDED));
|
|
|
- taskAttempt.eventHandler.handle
|
|
|
- (new SpeculatorEvent
|
|
|
- (taskAttempt.reportedStatus, taskAttempt.clock.getTime()));
|
|
|
- }
|
|
|
+ taskAttempt.appContext.getTaskAttemptFinishingMonitor().unregister(
|
|
|
+ taskAttempt.attemptId);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private static class FailedTransition implements
|
|
|
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
@SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
- public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
|
|
|
+ public void transition(TaskAttemptImpl taskAttempt,
|
|
|
+ TaskAttemptEvent event) {
|
|
|
// set the finish time
|
|
|
taskAttempt.setFinishTime();
|
|
|
-
|
|
|
- if (taskAttempt.getLaunchTime() != 0) {
|
|
|
- taskAttempt.eventHandler
|
|
|
- .handle(createJobCounterUpdateEventTAFailed(taskAttempt, false));
|
|
|
- TaskAttemptUnsuccessfulCompletionEvent tauce =
|
|
|
- createTaskAttemptUnsuccessfulCompletionEvent(taskAttempt,
|
|
|
- TaskAttemptStateInternal.FAILED);
|
|
|
- taskAttempt.eventHandler.handle(new JobHistoryEvent(
|
|
|
- taskAttempt.attemptId.getTaskId().getJobId(), tauce));
|
|
|
- // taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.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.attemptId, TaskEventType.T_ATTEMPT_FAILED));
|
|
|
+ notifyTaskAttemptFailed(taskAttempt);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static class FinalizeFailedTransition extends FailedTransition {
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Override
|
|
|
+ public void transition(TaskAttemptImpl taskAttempt,
|
|
|
+ TaskAttemptEvent event) {
|
|
|
+ finalizeProgress(taskAttempt);
|
|
|
+ sendContainerCompleted(taskAttempt);
|
|
|
+ super.transition(taskAttempt, event);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ private static void sendContainerCompleted(TaskAttemptImpl taskAttempt) {
|
|
|
+ taskAttempt.eventHandler.handle(new ContainerLauncherEvent(
|
|
|
+ taskAttempt.attemptId,
|
|
|
+ taskAttempt.container.getId(), StringInterner
|
|
|
+ .weakIntern(taskAttempt.container.getNodeId().toString()),
|
|
|
+ taskAttempt.container.getContainerToken(),
|
|
|
+ ContainerLauncher.EventType.CONTAINER_COMPLETED));
|
|
|
+ }
|
|
|
+
|
|
|
private static class RecoverTransition implements
|
|
|
MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptStateInternal> {
|
|
|
|
|
@@ -1832,6 +1975,35 @@ public abstract class TaskAttemptImpl implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static class KilledAfterSucceededFinishingTransition
|
|
|
+ implements MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent,
|
|
|
+ TaskAttemptStateInternal> {
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Override
|
|
|
+ public TaskAttemptStateInternal transition(TaskAttemptImpl taskAttempt,
|
|
|
+ TaskAttemptEvent event) {
|
|
|
+ taskAttempt.appContext.getTaskAttemptFinishingMonitor().unregister(
|
|
|
+ taskAttempt.attemptId);
|
|
|
+ sendContainerCleanup(taskAttempt, event);
|
|
|
+ if(taskAttempt.getID().getTaskId().getTaskType() == TaskType.REDUCE) {
|
|
|
+ // after a reduce task has succeeded, its outputs are in safe in HDFS.
|
|
|
+ // logically such a task should not be killed. we only come here when
|
|
|
+ // there is a race condition in the event queue. E.g. some logic sends
|
|
|
+ // a kill request to this attempt when the successful completion event
|
|
|
+ // for this task is already in the event queue. so the kill event will
|
|
|
+ // get executed immediately after the attempt is marked successful and
|
|
|
+ // result in this transition being exercised.
|
|
|
+ // ignore this for reduce tasks
|
|
|
+ LOG.info("Ignoring killed event for successful reduce task attempt" +
|
|
|
+ taskAttempt.getID().toString());
|
|
|
+ return TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP;
|
|
|
+ } else {
|
|
|
+ return TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private static class KilledTransition implements
|
|
|
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
|
|
@@ -1887,6 +2059,31 @@ public abstract class TaskAttemptImpl implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Transition from SUCCESS_FINISHING_CONTAINER or FAIL_FINISHING_CONTAINER
|
|
|
+ * state upon receiving TA_TIMED_OUT event
|
|
|
+ */
|
|
|
+ private static class ExitFinishingOnTimeoutTransition implements
|
|
|
+ SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Override
|
|
|
+ public void transition(TaskAttemptImpl taskAttempt,
|
|
|
+ TaskAttemptEvent event) {
|
|
|
+ taskAttempt.appContext.getTaskAttemptFinishingMonitor().unregister(
|
|
|
+ taskAttempt.attemptId);
|
|
|
+ // The attempt stays in finishing state for too long
|
|
|
+ String msg = "Task attempt " + taskAttempt.getID() + " is done from " +
|
|
|
+ "TaskUmbilicalProtocol's point of view. However, it stays in " +
|
|
|
+ "finishing state for too long";
|
|
|
+ LOG.warn(msg);
|
|
|
+ taskAttempt.addDiagnosticInfo(msg);
|
|
|
+ sendContainerCleanup(taskAttempt, event);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Finish and clean up the container
|
|
|
+ */
|
|
|
private static class CleanupContainerTransition implements
|
|
|
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
@SuppressWarnings("unchecked")
|
|
@@ -1894,27 +2091,103 @@ public abstract class TaskAttemptImpl implements
|
|
|
public void transition(TaskAttemptImpl taskAttempt,
|
|
|
TaskAttemptEvent event) {
|
|
|
// unregister it to TaskAttemptListener so that it stops listening
|
|
|
- // for it
|
|
|
- taskAttempt.taskAttemptListener.unregister(
|
|
|
- taskAttempt.attemptId, taskAttempt.jvmID);
|
|
|
+ // for it.
|
|
|
+ finalizeProgress(taskAttempt);
|
|
|
+ sendContainerCleanup(taskAttempt, event);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- if (event instanceof TaskAttemptKillEvent) {
|
|
|
- taskAttempt.addDiagnosticInfo(
|
|
|
- ((TaskAttemptKillEvent) event).getMessage());
|
|
|
- }
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ private static void sendContainerCleanup(TaskAttemptImpl taskAttempt,
|
|
|
+ TaskAttemptEvent event) {
|
|
|
+ if (event instanceof TaskAttemptKillEvent) {
|
|
|
+ taskAttempt.addDiagnosticInfo(
|
|
|
+ ((TaskAttemptKillEvent) event).getMessage());
|
|
|
+ }
|
|
|
+ //send the cleanup event to containerLauncher
|
|
|
+ taskAttempt.eventHandler.handle(new ContainerLauncherEvent(
|
|
|
+ taskAttempt.attemptId,
|
|
|
+ taskAttempt.container.getId(), StringInterner
|
|
|
+ .weakIntern(taskAttempt.container.getNodeId().toString()),
|
|
|
+ taskAttempt.container.getContainerToken(),
|
|
|
+ ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Transition to SUCCESS_FINISHING_CONTAINER upon receiving TA_DONE event
|
|
|
+ */
|
|
|
+ private static class MoveContainerToSucceededFinishingTransition implements
|
|
|
+ SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Override
|
|
|
+ public void transition(TaskAttemptImpl taskAttempt,
|
|
|
+ TaskAttemptEvent event) {
|
|
|
+ finalizeProgress(taskAttempt);
|
|
|
+
|
|
|
+ // register it to finishing state
|
|
|
+ taskAttempt.appContext.getTaskAttemptFinishingMonitor().register(
|
|
|
+ taskAttempt.attemptId);
|
|
|
+
|
|
|
+ // set the finish time
|
|
|
+ taskAttempt.setFinishTime();
|
|
|
+
|
|
|
+ // notify job history
|
|
|
+ taskAttempt.eventHandler.handle(
|
|
|
+ createJobCounterUpdateEventTASucceeded(taskAttempt));
|
|
|
+ taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
|
|
|
+
|
|
|
+ //notify the task even though the container might not have exited yet.
|
|
|
+ taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
|
|
|
+ taskAttempt.attemptId,
|
|
|
+ TaskEventType.T_ATTEMPT_SUCCEEDED));
|
|
|
+ taskAttempt.eventHandler.handle
|
|
|
+ (new SpeculatorEvent
|
|
|
+ (taskAttempt.reportedStatus, taskAttempt.clock.getTime()));
|
|
|
|
|
|
- taskAttempt.reportedStatus.progress = 1.0f;
|
|
|
- taskAttempt.updateProgressSplits();
|
|
|
- //send the cleanup event to containerLauncher
|
|
|
- taskAttempt.eventHandler.handle(new ContainerLauncherEvent(
|
|
|
- taskAttempt.attemptId,
|
|
|
- taskAttempt.container.getId(), StringInterner
|
|
|
- .weakIntern(taskAttempt.container.getNodeId().toString()),
|
|
|
- taskAttempt.container.getContainerToken(),
|
|
|
- ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Transition to FAIL_FINISHING_CONTAINER upon receiving TA_FAILMSG event
|
|
|
+ */
|
|
|
+ private static class MoveContainerToFailedFinishingTransition implements
|
|
|
+ SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Override
|
|
|
+ public void transition(TaskAttemptImpl taskAttempt,
|
|
|
+ TaskAttemptEvent event) {
|
|
|
+ finalizeProgress(taskAttempt);
|
|
|
+ // register it to finishing state
|
|
|
+ taskAttempt.appContext.getTaskAttemptFinishingMonitor().register(
|
|
|
+ taskAttempt.attemptId);
|
|
|
+ notifyTaskAttemptFailed(taskAttempt);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ private static void notifyTaskAttemptFailed(TaskAttemptImpl taskAttempt) {
|
|
|
+ // set the finish time
|
|
|
+ taskAttempt.setFinishTime();
|
|
|
+
|
|
|
+ if (taskAttempt.getLaunchTime() != 0) {
|
|
|
+ taskAttempt.eventHandler
|
|
|
+ .handle(createJobCounterUpdateEventTAFailed(taskAttempt, false));
|
|
|
+ TaskAttemptUnsuccessfulCompletionEvent tauce =
|
|
|
+ createTaskAttemptUnsuccessfulCompletionEvent(taskAttempt,
|
|
|
+ TaskAttemptStateInternal.FAILED);
|
|
|
+ taskAttempt.eventHandler.handle(new JobHistoryEvent(
|
|
|
+ taskAttempt.attemptId.getTaskId().getJobId(), tauce));
|
|
|
+ // taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.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.attemptId, TaskEventType.T_ATTEMPT_FAILED));
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
private void addDiagnosticInfo(String diag) {
|
|
|
if (diag != null && !diag.equals("")) {
|
|
|
diagnostics.add(diag);
|