|
@@ -35,7 +35,6 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
@@ -58,8 +57,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl.ProcessTreeInfo;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerState;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -128,8 +125,6 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
|
|
|
startAllocatedContainer(allocatedContInfo);
|
|
|
} else {
|
|
|
ContainerId cIdToStart = containerTokenIdentifier.getContainerID();
|
|
|
- this.context.getNMStateStore().storeContainer(cIdToStart, request);
|
|
|
- this.context.getNMStateStore().storeContainerQueued(cIdToStart);
|
|
|
LOG.info("No available resources for container {} to start its execution "
|
|
|
+ "immediately.", cIdToStart);
|
|
|
if (allocatedContInfo.getExecutionType() == ExecutionType.GUARANTEED) {
|
|
@@ -164,7 +159,6 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
|
|
|
this.context.getQueuingContext().getKilledQueuedContainers().put(
|
|
|
containerTokenId,
|
|
|
"Queued container request removed by ApplicationMaster.");
|
|
|
- this.context.getNMStateStore().storeContainerKilled(containerID);
|
|
|
} else {
|
|
|
// The container started execution in the meanwhile.
|
|
|
try {
|
|
@@ -474,38 +468,6 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
|
|
|
return super.getContainerStatusInternal(containerID, nmTokenIdentifier);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Recover running or queued container.
|
|
|
- */
|
|
|
- @Override
|
|
|
- protected void recoverActiveContainer(
|
|
|
- ContainerLaunchContext launchContext, ContainerTokenIdentifier token,
|
|
|
- RecoveredContainerState rcs) throws IOException {
|
|
|
- if (rcs.getStatus() ==
|
|
|
- RecoveredContainerStatus.QUEUED && !rcs.getKilled()) {
|
|
|
- LOG.info(token.getContainerID()
|
|
|
- + "will be added to the queued containers.");
|
|
|
-
|
|
|
- AllocatedContainerInfo allocatedContInfo = new AllocatedContainerInfo(
|
|
|
- token, rcs.getStartRequest(), token.getExecutionType(),
|
|
|
- token.getResource(), getConfig());
|
|
|
-
|
|
|
- this.context.getQueuingContext().getQueuedContainers().put(
|
|
|
- token.getContainerID(), token);
|
|
|
-
|
|
|
- if (allocatedContInfo.getExecutionType() == ExecutionType.GUARANTEED) {
|
|
|
- queuedGuaranteedContainers.add(allocatedContInfo);
|
|
|
- // Kill running opportunistic containers to make space for
|
|
|
- // guaranteed container.
|
|
|
- killOpportunisticContainers(allocatedContInfo);
|
|
|
- } else {
|
|
|
- queuedOpportunisticContainers.add(allocatedContInfo);
|
|
|
- }
|
|
|
- } else {
|
|
|
- super.recoverActiveContainer(launchContext, token, rcs);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
public int getNumAllocatedGuaranteedContainers() {
|
|
|
return allocatedGuaranteedContainers.size();
|