|
@@ -47,6 +47,7 @@ import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.VersionUtil;
|
|
import org.apache.hadoop.util.VersionUtil;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
@@ -61,6 +62,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factories.impl.pb.RecordFactoryPBImpl;
|
|
import org.apache.hadoop.yarn.factories.impl.pb.RecordFactoryPBImpl;
|
|
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
|
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
|
|
|
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
|
import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
|
|
import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
|
|
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
|
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
|
import org.apache.hadoop.yarn.server.api.ServerRMProxy;
|
|
import org.apache.hadoop.yarn.server.api.ServerRMProxy;
|
|
@@ -85,6 +87,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
|
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|
import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
|
|
import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
|
|
import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
|
|
import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
|
|
|
|
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.util.YarnVersionInfo;
|
|
import org.apache.hadoop.yarn.util.YarnVersionInfo;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
@@ -528,14 +531,56 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
containerStatuses.add(containerStatus);
|
|
containerStatuses.add(containerStatus);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // Account for all containers that got killed while they were still queued.
|
|
|
|
+ pendingCompletedContainers.putAll(getKilledQueuedContainerStatuses());
|
|
|
|
+
|
|
containerStatuses.addAll(pendingCompletedContainers.values());
|
|
containerStatuses.addAll(pendingCompletedContainers.values());
|
|
|
|
+
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Sending out " + containerStatuses.size()
|
|
LOG.debug("Sending out " + containerStatuses.size()
|
|
+ " container statuses: " + containerStatuses);
|
|
+ " container statuses: " + containerStatuses);
|
|
}
|
|
}
|
|
return containerStatuses;
|
|
return containerStatuses;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Add to the container statuses the status of the containers that got killed
|
|
|
|
+ * while they were queued.
|
|
|
|
+ */
|
|
|
|
+ private Map<ContainerId, ContainerStatus> getKilledQueuedContainerStatuses() {
|
|
|
|
+ Map<ContainerId, ContainerStatus> killedQueuedContainerStatuses =
|
|
|
|
+ new HashMap<>();
|
|
|
|
+ for (Map.Entry<ContainerTokenIdentifier, String> killedQueuedContainer :
|
|
|
|
+ this.context.getQueuingContext().
|
|
|
|
+ getKilledQueuedContainers().entrySet()) {
|
|
|
|
+ ContainerTokenIdentifier containerTokenId = killedQueuedContainer
|
|
|
|
+ .getKey();
|
|
|
|
+ ContainerId containerId = containerTokenId.getContainerID();
|
|
|
|
+ ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
|
|
|
|
+ containerId, ContainerState.COMPLETE,
|
|
|
|
+ killedQueuedContainer.getValue(), ContainerExitStatus.ABORTED,
|
|
|
|
+ containerTokenId.getResource(), containerTokenId.getExecutionType());
|
|
|
|
+ ApplicationId applicationId = containerId.getApplicationAttemptId()
|
|
|
|
+ .getApplicationId();
|
|
|
|
+ if (isApplicationStopped(applicationId)) {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug(applicationId + " is completing, " + " remove "
|
|
|
|
+ + containerId + " from NM context.");
|
|
|
|
+ }
|
|
|
|
+ this.context.getQueuingContext().getKilledQueuedContainers()
|
|
|
|
+ .remove(containerTokenId);
|
|
|
|
+ killedQueuedContainerStatuses.put(containerId, containerStatus);
|
|
|
|
+ } else {
|
|
|
|
+ if (!isContainerRecentlyStopped(containerId)) {
|
|
|
|
+ killedQueuedContainerStatuses.put(containerId, containerStatus);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ addCompletedContainer(containerId);
|
|
|
|
+ }
|
|
|
|
+ return killedQueuedContainerStatuses;
|
|
|
|
+ }
|
|
|
|
+
|
|
private List<ApplicationId> getRunningApplications() {
|
|
private List<ApplicationId> getRunningApplications() {
|
|
List<ApplicationId> runningApplications = new ArrayList<ApplicationId>();
|
|
List<ApplicationId> runningApplications = new ArrayList<ApplicationId>();
|
|
runningApplications.addAll(this.context.getApplications().keySet());
|
|
runningApplications.addAll(this.context.getApplications().keySet());
|
|
@@ -601,6 +646,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
public void removeOrTrackCompletedContainersFromContext(
|
|
public void removeOrTrackCompletedContainersFromContext(
|
|
List<ContainerId> containerIds) throws IOException {
|
|
List<ContainerId> containerIds) throws IOException {
|
|
Set<ContainerId> removedContainers = new HashSet<ContainerId>();
|
|
Set<ContainerId> removedContainers = new HashSet<ContainerId>();
|
|
|
|
+ Set<ContainerId> removedNullContainers = new HashSet<ContainerId>();
|
|
|
|
|
|
pendingContainersToRemove.addAll(containerIds);
|
|
pendingContainersToRemove.addAll(containerIds);
|
|
Iterator<ContainerId> iter = pendingContainersToRemove.iterator();
|
|
Iterator<ContainerId> iter = pendingContainersToRemove.iterator();
|
|
@@ -610,6 +656,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
Container nmContainer = context.getContainers().get(containerId);
|
|
Container nmContainer = context.getContainers().get(containerId);
|
|
if (nmContainer == null) {
|
|
if (nmContainer == null) {
|
|
iter.remove();
|
|
iter.remove();
|
|
|
|
+ removedNullContainers.add(containerId);
|
|
} else if (nmContainer.getContainerState().equals(
|
|
} else if (nmContainer.getContainerState().equals(
|
|
org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.DONE)) {
|
|
org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.DONE)) {
|
|
context.getContainers().remove(containerId);
|
|
context.getContainers().remove(containerId);
|
|
@@ -618,6 +665,17 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ // Remove null containers from queuing context for killed queued containers.
|
|
|
|
+ Iterator<ContainerTokenIdentifier> killedQueuedContIter =
|
|
|
|
+ context.getQueuingContext().getKilledQueuedContainers().keySet().
|
|
|
|
+ iterator();
|
|
|
|
+ while (killedQueuedContIter.hasNext()) {
|
|
|
|
+ if (removedNullContainers.contains(
|
|
|
|
+ killedQueuedContIter.next().getContainerID())) {
|
|
|
|
+ killedQueuedContIter.remove();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
if (!removedContainers.isEmpty()) {
|
|
if (!removedContainers.isEmpty()) {
|
|
LOG.info("Removed completed containers from NM context: "
|
|
LOG.info("Removed completed containers from NM context: "
|
|
+ removedContainers);
|
|
+ removedContainers);
|