diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java index 323d31d..582389f 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java @@ -34,5 +34,8 @@ RUNNING, /** Completed container */ - COMPLETE + COMPLETE, + + /** Queued at the NM. */ + QUEUED } \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceUtilization.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceUtilization.java index 5f52f85..2ae4872 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceUtilization.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceUtilization.java @@ -44,6 +44,14 @@ public static ResourceUtilization newInstance(int pmem, int vmem, float cpu) { return utilization; } + @Public + @Unstable + public static ResourceUtilization newInstance( + ResourceUtilization resourceUtil) { + return newInstance(resourceUtil.getPhysicalMemory(), + resourceUtil.getVirtualMemory(), resourceUtil.getCPU()); + } + /** * Get used virtual memory. * @@ -147,4 +155,18 @@ public void addTo(int pmem, int vmem, float cpu) { this.setVirtualMemory(this.getVirtualMemory() + vmem); this.setCPU(this.getCPU() + cpu); } + + /** + * Subtract utilization from the current one. + * @param pmem Physical memory to be subtracted. + * @param vmem Virtual memory to be subtracted. + * @param cpu CPU utilization to be subtracted. + */ + @Public + @Unstable + public void subtractFrom(int pmem, int vmem, float cpu) { + this.setPhysicalMemory(this.getPhysicalMemory() - pmem); + this.setVirtualMemory(this.getVirtualMemory() - vmem); + this.setCPU(this.getCPU() - cpu); + } } \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 61d1d72..bd116ab 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -659,6 +659,11 @@ public static boolean isAclEnabled(Configuration conf) { /** Prefix for all node manager configs.*/ public static final String NM_PREFIX = "yarn.nodemanager."; + /** Enable Queuing of OPPORTUNISTIC containers. */ + public static final String NM_CONTAINER_QUEUING_ENABLED = NM_PREFIX + + "container-queuing-enabled"; + public static final boolean NM_CONTAINER_QUEUING_ENABLED_DEFAULT = false; + /** Environment variables that will be sent to containers.*/ public static final String NM_ADMIN_USER_ENV = NM_PREFIX + "admin-env"; public static final String DEFAULT_NM_ADMIN_USER_ENV = "MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX"; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto index 9392efd..2fe4eda 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto @@ -82,6 +82,7 @@ enum ContainerStateProto { C_NEW = 1; C_RUNNING = 2; C_COMPLETE = 3; + C_QUEUED = 4; } message ContainerProto { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java index 4fdd43c..a70d143 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java @@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; @@ -215,6 +216,13 @@ public static NodeReport newNodeReport(NodeId nodeId, NodeState nodeState, public static ContainerStatus newContainerStatus(ContainerId containerId, ContainerState containerState, String diagnostics, int exitStatus, Resource capability) { + return newContainerStatus(containerId, containerState, diagnostics, + exitStatus, capability, ExecutionType.GUARANTEED); + } + + public static ContainerStatus newContainerStatus(ContainerId containerId, + ContainerState containerState, String diagnostics, int exitStatus, + Resource capability, ExecutionType executionType) { ContainerStatus containerStatus = recordFactory .newRecordInstance(ContainerStatus.class); containerStatus.setState(containerState); @@ -222,6 +230,7 @@ public static ContainerStatus newContainerStatus(ContainerId containerId, containerStatus.setDiagnostics(diagnostics); containerStatus.setExitStatus(exitStatus); containerStatus.setCapability(capability); + containerStatus.setExecutionType(executionType); return containerStatus; } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java index d3251ae..988aae9 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java @@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; @@ -43,6 +44,15 @@ public interface Context { /** + * Interface exposing methods related to the queuing of containers in the NM. + */ + interface QueuingContext { + ConcurrentMap getQueuedContainers(); + + ConcurrentMap getKilledQueuedContainers(); + } + + /** * Return the nodeId. Usable only when the ContainerManager is started. * * @return the NodeId @@ -89,4 +99,13 @@ getLogAggregationStatusForApps(); NodeStatusUpdater getNodeStatusUpdater(); + + /** + * Returns a QueuingContext that provides information about the number of + * Containers Queued as well as the number of Containers that were queued and + * killed. + * + * @return QueuingContext + */ + QueuingContext getQueuingContext(); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java index 7c104d5..fece288 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java @@ -57,11 +57,13 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing.QueuingContainerManagerImpl; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeLabelsProvider; import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider; @@ -170,8 +172,14 @@ protected ContainerManagerImpl createContainerManager(Context context, ContainerExecutor exec, DeletionService del, NodeStatusUpdater nodeStatusUpdater, ApplicationACLsManager aclsManager, LocalDirsHandlerService dirsHandler) { - return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater, - metrics, dirsHandler); + if (getConfig().getBoolean(YarnConfiguration.NM_CONTAINER_QUEUING_ENABLED, + YarnConfiguration.NM_CONTAINER_QUEUING_ENABLED_DEFAULT)) { + return new QueuingContainerManagerImpl(context, exec, del, + nodeStatusUpdater, metrics, dirsHandler); + } else { + return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater, + metrics, dirsHandler); + } } protected WebServer createWebServer(Context nmContext, @@ -188,8 +196,9 @@ protected NMContext createNMContext( NMContainerTokenSecretManager containerTokenSecretManager, NMTokenSecretManagerInNM nmTokenSecretManager, NMStateStoreService stateStore) { - return new NMContext(containerTokenSecretManager, nmTokenSecretManager, - dirsHandler, aclsManager, stateStore); + NMContext ctx = new NMContext(containerTokenSecretManager, + nmTokenSecretManager, dirsHandler, aclsManager, stateStore); + return ctx; } protected void doSecureLogin() throws IOException { @@ -461,6 +470,8 @@ public void run() { logAggregationReportForApps; private NodeStatusUpdater nodeStatusUpdater; + private final QueuingContext queuingContext; + public NMContext(NMContainerTokenSecretManager containerTokenSecretManager, NMTokenSecretManagerInNM nmTokenSecretManager, LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager, @@ -475,6 +486,7 @@ public NMContext(NMContainerTokenSecretManager containerTokenSecretManager, this.stateStore = stateStore; this.logAggregationReportForApps = new ConcurrentLinkedQueue< LogAggregationReport>(); + this.queuingContext = new QueuingNMContext(); } /** @@ -595,8 +607,35 @@ public NodeStatusUpdater getNodeStatusUpdater() { public void setNodeStatusUpdater(NodeStatusUpdater nodeStatusUpdater) { this.nodeStatusUpdater = nodeStatusUpdater; } + + @Override + public QueuingContext getQueuingContext() { + return this.queuingContext; + } } + /** + * Class that keeps the context for containers queued at the NM. + */ + public static class QueuingNMContext implements Context.QueuingContext { + protected final ConcurrentMap + queuedContainers = new ConcurrentSkipListMap<>(); + + protected final ConcurrentMap + killedQueuedContainers = new ConcurrentHashMap<>(); + + @Override + public ConcurrentMap + getQueuedContainers() { + return this.queuedContainers; + } + + @Override + public ConcurrentMap + getKilledQueuedContainers() { + return this.killedQueuedContainers; + } + } /** * @return the node health checker diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java index 94d5c1e..04e00bf 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java @@ -160,11 +160,11 @@ private static final Log LOG = LogFactory.getLog(ContainerManagerImpl.class); - static final String INVALID_NMTOKEN_MSG = "Invalid NMToken"; + public static final String INVALID_NMTOKEN_MSG = "Invalid NMToken"; static final String INVALID_CONTAINERTOKEN_MSG = "Invalid ContainerToken"; - final Context context; + protected final Context context; private final ContainersMonitor containersMonitor; private Server server; private final ResourceLocalizationService rsrcLocalizationSrvc; @@ -172,7 +172,7 @@ private final AuxServices auxiliaryServices; private final NodeManagerMetrics metrics; - private final NodeStatusUpdater nodeStatusUpdater; + protected final NodeStatusUpdater nodeStatusUpdater; protected LocalDirsHandlerService dirsHandler; protected final AsyncDispatcher dispatcher; @@ -213,8 +213,7 @@ public ContainerManagerImpl(Context context, ContainerExecutor exec, auxiliaryServices.registerServiceListener(this); addService(auxiliaryServices); - this.containersMonitor = - new ContainersMonitorImpl(exec, dispatcher, this.context); + this.containersMonitor = createContainersMonitor(exec); addService(this.containersMonitor); dispatcher.register(ContainerEventType.class, @@ -233,8 +232,13 @@ public ContainerManagerImpl(Context context, ContainerExecutor exec, this.writeLock = lock.writeLock(); } + protected ContainersMonitor createContainersMonitor(ContainerExecutor exec) { + return new ContainersMonitorImpl(exec, dispatcher, this.context); + } + @Override public void serviceInit(Configuration conf) throws Exception { + LogHandler logHandler = createLogHandler(conf, this.context, this.deletionService); addIfService(logHandler); @@ -814,9 +818,10 @@ public StartContainersResponse startContainers( .getContainerType().equals(ContainerType.APPLICATION_MASTER)) { this.amrmProxyService.processApplicationStartRequest(request); } - - startContainerInternal(nmTokenIdentifier, containerTokenIdentifier, - request); + performContainerPreStartChecks(nmTokenIdentifier, request, + containerTokenIdentifier); + performContainerStart(nmTokenIdentifier, request, containerId, + containerTokenIdentifier); succeededContainers.add(containerId); } catch (YarnException e) { failedContainers.put(containerId, SerializedException.newInstance(e)); @@ -834,6 +839,50 @@ public StartContainersResponse startContainers( } } + private void performContainerPreStartChecks( + NMTokenIdentifier nmTokenIdentifier, StartContainerRequest request, + ContainerTokenIdentifier containerTokenIdentifier) + throws YarnException, InvalidToken { + /* + * 1) It should save the NMToken into NMTokenSecretManager. This is done + * here instead of RPC layer because at the time of opening/authenticating + * the connection it doesn't know what all RPC calls user will make on it. + * Also new NMToken is issued only at startContainer (once it gets + * renewed). + * + * 2) It should validate containerToken. Need to check below things. a) It + * is signed by correct master key (part of retrieve password). b) It + * belongs to correct Node Manager (part of retrieve password). c) It has + * correct RMIdentifier. d) It is not expired. + */ + authorizeStartAndResourceIncreaseRequest( + nmTokenIdentifier, containerTokenIdentifier, true); + // update NMToken + updateNMTokenIdentifier(nmTokenIdentifier); + + ContainerLaunchContext launchContext = request.getContainerLaunchContext(); + + Map serviceData = getAuxServiceMetaData(); + if (launchContext.getServiceData()!=null && + !launchContext.getServiceData().isEmpty()) { + for (Entry meta : launchContext.getServiceData() + .entrySet()) { + if (null == serviceData.get(meta.getKey())) { + throw new InvalidAuxServiceException("The auxService:" + meta.getKey() + + " does not exist"); + } + } + } + } + + protected void performContainerStart(NMTokenIdentifier nmTokenIdentifier, + StartContainerRequest request, ContainerId containerId, + ContainerTokenIdentifier containerTokenIdentifier) + throws YarnException, IOException { + startContainerInternal(nmTokenIdentifier, containerTokenIdentifier, + request); + } + private ContainerManagerApplicationProto buildAppProto(ApplicationId appId, String user, Credentials credentials, Map appAcls, @@ -876,26 +925,10 @@ private ContainerManagerApplicationProto buildAppProto(ApplicationId appId, } @SuppressWarnings("unchecked") - private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier, + protected void startContainerInternal(NMTokenIdentifier nmTokenIdentifier, ContainerTokenIdentifier containerTokenIdentifier, StartContainerRequest request) throws YarnException, IOException { - /* - * 1) It should save the NMToken into NMTokenSecretManager. This is done - * here instead of RPC layer because at the time of opening/authenticating - * the connection it doesn't know what all RPC calls user will make on it. - * Also new NMToken is issued only at startContainer (once it gets renewed). - * - * 2) It should validate containerToken. Need to check below things. a) It - * is signed by correct master key (part of retrieve password). b) It - * belongs to correct Node Manager (part of retrieve password). c) It has - * correct RMIdentifier. d) It is not expired. - */ - authorizeStartAndResourceIncreaseRequest( - nmTokenIdentifier, containerTokenIdentifier, true); - // update NMToken - updateNMTokenIdentifier(nmTokenIdentifier); - ContainerId containerId = containerTokenIdentifier.getContainerID(); String containerIdStr = containerId.toString(); String user = containerTokenIdentifier.getApplicationSubmitter(); @@ -904,18 +937,6 @@ private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier, ContainerLaunchContext launchContext = request.getContainerLaunchContext(); - Map serviceData = getAuxServiceMetaData(); - if (launchContext.getServiceData()!=null && - !launchContext.getServiceData().isEmpty()) { - for (Map.Entry meta : launchContext.getServiceData() - .entrySet()) { - if (null == serviceData.get(meta.getKey())) { - throw new InvalidAuxServiceException("The auxService:" + meta.getKey() - + " does not exist"); - } - } - } - Credentials credentials = YarnServerSecurityUtils.parseCredentials(launchContext); @@ -935,13 +956,14 @@ private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier, this.readLock.lock(); try { - if (!serviceStopped) { + if (!isServiceStopped()) { // Create the application - Application application = - new ApplicationImpl(dispatcher, user, applicationID, credentials, context); + Application application = new ApplicationImpl(dispatcher, user, + applicationID, credentials, context); if (null == context.getApplications().putIfAbsent(applicationID, application)) { - LOG.info("Creating a new application reference for app " + applicationID); + LOG.info("Creating a new application reference for app " + + applicationID); LogAggregationContext logAggregationContext = containerTokenIdentifier.getLogAggregationContext(); Map appAcls = @@ -1159,7 +1181,9 @@ public StopContainersResponse stopContainers(StopContainersRequest requests) } for (ContainerId id : requests.getContainerIds()) { try { - stopContainerInternal(identifier, id); + Container container = this.context.getContainers().get(id); + authorizeGetAndStopContainerRequest(id, container, true, identifier); + stopContainerInternal(id); succeededRequests.add(id); } catch (YarnException e) { failedRequests.put(id, SerializedException.newInstance(e)); @@ -1170,13 +1194,11 @@ public StopContainersResponse stopContainers(StopContainersRequest requests) } @SuppressWarnings("unchecked") - private void stopContainerInternal(NMTokenIdentifier nmTokenIdentifier, - ContainerId containerID) throws YarnException, IOException { + protected void stopContainerInternal(ContainerId containerID) + throws YarnException, IOException { String containerIDStr = containerID.toString(); Container container = this.context.getContainers().get(containerID); LOG.info("Stopping container with container Id: " + containerIDStr); - authorizeGetAndStopContainerRequest(containerID, container, true, - nmTokenIdentifier); if (container == null) { if (!nodeStatusUpdater.isContainerRecentlyStopped(containerID)) { @@ -1223,7 +1245,7 @@ public GetContainerStatusesResponse getContainerStatuses( failedRequests); } - private ContainerStatus getContainerStatusInternal(ContainerId containerID, + protected ContainerStatus getContainerStatusInternal(ContainerId containerID, NMTokenIdentifier nmTokenIdentifier) throws YarnException { String containerIDStr = containerID.toString(); Container container = this.context.getContainers().get(containerID); @@ -1413,4 +1435,8 @@ public Context getContext() { public Map getAuxServiceMetaData() { return this.auxiliaryServices.getMetaData(); } + + protected boolean isServiceStopped() { + return serviceStopped; + } } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java index da8a3a6..62db4de 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java @@ -436,7 +436,8 @@ public ContainerStatus cloneAndGetContainerStatus() { this.readLock.lock(); try { return BuilderUtils.newContainerStatus(this.containerId, - getCurrentState(), diagnostics.toString(), exitCode, getResource()); + getCurrentState(), diagnostics.toString(), exitCode, getResource(), + this.containerTokenIdentifier.getExecutionType()); } finally { this.readLock.unlock(); } @@ -490,7 +491,8 @@ private void sendFinishedEvents() { EventHandler eventHandler = dispatcher.getEventHandler(); eventHandler.handle(new ApplicationContainerFinishedEvent(containerId)); // Remove the container from the resource-monitor - eventHandler.handle(new ContainerStopMonitoringEvent(containerId)); + eventHandler.handle(new ContainerStopMonitoringEvent(containerId, + containerTokenIdentifier.getExecutionType())); // Tell the logService too eventHandler.handle(new LogHandlerContainerFinishedEvent( containerId, exitCode)); @@ -524,10 +526,10 @@ private void sendContainerMonitorStartEvent() { int cpuVcores = getResource().getVirtualCores(); long localizationDuration = containerLaunchStartTime - containerLocalizationStartTime; - dispatcher.getEventHandler().handle( - new ContainerStartMonitoringEvent(containerId, - vmemBytes, pmemBytes, cpuVcores, launchDuration, - localizationDuration)); + dispatcher.getEventHandler() + .handle(new ContainerStartMonitoringEvent(containerId, + containerTokenIdentifier.getExecutionType(), vmemBytes, pmemBytes, + cpuVcores, launchDuration, localizationDuration)); } private void addDiagnostics(String... diags) { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.java index c09bebf..a2086de 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; public class ContainerStartMonitoringEvent extends ContainersMonitorEvent { @@ -29,9 +30,10 @@ private final long localizationDuration; public ContainerStartMonitoringEvent(ContainerId containerId, - long vmemLimit, long pmemLimit, int cpuVcores, long launchDuration, - long localizationDuration) { - super(containerId, ContainersMonitorEventType.START_MONITORING_CONTAINER); + ExecutionType executionType, long vmemLimit, long pmemLimit, + int cpuVcores, long launchDuration, long localizationDuration) { + super(containerId, executionType, + ContainersMonitorEventType.START_MONITORING_CONTAINER); this.vmemLimit = vmemLimit; this.pmemLimit = pmemLimit; this.cpuVcores = cpuVcores; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStopMonitoringEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStopMonitoringEvent.java index 240c5c0..d883281 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStopMonitoringEvent.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStopMonitoringEvent.java @@ -19,11 +19,14 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; public class ContainerStopMonitoringEvent extends ContainersMonitorEvent { - public ContainerStopMonitoringEvent(ContainerId containerId) { - super(containerId, ContainersMonitorEventType.STOP_MONITORING_CONTAINER); + public ContainerStopMonitoringEvent(ContainerId containerId, + ExecutionType executionType) { + super(containerId, executionType, + ContainersMonitorEventType.STOP_MONITORING_CONTAINER); } } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEvent.java index 56e578b..a48a635 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEvent.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEvent.java @@ -19,21 +19,33 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.event.AbstractEvent; public class ContainersMonitorEvent extends AbstractEvent { private final ContainerId containerId; + private final ExecutionType executionType; public ContainersMonitorEvent(ContainerId containerId, ContainersMonitorEventType eventType) { + this(containerId, ExecutionType.GUARANTEED, eventType); + } + + public ContainersMonitorEvent(ContainerId containerId, + ExecutionType executionType, ContainersMonitorEventType eventType) { super(eventType); this.containerId = containerId; + this.executionType = executionType; } public ContainerId getContainerId() { return this.containerId; } + public ExecutionType getExecutionType() { + return this.executionType; + } + } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index 446e7a1..8a801e9 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -62,10 +62,10 @@ new ConcurrentHashMap<>(); private final ContainerExecutor containerExecutor; - private final Dispatcher eventDispatcher; - private final Context context; - private ResourceCalculatorPlugin resourceCalculatorPlugin; - private Configuration conf; + protected final Dispatcher eventDispatcher; + protected final Context context; + protected ResourceCalculatorPlugin resourceCalculatorPlugin; + protected Configuration conf; private static float vmemRatio; private Class processTreeClass; @@ -76,10 +76,10 @@ private boolean vmemCheckEnabled; private boolean containersMonitorEnabled; - private long maxVCoresAllottedForContainers; + protected long maxVCoresAllottedForContainers; private static final long UNKNOWN_MEMORY_LIMIT = -1L; - private int nodeCpuPercentageForYARN; + protected int nodeCpuPercentageForYARN; private ResourceUtilization containersUtilization; @@ -132,10 +132,11 @@ protected void serviceInit(Configuration conf) throws Exception { YarnConfiguration.DEFAULT_NM_CONTAINER_METRICS_UNREGISTER_DELAY_MS); long configuredPMemForContainers = - NodeManagerHardwareUtils.getContainerMemoryMB(conf) * 1024 * 1024L; + NodeManagerHardwareUtils.getContainerMemoryMB( + this.resourceCalculatorPlugin, conf) * 1024 * 1024L; long configuredVCoresForContainers = - NodeManagerHardwareUtils.getVCores(conf); + NodeManagerHardwareUtils.getVCores(this.resourceCalculatorPlugin, conf); // Setting these irrespective of whether checks are enabled. Required in // the UI. @@ -191,7 +192,7 @@ protected void serviceInit(Configuration conf) throws Exception { super.serviceInit(conf); } - private boolean isEnabled() { + protected boolean isEnabled() { if (resourceCalculatorPlugin == null) { LOG.info("ResourceCalculatorPlugin is unavailable on this system. " + this.getClass().getName() + " is disabled."); @@ -234,7 +235,7 @@ protected void serviceStop() throws Exception { } @VisibleForTesting - static class ProcessTreeInfo { + protected static class ProcessTreeInfo { private ContainerId containerId; private String pid; private ResourceCalculatorProcessTree pTree; @@ -714,40 +715,56 @@ public void handle(ContainersMonitorEvent monitoringEvent) { switch (monitoringEvent.getType()) { case START_MONITORING_CONTAINER: - ContainerStartMonitoringEvent startEvent = - (ContainerStartMonitoringEvent) monitoringEvent; - LOG.info("Starting resource-monitoring for " + containerId); - updateContainerMetrics(monitoringEvent); - trackingContainers.put(containerId, - new ProcessTreeInfo(containerId, null, null, - startEvent.getVmemLimit(), startEvent.getPmemLimit(), - startEvent.getCpuVcores())); + onStartMonitoringContainer(monitoringEvent, containerId); break; case STOP_MONITORING_CONTAINER: - LOG.info("Stopping resource-monitoring for " + containerId); - updateContainerMetrics(monitoringEvent); - trackingContainers.remove(containerId); + onStopMonitoringContainer(monitoringEvent, containerId); break; case CHANGE_MONITORING_CONTAINER_RESOURCE: - ChangeMonitoringContainerResourceEvent changeEvent = - (ChangeMonitoringContainerResourceEvent) monitoringEvent; - ProcessTreeInfo processTreeInfo = trackingContainers.get(containerId); - if (processTreeInfo == null) { - LOG.warn("Failed to track container " - + containerId.toString() - + ". It may have already completed."); - break; - } - LOG.info("Changing resource-monitoring for " + containerId); - updateContainerMetrics(monitoringEvent); - long pmemLimit = changeEvent.getResource().getMemory() * 1024L * 1024L; - long vmemLimit = (long) (pmemLimit * vmemRatio); - int cpuVcores = changeEvent.getResource().getVirtualCores(); - processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores); - changeContainerResource(containerId, changeEvent.getResource()); + onChangeMonitoringContainerResource(monitoringEvent, containerId); break; default: // TODO: Wrong event. } } + + protected void onChangeMonitoringContainerResource( + ContainersMonitorEvent monitoringEvent, ContainerId containerId) { + ChangeMonitoringContainerResourceEvent changeEvent = + (ChangeMonitoringContainerResourceEvent) monitoringEvent; + ProcessTreeInfo processTreeInfo = trackingContainers.get(containerId); + if (processTreeInfo == null) { + LOG.warn("Failed to track container " + + containerId.toString() + + ". It may have already completed."); + return; + } + LOG.info("Changing resource-monitoring for " + containerId); + updateContainerMetrics(monitoringEvent); + long pmemLimit = changeEvent.getResource().getMemory() * 1024L * 1024L; + long vmemLimit = (long) (pmemLimit * vmemRatio); + int cpuVcores = changeEvent.getResource().getVirtualCores(); + processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores); + changeContainerResource(containerId, changeEvent.getResource()); + } + + protected void onStopMonitoringContainer( + ContainersMonitorEvent monitoringEvent, ContainerId containerId) { + LOG.info("Stopping resource-monitoring for " + containerId); + updateContainerMetrics(monitoringEvent); + trackingContainers.remove(containerId); + } + + protected void onStartMonitoringContainer( + ContainersMonitorEvent monitoringEvent, ContainerId containerId) { + ContainerStartMonitoringEvent startEvent = + (ContainerStartMonitoringEvent) monitoringEvent; + LOG.info("Starting resource-monitoring for " + containerId); + updateContainerMetrics(monitoringEvent); + trackingContainers.put(containerId, + new ProcessTreeInfo(containerId, null, null, + startEvent.getVmemLimit(), startEvent.getPmemLimit(), + startEvent.getCpuVcores())); + } + } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerExecutionEventType.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerExecutionEventType.java new file mode 100644 index 0000000..72ead5b --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerExecutionEventType.java @@ -0,0 +1,28 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing; + +/** + * Event Type handled by the QueuingContainerManagerImpl to start and stop + * execution of a Container. + */ +public enum ContainerExecutionEventType { + CONTAINER_EXECUTION_START, + CONTAINER_EXECUTION_STOP +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerQueuingEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerQueuingEvent.java new file mode 100644 index 0000000..e16f7a8 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerQueuingEvent.java @@ -0,0 +1,73 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing; + +import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; +import org.apache.hadoop.yarn.security.NMTokenIdentifier; + +/** + * These Event are sent from QueuingContainerManagerImpl to the + * QueuingContainersMonitorImpl to ask it queue/kill a Container. + */ +public class ContainerQueuingEvent + extends AbstractEvent { + + private final NMTokenIdentifier nmTokenIdentifier; + private final ContainerTokenIdentifier containerTokenIdentifier; + private final ContainerId containerId; + private final ExecutionType executionType; + private final StartContainerRequest startRequest; + + public ContainerQueuingEvent(NMTokenIdentifier nmTokenIdentifier, + ContainerTokenIdentifier containerTokenIdentifier, + ExecutionType executionType, StartContainerRequest startRequest, + ContainerQueuingEventType eventType) { + super(eventType); + this.nmTokenIdentifier = nmTokenIdentifier; + this.containerTokenIdentifier = containerTokenIdentifier; + this.containerId = containerTokenIdentifier.getContainerID(); + this.executionType = executionType; + this.startRequest = startRequest; + } + + public NMTokenIdentifier getNMTokenIdentifier() { + return nmTokenIdentifier; + } + + public ContainerTokenIdentifier getContainerTokenIdentifier() { + return containerTokenIdentifier; + } + + public ContainerId getContainerId() { + return containerId; + } + + public ExecutionType getExecutionType() { + return executionType; + } + + public StartContainerRequest getStartRequest() { + return startRequest; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerQueuingEventType.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerQueuingEventType.java new file mode 100644 index 0000000..61a10f5 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/ContainerQueuingEventType.java @@ -0,0 +1,29 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing; + +/** + * Event Type handled by the QueuingContainersMonitorImpl to queue or kill + * a Container. + */ +public enum ContainerQueuingEventType { + CONTAINER_REQUEST_ARRIVED, + CONTAINER_FAILED_TO_START, + QUEUED_CONTAINER_REMOVED +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerExecutionEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerExecutionEvent.java new file mode 100644 index 0000000..c5ab18b --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerExecutionEvent.java @@ -0,0 +1,74 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing; + +import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; +import org.apache.hadoop.yarn.security.NMTokenIdentifier; + +/** + * These Event are sent from QueuingContainersMonitor to the + * QueuingContainersManagerImpl to ask it start/stop a Container. + */ +public class QueuingContainerExecutionEvent + extends AbstractEvent { + + private final StartContainerRequest startRequest; + private final ContainerTokenIdentifier containerTokenIdentifier; + private final ContainerId containerId; + private final NMTokenIdentifier nmTokenIdentifier; + + public QueuingContainerExecutionEvent(StartContainerRequest startRequest, + ContainerTokenIdentifier containerTokenIdentifier, + NMTokenIdentifier nmTokenIdentifier, + ContainerExecutionEventType eventType) { + super(eventType); + this.startRequest = startRequest; + this.containerTokenIdentifier = containerTokenIdentifier; + this.containerId = containerTokenIdentifier.getContainerID(); + this.nmTokenIdentifier = nmTokenIdentifier; + } + + public QueuingContainerExecutionEvent(ContainerId containerId, + ContainerExecutionEventType eventType) { + super(eventType); + this.startRequest = null; + this.containerTokenIdentifier = null; + this.containerId = containerId; + this.nmTokenIdentifier = null; + } + + public StartContainerRequest getStartRequest() { + return this.startRequest; + } + + public ContainerTokenIdentifier getContainerTokenIdentifier() { + return this.containerTokenIdentifier; + } + + public ContainerId getContainerId() { + return this.containerId; + } + + public NMTokenIdentifier getNMTokenIdentifier() { + return this.nmTokenIdentifier; + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java new file mode 100644 index 0000000..25e8be9 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java @@ -0,0 +1,206 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing; + +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.ContainerStatus; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; +import org.apache.hadoop.yarn.security.NMTokenIdentifier; +import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.DeletionService; +import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; +import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; +import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * Class extending {@link ContainerManagerImpl} and is used when queuing at the + * NM is enabled. + */ +public class QueuingContainerManagerImpl extends ContainerManagerImpl { + + private static final Logger LOG = LoggerFactory + .getLogger(QueuingContainerManagerImpl.class); + + public QueuingContainerManagerImpl(Context context, ContainerExecutor exec, + DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater, + NodeManagerMetrics metrics, LocalDirsHandlerService dirsHandler) { + super(context, exec, deletionContext, nodeStatusUpdater, metrics, + dirsHandler); + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + dispatcher.register(ContainerExecutionEventType.class, + new QueuingContainerExecutionEventDispatcher()); + } + + @Override + protected ContainersMonitor createContainersMonitor(ContainerExecutor exec) { + return new QueuingContainersMonitorImpl(exec, dispatcher, this.context); + } + + @SuppressWarnings("unchecked") + @Override + protected void performContainerStart(NMTokenIdentifier nmTokenIdentifier, + StartContainerRequest request, ContainerId containerId, + ContainerTokenIdentifier containerTokenIdentifier) + throws YarnException, IOException { + this.context.getQueuingContext().getQueuedContainers().put(containerId, + containerTokenIdentifier); + + this.dispatcher.getEventHandler() + .handle(new ContainerQueuingEvent(nmTokenIdentifier, + containerTokenIdentifier, + containerTokenIdentifier.getExecutionType(), request, + ContainerQueuingEventType.CONTAINER_REQUEST_ARRIVED)); + } + + @Override + protected void startContainerInternal(NMTokenIdentifier nmTokenIdentifier, + ContainerTokenIdentifier containerTokenIdentifier, + StartContainerRequest request) throws YarnException, IOException { + ContainerId containerId = containerTokenIdentifier.getContainerID(); + this.context.getQueuingContext().getQueuedContainers().remove(containerId); + super.startContainerInternal(nmTokenIdentifier, containerTokenIdentifier, + request); + } + + @SuppressWarnings("unchecked") + @Override + protected void stopContainerInternal(ContainerId containerID) + throws YarnException, IOException { + Container container = this.context.getContainers().get(containerID); + // If container is null and distributed scheduling is enabled, container + // might be queued. Otherwise, container might not be handled by this NM. + if (container == null && this.context.getQueuingContext() + .getQueuedContainers().containsKey(containerID)) { + ContainerTokenIdentifier containerTokenId = this.context + .getQueuingContext().getQueuedContainers().remove(containerID); + + this.dispatcher.getEventHandler() + .handle(new ContainerQueuingEvent(null, containerTokenId, + containerTokenId.getExecutionType(), null, + ContainerQueuingEventType.QUEUED_CONTAINER_REMOVED)); + + this.context.getQueuingContext().getKilledQueuedContainers().put( + containerTokenId, + "Queued container request removed by ApplicationMaster."); + + nodeStatusUpdater.sendOutofBandHeartBeat(); + } + super.stopContainerInternal(containerID); + } + + @SuppressWarnings("unchecked") + private void containerFailedToStart(ContainerId containerId, + ContainerTokenIdentifier containerTokenId) { + this.context.getQueuingContext().getQueuedContainers().remove(containerId); + + this.dispatcher.getEventHandler() + .handle(new ContainerQueuingEvent(null, containerTokenId, + containerTokenId.getExecutionType(), null, + ContainerQueuingEventType.CONTAINER_FAILED_TO_START)); + + this.context.getQueuingContext().getKilledQueuedContainers().put( + containerTokenId, + "Container removed from queue as it failed to start."); + } + + private void stopContainerInternalIfNotQueued(ContainerId containerID) + throws YarnException, IOException { + if (this.context.getContainers().containsKey(containerID)) { + stopContainerInternal(containerID); + } + } + + @Override + protected ContainerStatus getContainerStatusInternal(ContainerId containerID, + NMTokenIdentifier nmTokenIdentifier) throws YarnException { + Container container = this.context.getContainers().get(containerID); + if (container == null) { + ContainerTokenIdentifier containerTokenId = this.context + .getQueuingContext().getQueuedContainers().get(containerID); + if (containerTokenId != null) { + ExecutionType executionType = this.context.getQueuingContext() + .getQueuedContainers().get(containerID).getExecutionType(); + return BuilderUtils.newContainerStatus(containerID, + org.apache.hadoop.yarn.api.records.ContainerState.QUEUED, "", + ContainerExitStatus.INVALID, this.context.getQueuingContext() + .getQueuedContainers().get(containerID).getResource(), + executionType); + } + } + return super.getContainerStatusInternal(containerID, nmTokenIdentifier); + } + + /** + * This class handles the {@link QueuingContainerExecutionEvent}s that are + * sent by the {@link QueuingContainersMonitorImpl}. + */ + class QueuingContainerExecutionEventDispatcher + implements EventHandler { + @Override + public void handle(QueuingContainerExecutionEvent containerExecutionEvent) { + StartContainerRequest containerReq = containerExecutionEvent + .getStartRequest(); + ContainerId containerId = containerExecutionEvent.getContainerId(); + + switch (containerExecutionEvent.getType()) { + case CONTAINER_EXECUTION_START: + try { + startContainerInternal(containerExecutionEvent.getNMTokenIdentifier(), + containerExecutionEvent.getContainerTokenIdentifier(), + containerReq); + } catch (YarnException | IOException e) { + containerFailedToStart(containerId, + containerExecutionEvent.getContainerTokenIdentifier()); + LOG.error("Container failed to start.", e); + } + break; + case CONTAINER_EXECUTION_STOP: + try { + stopContainerInternalIfNotQueued(containerId); + } catch (YarnException | IOException e) { + LOG.error("Container did not get removed successfully.", e); + } + break; + default: + throw new IllegalStateException( + "Got an unknown ContainerExecutionEvent type: " + + containerExecutionEvent.getType()); + } + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainersMonitorImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainersMonitorImpl.java new file mode 100644 index 0000000..a2fd7ce --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainersMonitorImpl.java @@ -0,0 +1,471 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Queue; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceUtilization; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; + +/** + * Class that extends the {@link ContainersMonitorImpl} and is used when queuing + * at the NMs is enabled. + */ +public class QueuingContainersMonitorImpl extends ContainersMonitorImpl { + + private static final Logger LOG = LoggerFactory + .getLogger(QueuingContainersMonitorImpl.class); + + private Map allocatedGuarContainers; + private Map allocatedOpportContainers; + + private ResourceUtilization allocatedContainersUtilization; + + private Queue queuedGuarRequests; + private Queue queuedOpportRequests; + + private Set opportContainersToKill; + + public QueuingContainersMonitorImpl(ContainerExecutor exec, + AsyncDispatcher dispatcher, Context context) { + super(exec, dispatcher, context); + this.allocatedGuarContainers = new HashMap<>(); + this.allocatedOpportContainers = new HashMap<>(); + this.allocatedContainersUtilization = ResourceUtilization.newInstance(0, 0, + 0.0f); + this.queuedGuarRequests = new LinkedList<>(); + this.queuedOpportRequests = new LinkedList<>(); + this.opportContainersToKill = new HashSet(); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + // The ContainerQueuingEventType and the ContainersMonitorEventType should + // be registered on the same dispatcher (which is currently the case). + // Otherwise, synchronization might be needed when accessing the fields of + // this class. + eventDispatcher.register(ContainerQueuingEventType.class, + new ContainerQueuingEventDispatcher()); + } + + @Override + protected void onStopMonitoringContainer( + ContainersMonitorEvent monitoringEvent, ContainerId containerId) { + super.onStopMonitoringContainer(monitoringEvent, containerId); + // Remove finished container from the allocated containers. + // Attempt to start new containers, if resources available. + removeAllocatedContainer(containerId, monitoringEvent.getExecutionType()); + opportContainersToKill.remove(containerId); + startPendingContainers(); + } + + /** + * @return true if there are available allocated resources for the given + * container to start. + */ + @VisibleForTesting + protected boolean hasAllocatedResourcesAvailable(ProcessTreeInfo pti) { + // Check physical memory. + if (this.allocatedContainersUtilization.getPhysicalMemory() + + (int) (pti.getPmemLimit() >> 20) > + (int) (getPmemAllocatedForContainers() >> 20)) { + return false; + } + // Check virtual memory. + if (this.allocatedContainersUtilization.getVirtualMemory() + + (int) (pti.getVmemLimit() >> 20) > + (int) (getVmemAllocatedForContainers() >> 20)) { + return false; + } + // Check CPU. + if (this.allocatedContainersUtilization.getCPU() + + allocatedCpuUsage(pti) > 1.0f) { + return false; + } + return true; + } + + /** + * Start the execution of the given container. Add it to the allocated + * containers, update allocated resource utilization and send event to + * ContainerManager to start the actual execution of the container. + */ + @SuppressWarnings("unchecked") + private void startAllocatedContainer( + AllocatedContainerInfo allocatedContainerInfo) { + ContainerQueuingEvent contQueuingEvent = allocatedContainerInfo + .getContQueuingEvent(); + ProcessTreeInfo pti = allocatedContainerInfo.getPti(); + + if (contQueuingEvent.getExecutionType() == ExecutionType.GUARANTEED) { + allocatedGuarContainers.put(pti.getContainerId(), pti); + } else { + allocatedOpportContainers.put(pti.getContainerId(), pti); + } + + increaseResourceUtil(this.allocatedContainersUtilization, pti); + + // Send event to ContainerManager to start execution of container. + eventDispatcher.getEventHandler() + .handle(new QueuingContainerExecutionEvent( + contQueuingEvent.getStartRequest(), + contQueuingEvent.getContainerTokenIdentifier(), + contQueuingEvent.getNMTokenIdentifier(), + ContainerExecutionEventType.CONTAINER_EXECUTION_START)); + } + + /** + * If there are available resources, try to start as many pending containers + * as possible. + */ + private void startPendingContainers() { + boolean resourcesAvailable = true; + // Start pending guaranteed containers, if resources available. + resourcesAvailable = startContainersFromQueue(queuedGuarRequests, + resourcesAvailable); + if (!resourcesAvailable) { + return; + } + // Start opportunistic containers, if resources available. + startContainersFromQueue(queuedOpportRequests, resourcesAvailable); + } + + private boolean startContainersFromQueue( + Queue queuedContainers, + boolean resourcesAvailable) { + Iterator guarIter = queuedContainers.iterator(); + while (guarIter.hasNext() && resourcesAvailable) { + AllocatedContainerInfo allocatedContInfo = guarIter.next(); + + if (hasAllocatedResourcesAvailable(allocatedContInfo.getPti())) { + startAllocatedContainer(allocatedContInfo); + guarIter.remove(); + } else { + resourcesAvailable = false; + } + } + return resourcesAvailable; + } + + /** + * Remove the given container from the allocated containers, and update + * allocated container utilization accordingly. + */ + private void removeAllocatedContainer(ContainerId containerId, + ExecutionType executionType) { + ProcessTreeInfo ptiToRemove = null; + if (executionType == ExecutionType.GUARANTEED) { + ptiToRemove = allocatedGuarContainers.remove(containerId); + } else { + ptiToRemove = allocatedOpportContainers.remove(containerId); + } + // If container was indeed running, update allocated resource utilization. + if (ptiToRemove != null) { + decreaseResourceUtil(this.allocatedContainersUtilization, ptiToRemove); + } + } + + /** + * Remove the given container from the container queues. + * + * @return true if the container was found in one of the queues. + */ + private boolean removeContainerFromQueues(ContainerId containerId, + ExecutionType executionType) { + Queue queue = + (executionType == ExecutionType.GUARANTEED) ? + queuedGuarRequests : queuedOpportRequests; + + boolean foundInQueue = false; + Iterator iter = queue.iterator(); + while (iter.hasNext() && !foundInQueue) { + if (iter.next().getPti().getContainerId().equals(containerId)) { + iter.remove(); + foundInQueue = true; + } + } + + return foundInQueue; + } + + /** + * Calculates the amount of resources that need to be free up (by killing + * opportunistic containers) in order for the given guaranteed container to + * start its execution. Resource utilization to be freed up = + * allocatedContainersUtilization - utilization of opportContainersToKill + + * utilization of pending guaranteed containers that will start before the + * given container + utilization of given container - total resources of node. + * + * @param containerToStartId + * the ContainerId of the guaranteed container for which we need to + * free resources, so that its execution can start. + * @return the resources that need to be freed up for the given guaranteed + * container to start. + */ + private ResourceUtilization resourcesToFreeUp( + ContainerId containerToStartId) { + // Get current utilization of allocated containers. + ResourceUtilization resourceUtilToFreeUp = ResourceUtilization + .newInstance(allocatedContainersUtilization); + + // Subtract from the utilization the utilization of the opportunistic + // containers that are marked for killing. + for (ContainerId opportContId : opportContainersToKill) { + if (this.allocatedOpportContainers.containsKey(opportContId)) { + decreaseResourceUtil(resourceUtilToFreeUp, + this.allocatedOpportContainers.get(opportContId)); + } + } + // Add to the utilization the utilization of the pending guaranteed + // containers that will start before the current container will be started. + for (AllocatedContainerInfo guarContInfo : queuedGuarRequests) { + increaseResourceUtil(resourceUtilToFreeUp, guarContInfo.getPti()); + if (guarContInfo.getContQueuingEvent().getContainerId() + .equals(containerToStartId)) { + break; + } + } + // Subtract the overall node resources. + resourceUtilToFreeUp.subtractFrom( + (int) (getPmemAllocatedForContainers() >> 20), + (int) (getVmemAllocatedForContainers() >> 20), 1.0f); + + return resourceUtilToFreeUp; + } + + /** + * Kill opportunistic containers to free up resources for running the given + * container. + * + * @param allocatedContInfo + * the container whose execution needs to start by freeing up + * resources occupied by opportunistic containers. + */ + @SuppressWarnings("unchecked") + private void killOpportContainers(AllocatedContainerInfo allocatedContInfo) { + ContainerId containerToStartId = allocatedContInfo.getContQueuingEvent() + .getContainerId(); + // Track resources that need to be freed. + ResourceUtilization resourcesToFreeUp = resourcesToFreeUp( + containerToStartId); + + // Go over the running opportunistic containers. Avoid containers that have + // already been marked for killing. + boolean hasSufficientResources = false; + for (Map.Entry runningOpportCont : + allocatedOpportContainers.entrySet()) { + ContainerId runningOpportContId = runningOpportCont.getKey(); + + // If there are sufficient resources to execute the given container, do + // not kill more opportunistic containers. + if (resourcesToFreeUp.getPhysicalMemory() <= 0 + && resourcesToFreeUp.getVirtualMemory() <= 0 + && resourcesToFreeUp.getCPU() <= 0.0f) { + hasSufficientResources = true; + break; + } + + if (!opportContainersToKill.contains(runningOpportContId)) { + opportContainersToKill.add(runningOpportContId); + decreaseResourceUtil(resourcesToFreeUp, runningOpportCont.getValue()); + // Send event to ContainerManager for actually killing the container. + eventDispatcher.getEventHandler() + .handle(new QueuingContainerExecutionEvent(runningOpportContId, + ContainerExecutionEventType.CONTAINER_EXECUTION_STOP)); + LOG.info( + "Opportunistic container {} will be killed in order to start the " + + "execution of guaranteed container {}.", + runningOpportContId, containerToStartId); + } + } + + if (LOG.isWarnEnabled() && !hasSufficientResources) { + LOG.warn( + "There are no sufficient resources to start guaranteed {} even after " + + "attempting to kill any running opportunistic containers.", + containerToStartId); + } + } + + private ProcessTreeInfo createProcessTreeInfo(ContainerId containerId, + Resource resource) { + long pmemBytes = resource.getMemory() * 1024 * 1024L; + float pmemRatio = conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO, + YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO); + long vmemBytes = (long) (pmemRatio * pmemBytes); + int cpuVcores = resource.getVirtualCores(); + + return new ProcessTreeInfo(containerId, null, null, vmemBytes, pmemBytes, + cpuVcores); + } + + private float allocatedCpuUsage(ProcessTreeInfo pti) { + float cpuUsagePercentPerCore = pti.getCpuVcores() * 100.0f; + float cpuUsageTotalCoresPercentage = cpuUsagePercentPerCore + / resourceCalculatorPlugin.getNumProcessors(); + return (cpuUsageTotalCoresPercentage * 1000 * maxVCoresAllottedForContainers + / nodeCpuPercentageForYARN) / 1000.0f; + } + + private void increaseResourceUtil(ResourceUtilization resourceUtil, + ProcessTreeInfo pti) { + resourceUtil.addTo((int) (pti.getPmemLimit() >> 20), + (int) (pti.getVmemLimit() >> 20), allocatedCpuUsage(pti)); + } + + private void decreaseResourceUtil(ResourceUtilization resourceUtil, + ProcessTreeInfo pti) { + resourceUtil.subtractFrom((int) (pti.getPmemLimit() >> 20), + (int) (pti.getVmemLimit() >> 20), allocatedCpuUsage(pti)); + } + + @VisibleForTesting + public int getNumAllocatedGuaranteedContainers() { + return allocatedGuarContainers.size(); + } + + @VisibleForTesting + public int getNumAllocatedOpportunisticContainers() { + return allocatedOpportContainers.size(); + } + + /** + * This class handles the {@link ContainerQueuingEvent}s that are sent + * by the {@link QueuingContainerManagerImpl}. + */ + public class ContainerQueuingEventDispatcher + implements EventHandler { + @SuppressWarnings("unchecked") + @Override + public void handle(ContainerQueuingEvent containerQueuingEvent) { + if (!isEnabled()) { + return; + } + + switch (containerQueuingEvent.getType()) { + case CONTAINER_REQUEST_ARRIVED: + ProcessTreeInfo pti = createProcessTreeInfo( + containerQueuingEvent.getContainerId(), + containerQueuingEvent.getContainerTokenIdentifier().getResource()); + + AllocatedContainerInfo allocatedContInfo = new AllocatedContainerInfo( + containerQueuingEvent, pti); + + // If there are already free resources for the container to start, and + // there are no queued containers waiting to be executed, start this + // container immediately. + if (queuedGuarRequests.isEmpty() && queuedOpportRequests.isEmpty() + && hasAllocatedResourcesAvailable(pti)) { + startAllocatedContainer(allocatedContInfo); + } else { + if (containerQueuingEvent + .getExecutionType() == ExecutionType.GUARANTEED) { + queuedGuarRequests.add(allocatedContInfo); + // Kill running opportunistic containers to make space for + // guaranteed container. + killOpportContainers(allocatedContInfo); + } else { + queuedOpportRequests.add(allocatedContInfo); + } + } + break; + case QUEUED_CONTAINER_REMOVED: + boolean foundInQueue = removeContainerFromQueues( + containerQueuingEvent.getContainerId(), + containerQueuingEvent.getExecutionType()); + + // The container started in the meanwhile, so the ContainerManager needs + // to be notified and stop it. + if (!foundInQueue) { + eventDispatcher.getEventHandler() + .handle(new QueuingContainerExecutionEvent( + containerQueuingEvent.getContainerTokenIdentifier() + .getContainerID(), + ContainerExecutionEventType.CONTAINER_EXECUTION_STOP)); + } + break; + case CONTAINER_FAILED_TO_START: + removeAllocatedContainer(containerQueuingEvent.getContainerId(), + containerQueuingEvent.getExecutionType()); + break; + default: + throw new IllegalStateException( + "Got an unknown ContainerQueuingEvent type: " + + containerQueuingEvent.getType()); + } + } + } + + static class AllocatedContainerInfo { + private final ContainerQueuingEvent contQueuingEvent; + private final ProcessTreeInfo pti; + + AllocatedContainerInfo(ContainerQueuingEvent contQueuingEvent, + ProcessTreeInfo pti) { + this.contQueuingEvent = contQueuingEvent; + this.pti = pti; + } + + ContainerQueuingEvent getContQueuingEvent() { + return this.contQueuingEvent; + } + + protected ProcessTreeInfo getPti() { + return this.pti; + } + + @Override + public boolean equals(Object obj) { + boolean equal = false; + if (obj instanceof AllocatedContainerInfo) { + AllocatedContainerInfo otherContInfo = (AllocatedContainerInfo) obj; + equal = this.getPti().getContainerId() + .equals(otherContInfo.getPti().getContainerId()); + } + return equal; + } + + @Override + public int hashCode() { + return this.getPti().getContainerId().hashCode(); + } + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/package-info.java new file mode 100644 index 0000000..1135932 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * This package contains classes related to the queuing of containers at + * the NM. + * + */ +package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java index fa8d131..dbf0228 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java @@ -41,7 +41,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -1584,7 +1583,7 @@ protected NodeStatusUpdater createNodeStatusUpdater(Context context, protected NMContext createNMContext( NMContainerTokenSecretManager containerTokenSecretManager, NMTokenSecretManagerInNM nmTokenSecretManager, - NMStateStoreService store) { + NMStateStoreService store, Context.QueuingContext queuingContext) { return new MyNMContext(containerTokenSecretManager, nmTokenSecretManager); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java index 1aea9d2..6c904eb 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java @@ -683,5 +683,10 @@ public NodeResourceMonitor getNodeResourceMonitor() { public NodeStatusUpdater getNodeStatusUpdater() { return null; } + + @Override + public QueuingContext getQueuingContext() { + return null; + } } -} \ No newline at end of file +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java index 787778e..526cc80 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java @@ -110,7 +110,8 @@ public BaseContainerManagerTest() throws UnsupportedFileSystemException { protected Configuration conf = new YarnConfiguration(); protected Context context = new NMContext(new NMContainerTokenSecretManager( conf), new NMTokenSecretManagerInNM(), null, - new ApplicationACLsManager(conf), new NMNullStateStoreService()) { + new ApplicationACLsManager(conf), new NMNullStateStoreService(), + new NodeManager.QueuingNMContext()) { public int getHttpPort() { return HTTP_PORT; }; @@ -280,21 +281,22 @@ public static void waitForContainerState(ContainerManagementProtocol containerMa list.add(containerID); GetContainerStatusesRequest request = GetContainerStatusesRequest.newInstance(list); - ContainerStatus containerStatus = - containerManager.getContainerStatuses(request).getContainerStatuses() - .get(0); + ContainerStatus containerStatus = null; int timeoutSecs = 0; - while (!containerStatus.getState().equals(finalState) - && timeoutSecs++ < timeOutMax) { - Thread.sleep(1000); - LOG.info("Waiting for container to get into state " + finalState - + ". Current state is " + containerStatus.getState()); - containerStatus = containerManager.getContainerStatuses(request).getContainerStatuses().get(0); - } - LOG.info("Container state is " + containerStatus.getState()); - Assert.assertEquals("ContainerState is not correct (timedout)", - finalState, containerStatus.getState()); - } + do { + Thread.sleep(2000); + containerStatus = + containerManager.getContainerStatuses(request) + .getContainerStatuses().get(0); + LOG.info("Waiting for container to get into state " + finalState + + ". Current state is " + containerStatus.getState()); + timeoutSecs += 2; + } while (!containerStatus.getState().equals(finalState) + && timeoutSecs < timeOutMax); + LOG.info("Container state is " + containerStatus.getState()); + Assert.assertEquals("ContainerState is not correct (timedout)", + finalState, containerStatus.getState()); + } static void waitForApplicationState(ContainerManagerImpl containerManager, ApplicationId appID, ApplicationState finalState) @@ -328,19 +330,24 @@ public static void waitForNMContainerState(ContainerManagerImpl org.apache.hadoop.yarn.server.nodemanager.containermanager .container.ContainerState finalState, int timeOutMax) throws InterruptedException, YarnException, IOException { - Container container = - containerManager.getContext().getContainers().get(containerID); + Container container = null; org.apache.hadoop.yarn.server.nodemanager - .containermanager.container.ContainerState currentState = - container.getContainerState(); + .containermanager.container.ContainerState currentState = null; int timeoutSecs = 0; - while (!currentState.equals(finalState) - && timeoutSecs++ < timeOutMax) { - Thread.sleep(1000); - LOG.info("Waiting for NM container to get into state " + finalState - + ". Current state is " + currentState); - currentState = container.getContainerState(); - } + do { + Thread.sleep(2000); + container = + containerManager.getContext().getContainers().get(containerID); + if (container != null) { + currentState = container.getContainerState(); + } + if (currentState != null) { + LOG.info("Waiting for NM container to get into state " + finalState + + ". Current state is " + currentState); + } + timeoutSecs += 2; + } while (!currentState.equals(finalState) + && timeoutSecs++ < timeOutMax); LOG.info("Container state is " + currentState); Assert.assertEquals("ContainerState is not correct (timedout)", finalState, currentState); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java index 3f5fc82..baedf86 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java @@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; @@ -74,6 +75,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.NMTokenIdentifier; +import org.apache.hadoop.yarn.server.api.ContainerType; import org.apache.hadoop.yarn.server.api.ResourceManagerConstants; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent; import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent; @@ -238,7 +240,7 @@ public void testContainerSetup() throws Exception { containerManager.startContainers(allRequests); BaseContainerManagerTest.waitForContainerState(containerManager, cId, - ContainerState.COMPLETE); + ContainerState.COMPLETE, 40); // Now ascertain that the resources are localised correctly. ApplicationId appId = cId.getApplicationAttemptId().getApplicationId(); @@ -375,7 +377,7 @@ public void testContainerLaunchAndStop() throws IOException, DefaultContainerExecutor.containerIsAlive(pid)); } - private void testContainerLaunchAndExit(int exitCode) throws IOException, + protected void testContainerLaunchAndExit(int exitCode) throws IOException, InterruptedException, YarnException { File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile"); @@ -679,6 +681,7 @@ public void testMultipleContainersLaunch() throws Exception { StartContainersResponse response = containerManager.startContainers(requestList); + Thread.sleep(5000); Assert.assertEquals(5, response.getSuccessfullyStartedContainers().size()); for (ContainerId id : response.getSuccessfullyStartedContainers()) { @@ -727,6 +730,7 @@ public void testMultipleContainersStopAndGetStatus() throws Exception { StartContainersRequest requestList = StartContainersRequest.newInstance(startRequest); containerManager.startContainers(requestList); + Thread.sleep(5000); // Get container statuses GetContainerStatusesRequest statusRequest = @@ -803,8 +807,8 @@ public void testStartContainerFailureWithUnknownAuxService() throws Exception { StartContainersResponse response = containerManager.startContainers(requestList); - Assert.assertTrue(response.getFailedRequests().size() == 1); - Assert.assertTrue(response.getSuccessfullyStartedContainers().size() == 0); + Assert.assertEquals(1, response.getFailedRequests().size()); + Assert.assertEquals(0, response.getSuccessfullyStartedContainers().size()); Assert.assertTrue(response.getFailedRequests().containsKey(cId)); Assert.assertTrue(response.getFailedRequests().get(cId).getMessage() .contains("The auxService:" + serviceName + " does not exist")); @@ -1184,6 +1188,21 @@ public static Token createContainerToken(ContainerId cId, long rmIdentifier, containerTokenIdentifier); } + public static Token createContainerToken(ContainerId cId, long rmIdentifier, + NodeId nodeId, String user, Resource resource, + NMContainerTokenSecretManager containerTokenSecretManager, + LogAggregationContext logAggregationContext, ExecutionType executionType) + throws IOException { + ContainerTokenIdentifier containerTokenIdentifier = + new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource, + System.currentTimeMillis() + 100000L, 123, rmIdentifier, + Priority.newInstance(0), 0, logAggregationContext, null, + ContainerType.TASK, executionType); + return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager + .retrievePassword(containerTokenIdentifier), + containerTokenIdentifier); + } + @Test public void testOutputThreadDumpSignal() throws IOException, InterruptedException, YarnException { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java index bbde9ed..0dc5c5b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java @@ -22,6 +22,10 @@ public class MockResourceCalculatorPlugin extends ResourceCalculatorPlugin { + public MockResourceCalculatorPlugin() { + super(null); + } + @Override public long getVirtualMemorySize() { return 0; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java index d7f89fc..c4247ac 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java @@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.AsyncDispatcher; @@ -153,7 +154,7 @@ public void testContainersResourceChange() throws Exception { containersMonitor.start(); // create container 1 containersMonitor.handle(new ContainerStartMonitoringEvent( - getContainerId(1), 2100L, 1000L, 1, 0, 0)); + getContainerId(1), ExecutionType.GUARANTEED, 2100L, 1000L, 1, 0, 0)); // verify that this container is properly tracked assertNotNull(getProcessTreeInfo(getContainerId(1))); assertEquals(1000L, getProcessTreeInfo(getContainerId(1)) @@ -173,8 +174,9 @@ public void testContainersResourceChange() throws Exception { assertTrue(containerEventHandler .isContainerKilled(getContainerId(1))); // create container 2 - containersMonitor.handle(new ContainerStartMonitoringEvent( - getContainerId(2), 2202009L, 1048576L, 1, 0, 0)); + containersMonitor + .handle(new ContainerStartMonitoringEvent(getContainerId(2), + ExecutionType.GUARANTEED, 2202009L, 1048576L, 1, 0, 0)); // verify that this container is properly tracked assertNotNull(getProcessTreeInfo(getContainerId(2))); assertEquals(1048576L, getProcessTreeInfo(getContainerId(2)) @@ -215,8 +217,9 @@ public void testContainersResourceChangeIsTriggeredImmediately() // now waiting for the next monitor cycle Thread.sleep(1000); // create a container with id 3 - containersMonitor.handle(new ContainerStartMonitoringEvent( - getContainerId(3), 2202009L, 1048576L, 1, 0, 0)); + containersMonitor + .handle(new ContainerStartMonitoringEvent(getContainerId(3), + ExecutionType.GUARANTEED, 2202009L, 1048576L, 1, 0, 0)); // Verify that this container has been tracked assertNotNull(getProcessTreeInfo(getContainerId(3))); // trigger a change resource event, check limit after change diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java new file mode 100644 index 0000000..da542a4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java @@ -0,0 +1,301 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing; + +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.UnsupportedFileSystemException; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; +import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; +import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.apache.hadoop.yarn.api.records.URL; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.NMTokenIdentifier; +import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.DeletionService; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; + +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorPlugin; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorProcessTree; +import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.junit.Assert; +import org.junit.Test; + + +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TestQueuingContainerManager extends TestContainerManager { + + interface HasResources { + boolean decide(Context context, ContainerId cId); + } + + public TestQueuingContainerManager() throws UnsupportedFileSystemException { + super(); + } + + static { + LOG = LogFactory.getLog(TestQueuingContainerManager.class); + } + + HasResources hasResources = null; + boolean shouldDeleteWait = false; + + @Override + protected ContainerManagerImpl + createContainerManager(DeletionService delSrvc) { + return new QueuingContainerManagerImpl(context, exec, delSrvc, + nodeStatusUpdater, metrics, dirsHandler) { + + @Override + public void serviceInit(Configuration conf) throws Exception { + conf.set( + YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR, + MockResourceCalculatorPlugin.class.getCanonicalName()); + conf.set( + YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, + MockResourceCalculatorProcessTree.class.getCanonicalName()); + super.serviceInit(conf); + } + + @Override + public void + setBlockNewContainerRequests(boolean blockNewContainerRequests) { + // do nothing + } + + @Override + protected UserGroupInformation getRemoteUgi() throws YarnException { + ApplicationId appId = ApplicationId.newInstance(0, 0); + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + UserGroupInformation ugi = + UserGroupInformation.createRemoteUser(appAttemptId.toString()); + ugi.addTokenIdentifier(new NMTokenIdentifier(appAttemptId, context + .getNodeId(), user, context.getNMTokenSecretManager().getCurrentKey() + .getKeyId())); + return ugi; + } + + @Override + protected void authorizeGetAndStopContainerRequest(ContainerId containerId, + Container container, boolean stopRequest, NMTokenIdentifier identifier) throws YarnException { + if(container == null || container.getUser().equals("Fail")){ + throw new YarnException("Reject this container"); + } + } + + @Override + protected ContainersMonitor createContainersMonitor(ContainerExecutor + exec) { + return new QueuingContainersMonitorImpl(exec, dispatcher, this.context) { + @Override + protected boolean hasAllocatedResourcesAvailable( + ProcessTreeInfo pti) { + return hasResources.decide(this.context, pti.getContainerId()); + } + }; + } + }; + } + + @Override + protected DeletionService createDeletionService() { + return new DeletionService(exec) { + @Override + public void delete(String user, Path subDir, Path... baseDirs) { + // Don't do any deletions. + if (shouldDeleteWait) { + try { + Thread.sleep(10000); + LOG.info("\n\nSleeping Pseudo delete : user - " + user + ", " + + "subDir - " + subDir + ", " + + "baseDirs - " + Arrays.asList(baseDirs)); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } else { + LOG.info("\n\nPseudo delete : user - " + user + ", " + + "subDir - " + subDir + ", " + + "baseDirs - " + Arrays.asList(baseDirs)); + } + } + }; + } + + @Override + public void setup() throws IOException { + super.setup(); + shouldDeleteWait = false; + hasResources = new HasResources() { + @Override + public boolean decide(Context context, ContainerId cId) { + return true; + } + }; + } + + /** + * Test to verify that an OPPORTUNISTIC container is killed when + * a GUARANTEED container arrives and all the Node Resources are used up + * + * For this specific test case, 4 containers are requested (last one being + * guaranteed). Assumptions : + * 1) The first OPPORTUNISTIC Container will start running + * 2) The second and third OPP containers will be queued + * 3) When the GUARANTEED container comes in, the running OPP container + * will be killed to make room + * 4) After the GUARANTEED container finishes, the remaining 2 OPP + * containers will be dequeued and run. + * 5) Only the first OPP container will be killed. + * + * @throws Exception + */ + @Test + public void testSimpleOpportunisticContainer() throws Exception { + shouldDeleteWait = true; + containerManager.start(); + + // ////// Create the resources for the container + File dir = new File(tmpDir, "dir"); + dir.mkdirs(); + File file = new File(dir, "file"); + PrintWriter fileWriter = new PrintWriter(file); + fileWriter.write("Hello World!"); + fileWriter.close(); + + // ////// Construct the container-spec. + ContainerLaunchContext containerLaunchContext = + recordFactory.newRecordInstance(ContainerLaunchContext.class); + URL resource_alpha = + ConverterUtils.getYarnUrlFromPath(localFS + .makeQualified(new Path(file.getAbsolutePath()))); + LocalResource rsrc_alpha = + recordFactory.newRecordInstance(LocalResource.class); + rsrc_alpha.setResource(resource_alpha); + rsrc_alpha.setSize(-1); + rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION); + rsrc_alpha.setType(LocalResourceType.FILE); + rsrc_alpha.setTimestamp(file.lastModified()); + String destinationFile = "dest_file"; + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); + + // Start 3 OPPORTUNISTIC containers and 1 GUARANTEED container + List list = new ArrayList<>(); + list.add(StartContainerRequest.newInstance( + containerLaunchContext, + createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER, + context.getNodeId(), + user, BuilderUtils.newResource(1024, 1), + context.getContainerTokenSecretManager(), null, + ExecutionType.OPPORTUNISTIC))); + list.add(StartContainerRequest.newInstance( + containerLaunchContext, + createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER, + context.getNodeId(), + user, BuilderUtils.newResource(1024, 1), + context.getContainerTokenSecretManager(), null, + ExecutionType.OPPORTUNISTIC))); + list.add(StartContainerRequest.newInstance( + containerLaunchContext, + createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER, + context.getNodeId(), + user, BuilderUtils.newResource(1024, 1), + context.getContainerTokenSecretManager(), null, + ExecutionType.OPPORTUNISTIC))); + // GUARANTEED + list.add(StartContainerRequest.newInstance( + containerLaunchContext, + createContainerToken(createContainerId(3), DUMMY_RM_IDENTIFIER, + context.getNodeId(), + user, context.getContainerTokenSecretManager()))); + StartContainersRequest allRequests = + StartContainersRequest.newInstance(list); + + // Plugin to simulate that the Node is full + // It only allows 1 container to run at a time. + hasResources = new HasResources() { + @Override + public boolean decide(Context context, ContainerId cId) { + int nOpp = + ((QueuingContainersMonitorImpl)containerManager + .getContainersMonitor()) + .getNumAllocatedOpportunisticContainers(); + int nGuar = + ((QueuingContainersMonitorImpl)containerManager + .getContainersMonitor()).getNumAllocatedGuaranteedContainers(); + boolean val = (nOpp + nGuar < 1); + System.out.println("\nHasResources : [" + cId + "]," + + "Opp[" + nOpp + "], Guar[" + nGuar + "], [" + val + "]\n"); + return val; + } + }; + + containerManager.startContainers(allRequests); + + BaseContainerManagerTest.waitForContainerState(containerManager, + createContainerId(3), + ContainerState.COMPLETE, 40); + List statList = new ArrayList(); + for (int i = 0; i < 4; i++) { + statList.add(createContainerId(i)); + } + GetContainerStatusesRequest statRequest = + GetContainerStatusesRequest.newInstance(statList); + List containerStatuses = containerManager + .getContainerStatuses(statRequest).getContainerStatuses(); + for (ContainerStatus status : containerStatuses) { + // Ensure that the first opportunistic container is killed + if (status.getContainerId().equals(createContainerId(0))) { + Assert.assertTrue(status.getDiagnostics() + .contains("Container killed by the ApplicationMaster")); + } + System.out.println("\nStatus : [" + status + "]\n"); + } + } +}