|
@@ -265,7 +265,8 @@ public abstract class TaskAttemptImpl implements
|
|
|
|
|
|
// Transitions from the UNASSIGNED state.
|
|
|
.addTransition(TaskAttemptStateInternal.UNASSIGNED,
|
|
|
- TaskAttemptStateInternal.ASSIGNED, TaskAttemptEventType.TA_ASSIGNED,
|
|
|
+ EnumSet.of(TaskAttemptStateInternal.ASSIGNED,
|
|
|
+ TaskAttemptStateInternal.RUNNING), TaskAttemptEventType.TA_ASSIGNED,
|
|
|
new ContainerAssignedTransition())
|
|
|
.addTransition(TaskAttemptStateInternal.UNASSIGNED, TaskAttemptStateInternal.KILLED,
|
|
|
TaskAttemptEventType.TA_KILL, new DeallocateContainerTransition(
|
|
@@ -1876,13 +1877,14 @@ public abstract class TaskAttemptImpl implements
|
|
|
}
|
|
|
|
|
|
private static class ContainerAssignedTransition implements
|
|
|
- SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
+ MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent,
|
|
|
+ TaskAttemptStateInternal> {
|
|
|
@SuppressWarnings({ "unchecked" })
|
|
|
@Override
|
|
|
- public void transition(final TaskAttemptImpl taskAttempt,
|
|
|
- TaskAttemptEvent event) {
|
|
|
- final TaskAttemptContainerAssignedEvent cEvent =
|
|
|
- (TaskAttemptContainerAssignedEvent) event;
|
|
|
+ public TaskAttemptStateInternal transition(
|
|
|
+ final TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
|
|
|
+ final TaskAttemptContainerAssignedEvent cEvent =
|
|
|
+ (TaskAttemptContainerAssignedEvent) event;
|
|
|
Container container = cEvent.getContainer();
|
|
|
taskAttempt.container = container;
|
|
|
// this is a _real_ Task (classic Hadoop mapred flavor):
|
|
@@ -1895,20 +1897,26 @@ public abstract class TaskAttemptImpl implements
|
|
|
taskAttempt.remoteTask, taskAttempt.jvmID);
|
|
|
|
|
|
taskAttempt.computeRackAndLocality();
|
|
|
-
|
|
|
- //launch the container
|
|
|
- //create the container object to be launched for a given Task attempt
|
|
|
- ContainerLaunchContext launchContext = createContainerLaunchContext(
|
|
|
- cEvent.getApplicationACLs(), taskAttempt.conf, taskAttempt.jobToken,
|
|
|
- taskAttempt.remoteTask, taskAttempt.oldJobId, taskAttempt.jvmID,
|
|
|
- taskAttempt.taskAttemptListener, taskAttempt.credentials);
|
|
|
- taskAttempt.eventHandler
|
|
|
- .handle(new ContainerRemoteLaunchEvent(taskAttempt.attemptId,
|
|
|
- launchContext, container, taskAttempt.remoteTask));
|
|
|
|
|
|
- // send event to speculator that our container needs are satisfied
|
|
|
- taskAttempt.eventHandler.handle
|
|
|
- (new SpeculatorEvent(taskAttempt.getID().getTaskId(), -1));
|
|
|
+ if (cEvent.getShufflePort() == -1) {
|
|
|
+ // launch the container
|
|
|
+ // create the container object to be launched for a given Task attempt
|
|
|
+ ContainerLaunchContext launchContext = createContainerLaunchContext(
|
|
|
+ cEvent.getApplicationACLs(), taskAttempt.conf, taskAttempt.jobToken,
|
|
|
+ taskAttempt.remoteTask, taskAttempt.oldJobId, taskAttempt.jvmID,
|
|
|
+ taskAttempt.taskAttemptListener, taskAttempt.credentials);
|
|
|
+ taskAttempt.eventHandler
|
|
|
+ .handle(new ContainerRemoteLaunchEvent(taskAttempt.attemptId,
|
|
|
+ launchContext, container, taskAttempt.remoteTask));
|
|
|
+
|
|
|
+ // send event to speculator that our container needs are satisfied
|
|
|
+ taskAttempt.eventHandler
|
|
|
+ .handle(new SpeculatorEvent(taskAttempt.getID().getTaskId(), -1));
|
|
|
+ return TaskAttemptStateInternal.ASSIGNED;
|
|
|
+ } else {
|
|
|
+ taskAttempt.onContainerLaunch(cEvent.getShufflePort());
|
|
|
+ return TaskAttemptStateInternal.RUNNING;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1982,7 +1990,6 @@ public abstract class TaskAttemptImpl implements
|
|
|
|
|
|
private static class LaunchedContainerTransition implements
|
|
|
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
public void transition(TaskAttemptImpl taskAttempt,
|
|
|
TaskAttemptEvent evnt) {
|
|
@@ -1990,34 +1997,34 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptContainerLaunchedEvent event =
|
|
|
(TaskAttemptContainerLaunchedEvent) evnt;
|
|
|
|
|
|
- //set the launch time
|
|
|
- taskAttempt.launchTime = taskAttempt.clock.getTime();
|
|
|
- taskAttempt.shufflePort = event.getShufflePort();
|
|
|
+ taskAttempt.onContainerLaunch(event.getShufflePort());
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- // register it to TaskAttemptListener so that it can start monitoring it.
|
|
|
- taskAttempt.taskAttemptListener
|
|
|
- .registerLaunchedTask(taskAttempt.attemptId, taskAttempt.jvmID);
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ private void onContainerLaunch(int shufflePortParam) {
|
|
|
+ // set the launch time
|
|
|
+ launchTime = clock.getTime();
|
|
|
+ this.shufflePort = shufflePortParam;
|
|
|
+
|
|
|
+ // register it to TaskAttemptListener so that it can start monitoring it.
|
|
|
+ taskAttemptListener.registerLaunchedTask(attemptId, jvmID);
|
|
|
+ // TODO Resolve to host / IP in case of a local address.
|
|
|
+ InetSocketAddress nodeHttpInetAddr = // TODO: Costly to create sock-addr?
|
|
|
+ NetUtils.createSocketAddr(container.getNodeHttpAddress());
|
|
|
+ trackerName = nodeHttpInetAddr.getHostName();
|
|
|
+ httpPort = nodeHttpInetAddr.getPort();
|
|
|
+ sendLaunchedEvents();
|
|
|
+ eventHandler.handle(new SpeculatorEvent(attemptId, true, clock.getTime()));
|
|
|
+ // make remoteTask reference as null as it is no more needed
|
|
|
+ // and free up the memory
|
|
|
+ remoteTask = null;
|
|
|
|
|
|
- //TODO Resolve to host / IP in case of a local address.
|
|
|
- InetSocketAddress nodeHttpInetAddr = // TODO: Costly to create sock-addr?
|
|
|
- NetUtils.createSocketAddr(taskAttempt.container.getNodeHttpAddress());
|
|
|
- taskAttempt.trackerName = nodeHttpInetAddr.getHostName();
|
|
|
- taskAttempt.httpPort = nodeHttpInetAddr.getPort();
|
|
|
- taskAttempt.sendLaunchedEvents();
|
|
|
- taskAttempt.eventHandler.handle
|
|
|
- (new SpeculatorEvent
|
|
|
- (taskAttempt.attemptId, true, taskAttempt.clock.getTime()));
|
|
|
- //make remoteTask reference as null as it is no more needed
|
|
|
- //and free up the memory
|
|
|
- taskAttempt.remoteTask = null;
|
|
|
-
|
|
|
- //tell the Task that attempt has started
|
|
|
- taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
|
|
|
- taskAttempt.attemptId,
|
|
|
- TaskEventType.T_ATTEMPT_LAUNCHED));
|
|
|
- }
|
|
|
+ // tell the Task that attempt has started
|
|
|
+ eventHandler.handle(
|
|
|
+ new TaskTAttemptEvent(attemptId, TaskEventType.T_ATTEMPT_LAUNCHED));
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private static class CommitPendingTransition implements
|
|
|
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
|
|
|
@SuppressWarnings("unchecked")
|