diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml index 01b1da7f1cb..ab36a4eec7f 100644 --- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -574,4 +574,13 @@      + + + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java index e5d120846b0..a244ad86f69 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java @@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerReport; 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.NodeId; import org.apache.hadoop.yarn.api.records.Priority; @@ -46,6 +47,8 @@ public interface RMContainer extends EventHandler { ContainerId getContainerId(); + void setContainerId(ContainerId containerId); + ApplicationAttemptId getApplicationAttemptId(); RMContainerState getState(); @@ -105,4 +108,14 @@ public interface RMContainer extends EventHandler { * @return If the container was allocated remotely. */ boolean isRemotelyAllocated(); + + /* + * Return reserved resource for reserved containers, return allocated resource + * for other container + */ + Resource getAllocatedOrReservedResource(); + + boolean completed(); + + NodeId getNodeId(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java index 706821ee565..4294ef04d1e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java @@ -161,7 +161,6 @@ public class RMContainerImpl implements RMContainer, Comparable { RMContainerEvent> stateMachine; private final ReadLock readLock; private final WriteLock writeLock; - private final ContainerId containerId; private final ApplicationAttemptId appAttemptId; private final NodeId nodeId; private final Container container; @@ -224,7 +223,6 @@ public class RMContainerImpl implements RMContainer, Comparable { RMContext rmContext, long creationTime, String nodeLabelExpression, boolean isExternallyAllocated) { this.stateMachine = stateMachineFactory.make(this); - this.containerId = container.getId(); this.nodeId = nodeId; this.container = container; this.allocatedSchedulerKey = SchedulerRequestKey.extractFrom(container); @@ -255,7 +253,7 @@ public class RMContainerImpl implements RMContainer, Comparable { // containers. If false, and if this container is marked as the AM, metrics // will still be published for this container, but that calculation happens // later. - if (saveNonAMContainerMetaInfo) { + if (saveNonAMContainerMetaInfo && null != container.getId()) { rmContext.getSystemMetricsPublisher().containerCreated( this, this.creationTime); } @@ -263,7 +261,7 @@ public class RMContainerImpl implements RMContainer, Comparable { @Override public ContainerId getContainerId() { - return this.containerId; + return this.container.getId(); } @Override @@ -356,8 +354,8 @@ public class RMContainerImpl implements RMContainer, Comparable { public String getDiagnosticsInfo() { try { readLock.lock(); - if (getFinishedStatus() != null) { - return getFinishedStatus().getDiagnostics(); + if (finishedStatus != null) { + return finishedStatus.getDiagnostics(); } else { return null; } @@ -374,7 +372,7 @@ public class RMContainerImpl implements RMContainer, Comparable { logURL.append(WebAppUtils.getHttpSchemePrefix(rmContext .getYarnConfiguration())); logURL.append(WebAppUtils.getRunningLogURL( - container.getNodeHttpAddress(), containerId.toString(), + container.getNodeHttpAddress(), getContainerId().toString(), user)); return logURL.toString(); } finally { @@ -386,8 +384,8 @@ public class RMContainerImpl implements RMContainer, Comparable { public int getContainerExitStatus() { try { readLock.lock(); - if (getFinishedStatus() != null) { - return getFinishedStatus().getExitStatus(); + if (finishedStatus != null) { + return finishedStatus.getExitStatus(); } else { return 0; } @@ -400,8 +398,8 @@ public class RMContainerImpl implements RMContainer, Comparable { public ContainerState getContainerState() { try { readLock.lock(); - if (getFinishedStatus() != null) { - return getFinishedStatus().getState(); + if (finishedStatus != null) { + return finishedStatus.getState(); } else { return ContainerState.RUNNING; } @@ -431,7 +429,7 @@ public class RMContainerImpl implements RMContainer, Comparable { @Override public String toString() { - return containerId.toString(); + return getContainerId().toString(); } @Override @@ -476,7 +474,7 @@ public class RMContainerImpl implements RMContainer, Comparable { } catch (InvalidStateTransitionException e) { LOG.error("Can't handle this event at current state", e); LOG.error("Invalid event " + event.getType() + - " on container " + this.containerId); + " on container " + this.getContainerId()); } if (oldState != getState()) { LOG.info(event.getContainerId() + " Container Transitioned from " @@ -489,10 +487,15 @@ public class RMContainerImpl implements RMContainer, Comparable { } } - public ContainerStatus getFinishedStatus() { - return finishedStatus; + public boolean completed() { + return finishedStatus != null; } - + + @Override + public NodeId getNodeId() { + return nodeId; + } + private static class BaseTransition implements SingleArcTransition { @@ -517,7 +520,7 @@ public class RMContainerImpl implements RMContainer, Comparable { report.getContainerExitStatus()); new FinishedTransition().transition(container, - new RMContainerFinishedEvent(container.containerId, status, + new RMContainerFinishedEvent(container.getContainerId(), status, RMContainerEventType.FINISHED)); return RMContainerState.COMPLETED; } else if (report.getContainerState().equals(ContainerState.RUNNING)) { @@ -654,11 +657,11 @@ public class RMContainerImpl implements RMContainer, Comparable { } else { // Something wrong happened, kill the container LOG.warn("Something wrong happened, container size reported by NM" - + " is not expected, ContainerID=" + container.containerId + + " is not expected, ContainerID=" + container.getContainerId() + " rm-size-resource:" + rmContainerResource + " nm-size-reosurce:" + nmContainerResource); container.eventHandler.handle(new RMNodeCleanContainerEvent( - container.nodeId, container.containerId)); + container.nodeId, container.getContainerId())); } } @@ -761,7 +764,7 @@ public class RMContainerImpl implements RMContainer, Comparable { // Inform node container.eventHandler.handle(new RMNodeCleanContainerEvent( - container.nodeId, container.containerId)); + container.nodeId, container.getContainerId())); // Inform appAttempt super.transition(container, event); @@ -831,8 +834,8 @@ public class RMContainerImpl implements RMContainer, Comparable { @Override public int compareTo(RMContainer o) { - if (containerId != null && o.getContainerId() != null) { - return containerId.compareTo(o.getContainerId()); + if (getContainerId() != null && o.getContainerId() != null) { + return getContainerId().compareTo(o.getContainerId()); } return -1; } @@ -865,4 +868,35 @@ public class RMContainerImpl implements RMContainer, Comparable { public boolean isRemotelyAllocated() { return isExternallyAllocated; } + + @Override + public Resource getAllocatedOrReservedResource() { + try { + readLock.lock(); + if (getState().equals(RMContainerState.RESERVED)) { + return getReservedResource(); + } else { + return getAllocatedResource(); + } + } finally { + readLock.unlock(); + } + } + + @Override + public void setContainerId(ContainerId containerId) { + // In some cases, for example, global scheduling. It is possible that + // container created without container-id assigned, so we will publish + // container creation event to timeline service when id assigned. + container.setId(containerId); + + // If saveNonAMContainerMetaInfo is true, store system metrics for all + // containers. If false, and if this container is marked as the AM, metrics + // will still be published for this container, but that calculation happens + // later. + if (saveNonAMContainerMetaInfo && null != container.getId()) { + rmContext.getSystemMetricsPublisher().containerCreated( + this, this.creationTime); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java index 59a6650d374..ffb188554e0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -32,6 +33,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.collections.IteratorUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -49,6 +51,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceRequestUpdateResult; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; import org.apache.hadoop.yarn.util.resource.Resources; /** @@ -691,6 +696,25 @@ public class AppSchedulingInfo { } } + public List allocate(NodeType type, + SchedulerNode node, SchedulerRequestKey schedulerKey, + Container containerAllocated) { + try { + writeLock.lock(); + ResourceRequest request; + if (type == NodeType.NODE_LOCAL) { + request = resourceRequestMap.get(schedulerKey).get(node.getNodeName()); + } else if (type == NodeType.RACK_LOCAL) { + request = resourceRequestMap.get(schedulerKey).get(node.getRackName()); + } else{ + request = resourceRequestMap.get(schedulerKey).get(ResourceRequest.ANY); + } + return allocate(type, node, schedulerKey, request, containerAllocated); + } finally { + writeLock.unlock(); + } + } + /** * Resources have been allocated to this application by the resource * scheduler. Track them. @@ -701,40 +725,26 @@ public class AppSchedulingInfo { * @param containerAllocated Container Allocated * @return List of ResourceRequests */ - public List allocate(NodeType type, SchedulerNode node, - SchedulerRequestKey schedulerKey, ResourceRequest request, - Container containerAllocated) { - List resourceRequests = new ArrayList<>(); + public List allocate(NodeType type, + SchedulerNode node, SchedulerRequestKey schedulerKey, + ResourceRequest request, Container containerAllocated) { try { - this.writeLock.lock(); + writeLock.lock(); + List resourceRequests = new ArrayList<>(); if (type == NodeType.NODE_LOCAL) { allocateNodeLocal(node, schedulerKey, request, resourceRequests); } else if (type == NodeType.RACK_LOCAL) { allocateRackLocal(node, schedulerKey, request, resourceRequests); - } else { + } else{ allocateOffSwitch(request, resourceRequests, schedulerKey); } - QueueMetrics metrics = queue.getMetrics(); - if (pending) { - // once an allocation is done we assume the application is - // running from scheduler's POV. - pending = false; - metrics.runAppAttempt(applicationId, user); - } - if (LOG.isDebugEnabled()) { - LOG.debug("allocate: applicationId=" + applicationId - + " container=" + containerAllocated.getId() - + " host=" + containerAllocated.getNodeId().toString() - + " user=" + user - + " resource=" + request.getCapability() - + " type=" + type); + if (null != containerAllocated) { + updateMetricsForAllocatedContainer(request, type, containerAllocated); } - metrics.allocateResources(user, 1, request.getCapability(), true); - metrics.incrNodeTypeAggregations(user, type); return resourceRequests; } finally { - this.writeLock.unlock(); + writeLock.unlock(); } } @@ -942,4 +952,116 @@ public class AppSchedulingInfo { request.getRelaxLocality(), request.getNodeLabelExpression()); return newRequest; } + + /* + * In async environment, pending resource request could be updated during + * scheduling, this method checks pending request before allocating + */ + public boolean checkAllocation(NodeType type, SchedulerNode node, + SchedulerRequestKey schedulerKey) { + try { + readLock.lock(); + ResourceRequest r = resourceRequestMap.get(schedulerKey).get( + ResourceRequest.ANY); + if (r == null || r.getNumContainers() <= 0) { + return false; + } + if (type == NodeType.RACK_LOCAL || type == NodeType.NODE_LOCAL) { + r = resourceRequestMap.get(schedulerKey).get(node.getRackName()); + if (r == null || r.getNumContainers() <= 0) { + return false; + } + if (type == NodeType.NODE_LOCAL) { + r = resourceRequestMap.get(schedulerKey).get(node.getNodeName()); + if (r == null || r.getNumContainers() <= 0) { + return false; + } + } + } + + return true; + } finally { + readLock.unlock(); + } + } + + public void updateMetricsForAllocatedContainer( + ResourceRequest request, NodeType type, Container containerAllocated) { + try { + writeLock.lock(); + QueueMetrics metrics = queue.getMetrics(); + if (pending) { + // once an allocation is done we assume the application is + // running from scheduler's POV. + pending = false; + metrics.runAppAttempt(applicationId, user); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("allocate: applicationId=" + applicationId + " container=" + + containerAllocated.getId() + " host=" + containerAllocated + .getNodeId().toString() + " user=" + user + " resource=" + request + .getCapability() + " type=" + type); + } + metrics.allocateResources(user, 1, request.getCapability(), true); + metrics.incrNodeTypeAggregations(user, type); + } finally { + writeLock.unlock(); + } + } + + // Get placement-set by specified schedulerKey + // Now simply return all node of the input clusterPlacementSet + // TODO, need update this when we support global scheduling + public SchedulingPlacementSet getSchedulingPlacementSet( + SchedulerRequestKey schedulerkey) { + return new SchedulingPlacementSet() { + @Override + @SuppressWarnings("unchecked") + public Iterator getPreferredNodeIterator( + PlacementSet clusterPlacementSet) { + return IteratorUtils.singletonIterator( + clusterPlacementSet.getAllNodes().values().iterator().next()); + } + + @Override + public ResourceRequestUpdateResult updateResourceRequests( + List requests, + boolean recoverPreemptedRequestForAContainer) { + return null; + } + + @Override + public Map getResourceRequests() { + return null; + } + + @Override + public ResourceRequest getResourceRequest(String resourceName, + SchedulerRequestKey requestKey) { + return null; + } + + @Override + public List allocate(NodeType type, SchedulerNode node, + ResourceRequest request) { + return null; + } + + @Override + public Map getAllNodes() { + return null; + } + + @Override + public long getVersion() { + return 0; + } + + @Override + public String getPartition() { + return null; + } + }; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java index d148132ce44..bb1d46127a1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java @@ -28,10 +28,12 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; import com.google.common.collect.ConcurrentHashMultiset; +import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.time.DateUtils; import org.apache.commons.lang.time.FastDateFormat; import org.apache.commons.logging.Log; @@ -49,12 +51,14 @@ import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.api.records.LogAggregationContext; import org.apache.hadoop.yarn.api.records.NMToken; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.server.api.ContainerType; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AggregateAppResourceUsage; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; @@ -69,6 +73,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpdatesAcquiredEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity; import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext; @@ -178,6 +185,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { protected ReentrantReadWriteLock.ReadLock readLock; protected ReentrantReadWriteLock.WriteLock writeLock; + // Not confirmed allocation resource, will be used to avoid too many proposal + // rejected because of duplicated allocation + private AtomicLong unconfirmedAllocatedMem = new AtomicLong(); + private AtomicInteger unconfirmedAllocatedVcores = new AtomicInteger(); + public SchedulerApplicationAttempt(ApplicationAttemptId applicationAttemptId, String user, Queue queue, ActiveUsersManager activeUsersManager, RMContext rmContext) { @@ -529,6 +541,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { if (rmContainer == null) { rmContainer = new RMContainerImpl(container, getApplicationAttemptId(), node.getNodeID(), appSchedulingInfo.getUser(), rmContext); + } + if (rmContainer.getState() == RMContainerState.NEW) { attemptResourceUsage.incReserved(node.getPartition(), container.getResource()); ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName()); @@ -839,16 +853,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { } // used for continuous scheduling - public void resetSchedulingOpportunities( - SchedulerRequestKey schedulerKey, long currentTimeMs) { - try { - writeLock.lock(); - lastScheduledContainer.put(schedulerKey, currentTimeMs); - schedulingOpportunities.setCount(schedulerKey, 0); - } finally { - writeLock.unlock(); - } - + public void resetSchedulingOpportunities(SchedulerRequestKey schedulerKey, + long currentTimeMs) { + lastScheduledContainer.put(schedulerKey, currentTimeMs); + schedulingOpportunities.setCount(schedulerKey, 0); } @VisibleForTesting @@ -998,6 +1006,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { public void incNumAllocatedContainers(NodeType containerType, NodeType requestType) { + if (containerType == null || requestType == null) { + // Sanity check + return; + } + RMAppAttempt attempt = rmContext.getRMApps().get(attemptId.getApplicationId()) .getCurrentAppAttempt(); @@ -1039,9 +1052,27 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { public boolean hasPendingResourceRequest(ResourceCalculator rc, String nodePartition, Resource cluster, SchedulingMode schedulingMode) { - return SchedulerUtils.hasPendingResourceRequest(rc, - this.attemptResourceUsage, nodePartition, cluster, - schedulingMode); + // We need to consider unconfirmed allocations + if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) { + nodePartition = RMNodeLabelsManager.NO_LABEL; + } + + Resource pending = attemptResourceUsage.getPending(nodePartition); + + // TODO, need consider node partition here + // To avoid too many allocation-proposals rejected for non-default + // partition allocation + if (StringUtils.equals(nodePartition, RMNodeLabelsManager.NO_LABEL)) { + pending = Resources.subtract(pending, Resources + .createResource(unconfirmedAllocatedMem.get(), + unconfirmedAllocatedVcores.get())); + } + + if (Resources.greaterThan(rc, cluster, pending, Resources.none())) { + return true; + } + + return false; } @VisibleForTesting @@ -1206,6 +1237,22 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { this.isAttemptRecovering = isRecovering; } + public SchedulingPlacementSet getSchedulingPlacementSet( + SchedulerRequestKey schedulerRequestKey) { + return appSchedulingInfo.getSchedulingPlacementSet(schedulerRequestKey); + } + + + public void incUnconfirmedRes(Resource res) { + unconfirmedAllocatedMem.addAndGet(res.getMemorySize()); + unconfirmedAllocatedVcores.addAndGet(res.getVirtualCores()); + } + + public void decUnconfirmedRes(Resource res) { + unconfirmedAllocatedMem.addAndGet(-res.getMemorySize()); + unconfirmedAllocatedVcores.addAndGet(-res.getVirtualCores()); + } + /** * Different state for Application Master, user can see this state from web UI */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java index 8fa1bb54aa6..3f8ed5552a4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java @@ -25,12 +25,14 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; /** * Utility for logging scheduler activities */ +// FIXME: make sure PlacementSet works with this class public class ActivitiesLogger { private static final Log LOG = LogFactory.getLog(ActivitiesLogger.class); @@ -112,7 +114,7 @@ public class ActivitiesLogger { */ public static void recordAppActivityWithAllocation( ActivitiesManager activitiesManager, SchedulerNode node, - SchedulerApplicationAttempt application, Container updatedContainer, + SchedulerApplicationAttempt application, RMContainer updatedContainer, ActivityState activityState) { if (activitiesManager == null) { return; @@ -122,9 +124,9 @@ public class ActivitiesLogger { // Add application-container activity into specific node allocation. activitiesManager.addSchedulingActivityForNode(node.getNodeID(), application.getApplicationId().toString(), - updatedContainer.getId().toString(), - updatedContainer.getPriority().toString(), activityState, - ActivityDiagnosticConstant.EMPTY, type); + updatedContainer.getContainer().toString(), + updatedContainer.getContainer().getPriority().toString(), + activityState, ActivityDiagnosticConstant.EMPTY, type); type = "app"; // Add queue-application activity into specific node allocation. activitiesManager.addSchedulingActivityForNode(node.getNodeID(), @@ -138,9 +140,10 @@ public class ActivitiesLogger { application.getApplicationId())) { String type = "container"; activitiesManager.addSchedulingActivityForApp( - application.getApplicationId(), updatedContainer.getId().toString(), - updatedContainer.getPriority().toString(), activityState, - ActivityDiagnosticConstant.EMPTY, type); + application.getApplicationId(), + updatedContainer.getContainerId(), + updatedContainer.getContainer().getPriority().toString(), + activityState, ActivityDiagnosticConstant.EMPTY, type); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java index 4fa5feba854..af73ae381a4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java @@ -200,12 +200,13 @@ public class ActivitiesManager extends AbstractService { // Add queue, application or container activity into specific application // allocation. void addSchedulingActivityForApp(ApplicationId applicationId, - String containerId, String priority, ActivityState state, + ContainerId containerId, String priority, ActivityState state, String diagnostic, String type) { if (shouldRecordThisApp(applicationId)) { AppAllocation appAllocation = appsAllocation.get(applicationId); - appAllocation.addAppAllocationActivity(containerId, priority, state, - diagnostic, type); + appAllocation.addAppAllocationActivity(containerId == null ? + "Container-Id-Not-Assigned" : + containerId.toString(), priority, state, diagnostic, type); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java index 096f5ea7e94..7e18b293734 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -54,6 +55,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -738,4 +745,68 @@ public abstract class AbstractCSQueue implements CSQueue { return csContext.getPreemptionManager().getKillableContainers(queueName, partition); } + + @VisibleForTesting + @Override + public CSAssignment assignContainers(Resource clusterResource, + FiCaSchedulerNode node, ResourceLimits resourceLimits, + SchedulingMode schedulingMode) { + return assignContainers(clusterResource, new SimplePlacementSet<>(node), + resourceLimits, schedulingMode); + } + + @Override + public boolean accept(Resource cluster, + ResourceCommitRequest request) { + // Do we need to check parent queue before making this decision? + boolean checkParentQueue = false; + + ContainerAllocationProposal allocation = + request.getFirstAllocatedOrReservedContainer(); + SchedulerContainer schedulerContainer = + allocation.getAllocatedOrReservedContainer(); + + // Do not check when allocating new container from a reserved container + if (allocation.getAllocateFromReservedContainer() == null) { + Resource required = allocation.getAllocatedOrReservedResource(); + Resource netAllocated = Resources.subtract(required, + request.getTotalReleasedResource()); + + try { + readLock.lock(); + + String partition = schedulerContainer.getNodePartition(); + Resource maxResourceLimit; + if (allocation.getSchedulingMode() + == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) { + maxResourceLimit = getQueueMaxResource(partition, cluster); + } else{ + maxResourceLimit = labelManager.getResourceByLabel( + schedulerContainer.getNodePartition(), cluster); + } + if (!Resources.fitsIn(resourceCalculator, cluster, + Resources.add(queueUsage.getUsed(partition), netAllocated), + maxResourceLimit)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Used resource=" + queueUsage.getUsed(partition) + + " exceeded maxResourceLimit of the queue =" + + maxResourceLimit); + } + return false; + } + } finally { + readLock.unlock(); + } + + // Only check parent queue when something new allocated or reserved. + checkParentQueue = true; + } + + + if (parent != null && checkParentQueue) { + return parent.accept(cluster, request); + } + + return true; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java index 7bea9afb1d4..2cae9a95080 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java @@ -38,7 +38,11 @@ public class CSAssignment { new CSAssignment(SkippedType.OTHER); private Resource resource; + // Container allocation locality type private NodeType type; + + // Pending request locality type + private NodeType requestLocalityType; private RMContainer excessReservation; private FiCaSchedulerApp application; private SkippedType skipped; @@ -57,6 +61,10 @@ public class CSAssignment { private boolean increaseAllocation; private List containersToKill; + // Set when fulfilledReservation = true + private RMContainer fulfilledReservedContainer; + private SchedulingMode schedulingMode; + public CSAssignment(Resource resource, NodeType type) { this(resource, type, null, null, SkippedType.NONE, false); } @@ -173,4 +181,29 @@ public class CSAssignment { public List getContainersToKill() { return containersToKill; } + + public RMContainer getFulfilledReservedContainer() { + return fulfilledReservedContainer; + } + + public void setFulfilledReservedContainer( + RMContainer fulfilledReservedContainer) { + this.fulfilledReservedContainer = fulfilledReservedContainer; + } + + public SchedulingMode getSchedulingMode() { + return schedulingMode; + } + + public void setSchedulingMode(SchedulingMode schedulingMode) { + this.schedulingMode = schedulingMode; + } + + public NodeType getRequestLocalityType() { + return requestLocalityType; + } + + public void setRequestLocalityType(NodeType requestLocalityType) { + this.requestLocalityType = requestLocalityType; + } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index daf77903fe5..e5cbd04682e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.List; import java.util.Set; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.security.AccessControlException; @@ -42,8 +43,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet; /** * CSQueue represents a node in the tree of @@ -195,14 +199,14 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue { /** * Assign containers to applications in the queue or it's children (if any). * @param clusterResource the resource of the cluster. - * @param node node on which resources are available + * @param ps {@link PlacementSet} of nodes which resources are available * @param resourceLimits how much overall resource of this queue can use. * @param schedulingMode Type of exclusive check when assign container on a * NodeManager, see {@link SchedulingMode}. * @return the assignment */ public CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, ResourceLimits resourceLimits, + PlacementSet ps, ResourceLimits resourceLimits, SchedulingMode schedulingMode); /** @@ -340,4 +344,15 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue { * @return valid node labels */ public Set getNodeLabelsForQueue(); + + @VisibleForTesting + CSAssignment assignContainers(Resource clusterResource, + FiCaSchedulerNode node, ResourceLimits resourceLimits, + SchedulingMode schedulingMode); + + boolean accept(Resource cluster, + ResourceCommitRequest request); + + void apply(Resource cluster, + ResourceCommitRequest request); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index d759d470ec6..7e98f10bb97 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -32,7 +32,9 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Random; import java.util.Set; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.lang.StringUtils; @@ -112,7 +114,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Alloca import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceAllocationCommitter; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; @@ -128,6 +134,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourc import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.utils.Lock; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; @@ -142,11 +151,12 @@ import com.google.common.base.Preconditions; @SuppressWarnings("unchecked") public class CapacityScheduler extends AbstractYarnScheduler implements - PreemptableResourceScheduler, CapacitySchedulerContext, Configurable { + PreemptableResourceScheduler, CapacitySchedulerContext, Configurable, + ResourceAllocationCommitter { private static final Log LOG = LogFactory.getLog(CapacityScheduler.class); private YarnAuthorizationProvider authorizer; - + private CSQueue root; // timeout to join when we stop this service protected final long THREAD_JOIN_TIMEOUT_MS = 1000; @@ -155,6 +165,8 @@ public class CapacityScheduler extends private volatile boolean isLazyPreemptionEnabled = false; + private int offswitchPerHeartbeatLimit; + static final Comparator nonPartitionedQueueComparator = new Comparator() { @Override @@ -176,7 +188,7 @@ public class CapacityScheduler extends public void setConf(Configuration conf) { yarnConf = conf; } - + private void validateConf(Configuration conf) { // validate scheduler memory allocation setting int minMem = conf.getInt( @@ -229,7 +241,8 @@ public class CapacityScheduler extends private boolean usePortForNodeName; private boolean scheduleAsynchronously; - private AsyncScheduleThread asyncSchedulerThread; + private List asyncSchedulerThreads; + private ResourceCommitterService resourceCommitterService; private RMNodeLabelsManager labelManager; /** @@ -253,7 +266,7 @@ public class CapacityScheduler extends public CSQueue getRootQueue() { return root; } - + @Override public CapacitySchedulerConfiguration getConfiguration() { return conf; @@ -269,11 +282,16 @@ public class CapacityScheduler extends return calculator; } + @VisibleForTesting + public void setResourceCalculator(ResourceCalculator rc) { + this.calculator = rc; + } + @Override public Comparator getNonPartitionedQueueComparator() { return nonPartitionedQueueComparator; } - + @Override public PartitionedQueueComparator getPartitionedQueueComparator() { return partitionedQueueComparator; @@ -294,7 +312,8 @@ public class CapacityScheduler extends this.rmContext = rmContext; } - private void initScheduler(Configuration configuration) throws + @VisibleForTesting + void initScheduler(Configuration configuration) throws IOException { try { writeLock.lock(); @@ -315,10 +334,24 @@ public class CapacityScheduler extends scheduleAsynchronously = this.conf.getScheduleAynschronously(); asyncScheduleInterval = this.conf.getLong(ASYNC_SCHEDULER_INTERVAL, DEFAULT_ASYNC_SCHEDULER_INTERVAL); + + // number of threads for async scheduling + int maxAsyncSchedulingThreads = this.conf.getInt( + CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD, + 1); + maxAsyncSchedulingThreads = Math.max(maxAsyncSchedulingThreads, 1); + if (scheduleAsynchronously) { - asyncSchedulerThread = new AsyncScheduleThread(this); + asyncSchedulerThreads = new ArrayList<>(); + for (int i = 0; i < maxAsyncSchedulingThreads; i++) { + asyncSchedulerThreads.add(new AsyncScheduleThread(this)); + } + resourceCommitterService = new ResourceCommitterService(this); } + // Setup how many containers we can allocate for each round + offswitchPerHeartbeatLimit = this.conf.getOffSwitchPerHeartbeatLimit(); + LOG.info("Initialized CapacityScheduler with " + "calculator=" + getResourceCalculator().getClass() + ", " + "minimumAllocation=<" + getMinimumResourceCapability() + ">, " + "maximumAllocation=<" @@ -335,9 +368,13 @@ public class CapacityScheduler extends writeLock.lock(); activitiesManager.start(); if (scheduleAsynchronously) { - Preconditions.checkNotNull(asyncSchedulerThread, - "asyncSchedulerThread is null"); - asyncSchedulerThread.start(); + Preconditions.checkNotNull(asyncSchedulerThreads, + "asyncSchedulerThreads is null"); + for (Thread t : asyncSchedulerThreads) { + t.start(); + } + + resourceCommitterService.start(); } } finally { writeLock.unlock(); @@ -361,9 +398,13 @@ public class CapacityScheduler extends public void serviceStop() throws Exception { try { writeLock.lock(); - if (scheduleAsynchronously && asyncSchedulerThread != null) { - asyncSchedulerThread.interrupt(); - asyncSchedulerThread.join(THREAD_JOIN_TIMEOUT_MS); + if (scheduleAsynchronously && asyncSchedulerThreads != null) { + for (Thread t : asyncSchedulerThreads) { + t.interrupt(); + t.join(THREAD_JOIN_TIMEOUT_MS); + } + resourceCommitterService.interrupt(); + resourceCommitterService.join(THREAD_JOIN_TIMEOUT_MS); } } finally { writeLock.unlock(); @@ -393,17 +434,20 @@ public class CapacityScheduler extends // update lazy preemption this.isLazyPreemptionEnabled = this.conf.getLazyPreemptionEnabled(); + + // Setup how many containers we can allocate for each round + offswitchPerHeartbeatLimit = this.conf.getOffSwitchPerHeartbeatLimit(); } finally { writeLock.unlock(); } } - + long getAsyncScheduleInterval() { return asyncScheduleInterval; } private final static Random random = new Random(System.currentTimeMillis()); - + /** * Schedule on all nodes by starting at a random point. * @param cs @@ -413,20 +457,22 @@ public class CapacityScheduler extends int current = 0; Collection nodes = cs.nodeTracker.getAllNodes(); int start = random.nextInt(nodes.size()); + for (FiCaSchedulerNode node : nodes) { if (current++ >= start) { - cs.allocateContainersToNode(node); + cs.allocateContainersToNode(node.getNodeID(), false); } } // Now, just get everyone to be safe for (FiCaSchedulerNode node : nodes) { - cs.allocateContainersToNode(node); + cs.allocateContainersToNode(node.getNodeID(), false); } + try { Thread.sleep(cs.getAsyncScheduleInterval()); } catch (InterruptedException e) {} } - + static class AsyncScheduleThread extends Thread { private final CapacityScheduler cs; @@ -440,12 +486,19 @@ public class CapacityScheduler extends @Override public void run() { while (true) { - if (!runSchedules.get()) { - try { + try { + if (!runSchedules.get() || Thread.currentThread().isInterrupted()) { Thread.sleep(100); - } catch (InterruptedException ie) {} - } else { - schedule(cs); + } else { + // Don't run schedule if we have some pending backlogs already + if (cs.getAsyncSchedulingPendingBacklogs() > 100) { + Thread.sleep(1); + } else{ + schedule(cs); + } + } + } catch (InterruptedException ie) { + // Do nothing } } } @@ -460,6 +513,46 @@ public class CapacityScheduler extends } + static class ResourceCommitterService extends Thread { + private final CapacityScheduler cs; + private BlockingQueue> + backlogs = new LinkedBlockingQueue<>(); + + public ResourceCommitterService(CapacityScheduler cs) { + this.cs = cs; + setDaemon(true); + } + + @Override + public void run() { + while (!Thread.currentThread().isInterrupted()) { + try { + ResourceCommitRequest request = + backlogs.take(); + + try { + cs.writeLock.lock(); + cs.tryCommit(cs.getClusterResource(), request); + } finally { + cs.writeLock.unlock(); + } + + } catch (InterruptedException e) { + LOG.error(e); + } + } + } + + public void addNewCommitRequest( + ResourceCommitRequest proposal) { + backlogs.add(proposal); + } + + public int getPendingBacklogs() { + return backlogs.size(); + } + } + static class QueueHook { public CSQueue hook(CSQueue queue) { return queue; @@ -507,14 +600,14 @@ public class CapacityScheduler extends private void updatePlacementRules() throws IOException { List placementRules = new ArrayList<>(); - + // Initialize UserGroupMappingPlacementRule // TODO, need make this defineable by configuration. UserGroupMappingPlacementRule ugRule = getUserGroupMappingPlacementRule(); if (null != ugRule) { placementRules.add(ugRule); } - + rmContext.getQueuePlacementManager().updateRules(placementRules); } @@ -522,8 +615,8 @@ public class CapacityScheduler extends private void initializeQueues(CapacitySchedulerConfiguration conf) throws IOException { - root = - parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT, + root = + parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, noop); labelManager.reinitializeQueueLabels(getQueueToLabels()); LOG.info("Initialized root queue " + root); @@ -539,16 +632,16 @@ public class CapacityScheduler extends throws IOException { // Parse new queues Map newQueues = new HashMap(); - CSQueue newRoot = + CSQueue newRoot = parseQueue(this, newConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues, noop); - + // Ensure all existing queues are still present validateExistingQueues(queues, newQueues); // Add new queues addNewQueues(queues, newQueues); - + // Re-configure queues root.reinitialize(newRoot, getClusterResource()); updatePlacementRules(); @@ -592,14 +685,14 @@ public class CapacityScheduler extends */ @Lock(CapacityScheduler.class) private void validateExistingQueues( - Map queues, Map newQueues) + Map queues, Map newQueues) throws IOException { // check that all static queues are included in the newQueues list for (Map.Entry e : queues.entrySet()) { if (!(e.getValue() instanceof ReservationQueue)) { String queueName = e.getKey(); CSQueue oldQueue = e.getValue(); - CSQueue newQueue = newQueues.get(queueName); + CSQueue newQueue = newQueues.get(queueName); if (null == newQueue) { throw new IOException(queueName + " cannot be found during refresh!"); } else if (!oldQueue.getQueuePath().equals(newQueue.getQueuePath())) { @@ -619,7 +712,7 @@ public class CapacityScheduler extends */ @Lock(CapacityScheduler.class) private void addNewQueues( - Map queues, Map newQueues) + Map queues, Map newQueues) { for (Map.Entry e : newQueues.entrySet()) { String queueName = e.getKey(); @@ -629,19 +722,19 @@ public class CapacityScheduler extends } } } - + @Lock(CapacityScheduler.class) static CSQueue parseQueue( CapacitySchedulerContext csContext, - CapacitySchedulerConfiguration conf, + CapacitySchedulerConfiguration conf, CSQueue parent, String queueName, Map queues, - Map oldQueues, + Map oldQueues, QueueHook hook) throws IOException { CSQueue queue; String fullQueueName = (parent == null) ? queueName : (parent.getQueuePath() + "." + queueName); - String[] childQueueNames = + String[] childQueueNames = conf.getQueues(fullQueueName); boolean isReservableQueue = conf.isReservable(fullQueueName); if (childQueueNames == null || childQueueNames.length == 0) { @@ -676,8 +769,8 @@ public class CapacityScheduler extends List childQueues = new ArrayList(); for (String childQueueName : childQueueNames) { - CSQueue childQueue = - parseQueue(csContext, conf, queue, childQueueName, + CSQueue childQueue = + parseQueue(csContext, conf, queue, childQueueName, queues, oldQueues, hook); childQueues.add(childQueue); } @@ -960,6 +1053,9 @@ public class CapacityScheduler extends private LeafQueue updateIncreaseRequests( List increaseRequests, FiCaSchedulerApp app) { + // When application has some pending to-be-removed resource requests, + app.removedToBeRemovedIncreaseRequests(); + if (null == increaseRequests || increaseRequests.isEmpty()) { return null; } @@ -1068,8 +1164,8 @@ public class CapacityScheduler extends @Override @Lock(Lock.NoLock.class) - public QueueInfo getQueueInfo(String queueName, - boolean includeChildQueues, boolean recursive) + public QueueInfo getQueueInfo(String queueName, + boolean includeChildQueues, boolean recursive) throws IOException { CSQueue queue = null; queue = this.queues.get(queueName); @@ -1094,20 +1190,33 @@ public class CapacityScheduler extends } @Override - protected synchronized void nodeUpdate(RMNode nm) { + protected synchronized void nodeUpdate(RMNode rmNode) { try { - writeLock.lock(); + readLock.lock(); setLastNodeUpdateTime(Time.now()); - super.nodeUpdate(nm); - if (!scheduleAsynchronously) { - ActivitiesLogger.NODE.startNodeUpdateRecording(activitiesManager, - nm.getNodeID()); - allocateContainersToNode(getNode(nm.getNodeID())); - ActivitiesLogger.NODE.finishNodeUpdateRecording(activitiesManager, - nm.getNodeID()); - } + super.nodeUpdate(rmNode); } finally { - writeLock.unlock(); + readLock.unlock(); + } + + // Try to do scheduling + if (!scheduleAsynchronously) { + try { + writeLock.lock(); + ActivitiesLogger.NODE.startNodeUpdateRecording(activitiesManager, + rmNode.getNodeID()); + + // reset allocation and reservation stats before we start doing any + // work + updateSchedulerHealth(lastNodeUpdateTime, rmNode.getNodeID(), + CSAssignment.NULL_ASSIGNMENT); + + allocateContainersToNode(rmNode.getNodeID(), true); + ActivitiesLogger.NODE.finishNodeUpdateRecording(activitiesManager, + rmNode.getNodeID()); + } finally { + writeLock.unlock(); + } } } @@ -1174,10 +1283,8 @@ public class CapacityScheduler extends node.updateLabels(newLabels); } - private void updateSchedulerHealth(long now, FiCaSchedulerNode node, + private void updateSchedulerHealth(long now, NodeId nodeId, CSAssignment assignment) { - - NodeId nodeId = node.getNodeID(); List allocations = assignment.getAssignmentInformation().getAllocationDetails(); List reservations = @@ -1203,137 +1310,248 @@ public class CapacityScheduler extends schedulerHealth.updateSchedulerRunDetails(now, assignment .getAssignmentInformation().getAllocated(), assignment .getAssignmentInformation().getReserved()); - } + } - @VisibleForTesting - public void allocateContainersToNode(FiCaSchedulerNode node) { - try { - writeLock.lock(); - if (rmContext.isWorkPreservingRecoveryEnabled() && !rmContext - .isSchedulerReadyForAllocatingContainers()) { - return; + private boolean canAllocateMore(CSAssignment assignment, int offswitchCount) { + if (null != assignment && Resources.greaterThan(getResourceCalculator(), + getClusterResource(), assignment.getResource(), Resources.none()) + && offswitchCount < offswitchPerHeartbeatLimit) { + // And it should not be a reserved container + if (assignment.getAssignmentInformation().getNumReservations() == 0) { + return true; } + } - if (!nodeTracker.exists(node.getNodeID())) { - LOG.info("Skipping scheduling as the node " + node.getNodeID() - + " has been removed"); - return; - } + return false; + } - // reset allocation and reservation stats before we start doing any work - updateSchedulerHealth(lastNodeUpdateTime, node, - new CSAssignment(Resources.none(), NodeType.NODE_LOCAL)); + /** + * We need to make sure when doing allocation, Node should be existed + * And we will construct a {@link PlacementSet} before proceeding + */ + private void allocateContainersToNode(NodeId nodeId, + boolean withNodeHeartbeat) { + FiCaSchedulerNode node = getNode(nodeId); + if (null != node) { + int offswitchCount = 0; - CSAssignment assignment; - - // Assign new containers... - // 1. Check for reserved applications - // 2. Schedule if there are no reservations - - RMContainer reservedContainer = node.getReservedContainer(); - if (reservedContainer != null) { - - FiCaSchedulerApp reservedApplication = getCurrentAttemptForContainer( - reservedContainer.getContainerId()); - - // Try to fulfill the reservation - LOG.info("Trying to fulfill reservation for application " - + reservedApplication.getApplicationId() + " on node: " + node - .getNodeID()); - - LeafQueue queue = ((LeafQueue) reservedApplication.getQueue()); - assignment = queue.assignContainers(getClusterResource(), node, - // TODO, now we only consider limits for parent for non-labeled - // resources, should consider labeled resources as well. - new ResourceLimits(labelManager - .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, - getClusterResource())), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - if (assignment.isFulfilledReservation()) { - CSAssignment tmp = new CSAssignment( - reservedContainer.getReservedResource(), assignment.getType()); - Resources.addTo(assignment.getAssignmentInformation().getAllocated(), - reservedContainer.getReservedResource()); - tmp.getAssignmentInformation().addAllocationDetails( - reservedContainer.getContainerId(), queue.getQueuePath()); - tmp.getAssignmentInformation().incrAllocations(); - updateSchedulerHealth(lastNodeUpdateTime, node, tmp); - schedulerHealth.updateSchedulerFulfilledReservationCounts(1); - - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - queue.getParent().getQueueName(), queue.getQueueName(), - ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); - ActivitiesLogger.NODE.finishAllocatedNodeAllocation(activitiesManager, - node, reservedContainer.getContainerId(), - AllocationState.ALLOCATED_FROM_RESERVED); - } else{ - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - queue.getParent().getQueueName(), queue.getQueueName(), - ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); - ActivitiesLogger.NODE.finishAllocatedNodeAllocation(activitiesManager, - node, reservedContainer.getContainerId(), - AllocationState.SKIPPED); + PlacementSet ps = new SimplePlacementSet<>(node); + CSAssignment assignment = allocateContainersToNode(ps, withNodeHeartbeat); + // Only check if we can allocate more container on the same node when + // scheduling is triggered by node heartbeat + if (null != assignment && withNodeHeartbeat) { + if (assignment.getType() == NodeType.OFF_SWITCH) { + offswitchCount++; } - } - // Try to schedule more if there are no reservations to fulfill - if (node.getReservedContainer() == null) { - if (calculator.computeAvailableContainers(Resources - .add(node.getUnallocatedResource(), - node.getTotalKillableResources()), minimumAllocation) > 0) { + while (canAllocateMore(assignment, offswitchCount)) { + // Try to see if it is possible to allocate multiple container for + // the same node heartbeat + assignment = allocateContainersToNode(ps, true); + if (null != assignment + && assignment.getType() == NodeType.OFF_SWITCH) { + offswitchCount++; + } + } + + if (offswitchCount >= offswitchPerHeartbeatLimit) { if (LOG.isDebugEnabled()) { - LOG.debug("Trying to schedule on node: " + node.getNodeName() - + ", available: " + node.getUnallocatedResource()); + LOG.debug("Assigned maximum number of off-switch containers: " + + offswitchCount + ", assignments so far: " + assignment); } - - assignment = root.assignContainers(getClusterResource(), node, - new ResourceLimits(labelManager - .getResourceByLabel(node.getPartition(), - getClusterResource())), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - if (Resources.greaterThan(calculator, getClusterResource(), - assignment.getResource(), Resources.none())) { - updateSchedulerHealth(lastNodeUpdateTime, node, assignment); - return; - } - - // Only do non-exclusive allocation when node has node-labels. - if (StringUtils.equals(node.getPartition(), - RMNodeLabelsManager.NO_LABEL)) { - return; - } - - // Only do non-exclusive allocation when the node-label supports that - try { - if (rmContext.getNodeLabelManager().isExclusiveNodeLabel( - node.getPartition())) { - return; - } - } catch (IOException e) { - LOG.warn( - "Exception when trying to get exclusivity of node label=" + node - .getPartition(), e); - return; - } - - // Try to use NON_EXCLUSIVE - assignment = root.assignContainers(getClusterResource(), node, - // TODO, now we only consider limits for parent for non-labeled - // resources, should consider labeled resources as well. - new ResourceLimits(labelManager - .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, - getClusterResource())), - SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY); - updateSchedulerHealth(lastNodeUpdateTime, node, assignment); } + } + } + } + + /* + * Logics of allocate container on a single node (Old behavior) + */ + private CSAssignment allocateContainerOnSingleNode(PlacementSet ps, + FiCaSchedulerNode node, boolean withNodeHeartbeat) { + // Backward compatible way to make sure previous behavior which allocation + // driven by node heartbeat works. + if (getNode(node.getNodeID()) != node) { + LOG.error("Trying to schedule on a removed node, please double check."); + return null; + } + + CSAssignment assignment; + + // Assign new containers... + // 1. Check for reserved applications + // 2. Schedule if there are no reservations + RMContainer reservedContainer = node.getReservedContainer(); + if (reservedContainer != null) { + FiCaSchedulerApp reservedApplication = getCurrentAttemptForContainer( + reservedContainer.getContainerId()); + + // Try to fulfill the reservation + LOG.info( + "Trying to fulfill reservation for application " + reservedApplication + .getApplicationId() + " on node: " + node.getNodeID()); + + LeafQueue queue = ((LeafQueue) reservedApplication.getQueue()); + assignment = queue.assignContainers(getClusterResource(), ps, + // TODO, now we only consider limits for parent for non-labeled + // resources, should consider labeled resources as well. + new ResourceLimits(labelManager + .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, + getClusterResource())), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + + if (assignment.isFulfilledReservation()) { + if (withNodeHeartbeat) { + // Only update SchedulerHealth in sync scheduling, existing + // Data structure of SchedulerHealth need to be updated for + // Async mode + updateSchedulerHealth(lastNodeUpdateTime, node.getNodeID(), + assignment); + } + + schedulerHealth.updateSchedulerFulfilledReservationCounts(1); + + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + queue.getParent().getQueueName(), queue.getQueueName(), + ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); + ActivitiesLogger.NODE.finishAllocatedNodeAllocation(activitiesManager, + node, reservedContainer.getContainerId(), + AllocationState.ALLOCATED_FROM_RESERVED); } else{ - LOG.info("Skipping scheduling since node " + node.getNodeID() + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + queue.getParent().getQueueName(), queue.getQueueName(), + ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); + ActivitiesLogger.NODE.finishAllocatedNodeAllocation(activitiesManager, + node, reservedContainer.getContainerId(), AllocationState.SKIPPED); + } + + assignment.setSchedulingMode( + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + submitResourceCommitRequest(getClusterResource(), assignment); + } + + // Do not schedule if there are any reservations to fulfill on the node + if (node.getReservedContainer() != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skipping scheduling since node " + node.getNodeID() + " is reserved by application " + node.getReservedContainer() .getContainerId().getApplicationAttemptId()); } - } finally { - writeLock.unlock(); + return null; + } + + // First check if we can schedule + // When this time look at one node only, try schedule if the node + // has any available or killable resource + if (calculator.computeAvailableContainers(Resources + .add(node.getUnallocatedResource(), node.getTotalKillableResources()), + minimumAllocation) <= 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("This node or this node partition doesn't have available or" + + "killable resource"); + } + return null; + } + + if (LOG.isDebugEnabled()) { + LOG.debug( + "Trying to schedule on node: " + node.getNodeName() + ", available: " + + node.getUnallocatedResource()); + } + + return allocateOrReserveNewContainers(ps, withNodeHeartbeat); + } + + private CSAssignment allocateOrReserveNewContainers( + PlacementSet ps, boolean withNodeHeartbeat) { + CSAssignment assignment = root.assignContainers(getClusterResource(), ps, + new ResourceLimits(labelManager + .getResourceByLabel(ps.getPartition(), getClusterResource())), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + + assignment.setSchedulingMode(SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + submitResourceCommitRequest(getClusterResource(), assignment); + + if (Resources.greaterThan(calculator, getClusterResource(), + assignment.getResource(), Resources.none())) { + if (withNodeHeartbeat) { + updateSchedulerHealth(lastNodeUpdateTime, + PlacementSetUtils.getSingleNode(ps).getNodeID(), assignment); + } + return assignment; + } + + // Only do non-exclusive allocation when node has node-labels. + if (StringUtils.equals(ps.getPartition(), RMNodeLabelsManager.NO_LABEL)) { + return null; + } + + // Only do non-exclusive allocation when the node-label supports that + try { + if (rmContext.getNodeLabelManager().isExclusiveNodeLabel( + ps.getPartition())) { + return null; + } + } catch (IOException e) { + LOG.warn("Exception when trying to get exclusivity of node label=" + ps + .getPartition(), e); + return null; + } + + // Try to use NON_EXCLUSIVE + assignment = root.assignContainers(getClusterResource(), ps, + // TODO, now we only consider limits for parent for non-labeled + // resources, should consider labeled resources as well. + new ResourceLimits(labelManager + .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, + getClusterResource())), + SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY); + assignment.setSchedulingMode(SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY); + submitResourceCommitRequest(getClusterResource(), assignment); + + return assignment; + } + + /* + * New behavior, allocate containers considering multiple nodes + */ + private CSAssignment allocateContainersOnMultiNodes( + PlacementSet ps) { + // When this time look at multiple nodes, try schedule if the + // partition has any available resource or killable resource + if (root.getQueueCapacities().getUsedCapacity(ps.getPartition()) >= 1.0f + && preemptionManager.getKillableResource( + CapacitySchedulerConfiguration.ROOT, ps.getPartition()) == Resources + .none()) { + if (LOG.isDebugEnabled()) { + LOG.debug("This node or this node partition doesn't have available or" + + "killable resource"); + } + return null; + } + + return allocateOrReserveNewContainers(ps, false); + } + + @VisibleForTesting + CSAssignment allocateContainersToNode(PlacementSet ps, + boolean withNodeHeartbeat) { + if (rmContext.isWorkPreservingRecoveryEnabled() && !rmContext + .isSchedulerReadyForAllocatingContainers()) { + return null; + } + + // Backward compatible way to make sure previous behavior which allocation + // driven by node heartbeat works. + FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps); + + // We have two different logics to handle allocation on single node / multi + // nodes. + if (null != node) { + return allocateContainerOnSingleNode(ps, node, withNodeHeartbeat); + } else { + return allocateContainersOnMultiNodes(ps); } } @@ -1356,7 +1574,7 @@ public class CapacityScheduler extends break; case NODE_RESOURCE_UPDATE: { - NodeResourceUpdateSchedulerEvent nodeResourceUpdatedEvent = + NodeResourceUpdateSchedulerEvent nodeResourceUpdatedEvent = (NodeResourceUpdateSchedulerEvent)event; updateNodeAndQueueResource(nodeResourceUpdatedEvent.getRMNode(), nodeResourceUpdatedEvent.getResourceOption()); @@ -1366,7 +1584,7 @@ public class CapacityScheduler extends { NodeLabelsUpdateSchedulerEvent labelUpdateEvent = (NodeLabelsUpdateSchedulerEvent) event; - + updateNodeLabelsAndQueueResource(labelUpdateEvent); } break; @@ -1420,7 +1638,7 @@ public class CapacityScheduler extends break; case CONTAINER_EXPIRED: { - ContainerExpiredSchedulerEvent containerExpiredEvent = + ContainerExpiredSchedulerEvent containerExpiredEvent = (ContainerExpiredSchedulerEvent) event; ContainerId containerId = containerExpiredEvent.getContainerId(); if (containerExpiredEvent.isIncrease()) { @@ -1515,7 +1733,9 @@ public class CapacityScheduler extends + clusterResource); if (scheduleAsynchronously && getNumClusterNodes() == 1) { - asyncSchedulerThread.beginSchedule(); + for (AsyncScheduleThread t : asyncSchedulerThreads) { + t.beginSchedule(); + } } } finally { writeLock.unlock(); @@ -1561,7 +1781,9 @@ public class CapacityScheduler extends int numNodes = nodeTracker.nodeCount(); if (scheduleAsynchronously && numNodes == 0) { - asyncSchedulerThread.suspendSchedule(); + for (AsyncScheduleThread t : asyncSchedulerThreads) { + t.suspendSchedule(); + } } LOG.info( @@ -1629,7 +1851,7 @@ public class CapacityScheduler extends queue.completedContainer(getClusterResource(), application, node, rmContainer, containerStatus, event, null, true); } - + @Override protected void decreaseContainer(SchedContainerChangeRequest decreaseRequest, SchedulerApplicationAttempt attempt) { @@ -1673,7 +1895,7 @@ public class CapacityScheduler extends public List getAllNodes() { return nodeTracker.getAllNodes(); } - + @Override @Lock(Lock.NoLock.class) public void recover(RMState state) throws Exception { @@ -2085,7 +2307,7 @@ public class CapacityScheduler extends } return EnumSet.of(SchedulerResourceTypes.MEMORY, SchedulerResourceTypes.CPU); } - + @Override public Resource getMaximumResourceCapability(String queueName) { CSQueue queue = getQueue(queueName); @@ -2223,4 +2445,207 @@ public class CapacityScheduler extends public ResourceUsage getClusterResourceUsage() { return root.getQueueResourceUsage(); } + + private SchedulerContainer getSchedulerContainer( + RMContainer rmContainer, boolean allocated) { + if (null == rmContainer) { + return null; + } + + FiCaSchedulerApp app = getApplicationAttempt( + rmContainer.getApplicationAttemptId()); + if (null == app) { return null; } + + NodeId nodeId; + // Get nodeId + if (rmContainer.getState() == RMContainerState.RESERVED) { + nodeId = rmContainer.getReservedNode(); + } else { + nodeId = rmContainer.getNodeId(); + } + + FiCaSchedulerNode node = getNode(nodeId); + if (null == node) { + return null; + } + return new SchedulerContainer<>(app, node, rmContainer, + // TODO, node partition should come from CSAssignment to avoid partition + // get updated before submitting the commit + node.getPartition(), allocated); + } + + private List> + getSchedulerContainersToRelease( + CSAssignment csAssignment) { + List> list = null; + + if (csAssignment.getContainersToKill() != null && !csAssignment + .getContainersToKill().isEmpty()) { + list = new ArrayList<>(); + for (RMContainer rmContainer : csAssignment.getContainersToKill()) { + list.add(getSchedulerContainer(rmContainer, false)); + } + } + + if (csAssignment.getExcessReservation() != null) { + if (null == list) { + list = new ArrayList<>(); + } + list.add( + getSchedulerContainer(csAssignment.getExcessReservation(), false)); + } + + return list; + } + + @VisibleForTesting + public void submitResourceCommitRequest(Resource cluster, + CSAssignment csAssignment) { + ResourceCommitRequest request = + createResourceCommitRequest(csAssignment); + + if (null == request) { + return; + } + + if (scheduleAsynchronously) { + // Submit to a commit thread and commit it async-ly + resourceCommitterService.addNewCommitRequest(request); + } else{ + // Otherwise do it sync-ly. + tryCommit(cluster, request); + } + } + + @VisibleForTesting + public ResourceCommitRequest + createResourceCommitRequest(CSAssignment csAssignment) { + ContainerAllocationProposal allocated = + null; + ContainerAllocationProposal reserved = + null; + List> released = + null; + + if (Resources.greaterThan(calculator, getClusterResource(), + csAssignment.getResource(), Resources.none())) { + // Allocated something + List allocations = + csAssignment.getAssignmentInformation().getAllocationDetails(); + if (!allocations.isEmpty()) { + RMContainer rmContainer = allocations.get(0).rmContainer; + allocated = new ContainerAllocationProposal<>( + getSchedulerContainer(rmContainer, true), + getSchedulerContainersToRelease(csAssignment), + getSchedulerContainer(csAssignment.getFulfilledReservedContainer(), + false), csAssignment.isIncreasedAllocation(), + csAssignment.getType(), csAssignment.getRequestLocalityType(), + csAssignment.getSchedulingMode() != null ? + csAssignment.getSchedulingMode() : + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY, + csAssignment.getResource()); + } + + // Reserved something + List reservation = + csAssignment.getAssignmentInformation().getReservationDetails(); + if (!reservation.isEmpty()) { + RMContainer rmContainer = reservation.get(0).rmContainer; + reserved = new ContainerAllocationProposal<>( + getSchedulerContainer(rmContainer, false), + getSchedulerContainersToRelease(csAssignment), + getSchedulerContainer(csAssignment.getFulfilledReservedContainer(), + false), csAssignment.isIncreasedAllocation(), + csAssignment.getType(), csAssignment.getRequestLocalityType(), + csAssignment.getSchedulingMode() != null ? + csAssignment.getSchedulingMode() : + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY, + csAssignment.getResource()); + } + } + + // When we don't need to allocate/reserve anything, we can feel free to + // kill all to-release containers in the request. + if (null == allocated && null == reserved) { + released = getSchedulerContainersToRelease(csAssignment); + } + + if (null != allocated || null != reserved || (null != released && !released + .isEmpty())) { + List> + allocationsList = null; + if (allocated != null) { + allocationsList = new ArrayList<>(); + allocationsList.add(allocated); + } + + List> + reservationsList = null; + if (reserved != null) { + reservationsList = new ArrayList<>(); + reservationsList.add(reserved); + } + + return new ResourceCommitRequest<>(allocationsList, reservationsList, + released); + } + + return null; + } + + @Override + public void tryCommit(Resource cluster, ResourceCommitRequest r) { + ResourceCommitRequest request = + (ResourceCommitRequest) r; + + ApplicationAttemptId attemptId = null; + + // We need to update unconfirmed allocated resource of application when + // any container allocated. + boolean updateUnconfirmedAllocatedResource = + request.getContainersToAllocate() != null && !request + .getContainersToAllocate().isEmpty(); + + // find the application to accept and apply the ResourceCommitRequest + if (request.anythingAllocatedOrReserved()) { + ContainerAllocationProposal c = + request.getFirstAllocatedOrReservedContainer(); + attemptId = + c.getAllocatedOrReservedContainer().getSchedulerApplicationAttempt() + .getApplicationAttemptId(); + } else { + if (!request.getContainersToRelease().isEmpty()) { + attemptId = request.getContainersToRelease().get(0) + .getSchedulerApplicationAttempt().getApplicationAttemptId(); + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Try to commit allocation proposal=" + request); + } + + if (attemptId != null) { + FiCaSchedulerApp app = getApplicationAttempt(attemptId); + if (app != null) { + if (app.accept(cluster, request)) { + app.apply(cluster, request); + LOG.info("Allocation proposal accepted"); + } else{ + LOG.info("Failed to accept allocation proposal"); + } + + // Update unconfirmed allocated resource. + if (updateUnconfirmedAllocatedResource) { + app.decUnconfirmedRes(request.getTotalAllocatedResource()); + } + } + } + } + + public int getAsyncSchedulingPendingBacklogs() { + if (scheduleAsynchronously) { + return resourceCommitterService.getPendingBacklogs(); + } + return 0; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java index cea5aa46301..c153c263aa7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java @@ -208,6 +208,10 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur public static final String SCHEDULE_ASYNCHRONOUSLY_ENABLE = SCHEDULE_ASYNCHRONOUSLY_PREFIX + ".enable"; + @Private + public static final String SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD = + SCHEDULE_ASYNCHRONOUSLY_PREFIX + ".maximum-threads"; + @Private public static final boolean DEFAULT_SCHEDULE_ASYNCHRONOUSLY_ENABLE = false; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index 8941fdf6493..161957f922a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -67,8 +67,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Activi import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicyForPendingApps; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy; import org.apache.hadoop.yarn.server.utils.Lock; @@ -914,54 +919,6 @@ public class LeafQueue extends AbstractCSQueue { ApplicationAttemptId applicationAttemptId) { return applicationAttemptMap.get(applicationAttemptId); } - - private void handleExcessReservedContainer(Resource clusterResource, - CSAssignment assignment, FiCaSchedulerNode node, FiCaSchedulerApp app) { - if (assignment.getExcessReservation() != null) { - RMContainer excessReservedContainer = assignment.getExcessReservation(); - - if (excessReservedContainer.hasIncreaseReservation()) { - unreserveIncreasedContainer(clusterResource, - app, node, excessReservedContainer); - } else { - completedContainer(clusterResource, assignment.getApplication(), - scheduler.getNode(excessReservedContainer.getAllocatedNode()), - excessReservedContainer, - SchedulerUtils.createAbnormalContainerStatus( - excessReservedContainer.getContainerId(), - SchedulerUtils.UNRESERVED_CONTAINER), - RMContainerEventType.RELEASED, null, false); - } - - assignment.setExcessReservation(null); - } - } - - private void killToPreemptContainers(Resource clusterResource, - FiCaSchedulerNode node, - CSAssignment assignment) { - if (assignment.getContainersToKill() != null) { - StringBuilder sb = new StringBuilder("Killing containers: ["); - - for (RMContainer c : assignment.getContainersToKill()) { - FiCaSchedulerApp application = csContext.getApplicationAttempt( - c.getApplicationAttemptId()); - LeafQueue q = application.getCSLeafQueue(); - q.completedContainer(clusterResource, application, node, c, SchedulerUtils - .createPreemptedContainerStatus(c.getContainerId(), - SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL, - null, false); - sb.append("(container=" + c.getContainerId() + " resource=" + c - .getAllocatedResource() + ")"); - } - - sb.append("] for container=" + assignment.getAssignmentInformation() - .getFirstAllocatedOrReservedContainerId() + " resource=" + assignment - .getResource()); - LOG.info(sb.toString()); - - } - } private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) { // Set preemption-allowed: @@ -971,174 +928,312 @@ public class LeafQueue extends AbstractCSQueue { limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity); } - @Override - public CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, ResourceLimits currentResourceLimits, - SchedulingMode schedulingMode) { - try { - writeLock.lock(); - updateCurrentResourceLimits(currentResourceLimits, clusterResource); + private CSAssignment allocateFromReservedContainer( + Resource clusterResource, PlacementSet ps, + ResourceLimits currentResourceLimits, SchedulingMode schedulingMode) { + FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps); + if (null == node) { + return null; + } - if (LOG.isDebugEnabled()) { - LOG.debug( - "assignContainers: node=" + node.getNodeName() + " #applications=" - + orderingPolicy.getNumSchedulableEntities()); - } - - setPreemptionAllowed(currentResourceLimits, node.getPartition()); - - // Check for reserved resources - RMContainer reservedContainer = node.getReservedContainer(); - if (reservedContainer != null) { - FiCaSchedulerApp application = getApplication( - reservedContainer.getApplicationAttemptId()); + RMContainer reservedContainer = node.getReservedContainer(); + if (reservedContainer != null) { + FiCaSchedulerApp application = getApplication( + reservedContainer.getApplicationAttemptId()); + if (null != application) { ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager, node.getNodeID(), SystemClock.getInstance().getTime(), application); - CSAssignment assignment = application.assignContainers(clusterResource, - node, currentResourceLimits, schedulingMode, reservedContainer); - handleExcessReservedContainer(clusterResource, assignment, node, - application); - killToPreemptContainers(clusterResource, node, assignment); + ps, currentResourceLimits, schedulingMode, reservedContainer); return assignment; } + } - // if our queue cannot access this node, just return - if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY - && !accessibleToPartition(node.getPartition())) { - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParent().getQueueName(), getQueueName(), ActivityState.REJECTED, - ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node - .getPartition()); - return CSAssignment.NULL_ASSIGNMENT; - } + return null; + } - // Check if this queue need more resource, simply skip allocation if this - // queue doesn't need more resources. - if (!hasPendingResourceRequest(node.getPartition(), clusterResource, - schedulingMode)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skip this queue=" + getQueuePath() - + ", because it doesn't need more resource, schedulingMode=" - + schedulingMode.name() + " node-partition=" + node + @Override + public CSAssignment assignContainers(Resource clusterResource, + PlacementSet ps, ResourceLimits currentResourceLimits, + SchedulingMode schedulingMode) { + updateCurrentResourceLimits(currentResourceLimits, clusterResource); + FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps); + + if (LOG.isDebugEnabled()) { + LOG.debug("assignContainers: partition=" + ps.getPartition() + + " #applications=" + orderingPolicy.getNumSchedulableEntities()); + } + + setPreemptionAllowed(currentResourceLimits, ps.getPartition()); + + // Check for reserved resources, try to allocate reserved container first. + CSAssignment assignment = allocateFromReservedContainer(clusterResource, + ps, currentResourceLimits, schedulingMode); + if (null != assignment) { + return assignment; + } + + // if our queue cannot access this node, just return + if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY + && !accessibleToPartition(ps.getPartition())) { + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParent().getQueueName(), getQueueName(), ActivityState.REJECTED, + ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + ps .getPartition()); - } - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.QUEUE_DO_NOT_NEED_MORE_RESOURCE); - return CSAssignment.NULL_ASSIGNMENT; - } + return CSAssignment.NULL_ASSIGNMENT; + } - for (Iterator assignmentIterator = - orderingPolicy.getAssignmentIterator(); - assignmentIterator.hasNext(); ) { - FiCaSchedulerApp application = assignmentIterator.next(); - - ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager, - node.getNodeID(), SystemClock.getInstance().getTime(), application); - - // Check queue max-capacity limit - if (!super.canAssignToThisQueue(clusterResource, node.getPartition(), - currentResourceLimits, application.getCurrentReservation(), - schedulingMode)) { - ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue( - activitiesManager, node, application, application.getPriority(), - ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT); - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.EMPTY); - return CSAssignment.NULL_ASSIGNMENT; - } - - Resource userLimit = computeUserLimitAndSetHeadroom(application, - clusterResource, node.getPartition(), schedulingMode); - - // Check user limit - if (!canAssignToUser(clusterResource, application.getUser(), userLimit, - application, node.getPartition(), currentResourceLimits)) { - application.updateAMContainerDiagnostics(AMState.ACTIVATED, - "User capacity has reached its maximum limit."); - ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue( - activitiesManager, node, application, application.getPriority(), - ActivityDiagnosticConstant.USER_CAPACITY_MAXIMUM_LIMIT); - continue; - } - - // Try to schedule - CSAssignment assignment = application.assignContainers(clusterResource, - node, currentResourceLimits, schedulingMode, null); - - if (LOG.isDebugEnabled()) { - LOG.debug("post-assignContainers for application " + application - .getApplicationId()); - application.showRequests(); - } - - // Did we schedule or reserve a container? - Resource assigned = assignment.getResource(); - - handleExcessReservedContainer(clusterResource, assignment, node, - application); - killToPreemptContainers(clusterResource, node, assignment); - - if (Resources.greaterThan(resourceCalculator, clusterResource, assigned, - Resources.none())) { - // Get reserved or allocated container from application - RMContainer reservedOrAllocatedRMContainer = - application.getRMContainer(assignment.getAssignmentInformation() - .getFirstAllocatedOrReservedContainerId()); - - // Book-keeping - // Note: Update headroom to account for current allocation too... - allocateResource(clusterResource, application, assigned, - node.getPartition(), reservedOrAllocatedRMContainer, - assignment.isIncreasedAllocation()); - - // Update reserved metrics - Resource reservedRes = - assignment.getAssignmentInformation().getReserved(); - if (reservedRes != null && !reservedRes.equals(Resources.none())) { - incReservedResource(node.getPartition(), reservedRes); - } - - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParent().getQueueName(), getQueueName(), - ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); - - // Done - return assignment; - } else if (assignment.getSkippedType() - == CSAssignment.SkippedType.OTHER) { - ActivitiesLogger.APP.finishSkippedAppAllocationRecording( - activitiesManager, application.getApplicationId(), - ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY); - application.updateNodeInfoForAMDiagnostics(node); - } else if (assignment.getSkippedType() - == CSAssignment.SkippedType.QUEUE_LIMIT) { - return assignment; - } else{ - // If we don't allocate anything, and it is not skipped by application, - // we will return to respect FIFO of applications - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.RESPECT_FIFO); - ActivitiesLogger.APP.finishSkippedAppAllocationRecording( - activitiesManager, application.getApplicationId(), - ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY); - return CSAssignment.NULL_ASSIGNMENT; - } + // Check if this queue need more resource, simply skip allocation if this + // queue doesn't need more resources. + if (!hasPendingResourceRequest(ps.getPartition(), clusterResource, + schedulingMode)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip this queue=" + getQueuePath() + + ", because it doesn't need more resource, schedulingMode=" + + schedulingMode.name() + " node-partition=" + ps.getPartition()); } ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.EMPTY); - + ActivityDiagnosticConstant.QUEUE_DO_NOT_NEED_MORE_RESOURCE); return CSAssignment.NULL_ASSIGNMENT; + } + + for (Iterator assignmentIterator = + orderingPolicy.getAssignmentIterator(); + assignmentIterator.hasNext(); ) { + FiCaSchedulerApp application = assignmentIterator.next(); + + ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager, + node.getNodeID(), SystemClock.getInstance().getTime(), application); + + // Check queue max-capacity limit + if (!super.canAssignToThisQueue(clusterResource, ps.getPartition(), + currentResourceLimits, application.getCurrentReservation(), + schedulingMode)) { + ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue( + activitiesManager, node, application, application.getPriority(), + ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT); + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED, + ActivityDiagnosticConstant.EMPTY); + return CSAssignment.NULL_ASSIGNMENT; + } + + Resource userLimit = computeUserLimitAndSetHeadroom(application, + clusterResource, ps.getPartition(), schedulingMode); + + // Check user limit + if (!canAssignToUser(clusterResource, application.getUser(), userLimit, + application, ps.getPartition(), currentResourceLimits)) { + application.updateAMContainerDiagnostics(AMState.ACTIVATED, + "User capacity has reached its maximum limit."); + ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue( + activitiesManager, node, application, application.getPriority(), + ActivityDiagnosticConstant.USER_CAPACITY_MAXIMUM_LIMIT); + continue; + } + + // Try to schedule + assignment = application.assignContainers(clusterResource, + ps, currentResourceLimits, schedulingMode, null); + + if (LOG.isDebugEnabled()) { + LOG.debug("post-assignContainers for application " + application + .getApplicationId()); + application.showRequests(); + } + + // Did we schedule or reserve a container? + Resource assigned = assignment.getResource(); + + if (Resources.greaterThan(resourceCalculator, clusterResource, assigned, + Resources.none())) { + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParent().getQueueName(), getQueueName(), + ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); + return assignment; + } else if (assignment.getSkippedType() + == CSAssignment.SkippedType.OTHER) { + ActivitiesLogger.APP.finishSkippedAppAllocationRecording( + activitiesManager, application.getApplicationId(), + ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY); + application.updateNodeInfoForAMDiagnostics(node); + } else if (assignment.getSkippedType() + == CSAssignment.SkippedType.QUEUE_LIMIT) { + return assignment; + } else{ + // If we don't allocate anything, and it is not skipped by application, + // we will return to respect FIFO of applications + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED, + ActivityDiagnosticConstant.RESPECT_FIFO); + ActivitiesLogger.APP.finishSkippedAppAllocationRecording( + activitiesManager, application.getApplicationId(), + ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY); + return CSAssignment.NULL_ASSIGNMENT; + } + } + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED, + ActivityDiagnosticConstant.EMPTY); + + return CSAssignment.NULL_ASSIGNMENT; + } + + @Override + public boolean accept(Resource cluster, + ResourceCommitRequest request) { + ContainerAllocationProposal allocation = + request.getFirstAllocatedOrReservedContainer(); + SchedulerContainer schedulerContainer = + allocation.getAllocatedOrReservedContainer(); + + // Do not check limits when allocation from a reserved container + if (allocation.getAllocateFromReservedContainer() == null) { + try { + readLock.lock(); + FiCaSchedulerApp app = + schedulerContainer.getSchedulerApplicationAttempt(); + String username = app.getUser(); + String p = schedulerContainer.getNodePartition(); + + // check user-limit + Resource userLimit = computeUserLimitAndSetHeadroom(app, cluster, p, + allocation.getSchedulingMode()); + + // Deduct resources that we can release + Resource usedResource = Resources.clone(getUser(username).getUsed(p)); + Resources.subtractFrom(usedResource, + request.getTotalReleasedResource()); + + if (Resources.greaterThan(resourceCalculator, cluster, usedResource, + userLimit)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Used resource=" + usedResource + " exceeded user-limit=" + + userLimit); + } + return false; + } + } finally { + readLock.unlock(); + } + } + + return super.accept(cluster, request); + } + + private void internalReleaseContainer(Resource clusterResource, + SchedulerContainer schedulerContainer) { + RMContainer rmContainer = schedulerContainer.getRmContainer(); + + LeafQueue targetLeafQueue = + schedulerContainer.getSchedulerApplicationAttempt().getCSLeafQueue(); + + if (targetLeafQueue == this) { + // When trying to preempt containers from the same queue + if (rmContainer.hasIncreaseReservation()) { + // Increased container reservation + unreserveIncreasedContainer(clusterResource, + schedulerContainer.getSchedulerApplicationAttempt(), + schedulerContainer.getSchedulerNode(), rmContainer); + } else if (rmContainer.getState() == RMContainerState.RESERVED) { + // For other reserved containers + // This is a reservation exchange, complete previous reserved container + completedContainer(clusterResource, + schedulerContainer.getSchedulerApplicationAttempt(), + schedulerContainer.getSchedulerNode(), rmContainer, SchedulerUtils + .createAbnormalContainerStatus(rmContainer.getContainerId(), + SchedulerUtils.UNRESERVED_CONTAINER), + RMContainerEventType.RELEASED, null, false); + } + } else{ + // When trying to preempt containers from different queue -- this + // is for lazy preemption feature (kill preemption candidate in scheduling + // cycle). + targetLeafQueue.completedContainer(clusterResource, + schedulerContainer.getSchedulerApplicationAttempt(), + schedulerContainer.getSchedulerNode(), + schedulerContainer.getRmContainer(), SchedulerUtils + .createPreemptedContainerStatus(rmContainer.getContainerId(), + SchedulerUtils.PREEMPTED_CONTAINER), + RMContainerEventType.KILL, null, false); + } + } + + private void releaseContainers(Resource clusterResource, + ResourceCommitRequest request) { + for (SchedulerContainer c : request + .getContainersToRelease()) { + internalReleaseContainer(clusterResource, c); + } + + // Handle container reservation looking, or lazy preemption case: + if (null != request.getContainersToAllocate() && !request + .getContainersToAllocate().isEmpty()) { + for (ContainerAllocationProposal context : request + .getContainersToAllocate()) { + if (null != context.getToRelease()) { + for (SchedulerContainer c : context + .getToRelease()) { + internalReleaseContainer(clusterResource, c); + } + } + } + } + } + + public void apply(Resource cluster, + ResourceCommitRequest request) { + // Do we need to call parent queue's apply? + boolean applyToParentQueue = false; + + releaseContainers(cluster, request); + + try { + writeLock.lock(); + + if (request.anythingAllocatedOrReserved()) { + ContainerAllocationProposal + allocation = request.getFirstAllocatedOrReservedContainer(); + SchedulerContainer + schedulerContainer = allocation.getAllocatedOrReservedContainer(); + + // Do not modify queue when allocation from reserved container + if (allocation.getAllocateFromReservedContainer() == null) { + // Only invoke apply() of ParentQueue when new allocation / + // reservation happen. + applyToParentQueue = true; + // Book-keeping + // Note: Update headroom to account for current allocation too... + allocateResource(cluster, + schedulerContainer.getSchedulerApplicationAttempt(), + allocation.getAllocatedOrReservedResource(), + schedulerContainer.getNodePartition(), + schedulerContainer.getRmContainer(), + allocation.isIncreasedAllocation()); + orderingPolicy.containerAllocated( + schedulerContainer.getSchedulerApplicationAttempt(), + schedulerContainer.getRmContainer()); + } + + // Update reserved resource + if (Resources.greaterThan(resourceCalculator, cluster, + request.getTotalReservedResource(), Resources.none())) { + incReservedResource(schedulerContainer.getNodePartition(), + request.getTotalReservedResource()); + } + } } finally { writeLock.unlock(); } + + if (parent != null && applyToParentQueue) { + parent.apply(cluster, request); + } } + protected Resource getHeadroom(User user, Resource queueCurrentLimit, Resource clusterResource, FiCaSchedulerApp application) { return getHeadroom(user, queueCurrentLimit, clusterResource, application, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java index a69af6ecdf4..fd0c68bbcbf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java @@ -47,8 +47,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Activi import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.AllocationState; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils; import org.apache.hadoop.yarn.util.resource.Resources; import java.io.IOException; @@ -71,12 +76,10 @@ public class ParentQueue extends AbstractCSQueue { protected final Set childQueues; private final boolean rootQueue; - final Comparator nonPartitionedQueueComparator; - final PartitionedQueueComparator partitionQueueComparator; - volatile int numApplications; + private final Comparator nonPartitionedQueueComparator; + private final PartitionedQueueComparator partitionQueueComparator; + private volatile int numApplications; private final CapacitySchedulerContext scheduler; - private boolean needToResortQueuesAtNextAllocation = false; - private int offswitchPerHeartbeatLimit; private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); @@ -86,7 +89,7 @@ public class ParentQueue extends AbstractCSQueue { super(cs, queueName, parent, old); this.scheduler = cs; this.nonPartitionedQueueComparator = cs.getNonPartitionedQueueComparator(); - this.partitionQueueComparator = cs.getPartitionedQueueComparator(); + this.partitionQueueComparator = new PartitionedQueueComparator(); this.rootQueue = (parent == null); @@ -126,16 +129,12 @@ public class ParentQueue extends AbstractCSQueue { } } - offswitchPerHeartbeatLimit = - csContext.getConfiguration().getOffSwitchPerHeartbeatLimit(); - LOG.info(queueName + ", capacity=" + this.queueCapacities.getCapacity() + ", absoluteCapacity=" + this.queueCapacities.getAbsoluteCapacity() + ", maxCapacity=" + this.queueCapacities.getMaximumCapacity() + ", absoluteMaxCapacity=" + this.queueCapacities .getAbsoluteMaximumCapacity() + ", state=" + state + ", acls=" + aclsString + ", labels=" + labelStrBuilder.toString() + "\n" - + ", offswitchPerHeartbeatLimit = " + getOffSwitchPerHeartbeatLimit() + ", reservationsContinueLooking=" + reservationsContinueLooking); } finally { writeLock.unlock(); @@ -215,11 +214,6 @@ public class ParentQueue extends AbstractCSQueue { } - @Private - public int getOffSwitchPerHeartbeatLimit() { - return offswitchPerHeartbeatLimit; - } - private QueueUserACLInfo getUserAclInfo( UserGroupInformation user) { try { @@ -435,156 +429,145 @@ public class ParentQueue extends AbstractCSQueue { @Override public CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, ResourceLimits resourceLimits, - SchedulingMode schedulingMode) { - int offswitchCount = 0; - try { - writeLock.lock(); - // if our queue cannot access this node, just return - if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY - && !accessibleToPartition(node.getPartition())) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skip this queue=" + getQueuePath() - + ", because it is not able to access partition=" + node - .getPartition()); - } + PlacementSet ps, ResourceLimits resourceLimits, + SchedulingMode schedulingMode) { + FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps); - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParentName(), getQueueName(), ActivityState.REJECTED, - ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node - .getPartition()); - if (rootQueue) { - ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager, - node); - } - - return CSAssignment.NULL_ASSIGNMENT; + // if our queue cannot access this node, just return + if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY + && !accessibleToPartition(ps.getPartition())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip this queue=" + getQueuePath() + + ", because it is not able to access partition=" + ps + .getPartition()); } - // Check if this queue need more resource, simply skip allocation if this - // queue doesn't need more resources. - if (!super.hasPendingResourceRequest(node.getPartition(), clusterResource, - schedulingMode)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skip this queue=" + getQueuePath() - + ", because it doesn't need more resource, schedulingMode=" - + schedulingMode.name() + " node-partition=" + node + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParentName(), getQueueName(), ActivityState.REJECTED, + ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node .getPartition()); - } + if (rootQueue) { + ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager, + node); + } + + return CSAssignment.NULL_ASSIGNMENT; + } + + // Check if this queue need more resource, simply skip allocation if this + // queue doesn't need more resources. + if (!super.hasPendingResourceRequest(ps.getPartition(), clusterResource, + schedulingMode)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip this queue=" + getQueuePath() + + ", because it doesn't need more resource, schedulingMode=" + + schedulingMode.name() + " node-partition=" + ps + .getPartition()); + } + + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParentName(), getQueueName(), ActivityState.SKIPPED, + ActivityDiagnosticConstant.QUEUE_DO_NOT_NEED_MORE_RESOURCE); + if (rootQueue) { + ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager, + node); + } + + return CSAssignment.NULL_ASSIGNMENT; + } + + CSAssignment assignment = new CSAssignment(Resources.createResource(0, 0), + NodeType.NODE_LOCAL); + + while (canAssign(clusterResource, node)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Trying to assign containers to child-queue of " + + getQueueName()); + } + + // Are we over maximum-capacity for this queue? + // This will also consider parent's limits and also continuous reservation + // looking + if (!super.canAssignToThisQueue(clusterResource, ps.getPartition(), + resourceLimits, Resources + .createResource(getMetrics().getReservedMB(), + getMetrics().getReservedVirtualCores()), schedulingMode)) { ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, getParentName(), getQueueName(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.QUEUE_DO_NOT_NEED_MORE_RESOURCE); + ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT); if (rootQueue) { ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager, node); } - return CSAssignment.NULL_ASSIGNMENT; + break; } - CSAssignment assignment = new CSAssignment(Resources.createResource(0, 0), - NodeType.NODE_LOCAL); + // Schedule + CSAssignment assignedToChild = assignContainersToChildQueues( + clusterResource, ps, resourceLimits, schedulingMode); + assignment.setType(assignedToChild.getType()); + assignment.setRequestLocalityType( + assignedToChild.getRequestLocalityType()); + assignment.setExcessReservation(assignedToChild.getExcessReservation()); + assignment.setContainersToKill(assignedToChild.getContainersToKill()); - while (canAssign(clusterResource, node)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Trying to assign containers to child-queue of " - + getQueueName()); - } + // Done if no child-queue assigned anything + if (Resources.greaterThan(resourceCalculator, clusterResource, + assignedToChild.getResource(), Resources.none())) { - // Are we over maximum-capacity for this queue? - // This will also consider parent's limits and also continuous reservation - // looking - if (!super.canAssignToThisQueue(clusterResource, node.getPartition(), - resourceLimits, Resources - .createResource(getMetrics().getReservedMB(), - getMetrics().getReservedVirtualCores()), schedulingMode)) { + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParentName(), getQueueName(), ActivityState.ACCEPTED, + ActivityDiagnosticConstant.EMPTY); - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParentName(), getQueueName(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT); + boolean isReserved = + assignedToChild.getAssignmentInformation().getReservationDetails() + != null && !assignedToChild.getAssignmentInformation() + .getReservationDetails().isEmpty(); + if (node != null && !isReserved) { if (rootQueue) { - ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager, - node); + ActivitiesLogger.NODE.finishAllocatedNodeAllocation( + activitiesManager, node, + assignedToChild.getAssignmentInformation() + .getFirstAllocatedOrReservedContainerId(), + AllocationState.ALLOCATED); } - - break; - } - - // Schedule - CSAssignment assignedToChild = assignContainersToChildQueues( - clusterResource, node, resourceLimits, schedulingMode); - assignment.setType(assignedToChild.getType()); - - // Done if no child-queue assigned anything - if (Resources.greaterThan(resourceCalculator, clusterResource, - assignedToChild.getResource(), Resources.none())) { - - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParentName(), getQueueName(), ActivityState.ACCEPTED, - ActivityDiagnosticConstant.EMPTY); - - if (node.getReservedContainer() == null) { - if (rootQueue) { - ActivitiesLogger.NODE.finishAllocatedNodeAllocation( - activitiesManager, node, - assignedToChild.getAssignmentInformation() - .getFirstAllocatedOrReservedContainerId(), - AllocationState.ALLOCATED); - } - } else{ - if (rootQueue) { - ActivitiesLogger.NODE.finishAllocatedNodeAllocation( - activitiesManager, node, - assignedToChild.getAssignmentInformation() - .getFirstAllocatedOrReservedContainerId(), - AllocationState.RESERVED); - } - } - - // Track resource utilization for the parent-queue - allocateResource(clusterResource, assignedToChild.getResource(), - node.getPartition(), assignedToChild.isIncreasedAllocation()); - - // Track resource utilization in this pass of the scheduler - Resources.addTo(assignment.getResource(), - assignedToChild.getResource()); - Resources.addTo(assignment.getAssignmentInformation().getAllocated(), - assignedToChild.getAssignmentInformation().getAllocated()); - Resources.addTo(assignment.getAssignmentInformation().getReserved(), - assignedToChild.getAssignmentInformation().getReserved()); - assignment.getAssignmentInformation().incrAllocations( - assignedToChild.getAssignmentInformation().getNumAllocations()); - assignment.getAssignmentInformation().incrReservations( - assignedToChild.getAssignmentInformation().getNumReservations()); - assignment.getAssignmentInformation().getAllocationDetails().addAll( - assignedToChild.getAssignmentInformation() - .getAllocationDetails()); - assignment.getAssignmentInformation().getReservationDetails().addAll( - assignedToChild.getAssignmentInformation() - .getReservationDetails()); - assignment.setIncreasedAllocation( - assignedToChild.isIncreasedAllocation()); - - LOG.info("assignedContainer" + " queue=" + getQueueName() - + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" - + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() - + " cluster=" + clusterResource); - } else{ - assignment.setSkippedType(assignedToChild.getSkippedType()); - - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, - getParentName(), getQueueName(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.EMPTY); if (rootQueue) { - ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager, - node); + ActivitiesLogger.NODE.finishAllocatedNodeAllocation( + activitiesManager, node, + assignedToChild.getAssignmentInformation() + .getFirstAllocatedOrReservedContainerId(), + AllocationState.RESERVED); } - - break; } + // Track resource utilization in this pass of the scheduler + Resources.addTo(assignment.getResource(), + assignedToChild.getResource()); + Resources.addTo(assignment.getAssignmentInformation().getAllocated(), + assignedToChild.getAssignmentInformation().getAllocated()); + Resources.addTo(assignment.getAssignmentInformation().getReserved(), + assignedToChild.getAssignmentInformation().getReserved()); + assignment.getAssignmentInformation().incrAllocations( + assignedToChild.getAssignmentInformation().getNumAllocations()); + assignment.getAssignmentInformation().incrReservations( + assignedToChild.getAssignmentInformation().getNumReservations()); + assignment.getAssignmentInformation().getAllocationDetails().addAll( + assignedToChild.getAssignmentInformation() + .getAllocationDetails()); + assignment.getAssignmentInformation().getReservationDetails().addAll( + assignedToChild.getAssignmentInformation() + .getReservationDetails()); + assignment.setIncreasedAllocation( + assignedToChild.isIncreasedAllocation()); + + LOG.info("assignedContainer" + " queue=" + getQueueName() + + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" + + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + + " cluster=" + clusterResource); + if (LOG.isDebugEnabled()) { LOG.debug( "ParentQ=" + getQueueName() + " assignedSoFarInThisIteration=" @@ -592,39 +575,47 @@ public class ParentQueue extends AbstractCSQueue { + getUsedCapacity() + " absoluteUsedCapacity=" + getAbsoluteUsedCapacity()); } + } else{ + assignment.setSkippedType(assignedToChild.getSkippedType()); - if (assignment.getType() == NodeType.OFF_SWITCH) { - offswitchCount++; + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + getParentName(), getQueueName(), ActivityState.SKIPPED, + ActivityDiagnosticConstant.EMPTY); + if (rootQueue) { + ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager, + node); } - // Do not assign more containers if this isn't the root queue - // or if we've already assigned enough OFF_SWITCH containers in - // this pass - if (!rootQueue || offswitchCount >= getOffSwitchPerHeartbeatLimit()) { - if (LOG.isDebugEnabled()) { - if (rootQueue && offswitchCount >= getOffSwitchPerHeartbeatLimit()) { - LOG.debug("Assigned maximum number of off-switch containers: " + - offswitchCount + ", assignments so far: " + assignment); - } - } - break; - } + break; } - return assignment; - } finally { - writeLock.unlock(); + /* + * Previously here, we can allocate more than one container for each + * allocation under rootQ. Now this logic is not proper any more + * in global scheduling world. + * + * So here do not try to allocate more than one container for each + * allocation, let top scheduler make the decision. + */ + break; } + + return assignment; } private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) { + // When node == null means global scheduling is enabled, always return true + if (null == node) { + return true; + } + // Two conditions need to meet when trying to allocate: // 1) Node doesn't have reserved container // 2) Node's available-resource + killable-resource should > 0 return node.getReservedContainer() == null && Resources.greaterThanOrEqual( resourceCalculator, clusterResource, Resources - .add(node.getUnallocatedResource(), node.getTotalKillableResources()), - minimumAllocation); + .add(node.getUnallocatedResource(), + node.getTotalKillableResources()), minimumAllocation); } private ResourceLimits getResourceLimitsOfChild(CSQueue child, @@ -662,28 +653,20 @@ public class ParentQueue extends AbstractCSQueue { return new ResourceLimits(childLimit); } - - private Iterator sortAndGetChildrenAllocationIterator(FiCaSchedulerNode node) { - if (node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) { - if (needToResortQueuesAtNextAllocation) { - // If we skipped resort queues last time, we need to re-sort queue - // before allocation - List childrenList = new ArrayList<>(childQueues); - childQueues.clear(); - childQueues.addAll(childrenList); - needToResortQueuesAtNextAllocation = false; - } - return childQueues.iterator(); - } - partitionQueueComparator.setPartitionToLookAt(node.getPartition()); + private Iterator sortAndGetChildrenAllocationIterator( + String partition) { + // Previously we keep a sorted list for default partition, it is not good + // when multi-threading scheduler is enabled, so to make a simpler code + // now re-sort queue every time irrespective to node partition. + partitionQueueComparator.setPartitionToLookAt(partition); List childrenList = new ArrayList<>(childQueues); Collections.sort(childrenList, partitionQueueComparator); return childrenList.iterator(); } - - private CSAssignment assignContainersToChildQueues( - Resource cluster, FiCaSchedulerNode node, ResourceLimits limits, + + private CSAssignment assignContainersToChildQueues(Resource cluster, + PlacementSet ps, ResourceLimits limits, SchedulingMode schedulingMode) { CSAssignment assignment = CSAssignment.NULL_ASSIGNMENT; @@ -691,8 +674,8 @@ public class ParentQueue extends AbstractCSQueue { printChildQueues(); // Try to assign to most 'under-served' sub-queue - for (Iterator iter = sortAndGetChildrenAllocationIterator(node); iter - .hasNext();) { + for (Iterator iter = sortAndGetChildrenAllocationIterator( + ps.getPartition()); iter.hasNext(); ) { CSQueue childQueue = iter.next(); if(LOG.isDebugEnabled()) { LOG.debug("Trying to assign to queue: " + childQueue.getQueuePath() @@ -702,9 +685,9 @@ public class ParentQueue extends AbstractCSQueue { // Get ResourceLimits of child queue before assign containers ResourceLimits childLimits = getResourceLimitsOfChild(childQueue, cluster, parentLimits, - node.getPartition()); + ps.getPartition()); - CSAssignment childAssignment = childQueue.assignContainers(cluster, node, + CSAssignment childAssignment = childQueue.assignContainers(cluster, ps, childLimits, schedulingMode); if(LOG.isDebugEnabled()) { LOG.debug("Assigned to queue: " + childQueue.getQueuePath() + @@ -712,22 +695,9 @@ public class ParentQueue extends AbstractCSQueue { childAssignment.getResource() + ", " + childAssignment.getType()); } - // If we do assign, remove the queue and re-insert in-order to re-sort if (Resources.greaterThan( resourceCalculator, cluster, childAssignment.getResource(), Resources.none())) { - // Only update childQueues when we doing non-partitioned node - // allocation. - if (RMNodeLabelsManager.NO_LABEL.equals(node.getPartition())) { - // Remove and re-insert to sort - iter.remove(); - LOG.info("Re-sorting assigned queue: " + childQueue.getQueuePath() - + " stats: " + childQueue); - childQueues.add(childQueue); - if (LOG.isDebugEnabled()) { - printChildQueues(); - } - } assignment = childAssignment; break; } else if (childAssignment.getSkippedType() == @@ -770,10 +740,10 @@ public class ParentQueue extends AbstractCSQueue { + " child-queues: " + getChildQueuesToPrint()); } } - + private void internalReleaseResource(Resource clusterResource, - FiCaSchedulerNode node, Resource releasedResource, boolean changeResource, - CSQueue completedChildQueue, boolean sortQueues) { + FiCaSchedulerNode node, Resource releasedResource, + boolean changeResource) { try { writeLock.lock(); super.releaseResource(clusterResource, releasedResource, @@ -784,29 +754,6 @@ public class ParentQueue extends AbstractCSQueue { "completedContainer " + this + ", cluster=" + clusterResource); } - // Note that this is using an iterator on the childQueues so this can't - // be called if already within an iterator for the childQueues. Like - // from assignContainersToChildQueues. - if (sortQueues) { - // reinsert the updated queue - for (Iterator iter = childQueues.iterator(); - iter.hasNext(); ) { - CSQueue csqueue = iter.next(); - if (csqueue.equals(completedChildQueue)) { - iter.remove(); - if (LOG.isDebugEnabled()) { - LOG.debug("Re-sorting completed queue: " + csqueue); - } - childQueues.add(csqueue); - break; - } - } - } - - // If we skipped sort queue this time, we need to resort queues to make - // sure we allocate from least usage (or order defined by queue policy) - // queues. - needToResortQueuesAtNextAllocation = !sortQueues; } finally { writeLock.unlock(); } @@ -821,8 +768,7 @@ public class ParentQueue extends AbstractCSQueue { Resources.negate(decreaseRequest.getDeltaCapacity()); internalReleaseResource(clusterResource, - csContext.getNode(decreaseRequest.getNodeId()), absDeltaCapacity, false, - null, false); + csContext.getNode(decreaseRequest.getNodeId()), absDeltaCapacity, false); // Inform the parent if (parent != null) { @@ -835,7 +781,7 @@ public class ParentQueue extends AbstractCSQueue { FiCaSchedulerApp app, FiCaSchedulerNode node, RMContainer rmContainer) { if (app != null) { internalReleaseResource(clusterResource, node, - rmContainer.getReservedResource(), false, null, false); + rmContainer.getReservedResource(), false); // Inform the parent if (parent != null) { @@ -853,8 +799,7 @@ public class ParentQueue extends AbstractCSQueue { boolean sortQueues) { if (application != null) { internalReleaseResource(clusterResource, node, - rmContainer.getContainer().getResource(), false, completedChildQueue, - sortQueues); + rmContainer.getContainer().getResource(), false); // Inform the parent if (parent != null) { @@ -1062,4 +1007,37 @@ public class ParentQueue extends AbstractCSQueue { } } } + + public void apply(Resource cluster, + ResourceCommitRequest request) { + if (request.anythingAllocatedOrReserved()) { + ContainerAllocationProposal + allocation = request.getFirstAllocatedOrReservedContainer(); + SchedulerContainer + schedulerContainer = allocation.getAllocatedOrReservedContainer(); + + // Do not modify queue when allocation from reserved container + if (allocation.getAllocateFromReservedContainer() == null) { + try { + writeLock.lock(); + // Book-keeping + // Note: Update headroom to account for current allocation too... + allocateResource(cluster, allocation.getAllocatedOrReservedResource(), + schedulerContainer.getNodePartition(), + allocation.isIncreasedAllocation()); + + LOG.info("assignedContainer" + " queue=" + getQueueName() + + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" + + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + + " cluster=" + cluster); + } finally { + writeLock.unlock(); + } + } + } + + if (parent != null) { + parent.apply(cluster, request); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java index fa13df460d1..5bb91e2a711 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssign import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -77,11 +78,13 @@ public abstract class AbstractContainerAllocator { // Handle excess reservation assignment.setExcessReservation(result.getContainerToBeUnreserved()); + assignment.setRequestLocalityType(result.requestLocalityType); + // If we allocated something if (Resources.greaterThan(rc, clusterResource, result.getResourceToBeAllocated(), Resources.none())) { Resource allocatedResource = result.getResourceToBeAllocated(); - Container updatedContainer = result.getUpdatedContainer(); + RMContainer updatedContainer = result.getUpdatedContainer(); assignment.setResource(allocatedResource); assignment.setType(result.getContainerNodeType()); @@ -92,8 +95,7 @@ public abstract class AbstractContainerAllocator { + application.getApplicationId() + " resource=" + allocatedResource + " queue=" + this.toString() + " cluster=" + clusterResource); assignment.getAssignmentInformation().addReservationDetails( - updatedContainer.getId(), - application.getCSLeafQueue().getQueuePath()); + updatedContainer, application.getCSLeafQueue().getQueuePath()); assignment.getAssignmentInformation().incrReservations(); Resources.addTo(assignment.getAssignmentInformation().getReserved(), allocatedResource); @@ -111,41 +113,37 @@ public abstract class AbstractContainerAllocator { ActivityState.RESERVED); ActivitiesLogger.APP.finishAllocatedAppAllocationRecording( activitiesManager, application.getApplicationId(), - updatedContainer.getId(), ActivityState.RESERVED, + updatedContainer.getContainerId(), ActivityState.RESERVED, ActivityDiagnosticConstant.EMPTY); } } else if (result.getAllocationState() == AllocationState.ALLOCATED){ // This is a new container // Inform the ordering policy - LOG.info("assignedContainer" + " application attempt=" - + application.getApplicationAttemptId() + " container=" - + updatedContainer.getId() + " queue=" + this + " clusterResource=" + LOG.info("assignedContainer" + " application attempt=" + application + .getApplicationAttemptId() + " container=" + updatedContainer + .getContainerId() + " queue=" + this + " clusterResource=" + clusterResource + " type=" + assignment.getType()); - application - .getCSLeafQueue() - .getOrderingPolicy() - .containerAllocated(application, - application.getRMContainer(updatedContainer.getId())); - assignment.getAssignmentInformation().addAllocationDetails( - updatedContainer.getId(), - application.getCSLeafQueue().getQueuePath()); + updatedContainer, application.getCSLeafQueue().getQueuePath()); assignment.getAssignmentInformation().incrAllocations(); Resources.addTo(assignment.getAssignmentInformation().getAllocated(), allocatedResource); if (rmContainer != null) { assignment.setFulfilledReservation(true); + assignment.setFulfilledReservedContainer(rmContainer); } ActivitiesLogger.APP.recordAppActivityWithAllocation(activitiesManager, node, application, updatedContainer, ActivityState.ALLOCATED); ActivitiesLogger.APP.finishAllocatedAppAllocationRecording( activitiesManager, application.getApplicationId(), - updatedContainer.getId(), ActivityState.ACCEPTED, + updatedContainer.getContainerId(), ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); + // Update unformed resource + application.incUnconfirmedRes(allocatedResource); } assignment.setContainersToKill(result.getToKillContainers()); @@ -170,8 +168,15 @@ public abstract class AbstractContainerAllocator { *
  • Do allocation: this will decide/create allocated/reserved * container, this will also update metrics
  • * + * + * @param clusterResource clusterResource + * @param ps PlacementSet + * @param schedulingMode scheduling mode (exclusive or nonexclusive) + * @param resourceLimits resourceLimits + * @param reservedContainer reservedContainer + * @return CSAssignemnt proposal */ public abstract CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, SchedulingMode schedulingMode, + PlacementSet ps, SchedulingMode schedulingMode, ResourceLimits resourceLimits, RMContainer reservedContainer); } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.java index 8f749f6d7f0..f4085085004 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.java @@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.util.resource.Resources; import java.util.List; @@ -57,8 +58,13 @@ public class ContainerAllocation { private Resource resourceToBeAllocated = Resources.none(); AllocationState state; NodeType containerNodeType = NodeType.NODE_LOCAL; - NodeType requestNodeType = NodeType.NODE_LOCAL; - Container updatedContainer; + NodeType requestLocalityType = null; + + /** + * When some (new) container allocated/reserved or some increase container + * request allocated/reserved, updatedContainer will be set. + */ + RMContainer updatedContainer; private List toKillContainers; public ContainerAllocation(RMContainer containerToBeUnreserved, @@ -87,7 +93,7 @@ public class ContainerAllocation { return containerNodeType; } - public Container getUpdatedContainer() { + public RMContainer getUpdatedContainer() { return updatedContainer; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocator.java index 4eaa24bb26d..57188d88771 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocator.java @@ -28,12 +28,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssign import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; public class ContainerAllocator extends AbstractContainerAllocator { - AbstractContainerAllocator increaseContainerAllocator; - AbstractContainerAllocator regularContainerAllocator; + private AbstractContainerAllocator increaseContainerAllocator; + private AbstractContainerAllocator regularContainerAllocator; public ContainerAllocator(FiCaSchedulerApp application, ResourceCalculator rc, RMContext rmContext) { @@ -52,17 +53,17 @@ public class ContainerAllocator extends AbstractContainerAllocator { @Override public CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, SchedulingMode schedulingMode, + PlacementSet ps, SchedulingMode schedulingMode, ResourceLimits resourceLimits, RMContainer reservedContainer) { if (reservedContainer != null) { if (reservedContainer.getState() == RMContainerState.RESERVED) { // It's a regular container return regularContainerAllocator.assignContainers(clusterResource, - node, schedulingMode, resourceLimits, reservedContainer); + ps, schedulingMode, resourceLimits, reservedContainer); } else { // It's a increase container return increaseContainerAllocator.assignContainers(clusterResource, - node, schedulingMode, resourceLimits, reservedContainer); + ps, schedulingMode, resourceLimits, reservedContainer); } } else { /* @@ -70,14 +71,14 @@ public class ContainerAllocator extends AbstractContainerAllocator { * anything, we will try to allocate regular container */ CSAssignment assign = - increaseContainerAllocator.assignContainers(clusterResource, node, + increaseContainerAllocator.assignContainers(clusterResource, ps, schedulingMode, resourceLimits, null); if (Resources.greaterThan(rc, clusterResource, assign.getResource(), Resources.none())) { return assign; } - return regularContainerAllocator.assignContainers(clusterResource, node, + return regularContainerAllocator.assignContainers(clusterResource, ps, schedulingMode, resourceLimits, null); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java index 509dfba1cef..74a64c16415 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java @@ -18,12 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -36,16 +30,21 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; - -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; + public class IncreaseContainerAllocator extends AbstractContainerAllocator { private static final Log LOG = LogFactory.getLog(IncreaseContainerAllocator.class); @@ -76,7 +75,7 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { request.getDeltaCapacity()); assignment.getAssignmentInformation().incrReservations(); assignment.getAssignmentInformation().addReservationDetails( - request.getContainerId(), application.getCSLeafQueue().getQueuePath()); + request.getRMContainer(), application.getCSLeafQueue().getQueuePath()); assignment.setIncreasedAllocation(true); LOG.info("Reserved increase container request:" + request.toString()); @@ -93,8 +92,12 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { request.getDeltaCapacity()); assignment.getAssignmentInformation().incrAllocations(); assignment.getAssignmentInformation().addAllocationDetails( - request.getContainerId(), application.getCSLeafQueue().getQueuePath()); + request.getRMContainer(), application.getCSLeafQueue().getQueuePath()); assignment.setIncreasedAllocation(true); + + if (fromReservation) { + assignment.setFulfilledReservedContainer(request.getRMContainer()); + } // notify application application @@ -114,19 +117,6 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { SchedContainerChangeRequest increaseRequest) { if (Resources.fitsIn(rc, cluster, increaseRequest.getDeltaCapacity(), node.getUnallocatedResource())) { - // OK, we can allocate this increase request - // Unreserve it first - application.unreserve( - increaseRequest.getRMContainer().getAllocatedSchedulerKey(), - (FiCaSchedulerNode) node, increaseRequest.getRMContainer()); - - // Notify application - application.increaseContainer(increaseRequest); - - // Notify node - node.increaseContainer(increaseRequest.getContainerId(), - increaseRequest.getDeltaCapacity()); - return createSuccessfullyIncreasedCSAssignment(increaseRequest, true); } else { if (LOG.isDebugEnabled()) { @@ -144,40 +134,26 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { Resource cluster, SchedContainerChangeRequest increaseRequest) { if (Resources.fitsIn(rc, cluster, increaseRequest.getDeltaCapacity(), node.getUnallocatedResource())) { - // Notify node - node.increaseContainer(increaseRequest.getContainerId(), - increaseRequest.getDeltaCapacity()); - - // OK, we can allocate this increase request - // Notify application - application.increaseContainer(increaseRequest); return createSuccessfullyIncreasedCSAssignment(increaseRequest, false); - } else { - boolean reservationSucceeded = - application.reserveIncreasedContainer( - increaseRequest.getRMContainer().getAllocatedSchedulerKey(), - node, increaseRequest.getRMContainer(), - increaseRequest.getDeltaCapacity()); - - if (reservationSucceeded) { - // We cannot allocate this container, but since queue capacity / - // user-limit matches, we can reserve this container on this node. - return createReservedIncreasedCSAssignment(increaseRequest); - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Reserve increase request=" + increaseRequest.toString() - + " failed. Skipping.."); - } - return CSAssignment.SKIP_ASSIGNMENT; - } + } else{ + // We cannot allocate this container, but since queue capacity / + // user-limit matches, we can reserve this container on this node. + return createReservedIncreasedCSAssignment(increaseRequest); } } @Override public CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, SchedulingMode schedulingMode, + PlacementSet ps, SchedulingMode schedulingMode, ResourceLimits resourceLimits, RMContainer reservedContainer) { AppSchedulingInfo sinfo = application.getAppSchedulingInfo(); + FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps); + + if (null == node) { + // This is global scheduling enabled + // FIXME, support container increase when global scheduling enabled + return CSAssignment.SKIP_ASSIGNMENT; + } NodeId nodeId = node.getNodeID(); if (reservedContainer == null) { @@ -258,8 +234,6 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { } Iterator> iter = increaseRequestMap.entrySet().iterator(); - List toBeRemovedRequests = - new ArrayList<>(); while (iter.hasNext()) { Entry entry = @@ -289,7 +263,7 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { if (LOG.isDebugEnabled()) { LOG.debug(" Container is not running any more, skip..."); } - toBeRemovedRequests.add(increaseRequest); + application.addToBeRemovedIncreaseRequest(increaseRequest); continue; } @@ -304,7 +278,7 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { LOG.debug(" Target capacity is more than what node can offer," + " node.resource=" + node.getTotalResource()); } - toBeRemovedRequests.add(increaseRequest); + application.addToBeRemovedIncreaseRequest(increaseRequest); continue; } @@ -318,15 +292,6 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator { break; } } - - // Remove invalid in request requests - if (!toBeRemovedRequests.isEmpty()) { - for (SchedContainerChangeRequest req : toBeRemovedRequests) { - sinfo.removeIncreaseRequest(req.getNodeId(), - req.getRMContainer().getAllocatedSchedulerKey(), - req.getContainerId()); - } - } // We may have allocated something if (assigned != null && assigned.getSkippedType() diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java index 1a3f71fe856..3e8282f15b2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java @@ -19,13 +19,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; @@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils; @@ -50,6 +51,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssign import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -71,12 +75,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { private boolean checkHeadroom(Resource clusterResource, ResourceLimits currentResourceLimits, Resource required, - FiCaSchedulerNode node) { + String nodePartition) { // If headroom + currentReservation < required, we cannot allocate this // require Resource resourceCouldBeUnReserved = application.getCurrentReservation(); if (!application.getCSLeafQueue().getReservationContinueLooking() - || !node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) { + || !nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) { // If we don't allow reservation continuous looking, OR we're looking at // non-default node partition, we won't allow to unreserve before // allocation. @@ -87,20 +91,17 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { required); } - - private ContainerAllocation preCheckForNewContainer(Resource clusterResource, - FiCaSchedulerNode node, SchedulingMode schedulingMode, + /* + * Pre-check if we can allocate a pending resource request + * (given schedulerKey) to a given PlacementSet. + * We will consider stuffs like exclusivity, pending resource, node partition, + * headroom, etc. + */ + private ContainerAllocation preCheckForPlacementSet(Resource clusterResource, + PlacementSet ps, SchedulingMode schedulingMode, ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey) { Priority priority = schedulerKey.getPriority(); - - if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) { - application.updateAppSkipNodeDiagnostics( - CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE); - ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( - activitiesManager, node, application, priority, - ActivityDiagnosticConstant.SKIP_BLACK_LISTED_NODE); - return ContainerAllocation.APP_SKIPPED; - } + FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps); ResourceRequest anyRequest = application.getResourceRequest(schedulerKey, ResourceRequest.ANY); @@ -144,7 +145,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // matches the node's label? // If not match, jump to next priority. if (!SchedulerUtils.checkResourceRequestMatchingNodePartition( - anyRequest.getNodeLabelExpression(), node.getPartition(), + anyRequest.getNodeLabelExpression(), ps.getPartition(), schedulingMode)) { ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( activitiesManager, node, application, priority, @@ -165,7 +166,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } } - if (!checkHeadroom(clusterResource, resourceLimits, required, node)) { + if (!checkHeadroom(clusterResource, resourceLimits, required, + ps.getPartition())) { if (LOG.isDebugEnabled()) { LOG.debug("cannot allocate required resource=" + required + " because of headroom"); @@ -176,9 +178,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { return ContainerAllocation.QUEUE_SKIPPED; } - // Inform the application it is about to get a scheduling opportunity - application.addSchedulingOpportunity(schedulerKey); - // Increase missed-non-partitioned-resource-request-opportunity. // This is to make sure non-partitioned-resource-request will prefer // to be allocated to non-partitioned nodes @@ -210,32 +209,43 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { return ContainerAllocation.APP_SKIPPED; } } - + return null; } - ContainerAllocation preAllocation(Resource clusterResource, + private ContainerAllocation checkIfNodeBlackListed(FiCaSchedulerNode node, + SchedulerRequestKey schedulerKey) { + Priority priority = schedulerKey.getPriority(); + + if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) { + application.updateAppSkipNodeDiagnostics( + CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE); + ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( + activitiesManager, node, application, priority, + ActivityDiagnosticConstant.SKIP_BLACK_LISTED_NODE); + return ContainerAllocation.APP_SKIPPED; + } + + return null; + } + + ContainerAllocation tryAllocateOnNode(Resource clusterResource, FiCaSchedulerNode node, SchedulingMode schedulingMode, ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey, RMContainer reservedContainer) { ContainerAllocation result; - if (null == reservedContainer) { - // pre-check when allocating new container - result = - preCheckForNewContainer(clusterResource, node, schedulingMode, - resourceLimits, schedulerKey); - if (null != result) { - return result; - } - } else { - // pre-check when allocating reserved container - if (application.getTotalRequiredResources(schedulerKey) == 0) { - // Release - return new ContainerAllocation(reservedContainer, null, - AllocationState.QUEUE_SKIPPED); - } + + // Sanity checks before assigning to this node + result = checkIfNodeBlackListed(node, schedulerKey); + if (null != result) { + return result; } + // Inform the application it is about to get a scheduling opportunity + // TODO, we may need to revisit here to see if we should add scheduling + // opportunity here + application.addSchedulingOpportunity(schedulerKey); + // Try to allocate containers on node result = assignContainersOnNode(clusterResource, node, schedulerKey, @@ -383,20 +393,20 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { Priority priority = schedulerKey.getPriority(); ContainerAllocation allocation; + NodeType requestLocalityType = null; - NodeType requestType = null; // Data-local ResourceRequest nodeLocalResourceRequest = application.getResourceRequest(schedulerKey, node.getNodeName()); if (nodeLocalResourceRequest != null) { - requestType = NodeType.NODE_LOCAL; + requestLocalityType = NodeType.NODE_LOCAL; allocation = assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest, node, schedulerKey, reservedContainer, schedulingMode, currentResoureLimits); if (Resources.greaterThan(rc, clusterResource, allocation.getResourceToBeAllocated(), Resources.none())) { - allocation.requestNodeType = requestType; + allocation.requestLocalityType = requestLocalityType; return allocation; } } @@ -412,9 +422,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { return ContainerAllocation.PRIORITY_SKIPPED; } - if (requestType != NodeType.NODE_LOCAL) { - requestType = NodeType.RACK_LOCAL; - } + requestLocalityType = requestLocalityType == null ? + NodeType.RACK_LOCAL : + requestLocalityType; allocation = assignRackLocalContainers(clusterResource, rackLocalResourceRequest, @@ -422,7 +432,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { currentResoureLimits); if (Resources.greaterThan(rc, clusterResource, allocation.getResourceToBeAllocated(), Resources.none())) { - allocation.requestNodeType = requestType; + allocation.requestLocalityType = requestLocalityType; return allocation; } } @@ -437,22 +447,22 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY); return ContainerAllocation.PRIORITY_SKIPPED; } - if (requestType != NodeType.NODE_LOCAL - && requestType != NodeType.RACK_LOCAL) { - requestType = NodeType.OFF_SWITCH; - } + + requestLocalityType = requestLocalityType == null ? + NodeType.OFF_SWITCH : + requestLocalityType; allocation = assignOffSwitchContainers(clusterResource, offSwitchResourceRequest, node, schedulerKey, reservedContainer, schedulingMode, currentResoureLimits); - allocation.requestNodeType = requestType; - + // When a returned allocation is LOCALITY_SKIPPED, since we're in // off-switch request now, we will skip this app w.r.t priorities if (allocation.state == AllocationState.LOCALITY_SKIPPED) { allocation.state = AllocationState.APP_SKIPPED; } + allocation.requestLocalityType = requestLocalityType; return allocation; } @@ -671,33 +681,27 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { private Container createContainer(FiCaSchedulerNode node, Resource capability, SchedulerRequestKey schedulerKey) { - NodeId nodeId = node.getRMNode().getNodeID(); - ContainerId containerId = - BuilderUtils.newContainerId(application.getApplicationAttemptId(), - application.getNewContainerId()); // Create the container - return BuilderUtils.newContainer(containerId, nodeId, + // Now set the containerId to null first, because it is possible the + // container will be rejected because of concurrent resource allocation. + // new containerId will be generated and assigned to the container + // after confirmed. + return BuilderUtils.newContainer(null, nodeId, node.getRMNode().getHttpAddress(), capability, schedulerKey.getPriority(), null, schedulerKey.getAllocationRequestId()); } - + private ContainerAllocation handleNewContainerAllocation( ContainerAllocation allocationResult, FiCaSchedulerNode node, - SchedulerRequestKey schedulerKey, RMContainer reservedContainer, - Container container) { - // Handling container allocation - // Did we previously reserve containers at this 'priority'? - if (reservedContainer != null) { - application.unreserve(schedulerKey, node, reservedContainer); - } - + SchedulerRequestKey schedulerKey, Container container) { // Inform the application - RMContainer allocatedContainer = - application.allocate(allocationResult.containerNodeType, node, - schedulerKey, lastResourceRequest, container); + RMContainer allocatedContainer = application.allocate(node, schedulerKey, + lastResourceRequest, container); + + allocationResult.updatedContainer = allocatedContainer; // Does the application need this resource? if (allocatedContainer == null) { @@ -710,13 +714,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { ActivityDiagnosticConstant.FAIL_TO_ALLOCATE, ActivityState.REJECTED); return ret; } - - // Inform the node - node.allocateContainer(allocatedContainer); - - // update locality statistics - application.incNumAllocatedContainers(allocationResult.containerNodeType, - allocationResult.requestNodeType); return allocationResult; } @@ -743,14 +740,18 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { if (allocationResult.getAllocationState() == AllocationState.ALLOCATED) { // When allocating container - allocationResult = - handleNewContainerAllocation(allocationResult, node, schedulerKey, - reservedContainer, container); + allocationResult = handleNewContainerAllocation(allocationResult, node, + schedulerKey, container); } else { // When reserving container - application.reserve(schedulerKey, node, reservedContainer, container); + RMContainer updatedContainer = reservedContainer; + if (updatedContainer == null) { + updatedContainer = new RMContainerImpl(container, + application.getApplicationAttemptId(), node.getNodeID(), + application.getAppSchedulingInfo().getUser(), rmContext); + } + allocationResult.updatedContainer = updatedContainer; } - allocationResult.updatedContainer = container; // Only reset opportunities when we FIRST allocate the container. (IAW, When // reservedContainer != null, it's not the first time) @@ -788,16 +789,46 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } private ContainerAllocation allocate(Resource clusterResource, - FiCaSchedulerNode node, SchedulingMode schedulingMode, + PlacementSet ps, SchedulingMode schedulingMode, ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey, RMContainer reservedContainer) { - ContainerAllocation result = - preAllocation(clusterResource, node, schedulingMode, resourceLimits, - schedulerKey, reservedContainer); + // Do checks before determining which node to allocate + // Directly return if this check fails. + ContainerAllocation result; + if (reservedContainer == null) { + result = preCheckForPlacementSet(clusterResource, ps, schedulingMode, + resourceLimits, schedulerKey); + if (null != result) { + return result; + } + } else { + // pre-check when allocating reserved container + if (application.getTotalRequiredResources(schedulerKey) == 0) { + // Release + return new ContainerAllocation(reservedContainer, null, + AllocationState.QUEUE_SKIPPED); + } + } - if (AllocationState.ALLOCATED == result.state - || AllocationState.RESERVED == result.state) { - result = doAllocation(result, node, schedulerKey, reservedContainer); + SchedulingPlacementSet schedulingPS = + application.getAppSchedulingInfo().getSchedulingPlacementSet( + schedulerKey); + + result = ContainerAllocation.PRIORITY_SKIPPED; + + Iterator iter = schedulingPS.getPreferredNodeIterator( + ps); + while (iter.hasNext()) { + FiCaSchedulerNode node = iter.next(); + + result = tryAllocateOnNode(clusterResource, node, schedulingMode, + resourceLimits, schedulerKey, reservedContainer); + + if (AllocationState.ALLOCATED == result.state + || AllocationState.RESERVED == result.state) { + result = doAllocation(result, node, schedulerKey, reservedContainer); + break; + } } return result; @@ -805,17 +836,19 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { @Override public CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, SchedulingMode schedulingMode, + PlacementSet ps, SchedulingMode schedulingMode, ResourceLimits resourceLimits, RMContainer reservedContainer) { + FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps); + if (reservedContainer == null) { // Check if application needs more resource, skip if it doesn't need more. if (!application.hasPendingResourceRequest(rc, - node.getPartition(), clusterResource, schedulingMode)) { + ps.getPartition(), clusterResource, schedulingMode)) { if (LOG.isDebugEnabled()) { LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId() + ", because it doesn't need more resource, schedulingMode=" - + schedulingMode.name() + " node-label=" + node.getPartition()); + + schedulingMode.name() + " node-label=" + ps.getPartition()); } ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( activitiesManager, node, application, application.getPriority(), @@ -826,7 +859,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // Schedule in priority order for (SchedulerRequestKey schedulerKey : application.getSchedulerKeys()) { ContainerAllocation result = - allocate(clusterResource, node, schedulingMode, resourceLimits, + allocate(clusterResource, ps, schedulingMode, resourceLimits, schedulerKey, null); AllocationState allocationState = result.getAllocationState(); @@ -845,7 +878,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { return CSAssignment.SKIP_ASSIGNMENT; } else { ContainerAllocation result = - allocate(clusterResource, node, schedulingMode, resourceLimits, + allocate(clusterResource, ps, schedulingMode, resourceLimits, reservedContainer.getReservedSchedulerKey(), reservedContainer); return getCSAssignmentFromAllocateResult(clusterResource, result, reservedContainer, node); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java index aad3bc71792..63d8a890e3e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/AssignmentInformation.java @@ -38,11 +38,13 @@ public class AssignmentInformation { } public static class AssignmentDetails { + public RMContainer rmContainer; public ContainerId containerId; public String queue; - public AssignmentDetails(ContainerId containerId, String queue) { - this.containerId = containerId; + public AssignmentDetails(RMContainer rmContainer, String queue) { + this.containerId = rmContainer.getContainerId(); + this.rmContainer = rmContainer; this.queue = queue; } } @@ -58,7 +60,7 @@ public class AssignmentInformation { for (Operation op : Operation.values()) { operationCounts.put(op, 0); operationResources.put(op, Resource.newInstance(0, 0)); - operationDetails.put(op, new ArrayList()); + operationDetails.put(op, new ArrayList<>()); } } @@ -98,17 +100,17 @@ public class AssignmentInformation { return operationResources.get(Operation.RESERVATION); } - private void addAssignmentDetails(Operation op, ContainerId containerId, + private void addAssignmentDetails(Operation op, RMContainer rmContainer, String queue) { - operationDetails.get(op).add(new AssignmentDetails(containerId, queue)); + operationDetails.get(op).add(new AssignmentDetails(rmContainer, queue)); } - public void addAllocationDetails(ContainerId containerId, String queue) { - addAssignmentDetails(Operation.ALLOCATION, containerId, queue); + public void addAllocationDetails(RMContainer rmContainer, String queue) { + addAssignmentDetails(Operation.ALLOCATION, rmContainer, queue); } - public void addReservationDetails(ContainerId containerId, String queue) { - addAssignmentDetails(Operation.RESERVATION, containerId, queue); + public void addReservationDetails(RMContainer rmContainer, String queue) { + addAssignmentDetails(Operation.RESERVATION, rmContainer, queue); } public List getAllocationDetails() { @@ -119,23 +121,31 @@ public class AssignmentInformation { return operationDetails.get(Operation.RESERVATION); } - private ContainerId getFirstContainerIdFromOperation(Operation op) { + private RMContainer getFirstRMContainerFromOperation(Operation op) { if (null != operationDetails.get(op)) { List assignDetails = operationDetails.get(op); if (!assignDetails.isEmpty()) { - return assignDetails.get(0).containerId; + return assignDetails.get(0).rmContainer; } } return null; } - public ContainerId getFirstAllocatedOrReservedContainerId() { - ContainerId containerId; - containerId = getFirstContainerIdFromOperation(Operation.ALLOCATION); - if (null != containerId) { - return containerId; + public RMContainer getFirstAllocatedOrReservedRMContainer() { + RMContainer rmContainer; + rmContainer = getFirstRMContainerFromOperation(Operation.ALLOCATION); + if (null != rmContainer) { + return rmContainer; } - return getFirstContainerIdFromOperation(Operation.RESERVATION); + return getFirstRMContainerFromOperation(Operation.RESERVATION); } -} + + public ContainerId getFirstAllocatedOrReservedContainerId() { + RMContainer rmContainer = getFirstAllocatedOrReservedRMContainer(); + if (null != rmContainer) { + return rmContainer.getContainerId(); + } + return null; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerAllocationProposal.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerAllocationProposal.java new file mode 100644 index 00000000000..ac83d6fc706 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerAllocationProposal.java @@ -0,0 +1,111 @@ +/** + * 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.resourcemanager.scheduler.common; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; + +import java.util.Collections; +import java.util.List; + +/** + * Proposal to allocate/reserve a new container + */ +public class ContainerAllocationProposal { + // Container we allocated or reserved + private SchedulerContainer allocatedOrReservedContainer; + + // Containers we need to release before allocating or reserving the + // new container + private List> toRelease = Collections.emptyList(); + + // When trying to allocate from a reserved container, set this, and this will + // not be included by toRelease list + private SchedulerContainer allocateFromReservedContainer; + + private boolean isIncreasedAllocation; + + private NodeType allocationLocalityType; + + private NodeType requestLocalityType; + + private SchedulingMode schedulingMode; + + private Resource allocatedResource; // newly allocated resource + + public ContainerAllocationProposal( + SchedulerContainer allocatedOrReservedContainer, + List> toRelease, + SchedulerContainer allocateFromReservedContainer, + boolean isIncreasedAllocation, NodeType allocationLocalityType, + NodeType requestLocalityType, SchedulingMode schedulingMode, + Resource allocatedResource) { + this.allocatedOrReservedContainer = allocatedOrReservedContainer; + if (null != toRelease) { + this.toRelease = toRelease; + } + this.allocateFromReservedContainer = allocateFromReservedContainer; + this.isIncreasedAllocation = isIncreasedAllocation; + this.allocationLocalityType = allocationLocalityType; + this.requestLocalityType = requestLocalityType; + this.schedulingMode = schedulingMode; + this.allocatedResource = allocatedResource; + } + + public SchedulingMode getSchedulingMode() { + return schedulingMode; + } + + public Resource getAllocatedOrReservedResource() { + return allocatedResource; + } + + public NodeType getAllocationLocalityType() { + return allocationLocalityType; + } + + public boolean isIncreasedAllocation() { + return isIncreasedAllocation; + } + + public SchedulerContainer getAllocateFromReservedContainer() { + return allocateFromReservedContainer; + } + + public SchedulerContainer getAllocatedOrReservedContainer() { + return allocatedOrReservedContainer; + } + + public List> getToRelease() { + return toRelease; + } + + @Override + public String toString() { + return allocatedOrReservedContainer.toString(); + } + + public NodeType getRequestLocalityType() { + return requestLocalityType; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ResourceAllocationCommitter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ResourceAllocationCommitter.java new file mode 100644 index 00000000000..bdea97db134 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ResourceAllocationCommitter.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.resourcemanager.scheduler.common; + +import org.apache.hadoop.yarn.api.records.Resource; + +/** + * Scheduler should implement this interface if it wants to have multi-threading + * plus global scheduling functionality + */ +public interface ResourceAllocationCommitter { + void tryCommit(Resource cluster, ResourceCommitRequest proposal); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ResourceCommitRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ResourceCommitRequest.java new file mode 100644 index 00000000000..5aca2028105 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ResourceCommitRequest.java @@ -0,0 +1,164 @@ +/** + * 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.resourcemanager.scheduler.common; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.util.resource.Resources; + +import java.util.Collections; +import java.util.List; + +public class ResourceCommitRequest { + // New containers to be allocated + private List> containersToAllocate = + Collections.emptyList(); + + // New containers to be released + private List> containersToReserve = + Collections.emptyList(); + + // We don't need these containers anymore + private List> toReleaseContainers = + Collections.emptyList(); + + private Resource totalAllocatedResource; + private Resource totalReservedResource; + private Resource totalReleasedResource; + + public ResourceCommitRequest( + List> containersToAllocate, + List> containersToReserve, + List> toReleaseContainers) { + if (null != containersToAllocate) { + this.containersToAllocate = containersToAllocate; + } + if (null != containersToReserve) { + this.containersToReserve = containersToReserve; + } + if (null != toReleaseContainers) { + this.toReleaseContainers = toReleaseContainers; + } + + totalAllocatedResource = Resources.createResource(0); + totalReservedResource = Resources.createResource(0); + + /* + * For total-release resource, it has two parts: + * 1) Unconditional release: for example, an app reserved a container, + * but the app doesn't has any pending resource. + * 2) Conditional release: for example, reservation continuous looking, or + * Lazy preemption -- which we need to kill some resource to allocate + * or reserve the new container. + * + * For the 2nd part, it is inside: + * ContainerAllocationProposal#toRelease, which means we will kill/release + * these containers to allocate/reserve the given container. + * + * So we need to account both of conditional/unconditional to-release + * containers to the total release-able resource. + */ + totalReleasedResource = Resources.createResource(0); + + for (ContainerAllocationProposal c : this.containersToAllocate) { + Resources.addTo(totalAllocatedResource, + c.getAllocatedOrReservedResource()); + for (SchedulerContainer r : c.getToRelease()) { + Resources.addTo(totalReleasedResource, + r.getRmContainer().getAllocatedOrReservedResource()); + } + } + + for (ContainerAllocationProposal c : this.containersToReserve) { + Resources.addTo(totalReservedResource, + c.getAllocatedOrReservedResource()); + for (SchedulerContainer r : c.getToRelease()) { + Resources.addTo(totalReleasedResource, + r.getRmContainer().getAllocatedOrReservedResource()); + } + } + + for (SchedulerContainer r : this.toReleaseContainers) { + Resources.addTo(totalReleasedResource, + r.getRmContainer().getAllocatedOrReservedResource()); + } + } + + public List> getContainersToAllocate() { + return containersToAllocate; + } + + public List> getContainersToReserve() { + return containersToReserve; + } + + public List> getContainersToRelease() { + return toReleaseContainers; + } + + public Resource getTotalAllocatedResource() { + return totalAllocatedResource; + } + + public Resource getTotalReservedResource() { + return totalReservedResource; + } + + public Resource getTotalReleasedResource() { + return totalReleasedResource; + } + + /* + * Util functions to make your life easier + */ + public boolean anythingAllocatedOrReserved() { + return (!containersToAllocate.isEmpty()) || (!containersToReserve + .isEmpty()); + } + + public ContainerAllocationProposal getFirstAllocatedOrReservedContainer() { + ContainerAllocationProposal c = null; + if (!containersToAllocate.isEmpty()) { + c = containersToAllocate.get(0); + } + if (c == null && !containersToReserve.isEmpty()) { + c = containersToReserve.get(0); + } + + return c; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("New " + getClass().getName() + ":" + "\n"); + if (null != containersToAllocate && !containersToAllocate.isEmpty()) { + sb.append("\t ALLOCATED=" + containersToAllocate.toString()); + } + if (null != containersToReserve && !containersToReserve.isEmpty()) { + sb.append("\t RESERVED=" + containersToReserve.toString()); + } + if (null != toReleaseContainers && !toReleaseContainers.isEmpty()) { + sb.append("\t RELEASED=" + toReleaseContainers.toString()); + } + return sb.toString(); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/SchedulerContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/SchedulerContainer.java new file mode 100644 index 00000000000..8b4907b76d1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/SchedulerContainer.java @@ -0,0 +1,80 @@ +/** + * 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.resourcemanager.scheduler.common; + +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; + +/** + * Contexts for a container inside scheduler + */ +public class SchedulerContainer { + private RMContainer rmContainer; + private String nodePartition; + private A schedulerApplicationAttempt; + private N schedulerNode; + private boolean allocated; // Allocated (True) or reserved (False) + + public SchedulerContainer(A app, N node, RMContainer rmContainer, + String nodePartition, boolean allocated) { + this.schedulerApplicationAttempt = app; + this.schedulerNode = node; + this.rmContainer = rmContainer; + this.nodePartition = nodePartition; + this.allocated = allocated; + } + + public String getNodePartition() { + return nodePartition; + } + + public RMContainer getRmContainer() { + return rmContainer; + } + + public A getSchedulerApplicationAttempt() { + return schedulerApplicationAttempt; + } + + public N getSchedulerNode() { + return schedulerNode; + } + + public boolean isAllocated() { + return allocated; + } + + public SchedulerRequestKey getSchedulerRequestKey() { + if (rmContainer.getState() == RMContainerState.RESERVED) { + return rmContainer.getReservedSchedulerKey(); + } + return rmContainer.getAllocatedSchedulerKey(); + } + + @Override + public String toString() { + return "(Application=" + schedulerApplicationAttempt + .getApplicationAttemptId() + "; Node=" + schedulerNode.getNodeID() + + "; Resource=" + rmContainer.getAllocatedOrReservedResource() + ")"; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index ebe70d4aaf9..6d9dda84a03 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -18,14 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.locks.ReentrantReadWriteLock; - +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -51,12 +44,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFinishedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager; @@ -70,11 +64,25 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCap import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; +import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; /** * Represents an application attempt from the viewpoint of the FIFO or Capacity @@ -101,6 +109,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { */ private String appSkipNodeDiagnostics; + private Map toBeRemovedIncRequests = + new ConcurrentHashMap<>(); + public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, String user, Queue queue, ActiveUsersManager activeUsersManager, RMContext rmContext) { @@ -193,11 +204,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { } } - public RMContainer allocate(NodeType type, FiCaSchedulerNode node, + public RMContainer allocate(FiCaSchedulerNode node, SchedulerRequestKey schedulerKey, ResourceRequest request, Container container) { try { - writeLock.lock(); + readLock.lock(); if (isStopped) { return null; @@ -216,41 +227,408 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { request.getNodeLabelExpression()); ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName()); + // FIXME, should set when confirmed updateAMContainerDiagnostics(AMState.ASSIGNED, null); - // Add it to allContainers list. - newlyAllocatedContainers.add(rmContainer); - - ContainerId containerId = container.getId(); - liveContainers.put(containerId, rmContainer); - - // Update consumption and track allocations - List resourceRequestList = appSchedulingInfo.allocate( - type, node, schedulerKey, request, container); - - attemptResourceUsage.incUsed(node.getPartition(), - container.getResource()); - - // Update resource requests related to "request" and store in RMContainer - ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList); - - // Inform the container - rmContainer.handle( - new RMContainerEvent(containerId, RMContainerEventType.START)); - - if (LOG.isDebugEnabled()) { - LOG.debug("allocate: applicationAttemptId=" + containerId - .getApplicationAttemptId() + " container=" + containerId + " host=" - + container.getNodeId().getHost() + " type=" + type); - } - RMAuditLogger.logSuccess(getUser(), AuditConstants.ALLOC_CONTAINER, - "SchedulerApp", getApplicationId(), containerId, - container.getResource()); - return rmContainer; + } finally { + readLock.unlock(); + } + } + + private boolean rmContainerInFinalState(RMContainer rmContainer) { + if (null == rmContainer) { + return false; + } + + return rmContainer.completed(); + } + + private boolean anyContainerInFinalState( + ResourceCommitRequest request) { + for (SchedulerContainer c : request + .getContainersToRelease()) { + if (rmContainerInFinalState(c.getRmContainer())) { + if (LOG.isDebugEnabled()) { + LOG.debug("To-release container=" + c.getRmContainer() + + " is in final state"); + } + return true; + } + } + + for (ContainerAllocationProposal c : request + .getContainersToAllocate()) { + for (SchedulerContainer r : c + .getToRelease()) { + if (rmContainerInFinalState(r.getRmContainer())) { + if (LOG.isDebugEnabled()) { + LOG.debug("To-release container=" + r.getRmContainer() + + ", for to a new allocated container, is in final state"); + } + return true; + } + } + + if (null != c.getAllocateFromReservedContainer()) { + if (rmContainerInFinalState( + c.getAllocateFromReservedContainer().getRmContainer())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Allocate from reserved container" + c + .getAllocateFromReservedContainer().getRmContainer() + + " is in final state"); + } + return true; + } + } + } + + for (ContainerAllocationProposal c : request + .getContainersToReserve()) { + for (SchedulerContainer r : c + .getToRelease()) { + if (rmContainerInFinalState(r.getRmContainer())) { + if (LOG.isDebugEnabled()) { + LOG.debug("To-release container=" + r.getRmContainer() + + ", for a reserved container, is in final state"); + } + return true; + } + } + } + + return false; + } + + private SchedContainerChangeRequest getResourceChangeRequest( + SchedulerContainer schedulerContainer) { + return appSchedulingInfo.getIncreaseRequest( + schedulerContainer.getSchedulerNode().getNodeID(), + schedulerContainer.getSchedulerRequestKey(), + schedulerContainer.getRmContainer().getContainerId()); + } + + private boolean checkIncreaseContainerAllocation( + ContainerAllocationProposal allocation, + SchedulerContainer schedulerContainer) { + // When increase a container + if (schedulerContainer.getRmContainer().getState() + != RMContainerState.RUNNING) { + if (LOG.isDebugEnabled()) { + LOG.debug("Trying to increase a container, but container=" + + schedulerContainer.getRmContainer().getContainerId() + + " is not in running state."); + } + return false; + } + + // Check if increase request is still valid + SchedContainerChangeRequest increaseRequest = getResourceChangeRequest( + schedulerContainer); + + if (null == increaseRequest || !Resources.equals( + increaseRequest.getDeltaCapacity(), + allocation.getAllocatedOrReservedResource())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Increase request has been changed, reject this proposal"); + } + return false; + } + + if (allocation.getAllocateFromReservedContainer() != null) { + // In addition, if allocation is from a reserved container, check + // if the reserved container has enough reserved space + if (!Resources.equals( + allocation.getAllocateFromReservedContainer().getRmContainer() + .getReservedResource(), increaseRequest.getDeltaCapacity())) { + return false; + } + } + + return true; + } + + private boolean commonCheckContainerAllocation( + Resource cluster, + ContainerAllocationProposal allocation, + SchedulerContainer schedulerContainer) { + // Make sure node is not reserved by anyone else + RMContainer reservedContainerOnNode = + schedulerContainer.getSchedulerNode().getReservedContainer(); + if (reservedContainerOnNode != null) { + RMContainer fromReservedContainer = + allocation.getAllocateFromReservedContainer().getRmContainer(); + + if (fromReservedContainer != reservedContainerOnNode) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "Try to allocate from a non-existed reserved container"); + } + return false; + } + } + + // Do we have enough space on this node? + Resource availableResource = Resources.clone( + schedulerContainer.getSchedulerNode().getUnallocatedResource()); + + // If we have any to-release container in non-reserved state, they are + // from lazy-preemption, add their consumption to available resource + // of this node + if (allocation.getToRelease() != null && !allocation.getToRelease() + .isEmpty()) { + for (SchedulerContainer + releaseContainer : allocation.getToRelease()) { + // Only consider non-reserved container (reserved container will + // not affect available resource of node) on the same node + if (releaseContainer.getRmContainer().getState() + != RMContainerState.RESERVED + && releaseContainer.getSchedulerNode() == schedulerContainer + .getSchedulerNode()) { + Resources.addTo(availableResource, + releaseContainer.getRmContainer().getAllocatedResource()); + } + } + } + if (!Resources.fitsIn(rc, cluster, + allocation.getAllocatedOrReservedResource(), + availableResource)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Node doesn't have enough available resource, asked=" + + allocation.getAllocatedOrReservedResource() + " available=" + + availableResource); + } + return false; + } + + return true; + } + + public boolean accept(Resource cluster, + ResourceCommitRequest request) { + List resourceRequests = null; + boolean reReservation = false; + + try { + readLock.lock(); + + // First make sure no container in release list in final state + if (anyContainerInFinalState(request)) { + return false; + } + + // TODO, make sure all scheduler nodes are existed + // TODO, make sure all node labels are not changed + + if (request.anythingAllocatedOrReserved()) { + /* + * 1) If this is a newly allocated container, check if the node is reserved + * / not-reserved by any other application + * 2) If this is a newly reserved container, check if the node is reserved or not + */ + // Assume we have only one container allocated or reserved + ContainerAllocationProposal + allocation = request.getFirstAllocatedOrReservedContainer(); + SchedulerContainer + schedulerContainer = allocation.getAllocatedOrReservedContainer(); + + if (schedulerContainer.isAllocated()) { + if (!allocation.isIncreasedAllocation()) { + // When allocate a new container + resourceRequests = + schedulerContainer.getRmContainer().getResourceRequests(); + + // Check pending resource request + if (!appSchedulingInfo.checkAllocation(allocation.getAllocationLocalityType(), + schedulerContainer.getSchedulerNode(), + schedulerContainer.getSchedulerRequestKey())) { + if (LOG.isDebugEnabled()) { + LOG.debug("No pending resource for: nodeType=" + allocation + .getAllocationLocalityType() + ", node=" + schedulerContainer + .getSchedulerNode() + ", requestKey=" + schedulerContainer + .getSchedulerRequestKey() + ", application=" + + getApplicationAttemptId()); + } + + return false; + } + } else { + if (!checkIncreaseContainerAllocation(allocation, + schedulerContainer)) { + return false; + } + } + + // Common part of check container allocation regardless if it is a + // increase container or regular container + commonCheckContainerAllocation(cluster, allocation, + schedulerContainer); + } else { + // Container reserved first time will be NEW, after the container + // accepted & confirmed, it will become RESERVED state + if (schedulerContainer.getRmContainer().getState() + == RMContainerState.RESERVED) { + // Set reReservation == true + reReservation = true; + } else { + // When reserve a resource (state == NEW is for new container, + // state == RUNNING is for increase container). + // Just check if the node is not already reserved by someone + if (schedulerContainer.getSchedulerNode().getReservedContainer() + != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Try to reserve a container, but the node is " + + "already reserved by another container=" + + schedulerContainer.getSchedulerNode() + .getReservedContainer().getContainerId()); + } + return false; + } + } + } + } + } finally { + readLock.unlock(); + } + + // Skip check parent if this is a re-reservation container + boolean accepted = true; + if (!reReservation) { + // Check parent if anything allocated or reserved + if (request.anythingAllocatedOrReserved()) { + accepted = getCSLeafQueue().accept(cluster, request); + } + } + + // When rejected, recover resource requests for this app + if (!accepted && resourceRequests != null) { + recoverResourceRequestsForContainer(resourceRequests); + } + + return accepted; + } + + public void apply(Resource cluster, + ResourceCommitRequest request) { + boolean reReservation = false; + + try { + writeLock.lock(); + + // If we allocated something + if (request.anythingAllocatedOrReserved()) { + ContainerAllocationProposal + allocation = request.getFirstAllocatedOrReservedContainer(); + SchedulerContainer + schedulerContainer = allocation.getAllocatedOrReservedContainer(); + RMContainer rmContainer = schedulerContainer.getRmContainer(); + + reReservation = + (!schedulerContainer.isAllocated()) && (rmContainer.getState() + == RMContainerState.RESERVED); + + // Generate new containerId if it is not an allocation for increasing + // Or re-reservation + if (!allocation.isIncreasedAllocation()) { + if (rmContainer.getContainer().getId() == null) { + rmContainer.setContainerId(BuilderUtils + .newContainerId(getApplicationAttemptId(), + getNewContainerId())); + } + } + ContainerId containerId = rmContainer.getContainerId(); + + if (schedulerContainer.isAllocated()) { + // This allocation is from a reserved container + // Unreserve it first + if (allocation.getAllocateFromReservedContainer() != null) { + RMContainer reservedContainer = + allocation.getAllocateFromReservedContainer().getRmContainer(); + // Handling container allocation + // Did we previously reserve containers at this 'priority'? + unreserve(schedulerContainer.getSchedulerRequestKey(), + schedulerContainer.getSchedulerNode(), reservedContainer); + } + + // Update this application for the allocated container + if (!allocation.isIncreasedAllocation()) { + // Allocate a new container + newlyAllocatedContainers.add(rmContainer); + liveContainers.put(containerId, rmContainer); + + // Deduct pending resource requests + List requests = appSchedulingInfo.allocate( + allocation.getAllocationLocalityType(), schedulerContainer.getSchedulerNode(), + schedulerContainer.getSchedulerRequestKey(), + schedulerContainer.getRmContainer().getContainer()); + ((RMContainerImpl) rmContainer).setResourceRequests(requests); + + attemptResourceUsage.incUsed(schedulerContainer.getNodePartition(), + allocation.getAllocatedOrReservedResource()); + + rmContainer.handle( + new RMContainerEvent(containerId, RMContainerEventType.START)); + + // Inform the node + schedulerContainer.getSchedulerNode().allocateContainer( + rmContainer); + + // update locality statistics, + incNumAllocatedContainers(allocation.getAllocationLocalityType(), + allocation.getRequestLocalityType()); + + if (LOG.isDebugEnabled()) { + LOG.debug("allocate: applicationAttemptId=" + containerId + .getApplicationAttemptId() + " container=" + containerId + + " host=" + rmContainer.getAllocatedNode().getHost() + + " type=" + allocation.getAllocationLocalityType()); + } + RMAuditLogger.logSuccess(getUser(), AuditConstants.ALLOC_CONTAINER, + "SchedulerApp", getApplicationId(), containerId, + allocation.getAllocatedOrReservedResource()); + } else{ + SchedContainerChangeRequest increaseRequest = + getResourceChangeRequest(schedulerContainer); + + // allocate resource for an increase request + // Notify node + schedulerContainer.getSchedulerNode().increaseContainer( + increaseRequest.getContainerId(), + increaseRequest.getDeltaCapacity()); + + // OK, we can allocate this increase request + // Notify application + increaseContainer(increaseRequest); + } + } else { + if (!allocation.isIncreasedAllocation()) { + // If the rmContainer's state is already updated to RESERVED, this is + // a reReservation + reserve(schedulerContainer.getSchedulerRequestKey(), + schedulerContainer.getSchedulerNode(), + schedulerContainer.getRmContainer(), + schedulerContainer.getRmContainer().getContainer(), + reReservation); + } else{ + SchedContainerChangeRequest increaseRequest = + getResourceChangeRequest(schedulerContainer); + + reserveIncreasedContainer( + schedulerContainer.getSchedulerRequestKey(), + schedulerContainer.getSchedulerNode(), + increaseRequest.getRMContainer(), + increaseRequest.getDeltaCapacity()); + } + } + } } finally { writeLock.unlock(); } + + // Don't bother CS leaf queue if it is a re-reservation + if (!reReservation) { + getCSLeafQueue().apply(cluster, request); + } } public boolean unreserve(SchedulerRequestKey schedulerKey, @@ -347,9 +725,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { * of the resources that will be allocated to and preempted from this * application. * - * @param resourceCalculator - * @param clusterResource - * @param minimumAllocation + * @param resourceCalculator resourceCalculator + * @param clusterResource clusterResource + * @param minimumAllocation minimumAllocation * @return an allocation */ public Allocation getAllocation(ResourceCalculator resourceCalculator, @@ -386,45 +764,40 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { public NodeId getNodeIdToUnreserve( SchedulerRequestKey schedulerKey, Resource resourceNeedUnreserve, ResourceCalculator rc, Resource clusterResource) { - try { - writeLock.lock(); - // first go around make this algorithm simple and just grab first - // reservation that has enough resources - Map reservedContainers = this.reservedContainers.get( - schedulerKey); + // first go around make this algorithm simple and just grab first + // reservation that has enough resources + Map reservedContainers = this.reservedContainers.get( + schedulerKey); - if ((reservedContainers != null) && (!reservedContainers.isEmpty())) { - for (Map.Entry entry : reservedContainers - .entrySet()) { - NodeId nodeId = entry.getKey(); - RMContainer reservedContainer = entry.getValue(); - if (reservedContainer.hasIncreaseReservation()) { - // Currently, only regular container allocation supports continuous - // reservation looking, we don't support canceling increase request - // reservation when allocating regular container. - continue; - } - - Resource reservedResource = reservedContainer.getReservedResource(); - - // make sure we unreserve one with at least the same amount of - // resources, otherwise could affect capacity limits - if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve, - reservedResource)) { - if (LOG.isDebugEnabled()) { - LOG.debug( - "unreserving node with reservation size: " + reservedResource - + " in order to allocate container with size: " - + resourceNeedUnreserve); - } - return nodeId; + if ((reservedContainers != null) && (!reservedContainers.isEmpty())) { + for (Map.Entry entry : reservedContainers + .entrySet()) { + NodeId nodeId = entry.getKey(); + RMContainer reservedContainer = entry.getValue(); + if (reservedContainer.hasIncreaseReservation()) { + // Currently, only regular container allocation supports continuous + // reservation looking, we don't support canceling increase request + // reservation when allocating regular container. + continue; + } + + Resource reservedResource = reservedContainer.getReservedResource(); + + // make sure we unreserve one with at least the same amount of + // resources, otherwise could affect capacity limits + if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve, + reservedResource)) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "unreserving node with reservation size: " + reservedResource + + " in order to allocate container with size: " + + resourceNeedUnreserve); } + return nodeId; } } - return null; - } finally { - writeLock.unlock(); } + return null; } public void setHeadroomProvider( @@ -482,10 +855,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { return false; } - public void reserve(SchedulerRequestKey schedulerKey, - FiCaSchedulerNode node, RMContainer rmContainer, Container container) { + public void reserve(SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, + RMContainer rmContainer, Container container, boolean reReservation) { // Update reserved metrics if this is the first reservation - if (rmContainer == null) { + // rmContainer will be moved to reserved in the super.reserve + if (!reReservation) { queue.getMetrics().reserveResource( getUser(), container.getResource()); } @@ -501,35 +875,39 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { public RMContainer findNodeToUnreserve(Resource clusterResource, FiCaSchedulerNode node, SchedulerRequestKey schedulerKey, Resource minimumUnreservedResource) { - // need to unreserve some other container first - NodeId idToUnreserve = - getNodeIdToUnreserve(schedulerKey, minimumUnreservedResource, - rc, clusterResource); - if (idToUnreserve == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("checked to see if could unreserve for app but nothing " - + "reserved that matches for this app"); + try { + readLock.lock(); + // need to unreserve some other container first + NodeId idToUnreserve = getNodeIdToUnreserve(schedulerKey, + minimumUnreservedResource, rc, clusterResource); + if (idToUnreserve == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("checked to see if could unreserve for app but nothing " + + "reserved that matches for this app"); + } + return null; + } + FiCaSchedulerNode nodeToUnreserve = + ((CapacityScheduler) scheduler).getNode(idToUnreserve); + if (nodeToUnreserve == null) { + LOG.error("node to unreserve doesn't exist, nodeid: " + idToUnreserve); + return null; + } + if (LOG.isDebugEnabled()) { + LOG.debug("unreserving for app: " + getApplicationId() + " on nodeId: " + + idToUnreserve + + " in order to replace reserved application and place it on node: " + + node.getNodeID() + " needing: " + minimumUnreservedResource); } - return null; - } - FiCaSchedulerNode nodeToUnreserve = - ((CapacityScheduler) scheduler).getNode(idToUnreserve); - if (nodeToUnreserve == null) { - LOG.error("node to unreserve doesn't exist, nodeid: " + idToUnreserve); - return null; - } - if (LOG.isDebugEnabled()) { - LOG.debug("unreserving for app: " + getApplicationId() - + " on nodeId: " + idToUnreserve - + " in order to replace reserved application and place it on node: " - + node.getNodeID() + " needing: " + minimumUnreservedResource); - } - // headroom - Resources.addTo(getHeadroom(), nodeToUnreserve - .getReservedContainer().getReservedResource()); + // headroom + Resources.addTo(getHeadroom(), + nodeToUnreserve.getReservedContainer().getReservedResource()); - return nodeToUnreserve.getReservedContainer(); + return nodeToUnreserve.getReservedContainer(); + } finally { + readLock.unlock(); + } } public LeafQueue getCSLeafQueue() { @@ -537,7 +915,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { } public CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, ResourceLimits currentResourceLimits, + PlacementSet ps, ResourceLimits currentResourceLimits, SchedulingMode schedulingMode, RMContainer reservedContainer) { if (LOG.isDebugEnabled()) { LOG.debug("pre-assignContainers for application " @@ -545,13 +923,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { showRequests(); } - try { - writeLock.lock(); - return containerAllocator.assignContainers(clusterResource, node, - schedulingMode, currentResourceLimits, reservedContainer); - } finally { - writeLock.unlock(); - } + return containerAllocator.assignContainers(clusterResource, ps, + schedulingMode, currentResourceLimits, reservedContainer); } public void nodePartitionUpdated(RMContainer rmContainer, String oldPartition, @@ -626,13 +999,18 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { /** * Set the message temporarily if the reason is known for why scheduling did * not happen for a given node, if not message will be over written - * @param message + * @param message Message of app skip diagnostics */ public void updateAppSkipNodeDiagnostics(String message) { this.appSkipNodeDiagnostics = message; } public void updateNodeInfoForAMDiagnostics(FiCaSchedulerNode node) { + // FIXME, update AM diagnostics when global scheduling is enabled + if (null == node) { + return; + } + if (isWaitingForAMContainer()) { StringBuilder diagnosticMessageBldr = new StringBuilder(); if (appSkipNodeDiagnostics != null) { @@ -653,6 +1031,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { } } + @Override + @SuppressWarnings("unchecked") + public SchedulingPlacementSet getSchedulingPlacementSet( + SchedulerRequestKey schedulerRequestKey) { + return super.getSchedulingPlacementSet(schedulerRequestKey); + } + /** * Recalculates the per-app, percent of queue metric, specific to the * Capacity Scheduler. @@ -690,4 +1075,29 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { public ReentrantReadWriteLock.WriteLock getWriteLock() { return this.writeLock; } + + public void addToBeRemovedIncreaseRequest( + SchedContainerChangeRequest request) { + toBeRemovedIncRequests.put(request.getContainerId(), request); + } + + public void removedToBeRemovedIncreaseRequests() { + // Remove invalid in request requests + if (!toBeRemovedIncRequests.isEmpty()) { + try { + writeLock.lock(); + Iterator> iter = + toBeRemovedIncRequests.entrySet().iterator(); + while (iter.hasNext()) { + SchedContainerChangeRequest req = iter.next().getValue(); + appSchedulingInfo.removeIncreaseRequest(req.getNodeId(), + req.getRMContainer().getAllocatedSchedulerKey(), + req.getContainerId()); + iter.remove(); + } + } finally { + writeLock.unlock(); + } + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java new file mode 100644 index 00000000000..d275bfdbf1f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java @@ -0,0 +1,110 @@ +/** + * 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.resourcemanager.scheduler.fifo; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; + +import java.util.List; + +public class FifoAppAttempt extends FiCaSchedulerApp { + private static final Log LOG = LogFactory.getLog(FifoAppAttempt.class); + + FifoAppAttempt(ApplicationAttemptId appAttemptId, String user, + Queue queue, ActiveUsersManager activeUsersManager, + RMContext rmContext) { + super(appAttemptId, user, queue, activeUsersManager, rmContext); + } + + public RMContainer allocate(NodeType type, FiCaSchedulerNode node, + SchedulerRequestKey schedulerKey, ResourceRequest request, + Container container) { + try { + writeLock.lock(); + + if (isStopped) { + return null; + } + + // Required sanity check - AM can call 'allocate' to update resource + // request without locking the scheduler, hence we need to check + if (getTotalRequiredResources(schedulerKey) <= 0) { + return null; + } + + // Create RMContainer + RMContainer rmContainer = new RMContainerImpl(container, + this.getApplicationAttemptId(), node.getNodeID(), + appSchedulingInfo.getUser(), this.rmContext, + request.getNodeLabelExpression()); + ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName()); + + updateAMContainerDiagnostics(AMState.ASSIGNED, null); + + // Add it to allContainers list. + newlyAllocatedContainers.add(rmContainer); + + ContainerId containerId = container.getId(); + liveContainers.put(containerId, rmContainer); + + // Update consumption and track allocations + List resourceRequestList = appSchedulingInfo.allocate( + type, node, schedulerKey, request, container); + + attemptResourceUsage.incUsed(node.getPartition(), + container.getResource()); + + // Update resource requests related to "request" and store in RMContainer + ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList); + + // Inform the container + rmContainer.handle( + new RMContainerEvent(containerId, RMContainerEventType.START)); + + if (LOG.isDebugEnabled()) { + LOG.debug("allocate: applicationAttemptId=" + containerId + .getApplicationAttemptId() + " container=" + containerId + " host=" + + container.getNodeId().getHost() + " type=" + type); + } + RMAuditLogger.logSuccess(getUser(), + RMAuditLogger.AuditConstants.ALLOC_CONTAINER, "SchedulerApp", + getApplicationId(), containerId, container.getResource()); + + return rmContainer; + } finally { + writeLock.unlock(); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index 92acf75ef8b..5ccde192934 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicat import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; @@ -103,7 +102,7 @@ import com.google.common.annotations.VisibleForTesting; @Evolving @SuppressWarnings("unchecked") public class FifoScheduler extends - AbstractYarnScheduler implements + AbstractYarnScheduler implements Configurable { private static final Log LOG = LogFactory.getLog(FifoScheduler.class); @@ -239,7 +238,7 @@ public class FifoScheduler extends validateConf(conf); //Use ConcurrentSkipListMap because applications need to be ordered this.applications = - new ConcurrentSkipListMap>(); + new ConcurrentSkipListMap<>(); this.minimumAllocation = Resources.createResource(conf.getInt( YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, @@ -328,7 +327,7 @@ public class FifoScheduler extends List blacklistAdditions, List blacklistRemovals, List increaseRequests, List decreaseRequests) { - FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId); + FifoAppAttempt application = getApplicationAttempt(applicationAttemptId); if (application == null) { LOG.error("Calling allocate on removed " + "or non existant application " + applicationAttemptId); @@ -384,8 +383,8 @@ public class FifoScheduler extends @VisibleForTesting public synchronized void addApplication(ApplicationId applicationId, String queue, String user, boolean isAppRecovering) { - SchedulerApplication application = - new SchedulerApplication(DEFAULT_QUEUE, user); + SchedulerApplication application = + new SchedulerApplication<>(DEFAULT_QUEUE, user); applications.put(applicationId, application); metrics.submitApp(user); LOG.info("Accepted application " + applicationId + " from user: " + user @@ -405,12 +404,12 @@ public class FifoScheduler extends addApplicationAttempt(ApplicationAttemptId appAttemptId, boolean transferStateFromPreviousAttempt, boolean isAttemptRecovering) { - SchedulerApplication application = + SchedulerApplication application = applications.get(appAttemptId.getApplicationId()); String user = application.getUser(); // TODO: Fix store - FiCaSchedulerApp schedulerApp = - new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE, + FifoAppAttempt schedulerApp = + new FifoAppAttempt(appAttemptId, user, DEFAULT_QUEUE, activeUsersManager, this.rmContext); if (transferStateFromPreviousAttempt) { @@ -436,7 +435,7 @@ public class FifoScheduler extends private synchronized void doneApplication(ApplicationId applicationId, RMAppState finalState) { - SchedulerApplication application = + SchedulerApplication application = applications.get(applicationId); if (application == null){ LOG.warn("Couldn't find application " + applicationId); @@ -454,8 +453,8 @@ public class FifoScheduler extends ApplicationAttemptId applicationAttemptId, RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) throws IOException { - FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId); - SchedulerApplication application = + FifoAppAttempt attempt = getApplicationAttempt(applicationAttemptId); + SchedulerApplication application = applications.get(applicationAttemptId.getApplicationId()); if (application == null || attempt == null) { throw new IOException("Unknown application " + applicationAttemptId + @@ -492,9 +491,9 @@ public class FifoScheduler extends " #applications=" + applications.size()); // Try to assign containers to applications in fifo order - for (Map.Entry> e : applications + for (Map.Entry> e : applications .entrySet()) { - FiCaSchedulerApp application = e.getValue().getCurrentAppAttempt(); + FifoAppAttempt application = e.getValue().getCurrentAppAttempt(); if (application == null) { continue; } @@ -536,9 +535,9 @@ public class FifoScheduler extends // Update the applications' headroom to correctly take into // account the containers assigned in this update. - for (SchedulerApplication application : applications.values()) { - FiCaSchedulerApp attempt = - (FiCaSchedulerApp) application.getCurrentAppAttempt(); + for (SchedulerApplication application : applications.values()) { + FifoAppAttempt attempt = + (FifoAppAttempt) application.getCurrentAppAttempt(); if (attempt == null) { continue; } @@ -546,7 +545,7 @@ public class FifoScheduler extends } } - private int getMaxAllocatableContainers(FiCaSchedulerApp application, + private int getMaxAllocatableContainers(FifoAppAttempt application, SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, NodeType type) { int maxContainers = 0; @@ -585,7 +584,7 @@ public class FifoScheduler extends private int assignContainersOnNode(FiCaSchedulerNode node, - FiCaSchedulerApp application, SchedulerRequestKey schedulerKey + FifoAppAttempt application, SchedulerRequestKey schedulerKey ) { // Data-local int nodeLocalContainers = @@ -612,7 +611,7 @@ public class FifoScheduler extends } private int assignNodeLocalContainers(FiCaSchedulerNode node, - FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) { + FifoAppAttempt application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; ResourceRequest request = application.getResourceRequest(schedulerKey, node.getNodeName()); @@ -638,7 +637,7 @@ public class FifoScheduler extends } private int assignRackLocalContainers(FiCaSchedulerNode node, - FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) { + FifoAppAttempt application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; ResourceRequest request = application.getResourceRequest(schedulerKey, node.getRMNode() @@ -664,7 +663,7 @@ public class FifoScheduler extends } private int assignOffSwitchContainers(FiCaSchedulerNode node, - FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) { + FifoAppAttempt application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; ResourceRequest request = application.getResourceRequest(schedulerKey, ResourceRequest.ANY); @@ -676,7 +675,7 @@ public class FifoScheduler extends return assignedContainers; } - private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application, + private int assignContainer(FiCaSchedulerNode node, FifoAppAttempt application, SchedulerRequestKey schedulerKey, int assignableContainers, ResourceRequest request, NodeType type) { LOG.debug("assignContainers:" + @@ -710,8 +709,8 @@ public class FifoScheduler extends // Allocate! // Inform the application - RMContainer rmContainer = - application.allocate(type, node, schedulerKey, request, container); + RMContainer rmContainer = application.allocate(type, node, schedulerKey, + request, container); // Inform the node node.allocateContainer(rmContainer); @@ -836,7 +835,7 @@ public class FifoScheduler extends // Get the application for the finished container Container container = rmContainer.getContainer(); - FiCaSchedulerApp application = + FifoAppAttempt application = getCurrentAttemptForContainer(container.getId()); ApplicationId appId = container.getId().getApplicationAttemptId().getApplicationId(); @@ -916,7 +915,7 @@ public class FifoScheduler extends @Override public RMContainer getRMContainer(ContainerId containerId) { - FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId); + FifoAppAttempt attempt = getCurrentAttemptForContainer(containerId); return (attempt == null) ? null : attempt.getRMContainer(containerId); } @@ -937,7 +936,7 @@ public class FifoScheduler extends if (queueName.equals(DEFAULT_QUEUE.getQueueName())) { List attempts = new ArrayList(applications.size()); - for (SchedulerApplication app : applications.values()) { + for (SchedulerApplication app : applications.values()) { attempts.add(app.getCurrentAppAttempt().getApplicationAttemptId()); } return attempts; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PlacementSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PlacementSet.java new file mode 100644 index 00000000000..2e6c3cab334 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PlacementSet.java @@ -0,0 +1,65 @@ +/** + * 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.resourcemanager.scheduler.placement; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; + +import java.util.Iterator; +import java.util.Map; + +/** + *

    + * PlacementSet is the central place that decide the order of node to fit + * asks by application. + *

    + * + *

    + * Also, PlacementSet can cache results (for example, ordered list) for + * better performance. + *

    + * + *

    + * PlacementSet can depend on one or more other PlacementSets. + *

    + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface PlacementSet { + /** + * Get all nodes for this PlacementSet + * @return all nodes for this PlacementSet + */ + Map getAllNodes(); + + /** + * Version of the PlacementSet, can help other PlacementSet with dependencies + * deciding if update is required + * @return version + */ + long getVersion(); + + /** + * Partition of the PlacementSet. + * @return node partition + */ + String getPartition(); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PlacementSetUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PlacementSetUtils.java new file mode 100644 index 00000000000..405122bee35 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PlacementSetUtils.java @@ -0,0 +1,36 @@ +/** + * 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.resourcemanager.scheduler.placement; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; + +public class PlacementSetUtils { + /* + * If the {@link PlacementSet} only has one entry, return it. otherwise + * return null + */ + public static N getSingleNode(PlacementSet ps) { + N node = null; + if (1 == ps.getAllNodes().size()) { + node = ps.getAllNodes().values().iterator().next(); + } + + return node; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.java new file mode 100644 index 00000000000..da356f5f548 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.java @@ -0,0 +1,43 @@ +/** + * 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.resourcemanager.scheduler.placement; + +import org.apache.hadoop.yarn.api.records.ResourceRequest; + +/** + * Result of ResourceRequest update + */ +public class ResourceRequestUpdateResult { + private final ResourceRequest lastAnyResourceRequest; + private final ResourceRequest newResourceRequest; + + public ResourceRequestUpdateResult(ResourceRequest lastAnyResourceRequest, + ResourceRequest newResourceRequest) { + this.lastAnyResourceRequest = lastAnyResourceRequest; + this.newResourceRequest = newResourceRequest; + } + + public ResourceRequest getLastAnyResourceRequest() { + return lastAnyResourceRequest; + } + + public ResourceRequest getNewResourceRequest() { + return newResourceRequest; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SchedulingPlacementSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SchedulingPlacementSet.java new file mode 100644 index 00000000000..f87f7647786 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SchedulingPlacementSet.java @@ -0,0 +1,90 @@ +/** + * 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.resourcemanager.scheduler.placement; + +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + *

    + * In addition to {@link PlacementSet}, this also maintains + * pending ResourceRequests: + * - When new ResourceRequest(s) added to scheduler, or, + * - Or new container allocated, scheduler can notify corresponding + * PlacementSet. + *

    + * + *

    + * Different set of resource requests (E.g., resource requests with the + * same schedulerKey) can have one instance of PlacementSet, each PlacementSet + * can have different ways to order nodes depends on requests. + *

    + */ +public interface SchedulingPlacementSet + extends PlacementSet { + /** + * Get iterator of preferred node depends on requirement and/or availability + * @param clusterPlacementSet input cluster PlacementSet + * @return iterator of preferred node + */ + Iterator getPreferredNodeIterator(PlacementSet clusterPlacementSet); + + /** + * Replace existing ResourceRequest by the new requests + * + * @param requests new ResourceRequests + * @param recoverPreemptedRequestForAContainer if we're recovering resource + * requests for preempted container + * @return true if total pending resource changed + */ + ResourceRequestUpdateResult updateResourceRequests( + List requests, + boolean recoverPreemptedRequestForAContainer); + + /** + * Get pending ResourceRequests by given schedulerRequestKey + * @return Map of resourceName to ResourceRequest + */ + Map getResourceRequests(); + + /** + * Get ResourceRequest by given schedulerKey and resourceName + * @param resourceName resourceName + * @param schedulerRequestKey schedulerRequestKey + * @return ResourceRequest + */ + ResourceRequest getResourceRequest(String resourceName, + SchedulerRequestKey schedulerRequestKey); + + /** + * Notify container allocated. + * @param type Type of the allocation + * @param node Which node this container allocated on + * @param request resource request + * @return list of ResourceRequests deducted + */ + List allocate(NodeType type, SchedulerNode node, + ResourceRequest request); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SimplePlacementSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SimplePlacementSet.java new file mode 100644 index 00000000000..48efaa14b27 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SimplePlacementSet.java @@ -0,0 +1,70 @@ +/** + * 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.resourcemanager.scheduler.placement; + +import com.google.common.collect.ImmutableMap; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; + +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; + +/** + * A simple PlacementSet which keeps an unordered map + */ +public class SimplePlacementSet + implements PlacementSet { + + private Map map; + private String partition; + + public SimplePlacementSet(N node) { + if (null != node) { + // Only one node in the initial PlacementSet + this.map = ImmutableMap.of(node.getNodeID(), node); + this.partition = node.getPartition(); + } else { + this.map = Collections.emptyMap(); + this.partition = NodeLabel.DEFAULT_NODE_LABEL_PARTITION; + } + } + + public SimplePlacementSet(Map map, String partition) { + this.map = map; + this.partition = partition; + } + + @Override + public Map getAllNodes() { + return map; + } + + @Override + public long getVersion() { + return 0L; + } + + @Override + public String getPartition() { + return partition; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java index 7bec03a17c0..b7cb1bff693 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java @@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy; import java.util.*; +import java.util.concurrent.ConcurrentSkipListSet; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; @@ -35,7 +37,7 @@ public abstract class AbstractComparatorOrderingPolicy schedulableEntities; + protected ConcurrentSkipListSet schedulableEntities; protected Comparator comparator; protected Map entitiesToReorder = new HashMap(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FairOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FairOrderingPolicy.java index 3cfcd7a6f1d..3371df803f0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FairOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FairOrderingPolicy.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy; import java.util.*; +import java.util.concurrent.ConcurrentSkipListSet; import com.google.common.annotations.VisibleForTesting; @@ -61,7 +62,7 @@ public class FairOrderingPolicy extends AbstractCom comparators ); this.comparator = fairComparator; - this.schedulableEntities = new TreeSet(comparator); + this.schedulableEntities = new ConcurrentSkipListSet(comparator); } private double getMagnitude(SchedulableEntity r) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicy.java index 10f8eebbf65..2d066bb4325 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicy.java @@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy; import java.util.*; +import java.util.concurrent.ConcurrentSkipListSet; + import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; /** @@ -32,7 +34,7 @@ public class FifoOrderingPolicy extends AbstractCom comparators.add(new PriorityComparator()); comparators.add(new FifoComparator()); this.comparator = new CompoundComparator(comparators); - this.schedulableEntities = new TreeSet(comparator); + this.schedulableEntities = new ConcurrentSkipListSet(comparator); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicyForPendingApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicyForPendingApps.java index 08912893942..6ced9e2dcb7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicyForPendingApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoOrderingPolicyForPendingApps.java @@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy; import java.util.*; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import java.util.concurrent.ConcurrentSkipListSet; /** * This ordering policy is used for pending applications only. @@ -46,7 +47,7 @@ public class FifoOrderingPolicyForPendingApps comparators.add(new PriorityComparator()); comparators.add(new FifoComparator()); this.comparator = new CompoundComparator(comparators); - this.schedulableEntities = new TreeSet(comparator); + this.schedulableEntities = new ConcurrentSkipListSet(comparator); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java index 58bb7215c75..38616242c0b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.security.PrivilegedAction; import java.util.Arrays; import java.util.Collection; +import java.util.EnumSet; import java.util.List; import java.util.Map; @@ -167,6 +168,28 @@ public class MockRM extends ResourceManager { } } + private void waitForState(ApplicationId appId, EnumSet finalStates) + throws InterruptedException { + RMApp app = getRMContext().getRMApps().get(appId); + Assert.assertNotNull("app shouldn't be null", app); + final int timeoutMsecs = 80 * SECOND; + int timeWaiting = 0; + while (!finalStates.contains(app.getState())) { + if (timeWaiting >= timeoutMsecs) { + break; + } + + LOG.info("App : " + appId + " State is : " + app.getState() + + " Waiting for state : " + finalStates); + Thread.sleep(WAIT_MS_PER_LOOP); + timeWaiting += WAIT_MS_PER_LOOP; + } + + LOG.info("App State is : " + app.getState()); + Assert.assertTrue("App State is not correct (timeout).", + finalStates.contains(app.getState())); + } + /** * Wait until an application has reached a specified state. * The timeout is 80 seconds. @@ -254,7 +277,7 @@ public class MockRM extends ResourceManager { RMAppAttemptState finalState, int timeoutMsecs) throws InterruptedException { int timeWaiting = 0; - while (!finalState.equals(attempt.getAppAttemptState())) { + while (finalState != attempt.getAppAttemptState()) { if (timeWaiting >= timeoutMsecs) { break; } @@ -267,7 +290,7 @@ public class MockRM extends ResourceManager { LOG.info("Attempt State is : " + attempt.getAppAttemptState()); Assert.assertEquals("Attempt state is not correct (timeout).", finalState, - attempt.getState()); + attempt.getState()); } public void waitForContainerToComplete(RMAppAttempt attempt, @@ -966,6 +989,26 @@ public class MockRM extends ResourceManager { rm.getResourceScheduler()).getApplicationAttempt(attemptId)); } + public static MockAM launchAMWhenAsyncSchedulingEnabled(RMApp app, MockRM rm) + throws Exception { + int i = 0; + while (app.getCurrentAppAttempt() == null) { + if (i < 100) { + i++; + } + Thread.sleep(50); + } + + RMAppAttempt attempt = app.getCurrentAppAttempt(); + + rm.waitForState(attempt.getAppAttemptId(), + RMAppAttemptState.ALLOCATED); + MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId()); + rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED); + + return am; + } + /** * NOTE: nm.nodeHeartbeat is explicitly invoked, * don't invoke it before calling launchAM diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java index 884e2368f42..e48d9d2c9d4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java @@ -1091,7 +1091,7 @@ public class TestClientRMService { rmContext.getScheduler().getSchedulerAppInfo(attemptId) .getLiveContainers()).thenReturn(rmContainers); ContainerStatus cs = mock(ContainerStatus.class); - when(containerimpl.getFinishedStatus()).thenReturn(cs); + when(containerimpl.completed()).thenReturn(false); when(containerimpl.getDiagnosticsInfo()).thenReturn("N/A"); when(containerimpl.getContainerExitStatus()).thenReturn(0); when(containerimpl.getContainerState()).thenReturn(ContainerState.COMPLETE); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java index 56d38fb9452..83a354de5a2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java @@ -238,8 +238,10 @@ public class TestSchedulerHealth { SchedulerHealth sh = ((CapacityScheduler) resourceManager.getResourceScheduler()) .getSchedulerHealth(); - Assert.assertEquals(2, sh.getAllocationCount().longValue()); - Assert.assertEquals(Resource.newInstance(3 * 1024, 2), + // Now SchedulerHealth records last container allocated, aggregated + // allocation account will not be changed + Assert.assertEquals(1, sh.getAllocationCount().longValue()); + Assert.assertEquals(Resource.newInstance(1 * 1024, 1), sh.getResourcesAllocated()); Assert.assertEquals(2, sh.getAggregateAllocationCount().longValue()); Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails() diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java index 865449f41ce..0aeedce98aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java @@ -134,6 +134,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSc import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; @@ -3453,7 +3454,7 @@ public class TestCapacityScheduler { scheduler.handle(new NodeRemovedSchedulerEvent( rm.getRMContext().getRMNodes().get(nm2.getNodeId()))); // schedulerNode is removed, try allocate a container - scheduler.allocateContainersToNode(node); + scheduler.allocateContainersToNode(new SimplePlacementSet<>(node), true); AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent( @@ -3699,4 +3700,57 @@ public class TestCapacityScheduler { cs.handle(addAttemptEvent1); return appAttemptId1; } + + @Test + public void testAppAttemptLocalityStatistics() throws Exception { + Configuration conf = + TestUtils.getConfigurationWithMultipleQueues(new Configuration(false)); + conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); + + final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager(); + mgr.init(conf); + + MockRM rm = new MockRM(conf) { + protected RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm.start(); + MockNM nm1 = + new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService()); + nm1.registerNode(); + + // Launch app1 in queue=a1 + RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a"); + + // Got one offswitch request and offswitch allocation + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1); + + // am1 asks for 1 GB resource on h1/default-rack/offswitch + am1.allocate(Arrays.asList(ResourceRequest + .newInstance(Priority.newInstance(1), "*", + Resources.createResource(1 * GB), 2), ResourceRequest + .newInstance(Priority.newInstance(1), "/default-rack", + Resources.createResource(1 * GB), 2), ResourceRequest + .newInstance(Priority.newInstance(1), "h1", + Resources.createResource(1 * GB), 1)), null); + + CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler(); + + // Got one nodelocal request and nodelocal allocation + cs.nodeUpdate(rm.getRMContext().getRMNodes().get(nm1.getNodeId())); + + // Got one nodelocal request and racklocal allocation + cs.nodeUpdate(rm.getRMContext().getRMNodes().get(nm1.getNodeId())); + + RMAppAttemptMetrics attemptMetrics = rm.getRMContext().getRMApps().get( + app1.getApplicationId()).getCurrentAppAttempt() + .getRMAppAttemptMetrics(); + + // We should get one node-local allocation, one rack-local allocation + // And one off-switch allocation + Assert.assertArrayEquals(new int[][] { { 1, 0, 0 }, { 0, 1, 0 }, { 0, 0, 1 } }, + attemptMetrics.getLocalityStatistics()); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java new file mode 100644 index 00000000000..9854a15e7a8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java @@ -0,0 +1,143 @@ +/** + * 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.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.MockAM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class TestCapacitySchedulerAsyncScheduling { + private final int GB = 1024; + + private YarnConfiguration conf; + + RMNodeLabelsManager mgr; + + @Before + public void setUp() throws Exception { + conf = new YarnConfiguration(); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + conf.setBoolean( + CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, true); + mgr = new NullRMNodeLabelsManager(); + mgr.init(conf); + } + + @Test(timeout = 300000) + public void testSingleThreadAsyncContainerAllocation() throws Exception { + testAsyncContainerAllocation(1); + } + + @Test(timeout = 300000) + public void testTwoThreadsAsyncContainerAllocation() throws Exception { + testAsyncContainerAllocation(2); + } + + @Test(timeout = 300000) + public void testThreeThreadsAsyncContainerAllocation() throws Exception { + testAsyncContainerAllocation(3); + } + + public void testAsyncContainerAllocation(int numThreads) throws Exception { + conf.setInt( + CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD, + numThreads); + conf.setInt(CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_PREFIX + + ".scheduling-interval-ms", 100); + + final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager(); + mgr.init(conf); + + // inject node label manager + MockRM rm = new MockRM(TestUtils.getConfigurationWithMultipleQueues(conf)) { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm.getRMContext().setNodeLabelManager(mgr); + rm.start(); + + List nms = new ArrayList<>(); + // Add 10 nodes to the cluster, in the cluster we have 200 GB resource + for (int i = 0; i < 10; i++) { + nms.add(rm.registerNode("h-" + i + ":1234", 20 * GB)); + } + + List ams = new ArrayList<>(); + // Add 3 applications to the cluster, one app in one queue + // the i-th app ask (20 * i) containers. So in total we will have + // 123G container allocated + int totalAsked = 3 * GB; // 3 AMs + + for (int i = 0; i < 3; i++) { + RMApp rmApp = rm.submitApp(1024, "app", "user", null, false, + Character.toString((char) (i % 34 + 97)), 1, null, null, false); + MockAM am = MockRM.launchAMWhenAsyncSchedulingEnabled(rmApp, rm); + am.registerAppAttempt(); + ams.add(am); + } + + for (int i = 0; i < 3; i++) { + ams.get(i).allocate("*", 1024, 20 * (i + 1), new ArrayList<>()); + totalAsked += 20 * (i + 1) * GB; + } + + // Wait for at most 15000 ms + int waitTime = 15000; // ms + while (waitTime > 0) { + if (rm.getResourceScheduler().getRootQueueMetrics().getAllocatedMB() + == totalAsked) { + break; + } + Thread.sleep(50); + waitTime -= 50; + } + + Assert.assertEquals( + rm.getResourceScheduler().getRootQueueMetrics().getAllocatedMB(), + totalAsked); + + // Wait for another 2 sec to make sure we will not allocate more than + // required + waitTime = 2000; // ms + while (waitTime > 0) { + Assert.assertEquals( + rm.getResourceScheduler().getRootQueueMetrics().getAllocatedMB(), + totalAsked); + waitTime -= 50; + Thread.sleep(50); + } + + rm.close(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java index 7f4fc2cda59..40e5d2a4a36 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java @@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; @@ -110,15 +111,16 @@ public class TestChildQueueOrder { return application; } - private void stubQueueAllocation(final CSQueue queue, - final Resource clusterResource, final FiCaSchedulerNode node, + private void stubQueueAllocation(final CSQueue queue, + final Resource clusterResource, final FiCaSchedulerNode node, final int allocation) { - stubQueueAllocation(queue, clusterResource, node, allocation, + stubQueueAllocation(queue, clusterResource, node, allocation, NodeType.NODE_LOCAL); } - private void stubQueueAllocation(final CSQueue queue, - final Resource clusterResource, final FiCaSchedulerNode node, + @SuppressWarnings("unchecked") + private void stubQueueAllocation(final CSQueue queue, + final Resource clusterResource, final FiCaSchedulerNode node, final int allocation, final NodeType type) { // Simulate the queue allocation @@ -145,7 +147,7 @@ public class TestChildQueueOrder { if (allocation > 0) { doReturn(new CSAssignment(Resources.none(), type)). when(queue) - .assignContainers(eq(clusterResource), eq(node), + .assignContainers(eq(clusterResource), any(PlacementSet.class), any(ResourceLimits.class), any(SchedulingMode.class)); // Mock the node's resource availability @@ -157,7 +159,7 @@ public class TestChildQueueOrder { return new CSAssignment(allocatedResource, type); } }). - when(queue).assignContainers(eq(clusterResource), eq(node), + when(queue).assignContainers(eq(clusterResource), any(PlacementSet.class), any(ResourceLimits.class), any(SchedulingMode.class)); doNothing().when(node).releaseContainer(any(Container.class)); } @@ -214,6 +216,7 @@ public class TestChildQueueOrder { } @Test + @SuppressWarnings("unchecked") public void testSortedQueues() throws Exception { // Setup queue configs setupSortedQueues(csConf); @@ -418,10 +421,10 @@ public class TestChildQueueOrder { clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); InOrder allocationOrder = inOrder(d,b); allocationOrder.verify(d).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), any(ResourceLimits.class), + any(PlacementSet.class), any(ResourceLimits.class), any(SchedulingMode.class)); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), any(ResourceLimits.class), + any(PlacementSet.class), any(ResourceLimits.class), any(SchedulingMode.class)); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java index e2b49526756..555e0fd7c23 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java @@ -81,7 +81,7 @@ public class TestContainerAllocation { mgr.init(conf); } - @Test(timeout = 3000000) + @Test(timeout = 60000) public void testExcessReservationThanNodeManagerCapacity() throws Exception { @SuppressWarnings("resource") MockRM rm = new MockRM(conf); @@ -598,4 +598,47 @@ public class TestContainerAllocation { rm1.close(); } + + @Test(timeout = 60000) + public void testAssignMultipleOffswitchContainers() throws Exception { + MockRM rm1 = new MockRM(); + + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 80 * GB); + + // launch an app to queue, AM container should be launched in nm1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + am1.allocate("*", 1 * GB, 5, new ArrayList()); + + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + + // Do node heartbeats once + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + FiCaSchedulerApp schedulerApp1 = + cs.getApplicationAttempt(am1.getApplicationAttemptId()); + + // App1 will get one container allocated (plus AM container + Assert.assertEquals(2, schedulerApp1.getLiveContainers().size()); + + // Set assign multiple off-switch containers to 3 + CapacitySchedulerConfiguration newCSConf = new CapacitySchedulerConfiguration(); + newCSConf.setInt( + CapacitySchedulerConfiguration.OFFSWITCH_PER_HEARTBEAT_LIMIT, 3); + + cs.reinitialize(newCSConf, rm1.getRMContext()); + + // Do node heartbeats once + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + // App1 will get 3 new container allocated (plus 2 previously allocated + // container) + Assert.assertEquals(5, schedulerApp1.getLiveContainers().size()); + + rm1.close(); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java index 26146301d4d..0696f572ea1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java @@ -59,9 +59,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica .FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet; import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; public class TestContainerResizing { @@ -97,13 +100,14 @@ public class TestContainerResizing { } @Override - public synchronized void allocateContainersToNode(FiCaSchedulerNode node) { + public CSAssignment allocateContainersToNode( + PlacementSet ps, boolean withNodeHeartbeat) { try { Thread.sleep(1000); } catch(InterruptedException e) { LOG.debug("Thread interrupted."); } - super.allocateContainersToNode(node); + return super.allocateContainersToNode(ps, withNodeHeartbeat); } } @@ -452,7 +456,7 @@ public class TestContainerResizing { ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); sentRMContainerLaunched(rm1, containerId1); - // am1 asks to change its AM container from 1GB to 3GB + // am1 asks to change its AM container from 1GB to 7GB am1.sendContainerResizingRequest(Arrays.asList( UpdateContainerRequest .newInstance(0, containerId1, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java index 51b567bcce4..8694efb0a5d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java @@ -41,6 +41,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CyclicBarrier; +import com.google.common.collect.ImmutableMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.security.UserGroupInformation; @@ -49,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueInfo; @@ -78,6 +80,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestK import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue.User; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; @@ -196,6 +199,7 @@ public class TestLeafQueue { cs.setRMContext(spyRMContext); cs.init(csConf); + cs.setResourceCalculator(rC); cs.start(); when(spyRMContext.getScheduler()).thenReturn(cs); @@ -268,6 +272,12 @@ public class TestLeafQueue { any(Resource.class), any(FiCaSchedulerApp.class), any(FiCaSchedulerNode.class), any(RMContainer.class), any(ContainerStatus.class), any(RMContainerEventType.class), any(CSQueue.class), anyBoolean()); + + // Stub out parent queue's accept and apply. + doReturn(true).when(parent).accept(any(Resource.class), + any(ResourceCommitRequest.class)); + doNothing().when(parent).apply(any(Resource.class), + any(ResourceCommitRequest.class)); return queue; } @@ -339,6 +349,12 @@ public class TestLeafQueue { FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0); + final int numNodes = 1; Resource clusterResource = Resources.createResource(numNodes * (8*GB), numNodes * 16); @@ -353,8 +369,10 @@ public class TestLeafQueue { // Start testing... // Only 1 container - a.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals( (int)(node_0.getTotalResource().getMemorySize() * a.getCapacity()) - (1*GB), a.getMetrics().getAvailableMB()); @@ -526,6 +544,12 @@ public class TestLeafQueue { FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0); + final int numNodes = 1; Resource clusterResource = Resources.createResource(numNodes * (8*GB), numNodes * 16); @@ -544,8 +568,10 @@ public class TestLeafQueue { // Start testing... // Only 1 container - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(1*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -555,8 +581,10 @@ public class TestLeafQueue { // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // you can get one container more than user-limit - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(2*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -564,8 +592,10 @@ public class TestLeafQueue { assertEquals(2*GB, a.getMetrics().getAllocatedMB()); // Can't allocate 3rd due to user-limit - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(2*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -574,8 +604,10 @@ public class TestLeafQueue { // Bump up user-limit-factor, now allocate should work a.setUserLimitFactor(10); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(3*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -583,8 +615,10 @@ public class TestLeafQueue { assertEquals(3*GB, a.getMetrics().getAllocatedMB()); // One more should work, for app_1, due to user-limit-factor - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(4*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -594,8 +628,10 @@ public class TestLeafQueue { // Test max-capacity // Now - no more allocs since we are at max-cap a.setMaxCapacity(0.5f); - a.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(4*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -687,6 +723,13 @@ public class TestLeafQueue { assign.getResource().getMemorySize() > 0); } + private void applyCSAssignment(Resource clusterResource, CSAssignment assign, + LeafQueue q, final Map nodes, + final Map apps) + throws IOException { + TestUtils.applyResourceCommitRequest(clusterResource, assign, nodes, apps); + } + @Test public void testDRFUserLimits() throws Exception { setUpWithDominantResourceCalculator(); @@ -723,6 +766,12 @@ public class TestLeafQueue { FiCaSchedulerNode node1 = TestUtils.getMockNode(host1, DEFAULT_RACK, 0, 8 * GB, 100); + Map nodes = ImmutableMap.of(node0.getNodeID(), + node0, node1.getNodeID(), node1); + Map apps = ImmutableMap.of( + app0.getApplicationAttemptId(), app0, app2.getApplicationAttemptId(), + app2); + int numNodes = 2; Resource clusterResource = Resources.createResource(numNodes * (8 * GB), numNodes * 100); @@ -758,12 +807,14 @@ public class TestLeafQueue { b.assignContainers(clusterResource, node0, new ResourceLimits( clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); LOG.info(assign.toString()); + applyCSAssignment(clusterResource, assign, b, nodes, apps); } while (assign.getResource().getMemorySize() > 0 && assign.getAssignmentInformation().getNumReservations() == 0); do { assign = b.assignContainers(clusterResource, node1, new ResourceLimits( clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assign, b, nodes, apps); } while (assign.getResource().getMemorySize() > 0 && assign.getAssignmentInformation().getNumReservations() == 0); //LOG.info("user_0: " + queueUser0.getUsed()); @@ -847,6 +898,12 @@ public class TestLeafQueue { TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 2, true, priority, recordFactory))); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1); + /** * Start testing... */ @@ -859,8 +916,10 @@ public class TestLeafQueue { assertEquals(2, a.getActiveUsersManager().getNumActiveUsers()); // 1 container to user_0 - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(3*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -868,8 +927,10 @@ public class TestLeafQueue { // Allocate one container to app_1. Even if app_0 // submit earlier, it cannot get this container assigned since user_0 // exceeded user-limit already. - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(4*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -877,8 +938,10 @@ public class TestLeafQueue { // Allocate one container to app_0, before allocating this container, // user-limit = ceil((4 + 1) / 2) = 3G. app_0's used resource (3G) <= // user-limit. - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(7*GB, a.getUsedResources().getMemorySize()); assertEquals(6*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -890,7 +953,7 @@ public class TestLeafQueue { } @Test - public void testComputeUserLimitAndSetHeadroom(){ + public void testComputeUserLimitAndSetHeadroom() throws IOException { LeafQueue qb = stubLeafQueue((LeafQueue)queues.get(B)); qb.setMaxCapacity(1.0f); // Users @@ -903,6 +966,9 @@ public class TestLeafQueue { String host_1 = "127.0.0.2"; FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1); + final int numNodes = 2; Resource clusterResource = Resources.createResource(numNodes * (8*GB), 1); when(csContext.getNumClusterNodes()).thenReturn(numNodes); @@ -925,6 +991,8 @@ public class TestLeafQueue { FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, qb, qb.getActiveUsersManager(), spyRMContext); + Map apps = new HashMap<>(); + apps.put(app_0.getApplicationAttemptId(), app_0); qb.submitApplicationAttempt(app_0, user_0); Priority u0Priority = TestUtils.createMockPriority(1); SchedulerRequestKey u0SchedKey = toSchedulerKey(u0Priority); @@ -935,8 +1003,10 @@ public class TestLeafQueue { assertEquals("There should only be 1 active user!", 1, qb.getActiveUsersManager().getNumActiveUsers()); //get headroom - qb.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + qb.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), qb, nodes, apps); qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); @@ -949,14 +1019,17 @@ public class TestLeafQueue { FiCaSchedulerApp app_2 = new FiCaSchedulerApp(appAttemptId_2, user_1, qb, qb.getActiveUsersManager(), spyRMContext); + apps.put(app_2.getApplicationAttemptId(), app_2); Priority u1Priority = TestUtils.createMockPriority(2); SchedulerRequestKey u1SchedKey = toSchedulerKey(u1Priority); app_2.updateResourceRequests(Collections.singletonList( TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true, u1Priority, recordFactory))); qb.submitApplicationAttempt(app_2, user_1); - qb.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + qb.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), qb, nodes, apps); qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); @@ -984,11 +1057,13 @@ public class TestLeafQueue { FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, qb, qb.getActiveUsersManager(), spyRMContext); + apps.put(app_1.getApplicationAttemptId(), app_1); final ApplicationAttemptId appAttemptId_3 = TestUtils.getMockApplicationAttemptId(3, 0); FiCaSchedulerApp app_3 = new FiCaSchedulerApp(appAttemptId_3, user_1, qb, qb.getActiveUsersManager(), spyRMContext); + apps.put(app_3.getApplicationAttemptId(), app_3); app_1.updateResourceRequests(Collections.singletonList( TestUtils.createResourceRequest(ResourceRequest.ANY, 2*GB, 1, true, u0Priority, recordFactory))); @@ -997,10 +1072,14 @@ public class TestLeafQueue { u1Priority, recordFactory))); qb.submitApplicationAttempt(app_1, user_0); qb.submitApplicationAttempt(app_3, user_1); - qb.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - qb.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + qb.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), qb, nodes, apps); + applyCSAssignment(clusterResource, + qb.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), qb, nodes, apps); qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); assertEquals(4*GB, qb.getUsedResources().getMemorySize()); @@ -1013,12 +1092,15 @@ public class TestLeafQueue { FiCaSchedulerApp app_4 = new FiCaSchedulerApp(appAttemptId_4, user_0, qb, qb.getActiveUsersManager(), spyRMContext); + apps.put(app_4.getApplicationAttemptId(), app_4); qb.submitApplicationAttempt(app_4, user_0); app_4.updateResourceRequests(Collections.singletonList( TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true, u0Priority, recordFactory))); - qb.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + qb.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), qb, nodes, apps); qb.computeUserLimitAndSetHeadroom(app_4, clusterResource, "", SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, @@ -1078,6 +1160,12 @@ public class TestLeafQueue { FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 16*GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1, app_2.getApplicationAttemptId(), app_2); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1); + final int numNodes = 2; Resource clusterResource = Resources.createResource(numNodes * (16*GB), 1); when(csContext.getNumClusterNodes()).thenReturn(numNodes); @@ -1088,8 +1176,10 @@ public class TestLeafQueue { TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true, priority, recordFactory))); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(1*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1105,8 +1195,10 @@ public class TestLeafQueue { TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 2, true, priority, recordFactory))); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(2*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1165,6 +1257,12 @@ public class TestLeafQueue { FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); String host_1 = "127.0.0.2"; FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB); + + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1, app_2.getApplicationAttemptId(), app_2); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1); final int numNodes = 2; Resource clusterResource = Resources.createResource(numNodes * (8*GB), 1); @@ -1194,8 +1292,10 @@ public class TestLeafQueue { 1, a.getActiveUsersManager().getNumActiveUsers()); // 1 container to user_0 - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(2*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1206,8 +1306,10 @@ public class TestLeafQueue { // the application is not yet active // Again one to user_0 since he hasn't exceeded user limit yet - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(3*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1223,8 +1325,10 @@ public class TestLeafQueue { // No more to user_0 since he is already over user-limit // and no more containers to queue since it's already at max-cap - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(3*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1237,8 +1341,10 @@ public class TestLeafQueue { TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 0, true, priority, recordFactory))); assertEquals(1, a.getActiveUsersManager().getNumActiveUsers()); - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(0*GB, app_2.getHeadroom().getMemorySize()); // hit queue max-cap } @@ -1283,10 +1389,17 @@ public class TestLeafQueue { new FiCaSchedulerApp(appAttemptId_3, user_2, a, a.getActiveUsersManager(), spyRMContext); a.submitApplicationAttempt(app_3, user_2); + + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1, app_2.getApplicationAttemptId(), app_2, + app_3.getApplicationAttemptId(), app_3); // Setup some nodes String host_0 = "127.0.0.1"; FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0); final int numNodes = 1; Resource clusterResource = @@ -1308,24 +1421,30 @@ public class TestLeafQueue { */ // Only 1 container - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(1*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // you can get one container more than user-limit - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(2*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); // Can't allocate 3rd due to user-limit a.setUserLimit(25); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(2*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1343,8 +1462,10 @@ public class TestLeafQueue { // Now allocations should goto app_2 since // user_0 is at limit inspite of high user-limit-factor a.setUserLimitFactor(10); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(5*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1353,8 +1474,10 @@ public class TestLeafQueue { // Now allocations should goto app_0 since // user_0 is at user-limit not above it - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(6*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1364,8 +1487,10 @@ public class TestLeafQueue { // Test max-capacity // Now - no more allocs since we are at max-cap a.setMaxCapacity(0.5f); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(6*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1376,8 +1501,10 @@ public class TestLeafQueue { // Now, allocations should goto app_3 since it's under user-limit a.setMaxCapacity(1.0f); a.setUserLimitFactor(1); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(7*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1385,8 +1512,10 @@ public class TestLeafQueue { assertEquals(1*GB, app_3.getCurrentConsumption().getMemorySize()); // Now we should assign to app_3 again since user_2 is under user-limit - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(8*GB, a.getUsedResources().getMemorySize()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1466,6 +1595,12 @@ public class TestLeafQueue { // Setup some nodes String host_0 = "127.0.0.1"; FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB); + + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0); final int numNodes = 2; Resource clusterResource = @@ -1485,8 +1620,10 @@ public class TestLeafQueue { // Start testing... // Only 1 container - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(1*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1496,8 +1633,10 @@ public class TestLeafQueue { // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // you can get one container more than user-limit - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(2*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1505,8 +1644,10 @@ public class TestLeafQueue { assertEquals(2*GB, a.getMetrics().getAllocatedMB()); // Now, reservation should kick in for app_1 - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(6*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1522,8 +1663,10 @@ public class TestLeafQueue { ContainerState.COMPLETE, "", ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(5*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1539,8 +1682,10 @@ public class TestLeafQueue { ContainerState.COMPLETE, "", ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(4*GB, a.getUsedResources().getMemorySize()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1587,6 +1732,12 @@ public class TestLeafQueue { when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0); when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1); + + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1); final int numNodes = 3; Resource clusterResource = @@ -1613,23 +1764,29 @@ public class TestLeafQueue { // Start testing... // Only 1 container - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(1*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also // you can get one container more than user-limit - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(2*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); // Now, reservation should kick in for app_1 - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(6*GB, a.getUsedResources().getMemorySize()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1643,8 +1800,10 @@ public class TestLeafQueue { ContainerState.COMPLETE, "", ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(5*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1653,8 +1812,10 @@ public class TestLeafQueue { assertEquals(1, app_1.getReReservations(toSchedulerKey(priority))); // Re-reserve - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(5*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1663,8 +1824,10 @@ public class TestLeafQueue { assertEquals(2, app_1.getReReservations(toSchedulerKey(priority))); // Try to schedule on node_1 now, should *move* the reservation - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(9*GB, a.getUsedResources().getMemorySize()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1681,8 +1844,10 @@ public class TestLeafQueue { ContainerState.COMPLETE, "", ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); assertEquals(4*GB, a.getUsedResources().getMemorySize()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -1735,6 +1900,15 @@ public class TestLeafQueue { String rack_2 = "rack_2"; FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB); + String host_3 = "127.0.0.4"; // on rack_1 + FiCaSchedulerNode node_3 = TestUtils.getMockNode(host_3, rack_1, 0, 8*GB); + + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2, + node_3.getNodeID(), node_3); + final int numNodes = 3; Resource clusterResource = Resources.createResource(numNodes * (8*GB), numNodes * 16); @@ -1767,6 +1941,7 @@ public class TestLeafQueue { // Start with off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(3, app_0.getTotalRequiredResources(schedulerKey)); @@ -1775,6 +1950,7 @@ public class TestLeafQueue { // Another off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(3, app_0.getTotalRequiredResources(schedulerKey)); @@ -1783,6 +1959,7 @@ public class TestLeafQueue { // Another off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(3, app_0.getTotalRequiredResources(schedulerKey)); @@ -1792,6 +1969,7 @@ public class TestLeafQueue { // since missedOpportunities=3 and reqdContainers=3 assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.OFF_SWITCH); // should NOT reset assertEquals(4, app_0.getSchedulingOpportunities(schedulerKey)); @@ -1800,6 +1978,7 @@ public class TestLeafQueue { // NODE_LOCAL - node_0 assignment = a.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.NODE_LOCAL); // should reset assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); @@ -1808,6 +1987,7 @@ public class TestLeafQueue { // NODE_LOCAL - node_1 assignment = a.assignContainers(clusterResource, node_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.NODE_LOCAL); // should reset assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); @@ -1828,9 +2008,6 @@ public class TestLeafQueue { app_0.updateResourceRequests(app_0_requests_0); assertEquals(4, app_0.getTotalRequiredResources(schedulerKey)); - String host_3 = "127.0.0.4"; // on rack_1 - FiCaSchedulerNode node_3 = TestUtils.getMockNode(host_3, rack_1, 0, 8*GB); - // Rack-delay doReturn(true).when(a).getRackLocalityFullReset(); doReturn(1).when(a).getNodeLocalityDelay(); @@ -1838,12 +2015,14 @@ public class TestLeafQueue { // Shouldn't assign RACK_LOCAL yet assignment = a.assignContainers(clusterResource, node_3, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(4, app_0.getTotalRequiredResources(schedulerKey)); // Should assign RACK_LOCAL now assignment = a.assignContainers(clusterResource, node_3, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.RACK_LOCAL); // should reset assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); @@ -1852,6 +2031,7 @@ public class TestLeafQueue { // Shouldn't assign RACK_LOCAL because schedulingOpportunities should have gotten reset. assignment = a.assignContainers(clusterResource, node_3, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(3, app_0.getTotalRequiredResources(schedulerKey)); @@ -1861,6 +2041,7 @@ public class TestLeafQueue { // Should assign RACK_LOCAL now assignment = a.assignContainers(clusterResource, node_3, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.RACK_LOCAL); // should NOT reset assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey)); @@ -1869,6 +2050,7 @@ public class TestLeafQueue { // Another RACK_LOCAL since schedulingOpportunities not reset assignment = a.assignContainers(clusterResource, node_3, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.RACK_LOCAL); // should NOT reset assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey)); @@ -1894,12 +2076,14 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits( clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); assertEquals(i+1, app_0.getSchedulingOpportunities(schedulerKey)); } // delay should be capped at numNodes so next one should allocate assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.OFF_SWITCH); assertEquals(numNodes+1, app_0.getSchedulingOpportunities(schedulerKey)); } @@ -1933,6 +2117,11 @@ public class TestLeafQueue { String rack_2 = "rack_2"; FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2); + final int numNodes = 3; Resource clusterResource = Resources.createResource(numNodes * (8*GB), 1); @@ -1981,6 +2170,7 @@ public class TestLeafQueue { // thus, no P2 either! CSAssignment assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey1)); assertEquals(2, app_0.getTotalRequiredResources(schedulerKey1)); @@ -1991,6 +2181,7 @@ public class TestLeafQueue { // thus, no P2 either! assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey1)); assertEquals(2, app_0.getTotalRequiredResources(schedulerKey1)); @@ -2000,6 +2191,7 @@ public class TestLeafQueue { // Another off-switch, shouldn't allocate OFF_SWITCH P1 assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.OFF_SWITCH); assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey1)); assertEquals(1, app_0.getTotalRequiredResources(schedulerKey1)); @@ -2009,6 +2201,7 @@ public class TestLeafQueue { // Now, DATA_LOCAL for P1 assignment = a.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.NODE_LOCAL); assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1)); assertEquals(0, app_0.getTotalRequiredResources(schedulerKey1)); @@ -2018,6 +2211,7 @@ public class TestLeafQueue { // Now, OFF_SWITCH for P2 assignment = a.assignContainers(clusterResource, node_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.OFF_SWITCH); assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1)); assertEquals(0, app_0.getTotalRequiredResources(schedulerKey1)); @@ -2054,6 +2248,12 @@ public class TestLeafQueue { String host_1_0 = "127.0.0.3"; String rack_1 = "rack_1"; FiCaSchedulerNode node_1_0 = TestUtils.getMockNode(host_1_0, rack_1, 0, 8*GB); + + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0); + Map nodes = ImmutableMap.of(node_0_0.getNodeID(), + node_0_0, node_0_1.getNodeID(), node_0_1, node_1_0.getNodeID(), + node_1_0); final int numNodes = 3; Resource clusterResource = Resources.createResource( @@ -2093,6 +2293,7 @@ public class TestLeafQueue { // NODE_LOCAL - node_0_1 CSAssignment assignment = a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.NODE_LOCAL); assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); // should reset @@ -2102,6 +2303,7 @@ public class TestLeafQueue { // required(ANY) == 0 assignment = a.assignContainers(clusterResource, node_1_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); // Still zero // since #req=0 @@ -2119,6 +2321,7 @@ public class TestLeafQueue { // required(rack_1) == 0 assignment = a.assignContainers(clusterResource, node_0_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(1, app_0.getTotalRequiredResources(schedulerKey)); @@ -2126,6 +2329,7 @@ public class TestLeafQueue { // NODE_LOCAL - node_1 assignment = a.assignContainers(clusterResource, node_1_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.NODE_LOCAL); // should reset assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); @@ -2319,18 +2523,25 @@ public class TestLeafQueue { mock(ActiveUsersManager.class), spyRMContext); a.submitApplicationAttempt(app_1, user_0); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + // Setup some nodes and racks String host_0_0 = "127.0.0.1"; String rack_0 = "rack_0"; String host_0_1 = "127.0.0.2"; FiCaSchedulerNode node_0_1 = TestUtils.getMockNode(host_0_1, rack_0, 0, 8*GB); - - + String host_1_0 = "127.0.0.3"; String rack_1 = "rack_1"; FiCaSchedulerNode node_1_0 = TestUtils.getMockNode(host_1_0, rack_1, 0, 8*GB); String host_1_1 = "127.0.0.4"; FiCaSchedulerNode node_1_1 = TestUtils.getMockNode(host_1_1, rack_1, 0, 8*GB); + + Map nodes = ImmutableMap.of(node_0_1.getNodeID(), + node_0_1, node_1_0.getNodeID(), node_1_0, node_1_1.getNodeID(), + node_1_1); final int numNodes = 4; Resource clusterResource = Resources.createResource( @@ -2380,6 +2591,7 @@ public class TestLeafQueue { CSAssignment assignment = a.assignContainers(clusterResource, node_0_1, new ResourceLimits( clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); // should be 0 assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); @@ -2403,6 +2615,7 @@ public class TestLeafQueue { // Shouldn't allocate since RR(rack_1) = relax: false assignment = a.assignContainers(clusterResource, node_1_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); // should be 0 assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); @@ -2434,6 +2647,7 @@ public class TestLeafQueue { // Shouldn't allocate since node_1_1 is blacklisted assignment = a.assignContainers(clusterResource, node_1_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); // should be 0 assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); @@ -2461,8 +2675,10 @@ public class TestLeafQueue { // node_1_1 // Shouldn't allocate since rack_1 is blacklisted - assignment = a.assignContainers(clusterResource, node_1_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + assignment = a.assignContainers(clusterResource, node_1_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); // should be 0 assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); @@ -2490,6 +2706,7 @@ public class TestLeafQueue { // Now, should allocate since RR(rack_1) = relax: true assignment = a.assignContainers(clusterResource, node_1_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyNoContainerAllocated(assignment); assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(1, app_0.getTotalRequiredResources(schedulerKey)); @@ -2520,6 +2737,7 @@ public class TestLeafQueue { assignment = a.assignContainers(clusterResource, node_1_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.NODE_LOCAL); assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(0, app_0.getTotalRequiredResources(schedulerKey)); @@ -2590,6 +2808,12 @@ public class TestLeafQueue { FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8 * GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0); + final int numNodes = 1; Resource clusterResource = Resources.createResource(numNodes * (8 * GB), numNodes * 16); @@ -2603,91 +2827,111 @@ public class TestLeafQueue { recordFactory))); try { - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); } catch (NullPointerException e) { Assert.fail("NPE when allocating container on node but " + "forget to set off-switch request should be handled"); } } - - @Test + + @Test public void testFifoAssignment() throws Exception { - LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A)); - + LeafQueue a = stubLeafQueue((LeafQueue) queues.get(A)); + a.setOrderingPolicy(new FifoOrderingPolicy()); String host_0_0 = "127.0.0.1"; String rack_0 = "rack_0"; - FiCaSchedulerNode node_0_0 = TestUtils.getMockNode(host_0_0, rack_0, 0, 16*GB); - + FiCaSchedulerNode node_0_0 = TestUtils.getMockNode(host_0_0, rack_0, 0, + 16 * GB); + final int numNodes = 4; - Resource clusterResource = Resources.createResource( - numNodes * (16*GB), numNodes * 16); + Resource clusterResource = Resources.createResource(numNodes * (16 * GB), + numNodes * 16); when(csContext.getNumClusterNodes()).thenReturn(numNodes); String user_0 = "user_0"; - - final ApplicationAttemptId appAttemptId_0 = - TestUtils.getMockApplicationAttemptId(0, 0); - FiCaSchedulerApp app_0 = - spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, - mock(ActiveUsersManager.class), spyRMContext, - Priority.newInstance(3), false)); + + final ApplicationAttemptId appAttemptId_0 = + TestUtils.getMockApplicationAttemptId(0, 0); + FiCaSchedulerApp app_0 = spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, + mock(ActiveUsersManager.class), spyRMContext, Priority.newInstance(3), + false)); a.submitApplicationAttempt(app_0, user_0); - - final ApplicationAttemptId appAttemptId_1 = - TestUtils.getMockApplicationAttemptId(1, 0); - FiCaSchedulerApp app_1 = - spy(new FiCaSchedulerApp(appAttemptId_1, user_0, a, - mock(ActiveUsersManager.class), spyRMContext, - Priority.newInstance(5), false)); + + final ApplicationAttemptId appAttemptId_1 = + TestUtils.getMockApplicationAttemptId(1, 0); + FiCaSchedulerApp app_1 = spy(new FiCaSchedulerApp(appAttemptId_1, user_0, a, + mock(ActiveUsersManager.class), spyRMContext, Priority.newInstance(5), + false)); a.submitApplicationAttempt(app_1, user_0); - + + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0_0.getNodeID(), + node_0_0); + Priority priority = TestUtils.createMockPriority(1); List app_0_requests_0 = new ArrayList(); List app_1_requests_0 = new ArrayList(); - + app_0_requests_0.clear(); - app_0_requests_0.add( - TestUtils.createResourceRequest(ResourceRequest.ANY, 2*GB, 1, - true, priority, recordFactory)); + app_0_requests_0.add(TestUtils + .createResourceRequest(ResourceRequest.ANY, 2 * GB, 1, true, priority, + recordFactory)); app_0.updateResourceRequests(app_0_requests_0); - + app_1_requests_0.clear(); - app_1_requests_0.add( - TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, - true, priority, recordFactory)); + app_1_requests_0.add(TestUtils + .createResourceRequest(ResourceRequest.ANY, 1 * GB, 1, true, priority, + recordFactory)); app_1.updateResourceRequests(app_1_requests_0); // app_1 will get containers as it has high priority - a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - Assert.assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); - a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - Assert.assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); + Assert.assertEquals(1 * GB, app_1.getCurrentConsumption().getMemorySize()); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); + Assert.assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); app_0_requests_0.clear(); - app_0_requests_0.add( - TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, - true, priority, recordFactory)); + app_0_requests_0.add(TestUtils + .createResourceRequest(ResourceRequest.ANY, 1 * GB, 1, true, priority, + recordFactory)); app_0.updateResourceRequests(app_0_requests_0); app_1_requests_0.clear(); - app_1_requests_0.add( - TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, - true, priority, recordFactory)); + app_1_requests_0.add(TestUtils + .createResourceRequest(ResourceRequest.ANY, 1 * GB, 1, true, priority, + recordFactory)); app_1.updateResourceRequests(app_1_requests_0); //app_1 will still get assigned first as priority is more. - a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - Assert.assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize()); - Assert.assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); + Assert.assertEquals(2 * GB, app_1.getCurrentConsumption().getMemorySize()); + Assert.assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); //and only then will app_2 - a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - Assert.assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); -} + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); + Assert.assertEquals(3 * GB, app_0.getCurrentConsumption().getMemorySize()); + } + @Test public void testConcurrentAccess() throws Exception { YarnConfiguration conf = new YarnConfiguration(); @@ -2792,6 +3036,12 @@ public class TestLeafQueue { mock(ActiveUsersManager.class), spyRMContext)); a.submitApplicationAttempt(app_1, user_0); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0_0.getNodeID(), + node_0_0); + Priority priority = TestUtils.createMockPriority(1); List app_0_requests_0 = new ArrayList(); List app_1_requests_0 = new ArrayList(); @@ -2809,9 +3059,15 @@ public class TestLeafQueue { app_1.updateResourceRequests(app_1_requests_0); // app_0 will get containers as its submitted first. - a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); Assert.assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); - a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); Assert.assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize()); app_0_requests_0.clear(); @@ -2828,12 +3084,18 @@ public class TestLeafQueue { //Since it already has more resources, app_0 will not get //assigned first, but app_1 will - a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); Assert.assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); Assert.assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize()); //and only then will app_0 - a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + a.assignContainers(clusterResource, node_0_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), a, nodes, apps); Assert.assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); } @@ -2874,6 +3136,12 @@ public class TestLeafQueue { String rack_2 = "rack_2"; FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2); + final int numNodes = 3; Resource clusterResource = Resources.createResource(numNodes * (8*GB), numNodes * 16); @@ -2922,6 +3190,7 @@ public class TestLeafQueue { // Check app_0's scheduling opportunities increased and app_1 get allocated assignment = a.assignContainers(clusterResource, node_2, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, assignment, a, nodes, apps); verifyContainerAllocated(assignment, NodeType.NODE_LOCAL); assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey)); assertEquals(3, app_0.getTotalRequiredResources(schedulerKey)); @@ -2965,6 +3234,12 @@ public class TestLeafQueue { FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 100*GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0); + final int numNodes = 1; Resource clusterResource = Resources.createResource(numNodes * (100*GB), numNodes * 128); @@ -2983,9 +3258,10 @@ public class TestLeafQueue { // Start testing... // Assign 1st Container of 1GB - e.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // With queue capacity set at 1% of 100GB and user-limit-factor set to 1.0, // all users (only user_0) queue 'e' should be able to consume 1GB. // The first container should be assigned to app_0 with no headroom left @@ -2996,9 +3272,10 @@ public class TestLeafQueue { clusterResource, RMNodeLabelsManager.NO_LABEL).getMemorySize()); // Assign 2nd container of 1GB - e.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // user_0 has no headroom due to user-limit-factor of 1.0. However capacity // scheduler will assign one container more than user-limit-factor. // This container also went to app_0. Still with no neadroom even though @@ -3009,9 +3286,10 @@ public class TestLeafQueue { clusterResource, RMNodeLabelsManager.NO_LABEL).getMemorySize()); // Can't allocate 3rd container due to user-limit. Headroom still 0. - e.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit( @@ -3025,9 +3303,10 @@ public class TestLeafQueue { assertEquals(3*GB, e.getTotalPendingResourcesConsideringUserLimit( clusterResource, RMNodeLabelsManager.NO_LABEL).getMemorySize()); - e.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // app_0 is now satisified, app_1 is still asking for 2GB. assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize()); @@ -3035,12 +3314,14 @@ public class TestLeafQueue { clusterResource, RMNodeLabelsManager.NO_LABEL).getMemorySize()); // Get the last 2 containers for app_1, no more pending requests. - e.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - e.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); assertEquals(3*GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(2*GB, app_1.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit( @@ -3112,10 +3393,17 @@ public class TestLeafQueue { mock(ActiveUsersManager.class), spyRMContext); e.submitApplicationAttempt(app_3, user_1); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1, app_2.getApplicationAttemptId(), app_2, + app_3.getApplicationAttemptId(), app_3); + // Setup 1 node with 100GB of memory resources. String host_0 = "127.0.0.1"; FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 100*GB); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0); final int numNodes = 1; Resource clusterResource = @@ -3156,9 +3444,10 @@ public class TestLeafQueue { assertEquals(0*GB, app_3.getCurrentConsumption().getMemorySize()); // Assign 1st Container of 1GB - e.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // The first container was assigned to user_0's app_0. Queues total headroom // has 1GB left for user_1. assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit( @@ -3171,9 +3460,10 @@ public class TestLeafQueue { assertEquals(0*GB, app_3.getCurrentConsumption().getMemorySize()); // Assign 2nd container of 1GB - e.assignContainers(clusterResource, node_0, + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // user_0 has no headroom due to user-limit-factor of 1.0. However capacity // scheduler will assign one container more than user-limit-factor. So, // this container went to user_0's app_1. so, headroom for queue 'e'e is @@ -3188,9 +3478,10 @@ public class TestLeafQueue { assertEquals(0*GB, app_3.getCurrentConsumption().getMemorySize()); // Assign 3rd container. - e.assignContainers(clusterResource, node_0, + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // Container was allocated to user_1's app_2 since user_1, Now, no headroom // is left. assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit( @@ -3203,9 +3494,10 @@ public class TestLeafQueue { assertEquals(0*GB, app_3.getCurrentConsumption().getMemorySize()); // Assign 4th container. - e.assignContainers(clusterResource, node_0, + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // Allocated to user_1's app_2 since scheduler allocates 1 container // above user resource limit. Available headroom still 0. assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit( @@ -3222,9 +3514,10 @@ public class TestLeafQueue { assertEquals(0*GB, app_3_consumption); // Attempt to assign 5th container. Will be a no-op. - e.assignContainers(clusterResource, node_0, + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // Cannot allocate 5th container because both users are above their allowed // user resource limit. Values should be the same as previously. assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit( @@ -3241,9 +3534,10 @@ public class TestLeafQueue { // factor is no longer the limiting factor. e.setUserLimitFactor(10.0f); - e.assignContainers(clusterResource, node_0, + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // Next container goes to user_0's app_1, since it still wanted 1GB. assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit( clusterResource, RMNodeLabelsManager.NO_LABEL).getMemorySize()); @@ -3254,9 +3548,10 @@ public class TestLeafQueue { assertEquals(2*GB, app_2.getCurrentConsumption().getMemorySize()); assertEquals(0*GB, app_3.getCurrentConsumption().getMemorySize()); - e.assignContainers(clusterResource, node_0, + applyCSAssignment(clusterResource, + e.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), e, nodes, apps); // Last container goes to user_1's app_3, since it still wanted 1GB. // user_0's apps: assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java index 42a88724cec..d8759693dd8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java @@ -49,8 +49,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -123,6 +127,27 @@ public class TestParentQueue { return application; } + private void applyAllocationToQueue(Resource clusterResource, + int allocatedMem, + CSQueue queue) { + // Call accept & apply for queue + ResourceCommitRequest request = mock(ResourceCommitRequest.class); + when(request.anythingAllocatedOrReserved()).thenReturn(true); + ContainerAllocationProposal allocation = mock( + ContainerAllocationProposal.class); + when(request.getTotalReleasedResource()).thenReturn(Resources.none()); + when(request.getFirstAllocatedOrReservedContainer()).thenReturn(allocation); + SchedulerContainer scontainer = mock(SchedulerContainer.class); + when(allocation.getAllocatedOrReservedContainer()).thenReturn(scontainer); + when(allocation.getAllocatedOrReservedResource()).thenReturn( + Resources.createResource(allocatedMem)); + when(scontainer.getNodePartition()).thenReturn(""); + + if (queue.accept(clusterResource, request)) { + queue.apply(clusterResource, request); + } + } + private void stubQueueAllocation(final CSQueue queue, final Resource clusterResource, final FiCaSchedulerNode node, final int allocation) { @@ -157,7 +182,7 @@ public class TestParentQueue { // Next call - nothing if (allocation > 0) { doReturn(new CSAssignment(Resources.none(), type)).when(queue) - .assignContainers(eq(clusterResource), eq(node), + .assignContainers(eq(clusterResource), any(PlacementSet.class), any(ResourceLimits.class), any(SchedulingMode.class)); // Mock the node's resource availability @@ -168,7 +193,7 @@ public class TestParentQueue { return new CSAssignment(allocatedResource, type); } - }).when(queue).assignContainers(eq(clusterResource), eq(node), + }).when(queue).assignContainers(eq(clusterResource), any(PlacementSet.class), any(ResourceLimits.class), any(SchedulingMode.class)); } @@ -205,8 +230,8 @@ public class TestParentQueue { setupSingleLevelQueues(csConf); Map queues = new HashMap(); - CSQueue root = - CapacityScheduler.parseQueue(csContext, csConf, null, + CSQueue root = + CapacityScheduler.parseQueue(csContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); @@ -245,13 +270,18 @@ public class TestParentQueue { // Now, A should get the scheduling opportunity since A=0G/6G, B=1G/14G stubQueueAllocation(a, clusterResource, node_1, 2*GB); stubQueueAllocation(b, clusterResource, node_1, 1*GB); - root.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + root.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); InOrder allocationOrder = inOrder(a, b); - allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + allocationOrder.verify(a).assignContainers(eq(clusterResource), + any(PlacementSet.class), anyResourceLimits(), + any(SchedulingMode.class)); + root.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); @@ -261,11 +291,13 @@ public class TestParentQueue { stubQueueAllocation(b, clusterResource, node_0, 2*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + root.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder = inOrder(b, a); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource); @@ -277,9 +309,9 @@ public class TestParentQueue { new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder = inOrder(b, a); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 8*GB, clusterResource); @@ -287,13 +319,17 @@ public class TestParentQueue { // since A has 3/6G while B has 8/14G stubQueueAllocation(a, clusterResource, node_1, 1*GB); stubQueueAllocation(b, clusterResource, node_1, 1*GB); - root.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + root.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + root.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder = inOrder(a, b); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); verifyQueueMetrics(a, 4*GB, clusterResource); verifyQueueMetrics(b, 9*GB, clusterResource); } @@ -430,9 +466,9 @@ public class TestParentQueue { setupMultiLevelQueues(csConf); Map queues = new HashMap(); - CSQueue root = - CapacityScheduler.parseQueue(csContext, csConf, null, - CapacitySchedulerConfiguration.ROOT, queues, queues, + CSQueue root = + CapacityScheduler.parseQueue(csContext, csConf, null, + CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); // Setup some nodes @@ -496,6 +532,8 @@ public class TestParentQueue { stubQueueAllocation(c, clusterResource, node_1, 0*GB); root.assignContainers(clusterResource, node_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + applyAllocationToQueue(clusterResource, 4*GB, + b); verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource); verifyQueueMetrics(c, 1*GB, clusterResource); @@ -506,15 +544,27 @@ public class TestParentQueue { stubQueueAllocation(a1, clusterResource, node_0, 1*GB); stubQueueAllocation(b3, clusterResource, node_0, 2*GB); stubQueueAllocation(c, clusterResource, node_0, 2*GB); + root.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); InOrder allocationOrder = inOrder(a, c, b); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + applyAllocationToQueue(clusterResource, 1*GB, a); + + root.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder.verify(c).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); - allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + applyAllocationToQueue(clusterResource, 2*GB, root); + + root.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + allocationOrder.verify(b).assignContainers(eq(clusterResource), + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + applyAllocationToQueue(clusterResource, 2*GB, b); verifyQueueMetrics(a, 1*GB, clusterResource); verifyQueueMetrics(b, 6*GB, clusterResource); verifyQueueMetrics(c, 3*GB, clusterResource); @@ -533,17 +583,28 @@ public class TestParentQueue { stubQueueAllocation(b3, clusterResource, node_2, 1*GB); stubQueueAllocation(b1, clusterResource, node_2, 1*GB); stubQueueAllocation(c, clusterResource, node_2, 1*GB); - root.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + root.assignContainers(clusterResource, node_2, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder = inOrder(a, a2, a1, b, c); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); allocationOrder.verify(a2).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + applyAllocationToQueue(clusterResource, 2*GB, a); + + root.assignContainers(clusterResource, node_2, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + applyAllocationToQueue(clusterResource, 2*GB, b); + + root.assignContainers(clusterResource, node_2, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder.verify(c).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 8*GB, clusterResource); verifyQueueMetrics(c, 4*GB, clusterResource); @@ -614,27 +675,25 @@ public class TestParentQueue { public void testOffSwitchScheduling() throws Exception { // Setup queue configs setupSingleLevelQueues(csConf); - csConf.setOffSwitchPerHeartbeatLimit(2); - Map queues = new HashMap(); - CSQueue root = - CapacityScheduler.parseQueue(csContext, csConf, null, - CapacitySchedulerConfiguration.ROOT, queues, queues, + CSQueue root = + CapacityScheduler.parseQueue(csContext, csConf, null, + CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); // Setup some nodes final int memoryPerNode = 10; final int coresPerNode = 16; final int numNodes = 2; - - FiCaSchedulerNode node_0 = + + FiCaSchedulerNode node_0 = TestUtils.getMockNode("host_0", DEFAULT_RACK, 0, memoryPerNode*GB); - FiCaSchedulerNode node_1 = + FiCaSchedulerNode node_1 = TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB); - - final Resource clusterResource = - Resources.createResource(numNodes * (memoryPerNode*GB), + + final Resource clusterResource = + Resources.createResource(numNodes * (memoryPerNode*GB), numNodes * coresPerNode); when(csContext.getNumClusterNodes()).thenReturn(numNodes); @@ -644,50 +703,46 @@ public class TestParentQueue { a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage() - .incPending(Resources.createResource(1 * GB)); - - // Simulate returning 2 containers on node_0 before offswitch limit - stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH); - stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH); + .incPending(Resources.createResource(1 * GB)); - root.assignContainers(clusterResource, node_0, + // Simulate B returning a container on node_0 + stubQueueAllocation(a, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH); + stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH); + root.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - InOrder allocationOrder = inOrder(a, b); - allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); - allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); - verifyQueueMetrics(a, 1*GB, clusterResource); + verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(b, 1*GB, clusterResource); - + // Now, A should get the scheduling opportunity since A=0G/6G, B=1G/14G // also, B gets a scheduling opportunity since A allocates RACK_LOCAL stubQueueAllocation(a, clusterResource, node_1, 2*GB, NodeType.RACK_LOCAL); stubQueueAllocation(b, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH); - root.assignContainers(clusterResource, node_1, + root.assignContainers(clusterResource, node_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); - allocationOrder = inOrder(a, b); - allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); - allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); - verifyQueueMetrics(a, 3*GB, clusterResource); + InOrder allocationOrder = inOrder(a); + allocationOrder.verify(a).assignContainers(eq(clusterResource), + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + root.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + allocationOrder = inOrder(b); + allocationOrder.verify(b).assignContainers(eq(clusterResource), + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); - - // Now, B should get the scheduling opportunity - // since A has 2/6G while B has 2/14G, - // A also gets an opportunity because offswitchlimit not reached + + // Now, B should get the scheduling opportunity + // since A has 2/6G while B has 2/14G, + // However, since B returns off-switch, A won't get an opportunity stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL); stubQueueAllocation(b, clusterResource, node_0, 2*GB, NodeType.OFF_SWITCH); - - root.assignContainers(clusterResource, node_0, + root.assignContainers(clusterResource, node_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder = inOrder(b, a); - allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); - allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); - verifyQueueMetrics(a, 4*GB, clusterResource); + allocationOrder.verify(b).assignContainers(eq(clusterResource), + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + allocationOrder.verify(a).assignContainers(eq(clusterResource), + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); + verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource); } @@ -743,11 +798,13 @@ public class TestParentQueue { stubQueueAllocation(b3, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH); root.assignContainers(clusterResource, node_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + root.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); InOrder allocationOrder = inOrder(b2, b3); allocationOrder.verify(b2).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); allocationOrder.verify(b3).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); verifyQueueMetrics(b2, 1*GB, clusterResource); verifyQueueMetrics(b3, 2*GB, clusterResource); @@ -760,9 +817,9 @@ public class TestParentQueue { new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); allocationOrder = inOrder(b3, b2); allocationOrder.verify(b3).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); allocationOrder.verify(b2).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class)); + any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class)); verifyQueueMetrics(b2, 1*GB, clusterResource); verifyQueueMetrics(b3, 3*GB, clusterResource); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java index 8fe85c971c5..f6caa50a315 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -30,6 +32,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import com.google.common.collect.ImmutableMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -59,6 +62,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicat import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; @@ -190,6 +194,15 @@ public class TestReservations { } static LeafQueue stubLeafQueue(LeafQueue queue) { + ParentQueue parent = (ParentQueue) queue.getParent(); + + if (parent != null) { + // Stub out parent queue's accept and apply. + doReturn(true).when(parent).accept(any(Resource.class), + any(ResourceCommitRequest.class)); + doNothing().when(parent).apply(any(Resource.class), + any(ResourceCommitRequest.class)); + } return queue; } @@ -239,6 +252,12 @@ public class TestReservations { FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, DEFAULT_RACK, 0, 8 * GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2); + when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0); when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1); when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2); @@ -268,8 +287,10 @@ public class TestReservations { // Start testing... // Only AM - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(2 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -280,8 +301,10 @@ public class TestReservations { assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize()); // Only 1 map - simulating reduce - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(5 * GB, a.getUsedResources().getMemorySize()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -292,8 +315,10 @@ public class TestReservations { assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize()); // Only 1 map to other node - simulating reduce - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(8 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -308,8 +333,10 @@ public class TestReservations { toSchedulerKey(priorityReduce))); // try to assign reducer (5G on node 0 and should reserve) - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(13 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -325,8 +352,10 @@ public class TestReservations { toSchedulerKey(priorityReduce))); // assign reducer to node 2 - a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_2, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(18 * GB, a.getUsedResources().getMemorySize()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -343,8 +372,10 @@ public class TestReservations { // node_1 heartbeat and unreserves from node_0 in order to allocate // on node_1 - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(18 * GB, a.getUsedResources().getMemorySize()); assertEquals(18 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -412,6 +443,12 @@ public class TestReservations { when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1); when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2); + cs.getNodeTracker().addNode(node_0); cs.getNodeTracker().addNode(node_1); cs.getNodeTracker().addNode(node_2); @@ -434,8 +471,10 @@ public class TestReservations { // Start testing... // Only AM - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(2 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, app_1.getCurrentConsumption().getMemorySize()); @@ -446,8 +485,10 @@ public class TestReservations { assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize()); assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize()); - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(4 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(2 * GB, app_1.getCurrentConsumption().getMemorySize()); @@ -467,8 +508,10 @@ public class TestReservations { priorityMap, recordFactory))); // add a reservation for app_0 - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(12 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(2 * GB, app_1.getCurrentConsumption().getMemorySize()); @@ -481,8 +524,10 @@ public class TestReservations { // next assignment is beyond user limit for user_0 but it should assign to // app_1 for user_1 - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(14 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(4 * GB, app_1.getCurrentConsumption().getMemorySize()); @@ -544,6 +589,12 @@ public class TestReservations { FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, DEFAULT_RACK, 0, 8 * GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2); + when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0); when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1); when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2); @@ -569,8 +620,10 @@ public class TestReservations { // Start testing... // Only AM - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(2 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -581,8 +634,10 @@ public class TestReservations { assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize()); // Only 1 map - simulating reduce - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(5 * GB, a.getUsedResources().getMemorySize()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -593,8 +648,10 @@ public class TestReservations { assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize()); // Only 1 map to other node - simulating reduce - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(8 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -609,8 +666,10 @@ public class TestReservations { toSchedulerKey(priorityReduce))); // try to assign reducer (5G on node 0 and should reserve) - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(13 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -626,8 +685,10 @@ public class TestReservations { toSchedulerKey(priorityReduce))); // assign reducer to node 2 - a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_2, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(18 * GB, a.getUsedResources().getMemorySize()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -644,8 +705,10 @@ public class TestReservations { // node_1 heartbeat and won't unreserve from node_0, potentially stuck // if AM doesn't handle - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(18 * GB, a.getUsedResources().getMemorySize()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -706,6 +769,12 @@ public class TestReservations { FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8 * GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1); + cs.getNodeTracker().addNode(node_0); cs.getNodeTracker().addNode(node_1); @@ -733,8 +802,10 @@ public class TestReservations { // Start testing... // Only AM - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(2 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -744,8 +815,10 @@ public class TestReservations { assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize()); // Only 1 map - simulating reduce - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(5 * GB, a.getUsedResources().getMemorySize()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -755,8 +828,10 @@ public class TestReservations { assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize()); // Only 1 map to other node - simulating reduce - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(8 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -770,8 +845,10 @@ public class TestReservations { toSchedulerKey(priorityReduce))); // try to assign reducer (5G on node 0 and should reserve) - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(13 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -786,8 +863,10 @@ public class TestReservations { toSchedulerKey(priorityReduce))); // could allocate but told need to unreserve first - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(13 * GB, a.getUsedResources().getMemorySize()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -983,6 +1062,12 @@ public class TestReservations { when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1); when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2); + final int numNodes = 2; Resource clusterResource = Resources.createResource(numNodes * (8 * GB)); when(csContext.getNumClusterNodes()).thenReturn(numNodes); @@ -1004,8 +1089,10 @@ public class TestReservations { // Start testing... // Only AM - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(2 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1015,8 +1102,10 @@ public class TestReservations { assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize()); // Only 1 map - simulating reduce - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(5 * GB, a.getUsedResources().getMemorySize()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1026,8 +1115,10 @@ public class TestReservations { assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize()); // Only 1 map to other node - simulating reduce - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(8 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1040,8 +1131,10 @@ public class TestReservations { // now add in reservations and make sure it continues if config set // allocate to queue so that the potential new capacity is greater then // absoluteMaxCapacity - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(13 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -1153,6 +1246,12 @@ public class TestReservations { FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, DEFAULT_RACK, 0, 8 * GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2); + when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0); when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1); when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2); @@ -1178,8 +1277,10 @@ public class TestReservations { // Start testing... // Only AM - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(2 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1189,8 +1290,10 @@ public class TestReservations { assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize()); // Only 1 map - simulating reduce - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(5 * GB, a.getUsedResources().getMemorySize()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1200,8 +1303,10 @@ public class TestReservations { assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize()); // Only 1 map to other node - simulating reduce - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(8 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1214,8 +1319,10 @@ public class TestReservations { // now add in reservations and make sure it continues if config set // allocate to queue so that the potential new capacity is greater then // absoluteMaxCapacity - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(13 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(5 * GB, app_0.getCurrentReservation().getMemorySize()); @@ -1301,6 +1408,12 @@ public class TestReservations { FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, DEFAULT_RACK, 0, 8 * GB); + Map apps = ImmutableMap.of( + app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(), + app_1); + Map nodes = ImmutableMap.of(node_0.getNodeID(), + node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2); + when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0); when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1); when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2); @@ -1330,8 +1443,10 @@ public class TestReservations { // Start testing... // Only AM - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(2 * GB, a.getUsedResources().getMemorySize()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1342,8 +1457,10 @@ public class TestReservations { assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize()); // Only 1 map - simulating reduce - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(5 * GB, a.getUsedResources().getMemorySize()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1354,8 +1471,10 @@ public class TestReservations { assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize()); // Only 1 map to other node - simulating reduce - a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_1, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(8 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1370,8 +1489,10 @@ public class TestReservations { // used (8G) + required (5G). It will not reserved since it has to unreserve // some resource. Even with continous reservation looking, we don't allow // unreserve resource to reserve container. - a.assignContainers(clusterResource, node_0, - new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(Resources.createResource(10 * GB)), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(8 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1386,8 +1507,10 @@ public class TestReservations { // try to assign reducer (5G on node 0), but tell it's resource limits < // used (8G) + required (5G). It will not reserved since it has to unreserve // some resource. Unfortunately, there's nothing to unreserve. - a.assignContainers(clusterResource, node_2, - new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_2, + new ResourceLimits(Resources.createResource(10 * GB)), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(8 * GB, a.getUsedResources().getMemorySize()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1400,8 +1523,10 @@ public class TestReservations { assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize()); // let it assign 5G to node_2 - a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_2, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(13 * GB, a.getUsedResources().getMemorySize()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1413,8 +1538,10 @@ public class TestReservations { assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize()); // reserve 8G node_0 - a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_0, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(21 * GB, a.getUsedResources().getMemorySize()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(8 * GB, a.getMetrics().getReservedMB()); @@ -1428,8 +1555,10 @@ public class TestReservations { // try to assign (8G on node 2). No room to allocate, // continued to try due to having reservation above, // but hits queue limits so can't reserve anymore. - a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); + TestUtils.applyResourceCommitRequest(clusterResource, + a.assignContainers(clusterResource, node_2, + new ResourceLimits(clusterResource), + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps); assertEquals(21 * GB, a.getUsedResources().getMemorySize()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize()); assertEquals(8 * GB, a.getMetrics().getReservedMB()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java index a60b7ed4bf8..e34ee3467aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java @@ -24,6 +24,8 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; +import java.io.IOException; +import java.util.Map; import java.util.Set; import org.apache.commons.logging.Log; @@ -430,4 +432,28 @@ public class TestUtils { req.setAllocationRequestId(allocationRequestId); return SchedulerRequestKey.create(req); } + + public static void applyResourceCommitRequest(Resource clusterResource, + CSAssignment csAssignment, + final Map nodes, + final Map apps) + throws IOException { + CapacityScheduler cs = new CapacityScheduler() { + @Override + public FiCaSchedulerNode getNode(NodeId nodeId) { + return nodes.get(nodeId); + } + + @Override + public FiCaSchedulerApp getApplicationAttempt( + ApplicationAttemptId applicationAttemptId) { + return apps.get(applicationAttemptId); + } + }; + + cs.setResourceCalculator(new DefaultResourceCalculator()); + + cs.submitResourceCommitRequest(clusterResource, + csAssignment); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java index a0bd951383a..1e61186c3ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java @@ -103,10 +103,6 @@ public class TestRMWebServicesSchedulerActivities verifyStateOfAllocations(allocations.getJSONObject(i), "finalAllocationState", "ALLOCATED"); verifyQueueOrder(allocations.getJSONObject(i), "root-a-b-b2-b3-b1"); - } else { - verifyStateOfAllocations(allocations.getJSONObject(i), - "finalAllocationState", "SKIPPED"); - verifyQueueOrder(allocations.getJSONObject(i), "root-a-b"); } } } @@ -409,9 +405,9 @@ public class TestRMWebServicesSchedulerActivities verifyStateOfAllocations(allocations, "finalAllocationState", "ALLOCATED"); - verifyNumberOfNodes(allocations, 6); + verifyNumberOfNodes(allocations, 5); - verifyQueueOrder(json.getJSONObject("allocations"), "root-a-b-b1"); + verifyQueueOrder(json.getJSONObject("allocations"), "root-b-b1"); } finally { rm.stop();