diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java index 1be1727..1071831 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java @@ -146,7 +146,7 @@ private ConcurrentMap> finishedContainersSentToAM = new ConcurrentHashMap>(); - private Container masterContainer; + private volatile Container masterContainer; private float progress = 0; private String host = "N/A"; @@ -762,13 +762,7 @@ public float getProgress() { @Override public Container getMasterContainer() { - this.readLock.lock(); - - try { - return this.masterContainer; - } finally { - this.readLock.unlock(); - } + return this.masterContainer; } @InterfaceAudience.Private 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 84ebe9c..1f2d65d 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 @@ -73,10 +73,11 @@ /* Allocated by scheduler */ boolean pending = true; // for app metrics + private ResourceUsage appResourceUsage; public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user, Queue queue, ActiveUsersManager activeUsersManager, - long epoch) { + long epoch, ResourceUsage appResourceUsage) { this.applicationAttemptId = appAttemptId; this.applicationId = appAttemptId.getApplicationId(); this.queue = queue; @@ -84,6 +85,7 @@ public AppSchedulingInfo(ApplicationAttemptId appAttemptId, this.user = user; this.activeUsersManager = activeUsersManager; this.containerIdCounter = new AtomicLong(epoch << EPOCH_BIT_SHIFT); + this.appResourceUsage = appResourceUsage; } public ApplicationId getApplicationId() { @@ -191,13 +193,19 @@ synchronized public void updateResourceRequests( lastRequestCapability); // update queue: + Resource increasedResource = Resources.multiply(request.getCapability(), + request.getNumContainers()); queue.incPendingResource( request.getNodeLabelExpression(), - Resources.multiply(request.getCapability(), - request.getNumContainers())); + increasedResource); + appResourceUsage.incPending(request.getNodeLabelExpression(), increasedResource); if (lastRequest != null) { + Resource decreasedResource = + Resources.multiply(lastRequestCapability, lastRequestContainers); queue.decPendingResource(lastRequest.getNodeLabelExpression(), - Resources.multiply(lastRequestCapability, lastRequestContainers)); + decreasedResource); + appResourceUsage.decPending(lastRequest.getNodeLabelExpression(), + decreasedResource); } } } @@ -385,6 +393,8 @@ synchronized private void decrementOutstanding( checkForDeactivation(); } + appResourceUsage.decPending(offSwitchRequest.getNodeLabelExpression(), + offSwitchRequest.getCapability()); queue.decPendingResource(offSwitchRequest.getNodeLabelExpression(), offSwitchRequest.getCapability()); } 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/ResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java index 36ee4da..5169b78 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java @@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.util.resource.Resources; /** @@ -250,6 +251,10 @@ private static Resource normalize(Resource res) { } private Resource _get(String label, ResourceType type) { + if (label == null) { + label = RMNodeLabelsManager.NO_LABEL; + } + try { readLock.lock(); UsageByLabel usage = usages.get(label); @@ -263,6 +268,9 @@ private Resource _get(String label, ResourceType type) { } private UsageByLabel getAndAddIfMissing(String label) { + if (label == null) { + label = RMNodeLabelsManager.NO_LABEL; + } if (!usages.containsKey(label)) { UsageByLabel u = new UsageByLabel(label); usages.put(label, u); 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 bf5641d..b0af3ea 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 @@ -108,13 +108,22 @@ private Set pendingRelease = null; /** + * Count how many times the application has been given an opportunity to + * schedule a task at each priority. Each time the scheduler asks the + * application for a task at this priority, it is incremented, and each time + * the application successfully schedules a task (at rack or node local), it + * is reset to 0. + */ + Multiset schedulingOpportunities = HashMultiset.create(); + + /** * Count how many times the application has been given an opportunity * to schedule a task at each priority. Each time the scheduler * asks the application for a task at this priority, it is incremented, * and each time the application successfully schedules a task, it - * is reset to 0. + * is reset to 0 when schedule any task at corresponding priority. */ - Multiset schedulingOpportunities = HashMultiset.create(); + Multiset nonExclusiveSchedulingOpportunities = HashMultiset.create(); // Time of the last container scheduled at the current allowed level protected Map lastScheduledContainer = @@ -132,7 +141,7 @@ public SchedulerApplicationAttempt(ApplicationAttemptId applicationAttemptId, this.rmContext = rmContext; this.appSchedulingInfo = new AppSchedulingInfo(applicationAttemptId, user, queue, - activeUsersManager, rmContext.getEpoch()); + activeUsersManager, rmContext.getEpoch(), attemptResourceUsage); this.queue = queue; this.pendingRelease = new HashSet(); this.attemptId = applicationAttemptId; @@ -489,6 +498,23 @@ public boolean isBlacklisted(String resourceName) { return this.appSchedulingInfo.isBlacklisted(resourceName); } + public synchronized int addNonExclusiveSchedulingOpportunity(Priority priority) { + return nonExclusiveSchedulingOpportunities.setCount(priority, + nonExclusiveSchedulingOpportunities.count(priority) + 1) + 1; + } + + /** + * Should be called when an application has successfully scheduled a container, + * or when the scheduling locality threshold is relaxed. + * Reset various internal counters which affect delay scheduling + * + * @param priority The priority of the container scheduled. + */ + public synchronized void resetNonExclusiveSchedulingOpportunities(Priority priority) { + nonExclusiveSchedulingOpportunities.setCount(priority, 0); + } + + public synchronized void addSchedulingOpportunity(Priority priority) { schedulingOpportunities.setCount(priority, schedulingOpportunities.count(priority) + 1); @@ -518,6 +544,7 @@ public synchronized int getSchedulingOpportunities(Priority priority) { public synchronized void resetSchedulingOpportunities(Priority priority) { resetSchedulingOpportunities(priority, System.currentTimeMillis()); } + // used for continuous scheduling public synchronized void resetSchedulingOpportunities(Priority priority, long currentTimeMs) { 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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java index f03663a..1abb809 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java @@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.util.resource.Resources; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; @@ -271,6 +272,11 @@ public int getNumContainers() { public synchronized List getRunningContainers() { return new ArrayList(launchedContainers.values()); } + + @VisibleForTesting + public synchronized RMContainer getRunningContainer(ContainerId containerId) { + return launchedContainers.get(containerId); + } public synchronized RMContainer getReservedContainer() { return reservedContainer; 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/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java index 248cc08..8314f1b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java @@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.security.AccessType; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ExclusiveType; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -235,7 +236,10 @@ public static void validateResourceRequest(ResourceRequest resReq, if (labelExp == null && queueInfo != null && ResourceRequest.ANY.equals(resReq.getResourceName())) { labelExp = queueInfo.getDefaultNodeLabelExpression(); - resReq.setNodeLabelExpression(labelExp); + // If labelExp still equals to null, set it to be NO_LABEL + resReq + .setNodeLabelExpression(labelExp == null ? RMNodeLabelsManager.NO_LABEL + : labelExp); } // we don't allow specify label expression other than resourceName=ANY now @@ -313,7 +317,12 @@ public static void checkIfLabelInClusterNodeLabels(RMNodeLabelsManager mgr, } public static boolean checkNodeLabelExpression(Set nodeLabels, - String labelExpression) { + String labelExpression, ExclusiveType exclusiveType) { + if (exclusiveType == ExclusiveType.NON_EXECLUSIVE + && labelExpression.equals(RMNodeLabelsManager.NO_LABEL)) { + return true; + } + // empty label expression can only allocate on node with empty labels if (labelExpression == null || labelExpression.trim().isEmpty()) { if (!nodeLabels.isEmpty()) { 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 3cd85ae..5582427 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 @@ -320,16 +320,8 @@ public Resource getMinimumAllocation() { } synchronized void allocateResource(Resource clusterResource, - Resource resource, Set nodeLabels) { - - // Update usedResources by labels - if (nodeLabels == null || nodeLabels.isEmpty()) { - queueUsage.incUsed(resource); - } else { - for (String label : Sets.intersection(accessibleLabels, nodeLabels)) { - queueUsage.incUsed(label, resource); - } - } + Resource resource, String nodePartition) { + queueUsage.incUsed(nodePartition, resource); ++numContainers; CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(), @@ -337,15 +329,8 @@ synchronized void allocateResource(Resource clusterResource, } protected synchronized void releaseResource(Resource clusterResource, - Resource resource, Set nodeLabels) { - // Update usedResources by labels - if (null == nodeLabels || nodeLabels.isEmpty()) { - queueUsage.decUsed(resource); - } else { - for (String label : Sets.intersection(accessibleLabels, nodeLabels)) { - queueUsage.decUsed(label, resource); - } - } + Resource resource, String nodePartition) { + queueUsage.decUsed(nodePartition, resource); CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(), clusterResource, minimumAllocation); @@ -412,27 +397,37 @@ private boolean isQueueHierarchyPreemptionDisabled(CSQueue q) { } private Resource getCurrentLimitResource(String nodeLabel, - Resource clusterResource, ResourceLimits currentResourceLimits) { - /* - * Current limit resource: For labeled resource: limit = queue-max-resource - * (TODO, this part need update when we support labeled-limit) For - * non-labeled resource: limit = min(queue-max-resource, - * limit-set-by-parent) - */ - Resource queueMaxResource = - Resources.multiplyAndNormalizeDown(resourceCalculator, - labelManager.getResourceByLabel(nodeLabel, clusterResource), - queueCapacities.getAbsoluteMaximumCapacity(nodeLabel), minimumAllocation); - if (nodeLabel.equals(RMNodeLabelsManager.NO_LABEL)) { - return Resources.min(resourceCalculator, clusterResource, - queueMaxResource, currentResourceLimits.getLimit()); + Resource clusterResource, ResourceLimits currentResourceLimits, + ExclusiveType exclusiveType) { + if (exclusiveType == ExclusiveType.EXCLUSIVE) { + /* + * Current limit resource: For labeled resource: limit = queue-max-resource + * (TODO, this part need update when we support labeled-limit) For + * non-labeled resource: limit = min(queue-max-resource, + * limit-set-by-parent) + */ + Resource queueMaxResource = + Resources.multiplyAndNormalizeDown(resourceCalculator, + labelManager.getResourceByLabel(nodeLabel, clusterResource), + queueCapacities.getAbsoluteMaximumCapacity(nodeLabel), minimumAllocation); + if (nodeLabel.equals(RMNodeLabelsManager.NO_LABEL)) { + return Resources.min(resourceCalculator, clusterResource, + queueMaxResource, currentResourceLimits.getLimit()); + } + return queueMaxResource; + } else if (exclusiveType == ExclusiveType.NON_EXECLUSIVE) { + // When we doing non-exclusive resource allocation, maximum capacity of + // all queues on this label equals to total resource with the label. + return labelManager.getResourceByLabel(nodeLabel, clusterResource); } - return queueMaxResource; + + return Resources.none(); } synchronized boolean canAssignToThisQueue(Resource clusterResource, Set nodeLabels, ResourceLimits currentResourceLimits, - Resource nowRequired, Resource resourceCouldBeUnreserved) { + Resource nowRequired, Resource resourceCouldBeUnreserved, + ExclusiveType exclusiveType) { // Get label of this queue can access, it's (nodeLabel AND queueLabel) Set labelCanAccess; if (null == nodeLabels || nodeLabels.isEmpty()) { @@ -440,9 +435,16 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource, // Any queue can always access any node without label labelCanAccess.add(RMNodeLabelsManager.NO_LABEL); } else { - labelCanAccess = new HashSet( - accessibleLabels.contains(CommonNodeLabelsManager.ANY) ? nodeLabels - : Sets.intersection(accessibleLabels, nodeLabels)); + if (exclusiveType == ExclusiveType.NON_EXECLUSIVE) { + // When we do non exclusive resource allocation, this node could be used + // by any queue. + labelCanAccess = nodeLabels; + } else { + labelCanAccess = + new HashSet( + accessibleLabels.contains(CommonNodeLabelsManager.ANY) ? nodeLabels + : Sets.intersection(accessibleLabels, nodeLabels)); + } } for (String label : labelCanAccess) { @@ -451,7 +453,8 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource, Resources.add(queueUsage.getUsed(label), nowRequired); Resource currentLimitResource = - getCurrentLimitResource(label, clusterResource, currentResourceLimits); + getCurrentLimitResource(label, clusterResource, + currentResourceLimits, exclusiveType); // if reservation continous looking enabled, check to see if could we // potentially use this node instead of a reserved node if the application @@ -533,4 +536,23 @@ public void decPendingResource(String nodeLabel, Resource resourceToDec) { parent.decPendingResource(nodeLabel, resourceToDec); } } + + private boolean needResource(String nodeLabelToLookAt, Resource cluster) { + if (Resources.greaterThan(resourceCalculator, cluster, + queueUsage.getPending(nodeLabelToLookAt), Resources.none())) { + return true; + } + return false; + } + + @Private + public boolean needResource(String nodeLabelToLookAt, + Resource cluster, ExclusiveType exclusiveType) { + if (exclusiveType == ExclusiveType.EXCLUSIVE) { + return needResource(nodeLabelToLookAt, cluster); + } else if (exclusiveType == ExclusiveType.NON_EXECLUSIVE) { + return needResource(RMNodeLabelsManager.NO_LABEL, cluster); + } + return 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/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 1a9448a..5256c9c 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 @@ -190,10 +190,13 @@ public void finishApplicationAttempt(FiCaSchedulerApp application, * @param clusterResource the resource of the cluster. * @param node node on which resources are available * @param resourceLimits how much overall resource of this queue can use. + * @param exclusiveType Type of exclusive check when assign container on a + * NodeManager, see {@link ExclusiveType}. * @return the assignment */ public CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, ResourceLimits resourceLimits); + FiCaSchedulerNode node, ResourceLimits resourceLimits, + ExclusiveType exclusiveType); /** * A container assigned to the queue has completed. 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 c86c0ff..4e8e04c 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 @@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; @@ -1068,7 +1069,8 @@ private synchronized void allocateContainersToNode(FiCaSchedulerNode 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, clusterResource))); + RMNodeLabelsManager.NO_LABEL, clusterResource)), + ExclusiveType.EXCLUSIVE); RMContainer excessReservation = assignment.getExcessReservation(); if (excessReservation != null) { @@ -1092,13 +1094,53 @@ private synchronized void allocateContainersToNode(FiCaSchedulerNode node) { LOG.debug("Trying to schedule on node: " + node.getNodeName() + ", available: " + node.getAvailableResource()); } - root.assignContainers( + CSAssignment assigned = null; + + // Try to schedule EXCLUSIVE resource request first + assigned = root.assignContainers( clusterResource, 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, clusterResource))); + RMNodeLabelsManager.NO_LABEL, clusterResource)), + ExclusiveType.EXCLUSIVE); + if (Resources.greaterThan(calculator, clusterResource, + assigned.getResource(), Resources.none())) { + 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 + assigned = root.assignContainers( + clusterResource, + 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, clusterResource)), + ExclusiveType.NON_EXECLUSIVE); + if (Resources.greaterThan(calculator, clusterResource, + assigned.getResource(), Resources.none())) { + return; + } } } else { LOG.info("Skipping scheduling since node " + node.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/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 102e553..4e8d617 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 @@ -319,6 +319,11 @@ public float getMaximumApplicationMasterResourcePerQueuePercent(String queue) { getMaximumApplicationMasterResourcePercent()); } + public void setMaximumApplicationMasterResourcePerQueuePercent(String queue, + float percent) { + setFloat(getQueuePrefix(queue) + MAXIMUM_AM_RESOURCE_SUFFIX, percent); + } + public float getNonLabeledQueueCapacity(String queue) { float capacity = queue.equals("root") ? 100.0f : getFloat( getQueuePrefix(queue) + CAPACITY, UNDEFINED); 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/ExclusiveType.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/ExclusiveType.java new file mode 100644 index 0000000..f57181e --- /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/capacity/ExclusiveType.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.capacity; + +/** + * Type of node label exclusive when assign a container to NMs + */ +public enum ExclusiveType { + /** + * The labeled node can only be used by queue accessible to the same node + * label request the same node label. + */ + EXCLUSIVE, + + /** + * The labeled node can be used by applications from all queues request + * non-labeled resource. + */ + NON_EXECLUSIVE +} 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 3e5405d..2a86361 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 @@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.security.AccessType; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; @@ -739,18 +740,19 @@ private synchronized FiCaSchedulerApp getApplication( return labels; } - private boolean checkResourceRequestMatchingNodeLabel(ResourceRequest offswitchResourceRequest, - FiCaSchedulerNode node) { + private boolean checkResourceRequestMatchingNodeLabel( + ResourceRequest offswitchResourceRequest, String nodeLabelToLookAt) { String askedNodeLabel = offswitchResourceRequest.getNodeLabelExpression(); if (null == askedNodeLabel) { askedNodeLabel = RMNodeLabelsManager.NO_LABEL; } - return askedNodeLabel.equals(node.getPartition()); + return askedNodeLabel.equals(nodeLabelToLookAt); } @Override public synchronized CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, ResourceLimits currentResourceLimits) { + FiCaSchedulerNode node, ResourceLimits currentResourceLimits, + ExclusiveType exclusiveType) { updateCurrentResourceLimits(currentResourceLimits, clusterResource); if(LOG.isDebugEnabled()) { @@ -759,8 +761,9 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, } // if our queue cannot access this node, just return - if (!SchedulerUtils.checkQueueAccessToNode(accessibleLabels, - node.getLabels())) { + if (exclusiveType == ExclusiveType.EXCLUSIVE + && !SchedulerUtils.checkQueueAccessToNode(accessibleLabels, + node.getLabels())) { return NULL_ASSIGNMENT; } @@ -771,10 +774,19 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, getApplication(reservedContainer.getApplicationAttemptId()); synchronized (application) { return assignReservedContainer(application, node, reservedContainer, - clusterResource); + clusterResource, exclusiveType); } } + // We will only look at node label = nodeLabelToLookAt according to + // exclusiveType and partition of node. + String nodeLabelToLookAt = null; + if (exclusiveType == ExclusiveType.EXCLUSIVE) { + nodeLabelToLookAt = node.getPartition(); + } else { + nodeLabelToLookAt = RMNodeLabelsManager.NO_LABEL; + } + // Try to assign containers to applications in order for (FiCaSchedulerApp application : activeApplications) { @@ -783,6 +795,17 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, + application.getApplicationId()); application.showRequests(); } + + // Check if application needs more resource, skip if it doesn't need more. + if (!application.needResource(resourceCalculator, node.getPartition(), + clusterResource, exclusiveType)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId() + + ", because it doesn't need more resource, exclusiveType=" + + exclusiveType.name() + " node-label=" + node.getPartition()); + } + continue; + } synchronized (application) { // Check if this resource is on the blacklist @@ -806,10 +829,27 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, continue; } + // AM container allocation doesn't support non-exclusive allocation to + // avoid painful of preempt an AM container + if (exclusiveType == ExclusiveType.NON_EXECLUSIVE) { + RMAppAttempt rmAppAttempt = + csContext.getRMContext().getRMApps() + .get(application.getApplicationId()).getCurrentAppAttempt(); + if (null == rmAppAttempt.getMasterContainer()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip allocating AM container to app_attempt=" + + application.getApplicationAttemptId() + + ", don't allow to allocate AM container in non-exclusive mode"); + } + break; + } + } + // Is the node-label-expression of this offswitch resource request // matches the node's label? // If not match, jump to next priority. - if (!checkResourceRequestMatchingNodeLabel(anyRequest, node)) { + if (!checkResourceRequestMatchingNodeLabel(anyRequest, + nodeLabelToLookAt)) { continue; } @@ -822,10 +862,6 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, } } - Set requestedNodeLabels = - getRequestLabelSetByExpression(anyRequest - .getNodeLabelExpression()); - // Compute user-limit & set headroom // Note: We compute both user-limit & headroom with the highest // priority request as the target. @@ -833,27 +869,52 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, // before all higher priority ones are serviced. Resource userLimit = computeUserLimitAndSetHeadroom(application, clusterResource, - required, requestedNodeLabels); + required, node.getPartition(), exclusiveType); // Check queue max-capacity limit if (!super.canAssignToThisQueue(clusterResource, node.getLabels(), - this.currentResourceLimits, required, application.getCurrentReservation())) { + this.currentResourceLimits, required, + application.getCurrentReservation(), exclusiveType)) { return NULL_ASSIGNMENT; } // Check user limit if (!canAssignToUser(clusterResource, application.getUser(), userLimit, - application, true, requestedNodeLabels)) { + application, true, node.getPartition())) { break; } // Inform the application it is about to get a scheduling opportunity application.addSchedulingOpportunity(priority); + int nonExclusiveSchedulingOpportunity = + application.addNonExclusiveSchedulingOpportunity(priority); + + if (exclusiveType == ExclusiveType.NON_EXECLUSIVE) { + // Before doing allocation, we need to check scheduling opportunity to + // make sure : non-partitioned resource request should be scheduled to + // non-partitioned partition first. + int minimumSchedulingOpportunityForNonExclusive = + scheduler.getNumClusterNodes(); + if (nonExclusiveSchedulingOpportunity < minimumSchedulingOpportunityForNonExclusive) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip app_attempt=" + + application.getApplicationAttemptId() + + " priority=" + + priority + + " for non-exclusive allocation because of schedule opportunity." + + " Now=" + nonExclusiveSchedulingOpportunity + + " required=" + + minimumSchedulingOpportunityForNonExclusive); + } + + break; + } + } // Try to schedule CSAssignment assignment = assignContainersOnNode(clusterResource, node, application, priority, - null); + null, exclusiveType); // Did the application skip this node? if (assignment.getSkipped()) { @@ -870,9 +931,9 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, // Book-keeping // Note: Update headroom to account for current allocation too... allocateResource(clusterResource, application, assigned, - node.getLabels()); + node.getPartition()); - // Don't reset scheduling opportunities for non-local assignments + // Don't reset scheduling opportunities for offswitch assignments // otherwise the app will be delayed for each non-local assignment. // This helps apps with many off-cluster requests schedule faster. if (assignment.getType() != NodeType.OFF_SWITCH) { @@ -881,6 +942,10 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, } application.resetSchedulingOpportunities(priority); } + // Non-exclusive scheduling opportunity is different: we need reset + // it every time to make sure non-labeled resource request will be + // most likely allocated on non-labeled nodes first. + application.resetNonExclusiveSchedulingOpportunities(priority); // Done return assignment; @@ -904,7 +969,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, private synchronized CSAssignment assignReservedContainer( FiCaSchedulerApp application, FiCaSchedulerNode node, - RMContainer rmContainer, Resource clusterResource) { + RMContainer rmContainer, Resource clusterResource, + ExclusiveType exclusiveType) { // Do we still need this reservation? Priority priority = rmContainer.getReservedPriority(); if (application.getTotalRequiredResources(priority) == 0) { @@ -914,7 +980,7 @@ private synchronized CSAssignment assignReservedContainer( // Try to assign if we have sufficient resources assignContainersOnNode(clusterResource, node, application, priority, - rmContainer); + rmContainer, exclusiveType); // Doesn't matter... since it's already charged for at time of reservation // "re-reservation" is *free* @@ -924,7 +990,8 @@ private synchronized CSAssignment assignReservedContainer( protected Resource getHeadroom(User user, Resource queueCurrentLimit, Resource clusterResource, FiCaSchedulerApp application, Resource required) { return getHeadroom(user, queueCurrentLimit, clusterResource, - computeUserLimit(application, clusterResource, required, user, null)); + computeUserLimit(application, clusterResource, required, user, + RMNodeLabelsManager.NO_LABEL, ExclusiveType.EXCLUSIVE)); } private Resource getHeadroom(User user, Resource currentResourceLimit, @@ -968,7 +1035,8 @@ private void setQueueResourceLimitsInfo( @Lock({LeafQueue.class, FiCaSchedulerApp.class}) Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application, - Resource clusterResource, Resource required, Set requestedLabels) { + Resource clusterResource, Resource required, String nodePartition, + ExclusiveType exclusiveType) { String user = application.getUser(); User queueUser = getUser(user); @@ -976,7 +1044,7 @@ Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application, // TODO, need consider headroom respect labels also Resource userLimit = computeUserLimit(application, clusterResource, required, - queueUser, requestedLabels); + queueUser, nodePartition, exclusiveType); setQueueResourceLimitsInfo(clusterResource); @@ -1005,7 +1073,7 @@ Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application, @Lock(NoLock.class) private Resource computeUserLimit(FiCaSchedulerApp application, Resource clusterResource, Resource required, User user, - Set requestedLabels) { + String nodePartition, ExclusiveType exclusiveType) { // What is our current capacity? // * It is equal to the max(required, queue-capacity) if // we're running below capacity. The 'max' ensures that jobs in queues @@ -1013,17 +1081,14 @@ private Resource computeUserLimit(FiCaSchedulerApp application, // * If we're running over capacity, then its // (usedResources + required) (which extra resources we are allocating) Resource queueCapacity = Resource.newInstance(0, 0); - if (requestedLabels != null && !requestedLabels.isEmpty()) { - // if we have multiple labels to request, we will choose to use the first - // label - String firstLabel = requestedLabels.iterator().next(); + if (!nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) { queueCapacity = Resources .max(resourceCalculator, clusterResource, queueCapacity, Resources.multiplyAndNormalizeUp(resourceCalculator, - labelManager.getResourceByLabel(firstLabel, + labelManager.getResourceByLabel(nodePartition, clusterResource), - queueCapacities.getAbsoluteCapacity(firstLabel), + queueCapacities.getAbsoluteCapacity(nodePartition), minimumAllocation)); } else { // else there's no label on request, just to use absolute capacity as @@ -1042,33 +1107,50 @@ private Resource computeUserLimit(FiCaSchedulerApp application, required); Resource currentCapacity = - Resources.lessThan(resourceCalculator, clusterResource, - queueUsage.getUsed(), queueCapacity) ? - queueCapacity : Resources.add(queueUsage.getUsed(), required); + Resources.lessThan(resourceCalculator, clusterResource, + queueUsage.getUsed(nodePartition), queueCapacity) ? queueCapacity + : Resources.add(queueUsage.getUsed(nodePartition), required); // Never allow a single user to take more than the // queue's configured capacity * user-limit-factor. // Also, the queue's configured capacity should be higher than // queue-hard-limit * ulMin - final int activeUsers = activeUsersManager.getNumActiveUsers(); - - Resource limit = + final int activeUsers = activeUsersManager.getNumActiveUsers(); + + // User limit resource is determined by: + // max{currentCapacity / #activeUsers, currentCapacity * user-limit-percentage%) + Resource userLimitResource = Resources.max( + resourceCalculator, clusterResource, + Resources.divideAndCeil( + resourceCalculator, currentCapacity, activeUsers), + Resources.divideAndCeil( + resourceCalculator, + Resources.multiplyAndRoundDown( + currentCapacity, userLimit), + 100) + ); + + // User limit is capped by maxUserLimit + // - maxUserLimit = queueCapacity * user-limit-factor (EXCLUSIVE) + // - maxUserLimit = total-partition-resource (NON-EXCLUSIVE) + Resource maxUserLimit = Resources.none(); + if (exclusiveType == ExclusiveType.EXCLUSIVE) { + maxUserLimit = + Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor); + } else if (exclusiveType == ExclusiveType.NON_EXECLUSIVE) { + maxUserLimit = + labelManager.getResourceByLabel(nodePartition, clusterResource); + } + + // Cap final user limit with maxUserLimit + userLimitResource = Resources.roundUp( resourceCalculator, Resources.min( resourceCalculator, clusterResource, - Resources.max( - resourceCalculator, clusterResource, - Resources.divideAndCeil( - resourceCalculator, currentCapacity, activeUsers), - Resources.divideAndCeil( - resourceCalculator, - Resources.multiplyAndRoundDown( - currentCapacity, userLimit), - 100) - ), - Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor) + userLimitResource, + maxUserLimit ), minimumAllocation); @@ -1076,11 +1158,11 @@ private Resource computeUserLimit(FiCaSchedulerApp application, String userName = application.getUser(); LOG.debug("User limit computation for " + userName + " in queue " + getQueueName() + - " userLimit=" + userLimit + + " userLimitPercent=" + userLimit + " userLimitFactor=" + userLimitFactor + " required: " + required + " consumed: " + user.getUsed() + - " limit: " + limit + + " user-limit-resource: " + userLimitResource + " queueCapacity: " + queueCapacity + " qconsumed: " + queueUsage.getUsed() + " currentCapacity: " + currentCapacity + @@ -1088,31 +1170,26 @@ private Resource computeUserLimit(FiCaSchedulerApp application, " clusterCapacity: " + clusterResource ); } - user.setUserResourceLimit(limit); - return limit; + user.setUserResourceLimit(userLimitResource); + return userLimitResource; } @Private protected synchronized boolean canAssignToUser(Resource clusterResource, String userName, Resource limit, FiCaSchedulerApp application, - boolean checkReservations, Set requestLabels) { + boolean checkReservations, String nodePartition) { User user = getUser(userName); - - String label = CommonNodeLabelsManager.NO_LABEL; - if (requestLabels != null && !requestLabels.isEmpty()) { - label = requestLabels.iterator().next(); - } // Note: We aren't considering the current request since there is a fixed // overhead of the AM, but it's a > check, not a >= check, so... if (Resources .greaterThan(resourceCalculator, clusterResource, - user.getUsed(label), + user.getUsed(nodePartition), limit)) { // if enabled, check to see if could we potentially use this node instead // of a reserved node if the application has reserved containers if (this.reservationsContinueLooking && checkReservations - && label.equals(CommonNodeLabelsManager.NO_LABEL)) { + && nodePartition.equals(CommonNodeLabelsManager.NO_LABEL)) { if (Resources.lessThanOrEqual( resourceCalculator, clusterResource, @@ -1131,7 +1208,7 @@ protected synchronized boolean canAssignToUser(Resource clusterResource, if (LOG.isDebugEnabled()) { LOG.debug("User " + userName + " in queue " + getQueueName() + " will exceed limit - " + " consumed: " - + user.getUsed() + " limit: " + limit); + + user.getUsed(nodePartition) + " limit: " + limit); } return false; } @@ -1171,7 +1248,7 @@ boolean shouldAllocOrReserveNewContainer(FiCaSchedulerApp application, private CSAssignment assignContainersOnNode(Resource clusterResource, FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, - RMContainer reservedContainer) { + RMContainer reservedContainer, ExclusiveType exclusiveType) { Resource assigned = Resources.none(); NodeType requestType = null; @@ -1184,7 +1261,7 @@ private CSAssignment assignContainersOnNode(Resource clusterResource, assigned = assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest, node, application, priority, reservedContainer, - allocatedContainer); + allocatedContainer, exclusiveType); if (Resources.greaterThan(resourceCalculator, clusterResource, assigned, Resources.none())) { @@ -1212,7 +1289,7 @@ private CSAssignment assignContainersOnNode(Resource clusterResource, assigned = assignRackLocalContainers(clusterResource, rackLocalResourceRequest, node, application, priority, reservedContainer, - allocatedContainer); + allocatedContainer, exclusiveType); if (Resources.greaterThan(resourceCalculator, clusterResource, assigned, Resources.none())) { @@ -1240,7 +1317,7 @@ private CSAssignment assignContainersOnNode(Resource clusterResource, assigned = assignOffSwitchContainers(clusterResource, offSwitchResourceRequest, node, application, priority, reservedContainer, - allocatedContainer); + allocatedContainer, exclusiveType); // update locality statistics if (allocatedContainer.getValue() != null) { @@ -1306,16 +1383,17 @@ protected boolean findNodeToUnreserve(Resource clusterResource, @Private protected boolean checkLimitsToReserve(Resource clusterResource, - FiCaSchedulerApp application, Resource capability) { + FiCaSchedulerApp application, Resource capability, String nodePartition, + ExclusiveType exclusiveType) { // we can't reserve if we got here based on the limit // checks assuming we could unreserve!!! Resource userLimit = computeUserLimitAndSetHeadroom(application, - clusterResource, capability, null); + clusterResource, capability, nodePartition, exclusiveType); // Check queue max-capacity limit, // TODO: Consider reservation on labels if (!canAssignToThisQueue(clusterResource, null, - this.currentResourceLimits, capability, Resources.none())) { + this.currentResourceLimits, capability, Resources.none(), exclusiveType)) { if (LOG.isDebugEnabled()) { LOG.debug("was going to reserve but hit queue limit"); } @@ -1324,7 +1402,7 @@ protected boolean checkLimitsToReserve(Resource clusterResource, // Check user limit if (!canAssignToUser(clusterResource, application.getUser(), userLimit, - application, false, null)) { + application, false, nodePartition)) { if (LOG.isDebugEnabled()) { LOG.debug("was going to reserve but hit user limit"); } @@ -1337,12 +1415,13 @@ protected boolean checkLimitsToReserve(Resource clusterResource, private Resource assignNodeLocalContainers(Resource clusterResource, ResourceRequest nodeLocalResourceRequest, FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, - RMContainer reservedContainer, MutableObject allocatedContainer) { + RMContainer reservedContainer, MutableObject allocatedContainer, + ExclusiveType exclusiveType) { if (canAssign(application, priority, node, NodeType.NODE_LOCAL, reservedContainer)) { return assignContainer(clusterResource, node, application, priority, nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer, - allocatedContainer); + allocatedContainer, exclusiveType); } return Resources.none(); @@ -1351,12 +1430,13 @@ private Resource assignNodeLocalContainers(Resource clusterResource, private Resource assignRackLocalContainers(Resource clusterResource, ResourceRequest rackLocalResourceRequest, FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, - RMContainer reservedContainer, MutableObject allocatedContainer) { + RMContainer reservedContainer, MutableObject allocatedContainer, + ExclusiveType exclusiveType) { if (canAssign(application, priority, node, NodeType.RACK_LOCAL, reservedContainer)) { return assignContainer(clusterResource, node, application, priority, rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer, - allocatedContainer); + allocatedContainer, exclusiveType); } return Resources.none(); @@ -1365,16 +1445,21 @@ private Resource assignRackLocalContainers(Resource clusterResource, private Resource assignOffSwitchContainers(Resource clusterResource, ResourceRequest offSwitchResourceRequest, FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, - RMContainer reservedContainer, MutableObject allocatedContainer) { + RMContainer reservedContainer, MutableObject allocatedContainer, + ExclusiveType exclusiveType) { if (canAssign(application, priority, node, NodeType.OFF_SWITCH, reservedContainer)) { return assignContainer(clusterResource, node, application, priority, offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer, - allocatedContainer); + allocatedContainer, exclusiveType); } return Resources.none(); } + + private int getActualNodeLocalityDelay() { + return Math.min(scheduler.getNumClusterNodes(), getNodeLocalityDelay()); + } boolean canAssign(FiCaSchedulerApp application, Priority priority, FiCaSchedulerNode node, NodeType type, RMContainer reservedContainer) { @@ -1409,10 +1494,7 @@ boolean canAssign(FiCaSchedulerApp application, Priority priority, if (type == NodeType.RACK_LOCAL) { // 'Delay' rack-local just a little bit... long missedOpportunities = application.getSchedulingOpportunities(priority); - return ( - Math.min(scheduler.getNumClusterNodes(), getNodeLocalityDelay()) < - missedOpportunities - ); + return getActualNodeLocalityDelay() < missedOpportunities; } // Check if we need containers on this host @@ -1454,7 +1536,7 @@ Container createContainer(FiCaSchedulerApp application, FiCaSchedulerNode node, private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, ResourceRequest request, NodeType type, RMContainer rmContainer, - MutableObject createdContainer) { + MutableObject createdContainer, ExclusiveType exclusiveType) { if (LOG.isDebugEnabled()) { LOG.debug("assignContainers: node=" + node.getNodeName() + " application=" + application.getApplicationId() @@ -1463,9 +1545,8 @@ private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode nod } // check if the resource request can access the label - if (!SchedulerUtils.checkNodeLabelExpression( - node.getLabels(), - request.getNodeLabelExpression())) { + if (!SchedulerUtils.checkNodeLabelExpression(node.getLabels(), + request.getNodeLabelExpression(), exclusiveType)) { // this is a reserved container, but we cannot allocate it now according // to label not match. This can be caused by node label changed // We should un-reserve this container. @@ -1565,7 +1646,7 @@ private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode nod // unreserved for reserving the new one. Check limits again before // reserve the new container if (!checkLimitsToReserve(clusterResource, - application, capability)) { + application, capability, node.getPartition(), exclusiveType)) { return Resources.none(); } } @@ -1648,7 +1729,7 @@ public void completedContainer(Resource clusterResource, // Book-keeping if (removed) { releaseResource(clusterResource, application, - container.getResource(), node.getLabels()); + container.getResource(), node.getPartition()); LOG.info("completedContainer" + " container=" + container + " queue=" + this + @@ -1666,13 +1747,13 @@ public void completedContainer(Resource clusterResource, synchronized void allocateResource(Resource clusterResource, SchedulerApplicationAttempt application, Resource resource, - Set nodeLabels) { - super.allocateResource(clusterResource, resource, nodeLabels); + String nodePartition) { + super.allocateResource(clusterResource, resource, nodePartition); // Update user metrics String userName = application.getUser(); User user = getUser(userName); - user.assignContainer(resource, nodeLabels); + user.assignContainer(resource, nodePartition); // Note this is a bit unconventional since it gets the object and modifies // it here, rather then using set routine Resources.subtractFrom(application.getHeadroom(), resource); // headroom @@ -1689,13 +1770,13 @@ synchronized void allocateResource(Resource clusterResource, } synchronized void releaseResource(Resource clusterResource, - FiCaSchedulerApp application, Resource resource, Set nodeLabels) { - super.releaseResource(clusterResource, resource, nodeLabels); + FiCaSchedulerApp application, Resource resource, String nodePartition) { + super.releaseResource(clusterResource, resource, nodePartition); // Update user metrics String userName = application.getUser(); User user = getUser(userName); - user.releaseContainer(resource, nodeLabels); + user.releaseContainer(resource, nodePartition); metrics.setAvailableResourcesToUser(userName, application.getHeadroom()); LOG.info(getQueueName() + @@ -1705,7 +1786,8 @@ synchronized void releaseResource(Resource clusterResource, private void updateAbsoluteCapacityResource(Resource clusterResource) { absoluteCapacityResource = - Resources.multiplyAndNormalizeUp(resourceCalculator, clusterResource, + Resources.multiplyAndNormalizeUp(resourceCalculator, labelManager + .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource), queueCapacities.getAbsoluteCapacity(), minimumAllocation); } @@ -1751,8 +1833,9 @@ public synchronized void updateClusterResource(Resource clusterResource, // Update application properties for (FiCaSchedulerApp application : activeApplications) { synchronized (application) { - computeUserLimitAndSetHeadroom(application, clusterResource, - Resources.none(), null); + computeUserLimitAndSetHeadroom(application, clusterResource, + Resources.none(), RMNodeLabelsManager.NO_LABEL, + ExclusiveType.EXCLUSIVE); } } } @@ -1810,25 +1893,12 @@ public synchronized void finishApplication(boolean wasActive) { } } - public void assignContainer(Resource resource, - Set nodeLabels) { - if (nodeLabels == null || nodeLabels.isEmpty()) { - userResourceUsage.incUsed(resource); - } else { - for (String label : nodeLabels) { - userResourceUsage.incUsed(label, resource); - } - } + public void assignContainer(Resource resource, String nodePartition) { + userResourceUsage.incUsed(nodePartition, resource); } - public void releaseContainer(Resource resource, Set nodeLabels) { - if (nodeLabels == null || nodeLabels.isEmpty()) { - userResourceUsage.decUsed(resource); - } else { - for (String label : nodeLabels) { - userResourceUsage.decUsed(label, resource); - } - } + public void releaseContainer(Resource resource, String nodePartition) { + userResourceUsage.decUsed(nodePartition, resource); } public Resource getUserResourceLimit() { @@ -1851,7 +1921,7 @@ public void recoverContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); allocateResource(clusterResource, attempt, rmContainer.getContainer() - .getResource(), node.getLabels()); + .getResource(), node.getPartition()); } getParent().recoverContainer(clusterResource, attempt, rmContainer); } @@ -1891,7 +1961,7 @@ public void attachContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); allocateResource(clusterResource, application, rmContainer.getContainer() - .getResource(), node.getLabels()); + .getResource(), node.getPartition()); LOG.info("movedContainer" + " container=" + rmContainer.getContainer() + " resource=" + rmContainer.getContainer().getResource() + " queueMoveIn=" + this + " usedCapacity=" + getUsedCapacity() @@ -1909,7 +1979,7 @@ public void detachContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); releaseResource(clusterResource, application, rmContainer.getContainer() - .getResource(), node.getLabels()); + .getResource(), node.getPartition()); LOG.info("movedContainer" + " container=" + rmContainer.getContainer() + " resource=" + rmContainer.getContainer().getResource() + " queueMoveOut=" + this + " usedCapacity=" + getUsedCapacity() 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 5ed6bb8..bc0f548 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 @@ -376,13 +376,15 @@ private synchronized void removeApplication(ApplicationId applicationId, @Override public synchronized CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, ResourceLimits resourceLimits) { + FiCaSchedulerNode node, ResourceLimits resourceLimits, + ExclusiveType exclusiveType) { CSAssignment assignment = new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL); Set nodeLabels = node.getLabels(); // if our queue cannot access this node, just return - if (!SchedulerUtils.checkQueueAccessToNode(accessibleLabels, nodeLabels)) { + if (exclusiveType == ExclusiveType.EXCLUSIVE + && !SchedulerUtils.checkQueueAccessToNode(accessibleLabels, nodeLabels)) { return assignment; } @@ -395,15 +397,17 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, // Are we over maximum-capacity for this queue? // This will also consider parent's limits and also continuous reservation // looking - if (!super.canAssignToThisQueue(clusterResource, nodeLabels, resourceLimits, - minimumAllocation, Resources.createResource(getMetrics() - .getReservedMB(), getMetrics().getReservedVirtualCores()))) { + if (!super.canAssignToThisQueue(clusterResource, nodeLabels, + resourceLimits, minimumAllocation, Resources.createResource( + getMetrics().getReservedMB(), getMetrics() + .getReservedVirtualCores()), exclusiveType)) { break; } // Schedule - CSAssignment assignedToChild = - assignContainersToChildQueues(clusterResource, node, resourceLimits); + CSAssignment assignedToChild = + assignContainersToChildQueues(clusterResource, node, resourceLimits, + exclusiveType); assignment.setType(assignedToChild.getType()); // Done if no child-queue assigned anything @@ -412,7 +416,7 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, assignedToChild.getResource(), Resources.none())) { // Track resource utilization for the parent-queue super.allocateResource(clusterResource, assignedToChild.getResource(), - nodeLabels); + node.getPartition()); // Track resource utilization in this pass of the scheduler Resources.addTo(assignment.getResource(), assignedToChild.getResource()); @@ -489,7 +493,8 @@ private ResourceLimits getResourceLimitsOfChild(CSQueue child, } private synchronized CSAssignment assignContainersToChildQueues( - Resource cluster, FiCaSchedulerNode node, ResourceLimits limits) { + Resource cluster, FiCaSchedulerNode node, ResourceLimits limits, + ExclusiveType exclusiveType) { CSAssignment assignment = new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL); @@ -503,11 +508,24 @@ private synchronized CSAssignment assignContainersToChildQueues( + " stats: " + childQueue); } + // Check if child need more resource, simply skip child doesn't need more + // resources. + if (!((AbstractCSQueue) childQueue).needResource(node.getPartition(), + cluster, exclusiveType)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip child queue=" + childQueue.getQueuePath() + + ", because it doesn't need more resource, exclusiveType=" + + exclusiveType.name() + " node-label=" + node.getPartition()); + } + continue; + } + // Get ResourceLimits of child queue before assign containers ResourceLimits childLimits = getResourceLimitsOfChild(childQueue, cluster, limits); - assignment = childQueue.assignContainers(cluster, node, childLimits); + assignment = childQueue.assignContainers(cluster, node, + childLimits, exclusiveType); if(LOG.isDebugEnabled()) { LOG.debug("Assigned to queue: " + childQueue.getQueuePath() + " stats: " + childQueue + " --> " + @@ -563,7 +581,7 @@ public void completedContainer(Resource clusterResource, // Book keeping synchronized (this) { super.releaseResource(clusterResource, rmContainer.getContainer() - .getResource(), node.getLabels()); + .getResource(), node.getPartition()); LOG.info("completedContainer" + " queue=" + getQueueName() + @@ -632,7 +650,7 @@ public void recoverContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); super.allocateResource(clusterResource, rmContainer.getContainer() - .getResource(), node.getLabels()); + .getResource(), node.getPartition()); } if (parent != null) { parent.recoverContainer(clusterResource, attempt, rmContainer); @@ -660,7 +678,7 @@ public void attachContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); super.allocateResource(clusterResource, rmContainer.getContainer() - .getResource(), node.getLabels()); + .getResource(), node.getPartition()); LOG.info("movedContainer" + " queueMoveIn=" + getQueueName() + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster=" @@ -680,7 +698,7 @@ public void detachContainer(Resource clusterResource, scheduler.getNode(rmContainer.getContainer().getNodeId()); super.releaseResource(clusterResource, rmContainer.getContainer().getResource(), - node.getLabels()); + node.getPartition()); LOG.info("movedContainer" + " queueMoveOut=" + getQueueName() + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster=" 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 e041389..015bf65 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 @@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; 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.RMContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; @@ -52,6 +53,7 @@ import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ExclusiveType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; /** @@ -69,7 +71,7 @@ private CapacityHeadroomProvider headroomProvider; - public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, + public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, String user, Queue queue, ActiveUsersManager activeUsersManager, RMContext rmContext) { super(applicationAttemptId, user, queue, activeUsersManager, rmContext); @@ -333,5 +335,23 @@ public synchronized void transferStateFromPreviousAttempt( ((FiCaSchedulerApp) appAttempt).getHeadroomProvider(); } - + private boolean needResource(ResourceCalculator calculator, + String nodeLabelToLookAt, Resource cluster) { + if (Resources.greaterThan(calculator, cluster, + attemptResourceUsage.getPending(nodeLabelToLookAt), Resources.none())) { + return true; + } + return false; + } + + @Private + public boolean needResource(ResourceCalculator calculator, + String nodeLabelToLookAt, Resource cluster, ExclusiveType exclusiveType) { + if (exclusiveType == ExclusiveType.EXCLUSIVE) { + return needResource(calculator, nodeLabelToLookAt, cluster); + } else if (exclusiveType == ExclusiveType.NON_EXECLUSIVE) { + return needResource(calculator, RMNodeLabelsManager.NO_LABEL, cluster); + } + return 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/common/fica/FiCaSchedulerNode.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/FiCaSchedulerNode.java index fe6db47..11599a5 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/FiCaSchedulerNode.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/FiCaSchedulerNode.java @@ -35,6 +35,8 @@ private static final Log LOG = LogFactory.getLog(FiCaSchedulerNode.class); + private boolean exclusive = false; + public FiCaSchedulerNode(RMNode node, boolean usePortForNodeName, Set nodeLabels) { super(node, usePortForNodeName, nodeLabels); @@ -115,4 +117,18 @@ public synchronized void unreserveResource( } setReservedContainer(null); } + + /** + * If this is a labeled node, return if this node can be shared to non-labeled + * resource request (Determined by scheduler). + * + * If this is a non-labeled node, always return false. + */ + public boolean exclusive() { + return exclusive; + } + + public void setExclusive(boolean exclusive) { + this.exclusive = exclusive; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java index f62fdb3..5c107aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java @@ -150,8 +150,14 @@ public AllocateResponse allocate( public AllocateResponse allocate( String host, int memory, int numContainers, List releases, String labelExpression) throws Exception { + return allocate(host, memory, numContainers, 1, releases, labelExpression); + } + + public AllocateResponse allocate( + String host, int memory, int numContainers, int priority, + List releases, String labelExpression) throws Exception { List reqs = - createReq(new String[] { host }, memory, 1, numContainers, + createReq(new String[] { host }, memory, priority, numContainers, labelExpression); return allocate(reqs, releases); } 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 06c6b32..6994d36 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 @@ -21,6 +21,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.security.PrivilegedAction; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; @@ -200,10 +202,18 @@ public boolean waitForState(MockNM nm, ContainerId containerId, public boolean waitForState(MockNM nm, ContainerId containerId, RMContainerState containerState, int timeoutMillisecs) throws Exception { + return waitForState(Arrays.asList(nm), containerId, containerState, + timeoutMillisecs); + } + + public boolean waitForState(Collection nms, ContainerId containerId, + RMContainerState containerState, int timeoutMillisecs) throws Exception { RMContainer container = getResourceScheduler().getRMContainer(containerId); int timeoutSecs = 0; while(container == null && timeoutSecs++ < timeoutMillisecs / 100) { - nm.nodeHeartbeat(true); + for (MockNM nm : nms) { + nm.nodeHeartbeat(true); + } container = getResourceScheduler().getRMContainer(containerId); System.out.println("Waiting for container " + containerId + " to be allocated."); Thread.sleep(100); @@ -217,9 +227,11 @@ public boolean waitForState(MockNM nm, ContainerId containerId, && timeoutSecs++ < timeoutMillisecs / 100) { System.out.println("Container : " + containerId + " State is : " + container.getState() + " Waiting for state : " + containerState); - nm.nodeHeartbeat(true); + for (MockNM nm : nms) { + nm.nodeHeartbeat(true); + } Thread.sleep(100); - + if (timeoutMillisecs <= timeoutSecs * 100) { return false; } 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/TestApplicationLimits.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/TestApplicationLimits.java index 1ca5c97..dc5cb71 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/TestApplicationLimits.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/TestApplicationLimits.java @@ -612,7 +612,7 @@ public void testHeadroom() throws Exception { // Schedule to compute queue.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); Resource expectedHeadroom = Resources.createResource(10*16*GB, 1); assertEquals(expectedHeadroom, app_0_0.getHeadroom()); @@ -632,7 +632,7 @@ public void testHeadroom() throws Exception { // Schedule to compute queue.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); // Schedule to compute + clusterResource), ExclusiveType.EXCLUSIVE); // Schedule to compute assertEquals(expectedHeadroom, app_0_0.getHeadroom()); assertEquals(expectedHeadroom, app_0_1.getHeadroom());// no change @@ -652,7 +652,7 @@ public void testHeadroom() throws Exception { // Schedule to compute queue.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); // Schedule to compute + clusterResource), ExclusiveType.EXCLUSIVE); // Schedule to compute expectedHeadroom = Resources.createResource(10*16*GB / 2, 1); // changes assertEquals(expectedHeadroom, app_0_0.getHeadroom()); assertEquals(expectedHeadroom, app_0_1.getHeadroom()); @@ -661,7 +661,7 @@ public void testHeadroom() throws Exception { // Now reduce cluster size and check for the smaller headroom clusterResource = Resources.createResource(90*16*GB); queue.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); // Schedule to compute + clusterResource), ExclusiveType.EXCLUSIVE); // Schedule to compute expectedHeadroom = Resources.createResource(9*16*GB / 2, 1); // changes assertEquals(expectedHeadroom, app_0_0.getHeadroom()); assertEquals(expectedHeadroom, app_0_1.getHeadroom()); 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 23b31fa..a62489e 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 @@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; @@ -133,7 +134,7 @@ public CSAssignment answer(InvocationOnMock invocation) throws Throwable { final Resource allocatedResource = Resources.createResource(allocation); if (queue instanceof ParentQueue) { ((ParentQueue)queue).allocateResource(clusterResource, - allocatedResource, null); + allocatedResource, RMNodeLabelsManager.NO_LABEL); } else { FiCaSchedulerApp app1 = getMockApplication(0, ""); ((LeafQueue)queue).allocateResource(clusterResource, app1, @@ -145,7 +146,7 @@ public CSAssignment answer(InvocationOnMock invocation) throws Throwable { doReturn(new CSAssignment(Resources.none(), type)). when(queue) .assignContainers(eq(clusterResource), eq(node), - any(ResourceLimits.class)); + any(ResourceLimits.class), any(ExclusiveType.class)); // Mock the node's resource availability Resource available = node.getAvailableResource(); @@ -157,7 +158,7 @@ public CSAssignment answer(InvocationOnMock invocation) throws Throwable { } }). when(queue).assignContainers(eq(clusterResource), eq(node), - any(ResourceLimits.class)); + any(ResourceLimits.class), any(ExclusiveType.class)); doNothing().when(node).releaseContainer(any(Container.class)); } @@ -241,6 +242,12 @@ public void testSortedQueues() throws Exception { CSQueue b = queues.get(B); CSQueue c = queues.get(C); CSQueue d = queues.get(D); + + // Make a/b/c/d has >0 pending resource, so that allocation will continue. + a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); + b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); + c.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); + d.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); final String user_0 = "user_0"; @@ -275,7 +282,7 @@ public void testSortedQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); for(int i=0; i < 2; i++) { stubQueueAllocation(a, clusterResource, node_0, 0*GB); @@ -283,7 +290,7 @@ public void testSortedQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); } for(int i=0; i < 3; i++) { @@ -292,7 +299,7 @@ public void testSortedQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 1*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); } for(int i=0; i < 4; i++) { @@ -301,7 +308,7 @@ public void testSortedQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 1*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); } verifyQueueMetrics(a, 1*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); @@ -335,7 +342,7 @@ public void testSortedQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); } verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); @@ -363,7 +370,7 @@ public void testSortedQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(b, 3*GB, clusterResource); verifyQueueMetrics(c, 3*GB, clusterResource); @@ -390,7 +397,7 @@ public void testSortedQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); verifyQueueMetrics(c, 3*GB, clusterResource); @@ -405,12 +412,14 @@ public void testSortedQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 0*GB); stubQueueAllocation(d, clusterResource, node_0, 1*GB); root.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); InOrder allocationOrder = inOrder(d,b); - allocationOrder.verify(d).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), any(ResourceLimits.class)); - allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), any(ResourceLimits.class)); + allocationOrder.verify(d).assignContainers(eq(clusterResource), + any(FiCaSchedulerNode.class), any(ResourceLimits.class), + any(ExclusiveType.class)); + allocationOrder.verify(b).assignContainers(eq(clusterResource), + any(FiCaSchedulerNode.class), any(ResourceLimits.class), + any(ExclusiveType.class)); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); verifyQueueMetrics(c, 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/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 03b8f5c..e2ea3dd 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 @@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Set; @@ -32,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.LogAggregationContext; 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; @@ -51,9 +54,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; 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.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.junit.Assert; @@ -710,4 +717,410 @@ public RMNodeLabelsManager createNodeLabelManager() { rm1.close(); } + + private void checkPendingResource(MockRM rm, int priority, + ApplicationAttemptId attemptId, int memory) { + CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler(); + FiCaSchedulerApp app = cs.getApplicationAttempt(attemptId); + ResourceRequest rr = + app.getAppSchedulingInfo().getResourceRequest( + Priority.newInstance(priority), "*"); + Assert.assertEquals(memory, + rr.getCapability().getMemory() * rr.getNumContainers()); + } + + private void checkLaunchedContainerNumOnNode(MockRM rm, NodeId nodeId, + int numContainers) { + CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler(); + SchedulerNode node = cs.getSchedulerNode(nodeId); + Assert.assertEquals(numContainers, node.getNumContainers()); + } + + @Test + public void testNonExclusiveNodeLabelsAllocationIgnoreAppSubmitOrder() throws Exception { + /** + * Test case: Submit two application to a queue (app1 first then app2), app1 + * asked for no-label, app2 asked for label=x, when node1 has label=x + * doing heart beat, app2 will get allocation first, even if app2 submits later + * than app1 + */ + + // set node -> label + mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y")); + // Makes y to be non-exclusive node labels + mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("y", false))); + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("y"))); + + // inject node label manager + MockRM rm1 = new MockRM(TestUtils.getConfigurationWithQueueLabels(conf)) { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); // label = y + MockNM nm2 = rm1.registerNode("h2:1234", 100 * GB); // label = + + ContainerId nextContainerId; + + // launch an app to queue b1 (label = y), AM container should be launched in nm3 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "b1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2); + + // launch another app to queue b1 (label = y), AM container should be launched in nm3 + RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "b1"); + MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2); + + // request container and nm1 do heartbeat (nm2 has label=y), note that app1 + // request non-labeled container, and app2 request labeled container, app2 + // will get allocated first even if app1 submitted first. + am1.allocate("*", 1 * GB, 1, new ArrayList()); + am2.allocate("*", 1 * GB, 1, new ArrayList(), "y"); + nextContainerId = + ContainerId.newContainerId(am2.getApplicationAttemptId(), 2); + Assert.assertTrue(rm1.waitForState(nm1, nextContainerId, + RMContainerState.ALLOCATED, 10 * 1000)); + nextContainerId = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue(rm1.waitForState(nm1, nextContainerId, + RMContainerState.ALLOCATED, 10 * 1000)); + + rm1.close(); + } + + @Test + public void testNonExclusiveNodeLabelsAllocationIgnorePriority() throws Exception { + /** + * Test case: Submit one application, it asks label="" in priority=1 and + * label="x" in priority=2, when a node with label=x heartbeat, priority=2 + * will get allocation first even if there're pending resource in priority=1 + */ + + // set node -> label + mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y")); + // Makes y to be non-exclusive node labels + mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("y", false))); + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("y"))); + + // inject node label manager + MockRM rm1 = new MockRM(TestUtils.getConfigurationWithQueueLabels(conf)) { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); // label = y + MockNM nm2 = rm1.registerNode("h2:1234", 100 * GB); // label = + + ContainerId nextContainerId; + + // launch an app to queue b1 (label = y), AM container should be launched in nm3 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "b1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2); + + // request containers from am2, priority=1 asks for "" and priority=2 asks + // for "y", "y" container should be allocated first + nextContainerId = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + am1.allocate("*", 1 * GB, 1, 1, new ArrayList(), ""); + am1.allocate("*", 1 * GB, 1, 2, new ArrayList(), "y"); + Assert.assertTrue(rm1.waitForState(nm1, nextContainerId, + RMContainerState.ALLOCATED, 10 * 1000)); + + // Check pending resource for am2, priority=1 doesn't get allocated before + // priority=2 allocated + checkPendingResource(rm1, 1, am1.getApplicationAttemptId(), 1 * GB); + checkPendingResource(rm1, 2, am1.getApplicationAttemptId(), 0 * GB); + + rm1.close(); + } + + @Test + public void testNonLabeledResourceRequestGetPreferrenceToNonLabeledNode() + throws Exception { + /** + * Test case: Submit one application, it asks 6 label="" containers, NM1 + * with label=y and NM2 has no label, NM1/NM2 doing heartbeat together. Even + * if NM1 has idle resource, containers are all allocated to NM2 since + * non-labeled request should get allocation on non-labeled nodes first. + */ + + // set node -> label + mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y")); + // Makes x to be non-exclusive node labels + mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false))); + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"))); + + // inject node label manager + MockRM rm1 = new MockRM(TestUtils.getConfigurationWithQueueLabels(conf)) { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); // label = y + MockNM nm2 = rm1.registerNode("h2:1234", 100 * GB); // label = + + ContainerId nextContainerId; + + // launch an app to queue b1 (label = y), AM container should be launched in nm3 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "b1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2); + + // request containers from am2, priority=1 asks for "" * 6 (id from 4 to 9), + // nm2/nm3 do + // heartbeat at the same time, check containers are always allocated to nm3. + // This is to verify when there's resource available in non-labeled + // partition, non-labeled resource should allocate to non-labeled partition + // first. + am1.allocate("*", 1 * GB, 6, 1, new ArrayList(), ""); + for (int i = 2; i < 2 + 6; i++) { + nextContainerId = + ContainerId.newContainerId(am1.getApplicationAttemptId(), i); + Assert.assertTrue(rm1.waitForState(Arrays.asList(nm1, nm2), + nextContainerId, RMContainerState.ALLOCATED, 10 * 1000)); + } + // no more container allocated on nm1 + checkLaunchedContainerNumOnNode(rm1, nm1.getNodeId(), 0); + // all 7 (1 AM container + 6 task container) containers allocated on nm2 + checkLaunchedContainerNumOnNode(rm1, nm2.getNodeId(), 7); + + rm1.close(); + } + + @Test + public void testLabeledResourceRequestsGetPreferrenceInHierarchyOfQueue() + throws Exception { + /** + * Test case: have a following queue structure: + * + *
+     *            root
+     *         /   |   \
+     *        a     b    c
+     *       / \   / \  /  \
+     *      a1 a2 b1 b2 c1 c2
+     *     (x)    (x)   (x)
+     * 
+ * + * Only a1, b1, c1 can access label=x, and their default label=x Each each + * has one application, asks for 5 containers. NM1 has label=x + * + * NM1/NM2 doing heartbeat for 15 times, it should allocate all 15 + * containers with label=x + */ + + CapacitySchedulerConfiguration csConf = + new CapacitySchedulerConfiguration(this.conf); + + // Define top-level queues + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"}); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + csConf.setCapacity(A, 33); + csConf.setAccessibleNodeLabels(A, toSet("x")); + csConf.setCapacityByLabel(A, "x", 33); + csConf.setQueues(A, new String[] {"a1", "a2"}); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + csConf.setCapacity(B, 33); + csConf.setAccessibleNodeLabels(B, toSet("x")); + csConf.setCapacityByLabel(B, "x", 33); + csConf.setQueues(B, new String[] {"b1", "b2"}); + + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; + csConf.setCapacity(C, 34); + csConf.setAccessibleNodeLabels(C, toSet("x")); + csConf.setCapacityByLabel(C, "x", 34); + csConf.setQueues(C, new String[] {"c1", "c2"}); + + // Define 2nd-level queues + final String A1 = A + ".a1"; + csConf.setCapacity(A1, 50); + csConf.setCapacityByLabel(A1, "x", 100); + csConf.setDefaultNodeLabelExpression(A1, "x"); + + final String A2 = A + ".a2"; + csConf.setCapacity(A2, 50); + csConf.setCapacityByLabel(A2, "x", 0); + + final String B1 = B + ".b1"; + csConf.setCapacity(B1, 50); + csConf.setCapacityByLabel(B1, "x", 100); + csConf.setDefaultNodeLabelExpression(B1, "x"); + + final String B2 = B + ".b2"; + csConf.setCapacity(B2, 50); + csConf.setCapacityByLabel(B2, "x", 0); + + final String C1 = C + ".c1"; + csConf.setCapacity(C1, 50); + csConf.setCapacityByLabel(C1, "x", 100); + csConf.setDefaultNodeLabelExpression(C1, "x"); + + final String C2 = C + ".c2"; + csConf.setCapacity(C2, 50); + csConf.setCapacityByLabel(C2, "x", 0); + + // set node -> label + mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y")); + // Makes x to be non-exclusive node labels + mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false))); + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"))); + + // inject node label manager + MockRM rm1 = new MockRM(csConf) { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 20 * GB); // label = x + MockNM nm2 = rm1.registerNode("h2:1234", 100 * GB); // label = + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + // app2 -> a2 + RMApp app2 = rm1.submitApp(1 * GB, "app", "user", null, "a2"); + MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2); + + // app3 -> b1 + RMApp app3 = rm1.submitApp(1 * GB, "app", "user", null, "b1"); + MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm1); + + // app4 -> b2 + RMApp app4 = rm1.submitApp(1 * GB, "app", "user", null, "b2"); + MockAM am4 = MockRM.launchAndRegisterAM(app4, rm1, nm2); + + // app5 -> c1 + RMApp app5 = rm1.submitApp(1 * GB, "app", "user", null, "c1"); + MockAM am5 = MockRM.launchAndRegisterAM(app5, rm1, nm1); + + // app6 -> b2 + RMApp app6 = rm1.submitApp(1 * GB, "app", "user", null, "c2"); + MockAM am6 = MockRM.launchAndRegisterAM(app6, rm1, nm2); + + // Each application request 5 * 1GB container + am1.allocate("*", 1 * GB, 5, new ArrayList()); + am2.allocate("*", 1 * GB, 5, new ArrayList()); + am3.allocate("*", 1 * GB, 5, new ArrayList()); + am4.allocate("*", 1 * GB, 5, new ArrayList()); + am5.allocate("*", 1 * GB, 5, new ArrayList()); + am6.allocate("*", 1 * GB, 5, new ArrayList()); + + // NM1 do 15 heartbeats + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + for (int i = 0; i < 15; i++) { + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + } + + // NM1 get 15 new containers (total is 18, 15 task containers and 3 AM + // containers) + checkLaunchedContainerNumOnNode(rm1, nm1.getNodeId(), 18); + + // Check pending resource each application + // APP1/APP3/APP5 get satisfied, and APP2/APP2/APP3 get nothing. + checkPendingResource(rm1, 1, am1.getApplicationAttemptId(), 0 * GB); + checkPendingResource(rm1, 1, am2.getApplicationAttemptId(), 5 * GB); + checkPendingResource(rm1, 1, am3.getApplicationAttemptId(), 0 * GB); + checkPendingResource(rm1, 1, am4.getApplicationAttemptId(), 5 * GB); + checkPendingResource(rm1, 1, am5.getApplicationAttemptId(), 0 * GB); + checkPendingResource(rm1, 1, am6.getApplicationAttemptId(), 5 * GB); + + rm1.close(); + } + + @Test + public void testNonLabeledQueueUsesLabeledResource() throws Exception { + /** + * Test case: have a following queue structure: + * + *
+     *            root
+     *         /      \
+     *        a        b
+     *        (x)
+     * 
+ * + * Only a can access label=x, two nodes in the cluster, n1 has x and n2 has + * no-label. + * + * When user-limit-factor=5, submit one application in queue b and request + * for infinite containers should be able to use up all cluster resources. + */ + + CapacitySchedulerConfiguration csConf = + new CapacitySchedulerConfiguration(this.conf); + + // Define top-level queues + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + csConf.setCapacity(A, 50); + csConf.setAccessibleNodeLabels(A, toSet("x")); + csConf.setCapacityByLabel(A, "x", 100); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + csConf.setCapacity(B, 50); + csConf.setAccessibleNodeLabels(B, new HashSet()); + csConf.setUserLimitFactor(B, 5); + + // set node -> label + mgr.addToCluserNodeLabels(ImmutableSet.of("x")); + // Makes x to be non-exclusive node labels + mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false))); + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"))); + + // inject node label manager + MockRM rm1 = new MockRM(csConf) { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 10 * GB); // label = x + MockNM nm2 = rm1.registerNode("h2:1234", 10 * GB); // label = + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "b"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm2); + + // Each application request 5 * 1GB container + am1.allocate("*", 1 * GB, 50, new ArrayList()); + + // NM1 do 50 heartbeats + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + RMNode rmNode2 = rm1.getRMContext().getRMNodes().get(nm2.getNodeId()); + for (int i = 0; i < 50; i++) { + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + cs.handle(new NodeUpdateSchedulerEvent(rmNode2)); + } + + // Both NM1/NM2 launched 10 containers, cluster resource is exhausted + checkLaunchedContainerNumOnNode(rm1, nm1.getNodeId(), 10); + checkLaunchedContainerNumOnNode(rm1, nm2.getNodeId(), 10); + + 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/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 972cabb..d3a718f 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 @@ -351,7 +351,7 @@ public void testSingleQueueOneUserMetrics() throws Exception { // Only 1 container a.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); assertEquals( (int)(node_0.getTotalResource().getMemory() * a.getCapacity()) - (1*GB), a.getMetrics().getAvailableMB()); @@ -487,7 +487,7 @@ public void testSingleQueueWithOneUser() throws Exception { // Only 1 container a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -498,7 +498,7 @@ public void testSingleQueueWithOneUser() throws Exception { // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -507,7 +507,7 @@ public void testSingleQueueWithOneUser() throws Exception { // Can't allocate 3rd due to user-limit a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -517,7 +517,7 @@ public void testSingleQueueWithOneUser() throws Exception { // Bump up user-limit-factor, now allocate should work a.setUserLimitFactor(10); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -526,7 +526,7 @@ public void testSingleQueueWithOneUser() throws Exception { // One more should work, for app_1, due to user-limit-factor a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); @@ -537,7 +537,7 @@ public void testSingleQueueWithOneUser() throws Exception { // Now - no more allocs since we are at max-cap a.setMaxCapacity(0.5f); a.assignContainers(clusterResource, node_0, new ResourceLimits( - clusterResource)); + clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); @@ -653,21 +653,21 @@ public void testUserLimits() throws Exception { // 1 container to user_0 a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); // Again one to user_0 since he hasn't exceeded user limit yet a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); // One more to user_0 since he is the only active user a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(2*GB, app_1.getCurrentConsumption().getMemory()); @@ -719,10 +719,10 @@ public void testComputeUserLimitAndSetHeadroom(){ 1, qb.getActiveUsersManager().getNumActiveUsers()); //get headroom qb.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0 .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(), - null); + "", ExclusiveType.EXCLUSIVE); //maxqueue 16G, userlimit 13G, - 4G used = 9G assertEquals(9*GB,app_0.getHeadroom().getMemory()); @@ -739,10 +739,10 @@ public void testComputeUserLimitAndSetHeadroom(){ u1Priority, recordFactory))); qb.submitApplicationAttempt(app_2, user_1); qb.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0 .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(), - null); + "", ExclusiveType.EXCLUSIVE); assertEquals(8*GB, qb.getUsedResources().getMemory()); assertEquals(4*GB, app_0.getCurrentConsumption().getMemory()); @@ -782,12 +782,12 @@ public void testComputeUserLimitAndSetHeadroom(){ qb.submitApplicationAttempt(app_1, user_0); qb.submitApplicationAttempt(app_3, user_1); qb.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); qb.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3 .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(), - null); + "", ExclusiveType.EXCLUSIVE); assertEquals(4*GB, qb.getUsedResources().getMemory()); //maxqueue 16G, userlimit 7G, used (by each user) 2G, headroom 5G (both) assertEquals(5*GB, app_3.getHeadroom().getMemory()); @@ -803,13 +803,13 @@ public void testComputeUserLimitAndSetHeadroom(){ TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true, u0Priority, recordFactory))); qb.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); qb.computeUserLimitAndSetHeadroom(app_4, clusterResource, app_4 .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(), - null); + "", ExclusiveType.EXCLUSIVE); qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3 .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(), - null); + "", ExclusiveType.EXCLUSIVE); //app3 is user1, active from last test case @@ -876,7 +876,7 @@ public void testUserHeadroomMultiApp() throws Exception { priority, recordFactory))); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -893,7 +893,7 @@ public void testUserHeadroomMultiApp() throws Exception { priority, recordFactory))); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); @@ -982,7 +982,7 @@ public void testHeadroomWithMaxCap() throws Exception { // 1 container to user_0 a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -993,7 +993,7 @@ public void testHeadroomWithMaxCap() throws Exception { // Again one to user_0 since he hasn't exceeded user limit yet a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); @@ -1010,7 +1010,7 @@ public void testHeadroomWithMaxCap() throws Exception { // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(3*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); @@ -1024,7 +1024,7 @@ public void testHeadroomWithMaxCap() throws Exception { priority, recordFactory))); assertEquals(1, a.getActiveUsersManager().getNumActiveUsers()); a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(0*GB, app_2.getHeadroom().getMemory()); // hit queue max-cap } @@ -1095,7 +1095,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // Only 1 container a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1103,7 +1103,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1111,7 +1111,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // Can't allocate 3rd due to user-limit a.setUserLimit(25); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1130,7 +1130,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // user_0 is at limit inspite of high user-limit-factor a.setUserLimitFactor(10); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1140,7 +1140,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // Now allocations should goto app_0 since // user_0 is at user-limit not above it a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1151,7 +1151,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // Now - no more allocs since we are at max-cap a.setMaxCapacity(0.5f); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1163,7 +1163,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { a.setMaxCapacity(1.0f); a.setUserLimitFactor(1); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(7*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1172,7 +1172,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception { // Now we should assign to app_3 again since user_2 is under user-limit a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8*GB, a.getUsedResources().getMemory()); assertEquals(3*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1272,7 +1272,7 @@ public void testReservation() throws Exception { // Only 1 container a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1283,7 +1283,7 @@ public void testReservation() throws Exception { // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1292,7 +1292,7 @@ public void testReservation() throws Exception { // Now, reservation should kick in for app_1 a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1309,7 +1309,7 @@ public void testReservation() throws Exception { ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1326,7 +1326,7 @@ public void testReservation() throws Exception { ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(4*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); @@ -1394,7 +1394,7 @@ public void testStolenReservedContainer() throws Exception { // Start testing... a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1404,7 +1404,7 @@ public void testStolenReservedContainer() throws Exception { // Now, reservation should kick in for app_1 a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1418,7 +1418,7 @@ public void testStolenReservedContainer() throws Exception { doReturn(-1).when(a).getNodeLocalityDelay(); a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(10*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); @@ -1435,7 +1435,7 @@ public void testStolenReservedContainer() throws Exception { ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8*GB, app_1.getCurrentConsumption().getMemory()); @@ -1504,7 +1504,7 @@ public void testReservationExchange() throws Exception { // Only 1 container a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(1*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1512,14 +1512,14 @@ public void testReservationExchange() throws Exception { // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); // Now, reservation should kick in for app_1 a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(6*GB, a.getUsedResources().getMemory()); assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1534,7 +1534,7 @@ public void testReservationExchange() throws Exception { ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1544,7 +1544,7 @@ public void testReservationExchange() throws Exception { // Re-reserve a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); @@ -1554,7 +1554,7 @@ public void testReservationExchange() throws Exception { // Try to schedule on node_1 now, should *move* the reservation a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(9*GB, a.getUsedResources().getMemory()); assertEquals(1*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); @@ -1572,7 +1572,7 @@ public void testReservationExchange() throws Exception { ContainerExitStatus.KILLED_BY_RESOURCEMANAGER), RMContainerEventType.KILL, null, true); CSAssignment assignment = a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8*GB, a.getUsedResources().getMemory()); assertEquals(0*GB, app_0.getCurrentConsumption().getMemory()); assertEquals(4*GB, app_1.getCurrentConsumption().getMemory()); @@ -1644,7 +1644,7 @@ public void testLocalityScheduling() throws Exception { // Start with off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(1, app_0.getSchedulingOpportunities(priority)); @@ -1653,7 +1653,7 @@ public void testLocalityScheduling() throws Exception { // Another off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(2, app_0.getSchedulingOpportunities(priority)); @@ -1662,7 +1662,7 @@ public void testLocalityScheduling() throws Exception { // Another off switch, shouldn't allocate due to delay scheduling assignment = a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(3, app_0.getSchedulingOpportunities(priority)); @@ -1672,7 +1672,7 @@ public void testLocalityScheduling() throws Exception { // Another off switch, now we should allocate // since missedOpportunities=3 and reqdContainers=3 assignment = a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(4, app_0.getSchedulingOpportunities(priority)); // should NOT reset @@ -1681,7 +1681,7 @@ public void testLocalityScheduling() throws Exception { // NODE_LOCAL - node_0 assignment = a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset @@ -1690,7 +1690,7 @@ public void testLocalityScheduling() throws Exception { // NODE_LOCAL - node_1 assignment = a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset @@ -1719,14 +1719,14 @@ public void testLocalityScheduling() throws Exception { // Shouldn't assign RACK_LOCAL yet assignment = a.assignContainers(clusterResource, node_3, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(1, app_0.getSchedulingOpportunities(priority)); assertEquals(2, app_0.getTotalRequiredResources(priority)); assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL // Should assign RACK_LOCAL now assignment = a.assignContainers(clusterResource, node_3, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.RACK_LOCAL), eq(node_3), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset @@ -1808,7 +1808,7 @@ public void testApplicationPriorityScheduling() throws Exception { // Start with off switch, shouldn't allocate P1 due to delay scheduling // thus, no P2 either! a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), eq(priority_1), any(ResourceRequest.class), any(Container.class)); assertEquals(1, app_0.getSchedulingOpportunities(priority_1)); @@ -1821,7 +1821,7 @@ public void testApplicationPriorityScheduling() throws Exception { // Another off-switch, shouldn't allocate P1 due to delay scheduling // thus, no P2 either! a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), eq(priority_1), any(ResourceRequest.class), any(Container.class)); assertEquals(2, app_0.getSchedulingOpportunities(priority_1)); @@ -1833,7 +1833,7 @@ public void testApplicationPriorityScheduling() throws Exception { // Another off-switch, shouldn't allocate OFF_SWITCH P1 a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2), eq(priority_1), any(ResourceRequest.class), any(Container.class)); assertEquals(3, app_0.getSchedulingOpportunities(priority_1)); @@ -1845,7 +1845,7 @@ public void testApplicationPriorityScheduling() throws Exception { // Now, DATA_LOCAL for P1 a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0), eq(priority_1), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority_1)); @@ -1857,7 +1857,7 @@ public void testApplicationPriorityScheduling() throws Exception { // Now, OFF_SWITCH for P2 a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_1), eq(priority_1), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority_1)); @@ -1934,7 +1934,7 @@ public void testSchedulingConstraints() throws Exception { // NODE_LOCAL - node_0_1 a.assignContainers(clusterResource, node_0_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0_0), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset @@ -1943,7 +1943,7 @@ public void testSchedulingConstraints() throws Exception { // No allocation on node_1_0 even though it's node/rack local since // required(ANY) == 0 a.assignContainers(clusterResource, node_1_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // Still zero @@ -1960,7 +1960,7 @@ public void testSchedulingConstraints() throws Exception { // No allocation on node_0_1 even though it's node/rack local since // required(rack_1) == 0 a.assignContainers(clusterResource, node_0_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(1, app_0.getSchedulingOpportunities(priority)); @@ -1968,7 +1968,7 @@ public void testSchedulingConstraints() throws Exception { // NODE_LOCAL - node_1 a.assignContainers(clusterResource, node_1_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset @@ -2221,7 +2221,7 @@ public void testLocalityConstraints() throws Exception { // node_0_1 // Shouldn't allocate since RR(rack_0) = null && RR(ANY) = relax: false a.assignContainers(clusterResource, node_0_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 @@ -2244,7 +2244,7 @@ public void testLocalityConstraints() throws Exception { // node_1_1 // Shouldn't allocate since RR(rack_1) = relax: false a.assignContainers(clusterResource, node_1_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 @@ -2275,7 +2275,7 @@ public void testLocalityConstraints() throws Exception { // node_1_1 // Shouldn't allocate since node_1_1 is blacklisted a.assignContainers(clusterResource, node_1_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 @@ -2304,7 +2304,7 @@ public void testLocalityConstraints() throws Exception { // node_1_1 // Shouldn't allocate since rack_1 is blacklisted a.assignContainers(clusterResource, node_1_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should be 0 @@ -2331,7 +2331,7 @@ public void testLocalityConstraints() throws Exception { // Now, should allocate since RR(rack_1) = relax: true a.assignContainers(clusterResource, node_1_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0,never()).allocate(eq(NodeType.RACK_LOCAL), eq(node_1_1), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); @@ -2362,7 +2362,7 @@ public void testLocalityConstraints() throws Exception { // host_1_1: 7G a.assignContainers(clusterResource, node_1_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), any(Priority.class), any(ResourceRequest.class), any(Container.class)); assertEquals(0, app_0.getSchedulingOpportunities(priority)); @@ -2445,7 +2445,7 @@ public void testAllocateContainerOnNodeWithoutOffSwitchSpecified() try { a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); } catch (NullPointerException e) { Assert.fail("NPE when allocating container on node but " + "forget to set off-switch request should be handled"); 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 7da1c97..8bcd025 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 @@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.security.YarnAuthorizationProvider; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; 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.common.fica.FiCaSchedulerApp; @@ -146,7 +147,7 @@ public CSAssignment answer(InvocationOnMock invocation) throws Throwable { final Resource allocatedResource = Resources.createResource(allocation); if (queue instanceof ParentQueue) { ((ParentQueue)queue).allocateResource(clusterResource, - allocatedResource, null); + allocatedResource, RMNodeLabelsManager.NO_LABEL); } else { FiCaSchedulerApp app1 = getMockApplication(0, ""); ((LeafQueue)queue).allocateResource(clusterResource, app1, @@ -157,7 +158,7 @@ public CSAssignment answer(InvocationOnMock invocation) throws Throwable { if (allocation > 0) { doReturn(new CSAssignment(Resources.none(), type)).when(queue) .assignContainers(eq(clusterResource), eq(node), - any(ResourceLimits.class)); + any(ResourceLimits.class), any(ExclusiveType.class)); // Mock the node's resource availability Resource available = node.getAvailableResource(); @@ -168,7 +169,7 @@ public CSAssignment answer(InvocationOnMock invocation) throws Throwable { return new CSAssignment(allocatedResource, type); } }).when(queue).assignContainers(eq(clusterResource), eq(node), - any(ResourceLimits.class)); + any(ResourceLimits.class), any(ExclusiveType.class)); } private float computeQueueAbsoluteUsedCapacity(CSQueue queue, @@ -228,11 +229,14 @@ public void testSingleLevelQueues() throws Exception { LeafQueue a = (LeafQueue)queues.get(A); LeafQueue b = (LeafQueue)queues.get(B); + a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); + b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); + // Simulate B returning a container on node_0 stubQueueAllocation(a, clusterResource, node_0, 0*GB); stubQueueAllocation(b, clusterResource, node_0, 1*GB); root.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(b, 1*GB, clusterResource); @@ -240,12 +244,12 @@ public void testSingleLevelQueues() throws Exception { stubQueueAllocation(a, clusterResource, node_1, 2*GB); stubQueueAllocation(b, clusterResource, node_1, 1*GB); root.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); InOrder allocationOrder = inOrder(a, b); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); @@ -254,12 +258,12 @@ public void testSingleLevelQueues() throws Exception { stubQueueAllocation(a, clusterResource, node_0, 1*GB); stubQueueAllocation(b, clusterResource, node_0, 2*GB); root.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); allocationOrder = inOrder(b, a); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource); @@ -268,12 +272,12 @@ public void testSingleLevelQueues() throws Exception { stubQueueAllocation(a, clusterResource, node_0, 0*GB); stubQueueAllocation(b, clusterResource, node_0, 4*GB); root.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); allocationOrder = inOrder(b, a); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 8*GB, clusterResource); @@ -282,12 +286,12 @@ public void testSingleLevelQueues() throws Exception { stubQueueAllocation(a, clusterResource, node_1, 1*GB); stubQueueAllocation(b, clusterResource, node_1, 1*GB); root.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); allocationOrder = inOrder(a, b); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(a, 4*GB, clusterResource); verifyQueueMetrics(b, 9*GB, clusterResource); } @@ -448,16 +452,25 @@ public void testMultiLevelQueues() throws Exception { // Start testing CSQueue a = queues.get(A); + a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); CSQueue b = queues.get(B); + b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); CSQueue c = queues.get(C); + c.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); CSQueue d = queues.get(D); + d.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); CSQueue a1 = queues.get(A1); + a1.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); CSQueue a2 = queues.get(A2); + a2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); CSQueue b1 = queues.get(B1); + b1.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); CSQueue b2 = queues.get(B2); + b2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); CSQueue b3 = queues.get(B3); + b3.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); // Simulate C returning a container on node_0 stubQueueAllocation(a, clusterResource, node_0, 0*GB); @@ -465,7 +478,7 @@ public void testMultiLevelQueues() throws Exception { stubQueueAllocation(c, clusterResource, node_0, 1*GB); stubQueueAllocation(d, clusterResource, node_0, 0*GB); root.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(b, 0*GB, clusterResource); verifyQueueMetrics(c, 1*GB, clusterResource); @@ -478,7 +491,7 @@ public void testMultiLevelQueues() throws Exception { stubQueueAllocation(b2, clusterResource, node_1, 4*GB); stubQueueAllocation(c, clusterResource, node_1, 0*GB); root.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource); verifyQueueMetrics(c, 1*GB, clusterResource); @@ -490,14 +503,14 @@ public void testMultiLevelQueues() throws Exception { stubQueueAllocation(b3, clusterResource, node_0, 2*GB); stubQueueAllocation(c, clusterResource, node_0, 2*GB); root.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); InOrder allocationOrder = inOrder(a, c, b); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(c).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(a, 1*GB, clusterResource); verifyQueueMetrics(b, 6*GB, clusterResource); verifyQueueMetrics(c, 3*GB, clusterResource); @@ -517,16 +530,16 @@ public void testMultiLevelQueues() throws Exception { stubQueueAllocation(b1, clusterResource, node_2, 1*GB); stubQueueAllocation(c, clusterResource, node_2, 1*GB); root.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); allocationOrder = inOrder(a, a2, a1, b, c); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(a2).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(c).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(a, 3*GB, clusterResource); verifyQueueMetrics(b, 8*GB, clusterResource); verifyQueueMetrics(c, 4*GB, clusterResource); @@ -622,12 +635,14 @@ public void testOffSwitchScheduling() throws Exception { // Start testing LeafQueue a = (LeafQueue)queues.get(A); LeafQueue b = (LeafQueue)queues.get(B); + a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); + b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verifyQueueMetrics(a, 0*GB, clusterResource); verifyQueueMetrics(b, 1*GB, clusterResource); @@ -636,12 +651,12 @@ public void testOffSwitchScheduling() throws Exception { 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, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); InOrder allocationOrder = inOrder(a, b); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(b, 2*GB, clusterResource); @@ -651,12 +666,12 @@ public void testOffSwitchScheduling() throws Exception { 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, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); allocationOrder = inOrder(b, a); allocationOrder.verify(b).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(a).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(a, 2*GB, clusterResource); verifyQueueMetrics(b, 4*GB, clusterResource); @@ -691,12 +706,17 @@ public void testOffSwitchSchedulingMultiLevelQueues() throws Exception { // Start testing LeafQueue b3 = (LeafQueue)queues.get(B3); LeafQueue b2 = (LeafQueue)queues.get(B2); + b2.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); + b3.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); + + CSQueue b = queues.get(B); + b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB)); // Simulate B3 returning a container on node_0 stubQueueAllocation(b2, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH); stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH); root.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); verifyQueueMetrics(b2, 0*GB, clusterResource); verifyQueueMetrics(b3, 1*GB, clusterResource); @@ -705,12 +725,12 @@ public void testOffSwitchSchedulingMultiLevelQueues() throws Exception { stubQueueAllocation(b2, clusterResource, node_1, 1*GB, NodeType.RACK_LOCAL); stubQueueAllocation(b3, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH); root.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); InOrder allocationOrder = inOrder(b2, b3); allocationOrder.verify(b2).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(b3).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); verifyQueueMetrics(b2, 1*GB, clusterResource); verifyQueueMetrics(b3, 2*GB, clusterResource); @@ -720,12 +740,12 @@ public void testOffSwitchSchedulingMultiLevelQueues() throws Exception { stubQueueAllocation(b2, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL); stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH); root.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); allocationOrder = inOrder(b3, b2); allocationOrder.verify(b3).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.class)); allocationOrder.verify(b2).assignContainers(eq(clusterResource), - any(FiCaSchedulerNode.class), anyResourceLimits()); + any(FiCaSchedulerNode.class), anyResourceLimits(), any(ExclusiveType.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 e8a8243..b03e31a 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 @@ -266,7 +266,7 @@ public void testReservation() throws Exception { // Start testing... // Only AM a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -278,7 +278,7 @@ public void testReservation() throws Exception { // Only 1 map - simulating reduce a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -290,7 +290,7 @@ public void testReservation() throws Exception { // Only 1 map to other node - simulating reduce a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -305,7 +305,7 @@ public void testReservation() throws Exception { // try to assign reducer (5G on node 0 and should reserve) a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -321,7 +321,7 @@ public void testReservation() throws Exception { // assign reducer to node 2 a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(18 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -338,7 +338,7 @@ public void testReservation() throws Exception { // node_1 heartbeat and unreserves from node_0 in order to allocate // on node_1 a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(18 * GB, a.getUsedResources().getMemory()); assertEquals(18 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -422,7 +422,7 @@ public void testReservationNoContinueLook() throws Exception { // Start testing... // Only AM a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -434,7 +434,7 @@ public void testReservationNoContinueLook() throws Exception { // Only 1 map - simulating reduce a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -446,7 +446,7 @@ public void testReservationNoContinueLook() throws Exception { // Only 1 map to other node - simulating reduce a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -461,7 +461,7 @@ public void testReservationNoContinueLook() throws Exception { // try to assign reducer (5G on node 0 and should reserve) a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -477,7 +477,7 @@ public void testReservationNoContinueLook() throws Exception { // assign reducer to node 2 a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(18 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -494,7 +494,7 @@ public void testReservationNoContinueLook() throws Exception { // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(18 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -570,7 +570,7 @@ public void testAssignContainersNeedToUnreserve() throws Exception { // Start testing... // Only AM a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -581,7 +581,7 @@ public void testAssignContainersNeedToUnreserve() throws Exception { // Only 1 map - simulating reduce a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -592,7 +592,7 @@ public void testAssignContainersNeedToUnreserve() throws Exception { // Only 1 map to other node - simulating reduce a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -606,7 +606,7 @@ public void testAssignContainersNeedToUnreserve() throws Exception { // try to assign reducer (5G on node 0 and should reserve) a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -621,7 +621,7 @@ public void testAssignContainersNeedToUnreserve() throws Exception { // could allocate but told need to unreserve first a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -823,7 +823,7 @@ public void testAssignToQueue() throws Exception { // Start testing... // Only AM a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -834,7 +834,7 @@ public void testAssignToQueue() throws Exception { // Only 1 map - simulating reduce a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -845,7 +845,7 @@ public void testAssignToQueue() throws Exception { // Only 1 map to other node - simulating reduce a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -861,14 +861,15 @@ public void testAssignToQueue() throws Exception { boolean res = a.canAssignToThisQueue(clusterResource, CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( - clusterResource), capability, Resources.none()); + clusterResource), capability, Resources.none(), + ExclusiveType.EXCLUSIVE); assertFalse(res); // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, a.getMetrics().getReservedMB()); @@ -882,15 +883,16 @@ public void testAssignToQueue() throws Exception { res = a.canAssignToThisQueue(clusterResource, CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( - clusterResource), capability, Resources - .createResource(5 * GB)); + clusterResource), capability, Resources.createResource(5 * GB), + ExclusiveType.EXCLUSIVE); assertTrue(res); // tell to not check reservations res = a.canAssignToThisQueue(clusterResource, CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( - clusterResource), capability, Resources.none()); + clusterResource), capability, Resources.none(), + ExclusiveType.EXCLUSIVE); assertFalse(res); refreshQueuesTurnOffReservationsContLook(a, csConf); @@ -900,14 +902,15 @@ public void testAssignToQueue() throws Exception { res = a.canAssignToThisQueue(clusterResource, CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( - clusterResource), capability, Resources.none()); + clusterResource), capability, Resources.none(), + ExclusiveType.EXCLUSIVE); assertFalse(res); res = a.canAssignToThisQueue(clusterResource, CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits( - clusterResource), capability, Resources - .createResource(5 * GB)); + clusterResource), capability, Resources.createResource(5 * GB), + ExclusiveType.EXCLUSIVE); assertFalse(res); } @@ -1008,7 +1011,7 @@ public void testAssignToUser() throws Exception { // Start testing... // Only AM a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1019,7 +1022,7 @@ public void testAssignToUser() throws Exception { // Only 1 map - simulating reduce a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1030,7 +1033,7 @@ public void testAssignToUser() throws Exception { // Only 1 map to other node - simulating reduce a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1044,7 +1047,7 @@ public void testAssignToUser() throws Exception { // allocate to queue so that the potential new capacity is greater then // absoluteMaxCapacity a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(5 * GB, app_0.getCurrentReservation().getMemory()); @@ -1059,18 +1062,18 @@ public void testAssignToUser() throws Exception { // set limit so subtrace reservations it can continue Resource limit = Resources.createResource(12 * GB, 0); boolean res = a.canAssignToUser(clusterResource, user_0, limit, app_0, - true, null); + true, ""); assertTrue(res); // tell it not to check for reservations and should fail as already over // limit - res = a.canAssignToUser(clusterResource, user_0, limit, app_0, false, null); + res = a.canAssignToUser(clusterResource, user_0, limit, app_0, false, ""); assertFalse(res); refreshQueuesTurnOffReservationsContLook(a, csConf); // should now return false since feature off - res = a.canAssignToUser(clusterResource, user_0, limit, app_0, true, null); + res = a.canAssignToUser(clusterResource, user_0, limit, app_0, true, ""); assertFalse(res); } @@ -1143,7 +1146,7 @@ public void testReservationsNoneAvailable() throws Exception { // Start testing... // Only AM a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(2 * GB, a.getUsedResources().getMemory()); assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1155,7 +1158,7 @@ public void testReservationsNoneAvailable() throws Exception { // Only 1 map - simulating reduce a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(5 * GB, a.getUsedResources().getMemory()); assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1167,7 +1170,7 @@ public void testReservationsNoneAvailable() throws Exception { // Only 1 map to other node - simulating reduce a.assignContainers(clusterResource, node_1, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1183,7 +1186,7 @@ public void testReservationsNoneAvailable() throws Exception { // 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))); + new ResourceLimits(Resources.createResource(10 * GB)), ExclusiveType.EXCLUSIVE); assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1199,7 +1202,7 @@ public void testReservationsNoneAvailable() throws Exception { // 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))); + new ResourceLimits(Resources.createResource(10 * GB)), ExclusiveType.EXCLUSIVE); assertEquals(8 * GB, a.getUsedResources().getMemory()); assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1213,7 +1216,7 @@ public void testReservationsNoneAvailable() throws Exception { // let it assign 5G to node_2 a.assignContainers(clusterResource, node_2, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(0 * GB, a.getMetrics().getReservedMB()); @@ -1226,7 +1229,7 @@ public void testReservationsNoneAvailable() throws Exception { // reserve 8G node_0 a.assignContainers(clusterResource, node_0, - new ResourceLimits(clusterResource)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(21 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); assertEquals(8 * GB, a.getMetrics().getReservedMB()); @@ -1241,7 +1244,7 @@ public void testReservationsNoneAvailable() throws Exception { // 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)); + new ResourceLimits(clusterResource), ExclusiveType.EXCLUSIVE); assertEquals(21 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); 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 62135b9..ec9e96f 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 @@ -273,6 +273,7 @@ public static Configuration getConfigurationWithQueueLabels(Configuration config conf.setCapacity(B1, 100); conf.setMaximumCapacity(B1, 100); conf.setCapacityByLabel(B1, "y", 100); + conf.setMaximumApplicationMasterResourcePerQueuePercent(B1, 1f); final String C1 = C + ".c1"; conf.setQueues(C, new String[] {"c1"});