|
@@ -22,9 +22,11 @@ import java.util.ArrayList;
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
import java.util.Comparator;
|
|
import java.util.Comparator;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
|
|
+import java.util.HashSet;
|
|
import java.util.LinkedHashMap;
|
|
import java.util.LinkedHashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.Set;
|
|
import java.util.concurrent.locks.Lock;
|
|
import java.util.concurrent.locks.Lock;
|
|
import java.util.concurrent.locks.ReadWriteLock;
|
|
import java.util.concurrent.locks.ReadWriteLock;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
@@ -118,9 +120,18 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
protected Credentials credentials;
|
|
protected Credentials credentials;
|
|
protected Token<JobTokenIdentifier> jobToken;
|
|
protected Token<JobTokenIdentifier> jobToken;
|
|
|
|
|
|
|
|
+ //should be set to one which comes first
|
|
|
|
+ //saying COMMIT_PENDING
|
|
|
|
+ private TaskAttemptId commitAttempt;
|
|
|
|
+
|
|
|
|
+ private TaskAttemptId successfulAttempt;
|
|
|
|
+
|
|
|
|
+ private final Set<TaskAttemptId> failedAttempts;
|
|
|
|
+ // Track the finished attempts - successful, failed and killed
|
|
|
|
+ private final Set<TaskAttemptId> finishedAttempts;
|
|
// counts the number of attempts that are either running or in a state where
|
|
// counts the number of attempts that are either running or in a state where
|
|
// 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 final Set<TaskAttemptId> inProgressAttempts;
|
|
|
|
|
|
private boolean historyTaskStartGenerated = false;
|
|
private boolean historyTaskStartGenerated = false;
|
|
|
|
|
|
@@ -182,6 +193,14 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
EnumSet.of(TaskStateInternal.KILL_WAIT, TaskStateInternal.KILLED),
|
|
EnumSet.of(TaskStateInternal.KILL_WAIT, TaskStateInternal.KILLED),
|
|
TaskEventType.T_ATTEMPT_KILLED,
|
|
TaskEventType.T_ATTEMPT_KILLED,
|
|
new KillWaitAttemptKilledTransition())
|
|
new KillWaitAttemptKilledTransition())
|
|
|
|
+ .addTransition(TaskStateInternal.KILL_WAIT,
|
|
|
|
+ EnumSet.of(TaskStateInternal.KILL_WAIT, TaskStateInternal.KILLED),
|
|
|
|
+ TaskEventType.T_ATTEMPT_SUCCEEDED,
|
|
|
|
+ new KillWaitAttemptSucceededTransition())
|
|
|
|
+ .addTransition(TaskStateInternal.KILL_WAIT,
|
|
|
|
+ EnumSet.of(TaskStateInternal.KILL_WAIT, TaskStateInternal.KILLED),
|
|
|
|
+ TaskEventType.T_ATTEMPT_FAILED,
|
|
|
|
+ new KillWaitAttemptFailedTransition())
|
|
// Ignore-able transitions.
|
|
// Ignore-able transitions.
|
|
.addTransition(
|
|
.addTransition(
|
|
TaskStateInternal.KILL_WAIT,
|
|
TaskStateInternal.KILL_WAIT,
|
|
@@ -189,8 +208,6 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
EnumSet.of(TaskEventType.T_KILL,
|
|
EnumSet.of(TaskEventType.T_KILL,
|
|
TaskEventType.T_ATTEMPT_LAUNCHED,
|
|
TaskEventType.T_ATTEMPT_LAUNCHED,
|
|
TaskEventType.T_ATTEMPT_COMMIT_PENDING,
|
|
TaskEventType.T_ATTEMPT_COMMIT_PENDING,
|
|
- TaskEventType.T_ATTEMPT_FAILED,
|
|
|
|
- TaskEventType.T_ATTEMPT_SUCCEEDED,
|
|
|
|
TaskEventType.T_ADD_SPEC_ATTEMPT))
|
|
TaskEventType.T_ADD_SPEC_ATTEMPT))
|
|
|
|
|
|
// Transitions from SUCCEEDED state
|
|
// Transitions from SUCCEEDED state
|
|
@@ -242,15 +259,6 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
private static final RecoverdAttemptsComparator RECOVERED_ATTEMPTS_COMPARATOR =
|
|
private static final RecoverdAttemptsComparator RECOVERED_ATTEMPTS_COMPARATOR =
|
|
new RecoverdAttemptsComparator();
|
|
new RecoverdAttemptsComparator();
|
|
|
|
|
|
- //should be set to one which comes first
|
|
|
|
- //saying COMMIT_PENDING
|
|
|
|
- private TaskAttemptId commitAttempt;
|
|
|
|
-
|
|
|
|
- private TaskAttemptId successfulAttempt;
|
|
|
|
-
|
|
|
|
- private int failedAttempts;
|
|
|
|
- private int finishedAttempts;//finish are total of success, failed and killed
|
|
|
|
-
|
|
|
|
@Override
|
|
@Override
|
|
public TaskState getState() {
|
|
public TaskState getState() {
|
|
readLock.lock();
|
|
readLock.lock();
|
|
@@ -275,6 +283,9 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
readLock = readWriteLock.readLock();
|
|
readLock = readWriteLock.readLock();
|
|
writeLock = readWriteLock.writeLock();
|
|
writeLock = readWriteLock.writeLock();
|
|
this.attempts = Collections.emptyMap();
|
|
this.attempts = Collections.emptyMap();
|
|
|
|
+ this.finishedAttempts = new HashSet<TaskAttemptId>(2);
|
|
|
|
+ this.failedAttempts = new HashSet<TaskAttemptId>(2);
|
|
|
|
+ this.inProgressAttempts = new HashSet<TaskAttemptId>(2);
|
|
// This overridable method call is okay in a constructor because we
|
|
// This overridable method call is okay in a constructor because we
|
|
// have a convention that none of the overrides depends on any
|
|
// have a convention that none of the overrides depends on any
|
|
// fields that need initialization.
|
|
// fields that need initialization.
|
|
@@ -611,9 +622,9 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
taskAttemptsFromPreviousGeneration.remove(0).getAttemptId().getId();
|
|
taskAttemptsFromPreviousGeneration.remove(0).getAttemptId().getId();
|
|
}
|
|
}
|
|
|
|
|
|
- ++numberUncompletedAttempts;
|
|
|
|
|
|
+ inProgressAttempts.add(attempt.getID());
|
|
//schedule the nextAttemptNumber
|
|
//schedule the nextAttemptNumber
|
|
- if (failedAttempts > 0) {
|
|
|
|
|
|
+ if (failedAttempts.size() > 0) {
|
|
eventHandler.handle(new TaskAttemptEvent(attempt.getID(),
|
|
eventHandler.handle(new TaskAttemptEvent(attempt.getID(),
|
|
TaskAttemptEventType.TA_RESCHEDULE));
|
|
TaskAttemptEventType.TA_RESCHEDULE));
|
|
} else {
|
|
} else {
|
|
@@ -788,12 +799,14 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
implements SingleArcTransition<TaskImpl, TaskEvent> {
|
|
implements SingleArcTransition<TaskImpl, TaskEvent> {
|
|
@Override
|
|
@Override
|
|
public void transition(TaskImpl task, TaskEvent event) {
|
|
public void transition(TaskImpl task, TaskEvent event) {
|
|
|
|
+ TaskTAttemptEvent taskTAttemptEvent = (TaskTAttemptEvent) event;
|
|
|
|
+ TaskAttemptId taskAttemptId = taskTAttemptEvent.getTaskAttemptID();
|
|
task.handleTaskAttemptCompletion(
|
|
task.handleTaskAttemptCompletion(
|
|
- ((TaskTAttemptEvent) event).getTaskAttemptID(),
|
|
|
|
|
|
+ taskAttemptId,
|
|
TaskAttemptCompletionEventStatus.SUCCEEDED);
|
|
TaskAttemptCompletionEventStatus.SUCCEEDED);
|
|
- task.finishedAttempts++;
|
|
|
|
- --task.numberUncompletedAttempts;
|
|
|
|
- task.successfulAttempt = ((TaskTAttemptEvent) event).getTaskAttemptID();
|
|
|
|
|
|
+ task.finishedAttempts.add(taskAttemptId);
|
|
|
|
+ task.inProgressAttempts.remove(taskAttemptId);
|
|
|
|
+ task.successfulAttempt = taskAttemptId;
|
|
task.eventHandler.handle(new JobTaskEvent(
|
|
task.eventHandler.handle(new JobTaskEvent(
|
|
task.taskId, TaskState.SUCCEEDED));
|
|
task.taskId, TaskState.SUCCEEDED));
|
|
LOG.info("Task succeeded with attempt " + task.successfulAttempt);
|
|
LOG.info("Task succeeded with attempt " + task.successfulAttempt);
|
|
@@ -824,11 +837,13 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
SingleArcTransition<TaskImpl, TaskEvent> {
|
|
SingleArcTransition<TaskImpl, TaskEvent> {
|
|
@Override
|
|
@Override
|
|
public void transition(TaskImpl task, TaskEvent event) {
|
|
public void transition(TaskImpl task, TaskEvent event) {
|
|
|
|
+ TaskAttemptId taskAttemptId =
|
|
|
|
+ ((TaskTAttemptEvent) event).getTaskAttemptID();
|
|
task.handleTaskAttemptCompletion(
|
|
task.handleTaskAttemptCompletion(
|
|
- ((TaskTAttemptEvent) event).getTaskAttemptID(),
|
|
|
|
|
|
+ taskAttemptId,
|
|
TaskAttemptCompletionEventStatus.KILLED);
|
|
TaskAttemptCompletionEventStatus.KILLED);
|
|
- task.finishedAttempts++;
|
|
|
|
- --task.numberUncompletedAttempts;
|
|
|
|
|
|
+ task.finishedAttempts.add(taskAttemptId);
|
|
|
|
+ task.inProgressAttempts.remove(taskAttemptId);
|
|
if (task.successfulAttempt == null) {
|
|
if (task.successfulAttempt == null) {
|
|
task.addAndScheduleAttempt();
|
|
task.addAndScheduleAttempt();
|
|
}
|
|
}
|
|
@@ -840,15 +855,25 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
|
|
MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
|
|
|
|
|
|
protected TaskStateInternal finalState = TaskStateInternal.KILLED;
|
|
protected TaskStateInternal finalState = TaskStateInternal.KILLED;
|
|
|
|
+ protected final TaskAttemptCompletionEventStatus taCompletionEventStatus;
|
|
|
|
+
|
|
|
|
+ public KillWaitAttemptKilledTransition() {
|
|
|
|
+ this(TaskAttemptCompletionEventStatus.KILLED);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public KillWaitAttemptKilledTransition(
|
|
|
|
+ TaskAttemptCompletionEventStatus taCompletionEventStatus) {
|
|
|
|
+ this.taCompletionEventStatus = taCompletionEventStatus;
|
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
|
|
public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
|
|
- task.handleTaskAttemptCompletion(
|
|
|
|
- ((TaskTAttemptEvent) event).getTaskAttemptID(),
|
|
|
|
- TaskAttemptCompletionEventStatus.KILLED);
|
|
|
|
- task.finishedAttempts++;
|
|
|
|
|
|
+ TaskAttemptId taskAttemptId =
|
|
|
|
+ ((TaskTAttemptEvent) event).getTaskAttemptID();
|
|
|
|
+ task.handleTaskAttemptCompletion(taskAttemptId, taCompletionEventStatus);
|
|
|
|
+ task.finishedAttempts.add(taskAttemptId);
|
|
// check whether all attempts are finished
|
|
// check whether all attempts are finished
|
|
- if (task.finishedAttempts == task.attempts.size()) {
|
|
|
|
|
|
+ if (task.finishedAttempts.size() == task.attempts.size()) {
|
|
if (task.historyTaskStartGenerated) {
|
|
if (task.historyTaskStartGenerated) {
|
|
TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, null,
|
|
TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, null,
|
|
finalState, null); // TODO JH verify failedAttempt null
|
|
finalState, null); // TODO JH verify failedAttempt null
|
|
@@ -867,43 +892,57 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private static class KillWaitAttemptSucceededTransition extends
|
|
|
|
+ KillWaitAttemptKilledTransition {
|
|
|
|
+ public KillWaitAttemptSucceededTransition() {
|
|
|
|
+ super(TaskAttemptCompletionEventStatus.SUCCEEDED);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private static class KillWaitAttemptFailedTransition extends
|
|
|
|
+ KillWaitAttemptKilledTransition {
|
|
|
|
+ public KillWaitAttemptFailedTransition() {
|
|
|
|
+ super(TaskAttemptCompletionEventStatus.FAILED);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private static class AttemptFailedTransition implements
|
|
private static class AttemptFailedTransition implements
|
|
MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
|
|
MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
|
|
public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
|
|
- task.failedAttempts++;
|
|
|
|
TaskTAttemptEvent castEvent = (TaskTAttemptEvent) event;
|
|
TaskTAttemptEvent castEvent = (TaskTAttemptEvent) event;
|
|
- if (castEvent.getTaskAttemptID().equals(task.commitAttempt)) {
|
|
|
|
|
|
+ TaskAttemptId taskAttemptId = castEvent.getTaskAttemptID();
|
|
|
|
+ task.failedAttempts.add(taskAttemptId);
|
|
|
|
+ if (taskAttemptId.equals(task.commitAttempt)) {
|
|
task.commitAttempt = null;
|
|
task.commitAttempt = null;
|
|
}
|
|
}
|
|
- TaskAttempt attempt = task.attempts.get(castEvent.getTaskAttemptID());
|
|
|
|
|
|
+ TaskAttempt attempt = task.attempts.get(taskAttemptId);
|
|
if (attempt.getAssignedContainerMgrAddress() != null) {
|
|
if (attempt.getAssignedContainerMgrAddress() != null) {
|
|
//container was assigned
|
|
//container was assigned
|
|
task.eventHandler.handle(new ContainerFailedEvent(attempt.getID(),
|
|
task.eventHandler.handle(new ContainerFailedEvent(attempt.getID(),
|
|
attempt.getAssignedContainerMgrAddress()));
|
|
attempt.getAssignedContainerMgrAddress()));
|
|
}
|
|
}
|
|
|
|
|
|
- task.finishedAttempts++;
|
|
|
|
- if (task.failedAttempts < task.maxAttempts) {
|
|
|
|
|
|
+ task.finishedAttempts.add(taskAttemptId);
|
|
|
|
+ if (task.failedAttempts.size() < task.maxAttempts) {
|
|
task.handleTaskAttemptCompletion(
|
|
task.handleTaskAttemptCompletion(
|
|
- ((TaskTAttemptEvent) event).getTaskAttemptID(),
|
|
|
|
|
|
+ taskAttemptId,
|
|
TaskAttemptCompletionEventStatus.FAILED);
|
|
TaskAttemptCompletionEventStatus.FAILED);
|
|
// we don't need a new event if we already have a spare
|
|
// we don't need a new event if we already have a spare
|
|
- if (--task.numberUncompletedAttempts == 0
|
|
|
|
|
|
+ task.inProgressAttempts.remove(taskAttemptId);
|
|
|
|
+ if (task.inProgressAttempts.size() == 0
|
|
&& task.successfulAttempt == null) {
|
|
&& task.successfulAttempt == null) {
|
|
task.addAndScheduleAttempt();
|
|
task.addAndScheduleAttempt();
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
task.handleTaskAttemptCompletion(
|
|
task.handleTaskAttemptCompletion(
|
|
- ((TaskTAttemptEvent) event).getTaskAttemptID(),
|
|
|
|
|
|
+ taskAttemptId,
|
|
TaskAttemptCompletionEventStatus.TIPFAILED);
|
|
TaskAttemptCompletionEventStatus.TIPFAILED);
|
|
- TaskTAttemptEvent ev = (TaskTAttemptEvent) event;
|
|
|
|
- TaskAttemptId taId = ev.getTaskAttemptID();
|
|
|
|
|
|
|
|
if (task.historyTaskStartGenerated) {
|
|
if (task.historyTaskStartGenerated) {
|
|
TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, attempt.getDiagnostics(),
|
|
TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, attempt.getDiagnostics(),
|
|
- TaskStateInternal.FAILED, taId);
|
|
|
|
|
|
+ TaskStateInternal.FAILED, taskAttemptId);
|
|
task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
|
|
task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
|
|
taskFailedEvent));
|
|
taskFailedEvent));
|
|
} else {
|
|
} else {
|
|
@@ -927,14 +966,12 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
|
|
public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
|
|
- if (event instanceof TaskTAttemptEvent) {
|
|
|
|
- TaskTAttemptEvent castEvent = (TaskTAttemptEvent) event;
|
|
|
|
- if (task.getInternalState() == TaskStateInternal.SUCCEEDED &&
|
|
|
|
- !castEvent.getTaskAttemptID().equals(task.successfulAttempt)) {
|
|
|
|
- // don't allow a different task attempt to override a previous
|
|
|
|
- // succeeded state
|
|
|
|
- return TaskStateInternal.SUCCEEDED;
|
|
|
|
- }
|
|
|
|
|
|
+ TaskTAttemptEvent castEvent = (TaskTAttemptEvent) event;
|
|
|
|
+ if (task.getInternalState() == TaskStateInternal.SUCCEEDED &&
|
|
|
|
+ !castEvent.getTaskAttemptID().equals(task.successfulAttempt)) {
|
|
|
|
+ // don't allow a different task attempt to override a previous
|
|
|
|
+ // succeeded state
|
|
|
|
+ return TaskStateInternal.SUCCEEDED;
|
|
}
|
|
}
|
|
|
|
|
|
// a successful REDUCE task should not be overridden
|
|
// a successful REDUCE task should not be overridden
|
|
@@ -953,7 +990,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
// believe that there's no redundancy.
|
|
// believe that there's no redundancy.
|
|
unSucceed(task);
|
|
unSucceed(task);
|
|
// fake increase in Uncomplete attempts for super.transition
|
|
// fake increase in Uncomplete attempts for super.transition
|
|
- ++task.numberUncompletedAttempts;
|
|
|
|
|
|
+ task.inProgressAttempts.add(castEvent.getTaskAttemptID());
|
|
return super.transition(task, event);
|
|
return super.transition(task, event);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1045,7 +1082,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|
(attempt, "Task KILL is received. Killing attempt!");
|
|
(attempt, "Task KILL is received. Killing attempt!");
|
|
}
|
|
}
|
|
|
|
|
|
- task.numberUncompletedAttempts = 0;
|
|
|
|
|
|
+ task.inProgressAttempts.clear();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|