|
@@ -117,7 +117,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
-import org.apache.hadoop.yarn.api.records.ContainerToken;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
|
@@ -490,14 +489,10 @@ public abstract class TaskAttemptImpl implements
|
|
|
<TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent>
|
|
|
stateMachine;
|
|
|
|
|
|
- private ContainerId containerID;
|
|
|
- private NodeId containerNodeId;
|
|
|
- private String containerMgrAddress;
|
|
|
- private String nodeHttpAddress;
|
|
|
+ @VisibleForTesting
|
|
|
+ public Container container;
|
|
|
private String nodeRackName;
|
|
|
private WrappedJvmID jvmID;
|
|
|
- private ContainerToken containerToken;
|
|
|
- private Resource assignedCapability;
|
|
|
|
|
|
//this takes good amount of memory ~ 30KB. Instantiate it lazily
|
|
|
//and make it null once task is launched.
|
|
@@ -825,7 +820,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
public ContainerId getAssignedContainerID() {
|
|
|
readLock.lock();
|
|
|
try {
|
|
|
- return containerID;
|
|
|
+ return container == null ? null : container.getId();
|
|
|
} finally {
|
|
|
readLock.unlock();
|
|
|
}
|
|
@@ -835,7 +830,8 @@ public abstract class TaskAttemptImpl implements
|
|
|
public String getAssignedContainerMgrAddress() {
|
|
|
readLock.lock();
|
|
|
try {
|
|
|
- return containerMgrAddress;
|
|
|
+ return container == null ? null : StringInterner.weakIntern(container
|
|
|
+ .getNodeId().toString());
|
|
|
} finally {
|
|
|
readLock.unlock();
|
|
|
}
|
|
@@ -895,7 +891,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
public NodeId getNodeId() {
|
|
|
readLock.lock();
|
|
|
try {
|
|
|
- return containerNodeId;
|
|
|
+ return container == null ? null : container.getNodeId();
|
|
|
} finally {
|
|
|
readLock.unlock();
|
|
|
}
|
|
@@ -907,7 +903,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
public String getNodeHttpAddress() {
|
|
|
readLock.lock();
|
|
|
try {
|
|
|
- return nodeHttpAddress;
|
|
|
+ return container == null ? null : container.getNodeHttpAddress();
|
|
|
} finally {
|
|
|
readLock.unlock();
|
|
|
}
|
|
@@ -967,8 +963,8 @@ public abstract class TaskAttemptImpl implements
|
|
|
result.setContainerId(this.getAssignedContainerID());
|
|
|
result.setNodeManagerHost(trackerName);
|
|
|
result.setNodeManagerHttpPort(httpPort);
|
|
|
- if (this.containerNodeId != null) {
|
|
|
- result.setNodeManagerPort(this.containerNodeId.getPort());
|
|
|
+ if (this.container != null) {
|
|
|
+ result.setNodeManagerPort(this.container.getNodeId().getPort());
|
|
|
}
|
|
|
return result;
|
|
|
} finally {
|
|
@@ -1093,13 +1089,17 @@ public abstract class TaskAttemptImpl implements
|
|
|
@SuppressWarnings("unchecked")
|
|
|
public TaskAttemptStateInternal recover(TaskAttemptInfo taInfo,
|
|
|
OutputCommitter committer, boolean recoverOutput) {
|
|
|
- containerID = taInfo.getContainerId();
|
|
|
- containerNodeId = ConverterUtils.toNodeId(taInfo.getHostname() + ":"
|
|
|
+ ContainerId containerId = taInfo.getContainerId();
|
|
|
+ NodeId containerNodeId = ConverterUtils.toNodeId(taInfo.getHostname() + ":"
|
|
|
+ taInfo.getPort());
|
|
|
- containerMgrAddress = StringInterner.weakIntern(
|
|
|
- containerNodeId.toString());
|
|
|
- nodeHttpAddress = StringInterner.weakIntern(taInfo.getHostname() + ":"
|
|
|
+ String nodeHttpAddress = StringInterner.weakIntern(taInfo.getHostname() + ":"
|
|
|
+ taInfo.getHttpPort());
|
|
|
+ // Resource/Priority/Tokens are only needed while launching the
|
|
|
+ // container on an NM, these are already completed tasks, so setting them to
|
|
|
+ // null
|
|
|
+ container =
|
|
|
+ BuilderUtils.newContainer(containerId, containerNodeId,
|
|
|
+ nodeHttpAddress, null, null, null);
|
|
|
computeRackAndLocality();
|
|
|
launchTime = taInfo.getStartTime();
|
|
|
finishTime = (taInfo.getFinishTime() != -1) ?
|
|
@@ -1227,6 +1227,7 @@ public abstract class TaskAttemptImpl implements
|
|
|
}
|
|
|
|
|
|
private void computeRackAndLocality() {
|
|
|
+ NodeId containerNodeId = container.getNodeId();
|
|
|
nodeRackName = RackResolver.resolve(
|
|
|
containerNodeId.getHost()).getNetworkLocation();
|
|
|
|
|
@@ -1331,10 +1332,10 @@ public abstract class TaskAttemptImpl implements
|
|
|
TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId()
|
|
|
.getTaskType()), attemptState.toString(),
|
|
|
taskAttempt.finishTime,
|
|
|
- taskAttempt.containerNodeId == null ? "UNKNOWN"
|
|
|
- : taskAttempt.containerNodeId.getHost(),
|
|
|
- taskAttempt.containerNodeId == null ? -1
|
|
|
- : taskAttempt.containerNodeId.getPort(),
|
|
|
+ taskAttempt.container == null ? "UNKNOWN"
|
|
|
+ : taskAttempt.container.getNodeId().getHost(),
|
|
|
+ taskAttempt.container == null ? -1
|
|
|
+ : taskAttempt.container.getNodeId().getPort(),
|
|
|
taskAttempt.nodeRackName == null ? "UNKNOWN"
|
|
|
: taskAttempt.nodeRackName,
|
|
|
StringUtils.join(
|
|
@@ -1353,12 +1354,12 @@ public abstract class TaskAttemptImpl implements
|
|
|
eventHandler.handle(jce);
|
|
|
|
|
|
LOG.info("TaskAttempt: [" + attemptId
|
|
|
- + "] using containerId: [" + containerID + " on NM: ["
|
|
|
- + containerMgrAddress + "]");
|
|
|
+ + "] using containerId: [" + container.getId() + " on NM: ["
|
|
|
+ + StringInterner.weakIntern(container.getNodeId().toString()) + "]");
|
|
|
TaskAttemptStartedEvent tase =
|
|
|
new TaskAttemptStartedEvent(TypeConverter.fromYarn(attemptId),
|
|
|
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
|
|
|
- launchTime, trackerName, httpPort, shufflePort, containerID,
|
|
|
+ launchTime, trackerName, httpPort, shufflePort, container.getId(),
|
|
|
locality.toString(), avataar.toString());
|
|
|
eventHandler.handle(
|
|
|
new JobHistoryEvent(attemptId.getTaskId().getJobId(), tase));
|
|
@@ -1490,19 +1491,14 @@ public abstract class TaskAttemptImpl implements
|
|
|
TaskAttemptEvent event) {
|
|
|
final TaskAttemptContainerAssignedEvent cEvent =
|
|
|
(TaskAttemptContainerAssignedEvent) event;
|
|
|
- taskAttempt.containerID = cEvent.getContainer().getId();
|
|
|
- taskAttempt.containerNodeId = cEvent.getContainer().getNodeId();
|
|
|
- taskAttempt.containerMgrAddress = StringInterner.weakIntern(
|
|
|
- taskAttempt.containerNodeId.toString());
|
|
|
- taskAttempt.nodeHttpAddress = StringInterner.weakIntern(
|
|
|
- cEvent.getContainer().getNodeHttpAddress());
|
|
|
- taskAttempt.containerToken = cEvent.getContainer().getContainerToken();
|
|
|
- taskAttempt.assignedCapability = cEvent.getContainer().getResource();
|
|
|
+ Container container = cEvent.getContainer();
|
|
|
+ taskAttempt.container = container;
|
|
|
// this is a _real_ Task (classic Hadoop mapred flavor):
|
|
|
taskAttempt.remoteTask = taskAttempt.createRemoteTask();
|
|
|
- taskAttempt.jvmID = new WrappedJvmID(
|
|
|
- taskAttempt.remoteTask.getTaskID().getJobID(),
|
|
|
- taskAttempt.remoteTask.isMapTask(), taskAttempt.containerID.getId());
|
|
|
+ taskAttempt.jvmID =
|
|
|
+ new WrappedJvmID(taskAttempt.remoteTask.getTaskID().getJobID(),
|
|
|
+ taskAttempt.remoteTask.isMapTask(), taskAttempt.container.getId()
|
|
|
+ .getId());
|
|
|
taskAttempt.taskAttemptListener.registerPendingTask(
|
|
|
taskAttempt.remoteTask, taskAttempt.jvmID);
|
|
|
|
|
@@ -1514,10 +1510,9 @@ public abstract class TaskAttemptImpl implements
|
|
|
cEvent.getApplicationACLs(), taskAttempt.conf, taskAttempt.jobToken,
|
|
|
taskAttempt.remoteTask, taskAttempt.oldJobId, taskAttempt.jvmID,
|
|
|
taskAttempt.taskAttemptListener, taskAttempt.credentials);
|
|
|
- taskAttempt.eventHandler.handle(new ContainerRemoteLaunchEvent(
|
|
|
- taskAttempt.attemptId, taskAttempt.containerID,
|
|
|
- taskAttempt.containerMgrAddress, taskAttempt.containerToken,
|
|
|
- launchContext, taskAttempt.assignedCapability, taskAttempt.remoteTask));
|
|
|
+ taskAttempt.eventHandler
|
|
|
+ .handle(new ContainerRemoteLaunchEvent(taskAttempt.attemptId,
|
|
|
+ launchContext, container, taskAttempt.remoteTask));
|
|
|
|
|
|
// send event to speculator that our container needs are satisfied
|
|
|
taskAttempt.eventHandler.handle
|
|
@@ -1604,9 +1599,8 @@ public abstract class TaskAttemptImpl implements
|
|
|
taskAttempt.taskAttemptListener
|
|
|
.registerLaunchedTask(taskAttempt.attemptId, taskAttempt.jvmID);
|
|
|
//TODO Resolve to host / IP in case of a local address.
|
|
|
- InetSocketAddress nodeHttpInetAddr =
|
|
|
- NetUtils.createSocketAddr(taskAttempt.nodeHttpAddress); // TODO:
|
|
|
- // Costly?
|
|
|
+ InetSocketAddress nodeHttpInetAddr = // TODO: Costly to create sock-addr?
|
|
|
+ NetUtils.createSocketAddr(taskAttempt.container.getNodeHttpAddress());
|
|
|
taskAttempt.trackerName = nodeHttpInetAddr.getHostName();
|
|
|
taskAttempt.httpPort = nodeHttpInetAddr.getPort();
|
|
|
taskAttempt.sendLaunchedEvents();
|
|
@@ -1713,6 +1707,10 @@ public abstract class TaskAttemptImpl implements
|
|
|
private void logAttemptFinishedEvent(TaskAttemptStateInternal state) {
|
|
|
//Log finished events only if an attempt started.
|
|
|
if (getLaunchTime() == 0) return;
|
|
|
+ String containerHostName = this.container == null ? "UNKNOWN"
|
|
|
+ : this.container.getNodeId().getHost();
|
|
|
+ int containerNodePort =
|
|
|
+ this.container == null ? -1 : this.container.getNodeId().getPort();
|
|
|
if (attemptId.getTaskId().getTaskType() == TaskType.MAP) {
|
|
|
MapAttemptFinishedEvent mfe =
|
|
|
new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
|
|
@@ -1720,9 +1718,8 @@ public abstract class TaskAttemptImpl implements
|
|
|
state.toString(),
|
|
|
this.reportedStatus.mapFinishTime,
|
|
|
finishTime,
|
|
|
- this.containerNodeId == null ? "UNKNOWN"
|
|
|
- : this.containerNodeId.getHost(),
|
|
|
- this.containerNodeId == null ? -1 : this.containerNodeId.getPort(),
|
|
|
+ containerHostName,
|
|
|
+ containerNodePort,
|
|
|
this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName,
|
|
|
this.reportedStatus.stateString,
|
|
|
getCounters(),
|
|
@@ -1737,9 +1734,8 @@ public abstract class TaskAttemptImpl implements
|
|
|
this.reportedStatus.shuffleFinishTime,
|
|
|
this.reportedStatus.sortFinishTime,
|
|
|
finishTime,
|
|
|
- this.containerNodeId == null ? "UNKNOWN"
|
|
|
- : this.containerNodeId.getHost(),
|
|
|
- this.containerNodeId == null ? -1 : this.containerNodeId.getPort(),
|
|
|
+ containerHostName,
|
|
|
+ containerNodePort,
|
|
|
this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName,
|
|
|
this.reportedStatus.stateString,
|
|
|
getCounters(),
|
|
@@ -1864,8 +1860,9 @@ public abstract class TaskAttemptImpl implements
|
|
|
//send the cleanup event to containerLauncher
|
|
|
taskAttempt.eventHandler.handle(new ContainerLauncherEvent(
|
|
|
taskAttempt.attemptId,
|
|
|
- taskAttempt.containerID, taskAttempt.containerMgrAddress,
|
|
|
- taskAttempt.containerToken,
|
|
|
+ taskAttempt.container.getId(), StringInterner
|
|
|
+ .weakIntern(taskAttempt.container.getNodeId().toString()),
|
|
|
+ taskAttempt.container.getContainerToken(),
|
|
|
ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP));
|
|
|
}
|
|
|
}
|