diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java index 7962d8e30f55f..910079171fa2f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java @@ -95,7 +95,7 @@ protected void addReservationQueue( PlanQueue planQueue = (PlanQueue)queue; try { ReservationQueue resQueue = - new ReservationQueue(cs, currResId, planQueue); + new ReservationQueue(cs.getQueueContext(), currResId, planQueue); cs.addQueue(resQueue); } catch (SchedulerDynamicEditException e) { LOG.warn( @@ -115,7 +115,7 @@ protected void createDefaultReservationQueue( if (cs.getQueue(defReservationId) == null) { try { ReservationQueue defQueue = - new ReservationQueue(cs, defReservationId, planQueue); + new ReservationQueue(cs.getQueueContext(), defReservationId, planQueue); cs.addQueue(defQueue); } catch (SchedulerDynamicEditException e) { LOG.warn( 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/AbstractAutoCreatedLeafQueue.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/AbstractAutoCreatedLeafQueue.java index 36d2aef4806ed..15960c8de8494 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/AbstractAutoCreatedLeafQueue.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/AbstractAutoCreatedLeafQueue.java @@ -35,23 +35,15 @@ * of AbstractManagedParentQueue */ public class AbstractAutoCreatedLeafQueue extends AbstractLeafQueue { + private static final Logger LOG = LoggerFactory.getLogger( + AbstractAutoCreatedLeafQueue.class); protected AbstractManagedParentQueue parent; - public AbstractAutoCreatedLeafQueue(CapacitySchedulerContext cs, + public AbstractAutoCreatedLeafQueue(CapacitySchedulerQueueContext queueContext, String queueName, AbstractManagedParentQueue parent, CSQueue old) throws IOException { - super(cs, queueName, parent, old); - this.parent = parent; - } - - private static final Logger LOG = LoggerFactory.getLogger( - AbstractAutoCreatedLeafQueue.class); - - public AbstractAutoCreatedLeafQueue(CapacitySchedulerContext cs, - CapacitySchedulerConfiguration leafQueueConfigs, String queueName, - AbstractManagedParentQueue parent, CSQueue old) throws IOException { - super(cs, leafQueueConfigs, queueName, parent, old); + super(queueContext, queueName, parent, old); this.parent = parent; } @@ -71,7 +63,7 @@ public void setEntitlement(QueueEntitlement entitlement) @Override protected Resource getMinimumAbsoluteResource(String queuePath, String label) { - return super.getMinimumAbsoluteResource(csContext.getConfiguration() + return super.getMinimumAbsoluteResource(queueContext.getConfiguration() .getAutoCreatedQueueTemplateConfPrefix(this.getParent().getQueuePath()), label); } @@ -79,7 +71,7 @@ protected Resource getMinimumAbsoluteResource(String queuePath, @Override protected Resource getMaximumAbsoluteResource(String queuePath, String label) { - return super.getMaximumAbsoluteResource(csContext.getConfiguration() + return super.getMaximumAbsoluteResource(queueContext.getConfiguration() .getAutoCreatedQueueTemplateConfPrefix(this.getParent().getQueuePath()), label); } @@ -87,7 +79,7 @@ protected Resource getMaximumAbsoluteResource(String queuePath, @Override protected boolean checkConfigTypeIsAbsoluteResource(String queuePath, String label) { - return super.checkConfigTypeIsAbsoluteResource(csContext.getConfiguration() + return super.checkConfigTypeIsAbsoluteResource(queueContext.getConfiguration() .getAutoCreatedQueueTemplateConfPrefix(this.getParent().getQueuePath()), label); } @@ -122,7 +114,7 @@ public void setEntitlement(String nodeLabel, QueueEntitlement entitlement) //update queue used capacity etc CSQueueUtils.updateQueueStatistics(resourceCalculator, - csContext.getClusterResource(), + queueContext.getClusterResource(), this, labelManager, nodeLabel); } finally { writeLock.unlock(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/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 097a9dfbc5776..5040b027003d1 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 @@ -117,7 +117,7 @@ public enum CapacityConfigType { private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); - protected CapacitySchedulerContext csContext; + protected CapacitySchedulerQueueContext queueContext; protected YarnAuthorizationProvider authorizer = null; protected ActivitiesManager activitiesManager; @@ -131,33 +131,39 @@ public enum CapacityConfigType { // is it a dynamic queue? private boolean dynamicQueue = false; - public AbstractCSQueue(CapacitySchedulerContext cs, - String queueName, CSQueue parent, CSQueue old) throws IOException { - this(cs, cs.getConfiguration(), queueName, parent, old); - } - - public AbstractCSQueue(CapacitySchedulerContext cs, - CapacitySchedulerConfiguration configuration, String queueName, + public AbstractCSQueue(CapacitySchedulerQueueContext queueContext, String queueName, CSQueue parent, CSQueue old) { - this.labelManager = cs.getRMContext().getNodeLabelManager(); this.parent = parent; this.queuePath = createQueuePath(parent, queueName); - this.resourceCalculator = cs.getResourceCalculator(); - this.activitiesManager = cs.getActivitiesManager(); + + this.queueContext = queueContext; + this.resourceCalculator = queueContext.getResourceCalculator(); + this.activitiesManager = queueContext.getActivitiesManager(); + this.labelManager = queueContext.getLabelManager(); // must be called after parent and queueName is set CSQueueMetrics metrics = old != null ? (CSQueueMetrics) old.getMetrics() : CSQueueMetrics.forQueue(getQueuePath(), parent, - cs.getConfiguration().getEnableUserMetrics(), configuration); - usageTracker = new CSQueueUsageTracker(metrics); - this.csContext = cs; - this.queueAllocationSettings = new QueueAllocationSettings(csContext); - queueEntity = new PrivilegedEntity(EntityType.QUEUE, getQueuePath()); - queueCapacities = new QueueCapacities(parent == null); + queueContext.getConfiguration().getEnableUserMetrics(), queueContext.getConfiguration()); + this.usageTracker = new CSQueueUsageTracker(metrics); + + this.queueCapacities = new QueueCapacities(parent == null); + this.queueAllocationSettings = new QueueAllocationSettings(queueContext.getMinimumAllocation()); + + this.queueEntity = new PrivilegedEntity(EntityType.QUEUE, getQueuePath()); + + this.resourceTypes = new HashSet<>(); + for (AbsoluteResourceType type : AbsoluteResourceType.values()) { + this.resourceTypes.add(type.toString().toLowerCase()); + } + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - readLock = lock.readLock(); - writeLock = lock.writeLock(); + this.readLock = lock.readLock(); + this.writeLock = lock.writeLock(); + + LOG.debug("Initialized {}: name={}, fullname={}", this.getClass().getSimpleName(), + queueName, getQueuePath()); } private static QueuePath createQueuePath(CSQueue parent, String queueName) { @@ -167,11 +173,6 @@ private static QueuePath createQueuePath(CSQueue parent, String queueName) { return new QueuePath(parent.getQueuePath(), queueName); } - @VisibleForTesting - protected void setupConfigurableCapacities() { - setupConfigurableCapacities(csContext.getConfiguration()); - } - protected void setupConfigurableCapacities( CapacitySchedulerConfiguration configuration) { CSQueueUtils.loadCapacitiesByLabelsFromConf(queuePath, queueCapacities, @@ -262,6 +263,10 @@ public PrivilegedEntity getPrivilegedEntity() { return queueEntity; } + public CapacitySchedulerQueueContext getQueueContext() { + return queueContext; + } + public Set getAccessibleNodeLabels() { return queueNodeLabelsSettings.getAccessibleNodeLabels(); } @@ -336,26 +341,24 @@ protected void setupQueueConfigs(Resource clusterResource, // Collect and set the Node label configuration this.queueNodeLabelsSettings = new QueueNodeLabelsSettings(configuration, parent, - getQueuePath(), csContext); + getQueuePath(), queueContext.getQueueManager().getConfiguredNodeLabelsForAllQueues()); // Initialize the queue capacities setupConfigurableCapacities(configuration); updateAbsoluteCapacities(); - updateCapacityConfigType(); // Fetch minimum/maximum resource limits for this queue if // configured - this.resourceTypes = new HashSet<>(); - for (AbsoluteResourceType type : AbsoluteResourceType.values()) { - resourceTypes.add(type.toString().toLowerCase()); - } updateConfigurableResourceLimits(clusterResource); // Setup queue's maximumAllocation respecting the global // and the queue settings - this.queueAllocationSettings.setupMaximumAllocation(configuration, getQueuePath(), - parent, csContext); + // TODO remove the getConfiguration() param after the AQC configuration duplication + // removal is resolved + this.queueAllocationSettings.setupMaximumAllocation(configuration, + queueContext.getConfiguration(), getQueuePath(), + parent); // Initialize the queue state based on previous state, configured state // and its parent state @@ -369,7 +372,8 @@ protected void setupQueueConfigs(Resource clusterResource, this.reservationsContinueLooking = configuration.getReservationContinueLook(); - this.configuredCapacityVectors = csContext.getConfiguration() + + this.configuredCapacityVectors = queueContext.getConfiguration() .parseConfiguredResourceVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels()); @@ -378,7 +382,10 @@ protected void setupQueueConfigs(Resource clusterResource, this, labelManager, null); // Store preemption settings - this.preemptionSettings = new CSQueuePreemptionSettings(this, csContext, configuration); + // TODO remove the getConfiguration() param after the AQC configuration duplication + // removal is resolved + this.preemptionSettings = new CSQueuePreemptionSettings(this, configuration, + queueContext.getConfiguration()); this.priority = configuration.getQueuePriority( getQueuePath()); @@ -409,12 +416,13 @@ protected void setDynamicQueueProperties( AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX); parentTemplate = parentTemplate.substring(0, parentTemplate.lastIndexOf( DOT)); - Set parentNodeLabels = csContext - .getCapacitySchedulerQueueManager().getConfiguredNodeLabels() + Set parentNodeLabels = queueContext.getQueueManager() + .getConfiguredNodeLabelsForAllQueues() .getLabelsByQueue(parentTemplate); if (parentNodeLabels != null && parentNodeLabels.size() > 1) { - csContext.getCapacitySchedulerQueueManager().getConfiguredNodeLabels() + queueContext.getQueueManager() + .getConfiguredNodeLabelsForAllQueues() .setLabelsByQueue(getQueuePath(), new HashSet<>(parentNodeLabels)); } } @@ -436,20 +444,18 @@ private UserWeights getUserWeightsFromHierarchy( } protected Resource getMinimumAbsoluteResource(String queuePath, String label) { - Resource minResource = csContext.getConfiguration() + return queueContext.getConfiguration() .getMinimumResourceRequirement(label, queuePath, resourceTypes); - return minResource; } protected Resource getMaximumAbsoluteResource(String queuePath, String label) { - Resource maxResource = csContext.getConfiguration() + return queueContext.getConfiguration() .getMaximumResourceRequirement(label, queuePath, resourceTypes); - return maxResource; } protected boolean checkConfigTypeIsAbsoluteResource(String queuePath, String label) { - return csContext.getConfiguration().checkConfigTypeIsAbsoluteResource(label, + return queueContext.getConfiguration().checkConfigTypeIsAbsoluteResource(label, queuePath, resourceTypes); } @@ -743,7 +749,7 @@ protected void releaseResource(Resource clusterResource, } @Private - public boolean getReservationContinueLooking() { + public boolean isReservationsContinueLooking() { return reservationsContinueLooking; } @@ -764,7 +770,7 @@ public boolean getPreemptionDisabled() { @Private public boolean getIntraQueuePreemptionDisabled() { - return preemptionSettings.getIntraQueuePreemptionDisabled(); + return preemptionSettings.isIntraQueuePreemptionDisabled(); } @Private @@ -1026,12 +1032,12 @@ public Set getNodeLabelsForQueue() { } public Resource getTotalKillableResource(String partition) { - return csContext.getPreemptionManager().getKillableResource(getQueuePath(), + return queueContext.getPreemptionManager().getKillableResource(getQueuePath(), partition); } public Iterator getKillableContainers(String partition) { - return csContext.getPreemptionManager().getKillableContainers( + return queueContext.getPreemptionManager().getKillableContainers( getQueuePath(), partition); } @@ -1383,7 +1389,7 @@ public boolean isInactiveDynamicQueue() { long idleDurationSeconds = (Time.monotonicNow() - getLastSubmittedTimestamp())/1000; return isDynamicQueue() && isEligibleForAutoDeletion() && - (idleDurationSeconds > this.csContext.getConfiguration(). + (idleDurationSeconds > queueContext.getConfiguration(). getAutoExpiredDeletionTime()); } 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/AbstractLeafQueue.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/AbstractLeafQueue.java index 99911400f26e4..dff4ade9b9ef1 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/AbstractLeafQueue.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/AbstractLeafQueue.java @@ -19,7 +19,16 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -138,23 +147,18 @@ public class AbstractLeafQueue extends AbstractCSQueue { private final List runnableApps = new ArrayList<>(); private final List nonRunnableApps = new ArrayList<>(); - @SuppressWarnings({ "unchecked", "rawtypes" }) - public AbstractLeafQueue(CapacitySchedulerContext cs, String queueName, - CSQueue parent, CSQueue old) { - this(cs, cs.getConfiguration(), queueName, parent, old, false); + public AbstractLeafQueue(CapacitySchedulerQueueContext queueContext, + String queueName, CSQueue parent, CSQueue old) throws IOException { + this(queueContext, queueName, parent, old, false); } - public AbstractLeafQueue(CapacitySchedulerContext cs, CapacitySchedulerConfiguration configuration, - String queueName, CSQueue parent, CSQueue old) { - this(cs, configuration, queueName, parent, old, false); - } - - public AbstractLeafQueue(CapacitySchedulerContext cs, CapacitySchedulerConfiguration configuration, - String queueName, CSQueue parent, CSQueue old, boolean isDynamic) { - super(cs, configuration, queueName, parent, old); + public AbstractLeafQueue(CapacitySchedulerQueueContext queueContext, + String queueName, CSQueue parent, CSQueue old, boolean isDynamic) throws + IOException { + super(queueContext, queueName, parent, old); setDynamicQueue(isDynamic); - this.usersManager = new UsersManager(usageTracker.getMetrics(), this, labelManager, csContext, + this.usersManager = new UsersManager(usageTracker.getMetrics(), this, labelManager, resourceCalculator); // One time initialization is enough since it is static ordering policy @@ -162,16 +166,20 @@ public AbstractLeafQueue(CapacitySchedulerContext cs, CapacitySchedulerConfigura } @SuppressWarnings("checkstyle:nowhitespaceafter") - protected void setupQueueConfigs( - Resource clusterResource, CapacitySchedulerConfiguration conf) throws IOException { + protected void setupQueueConfigs(Resource clusterResource, + CapacitySchedulerConfiguration conf) throws + IOException { writeLock.lock(); try { - CapacitySchedulerConfiguration schedConf = csContext.getConfiguration(); + // TODO conf parameter can be a modified configuration with template entries and missing + // some global configs. This config duplication needs to be removed. + CapacitySchedulerConfiguration originalConfiguration = queueContext.getConfiguration(); super.setupQueueConfigs(clusterResource, conf); this.lastClusterResource = clusterResource; - this.cachedResourceLimitsForHeadroom = new ResourceLimits(clusterResource); + this.cachedResourceLimitsForHeadroom = new ResourceLimits( + clusterResource); // Initialize headroom info, also used for calculating application // master resource limits. Since this happens during queue initialization @@ -180,44 +188,52 @@ protected void setupQueueConfigs( // absoluteMaxAvailCapacity during headroom/userlimit/allocation events) setQueueResourceLimitsInfo(clusterResource); - setOrderingPolicy(conf.getAppOrderingPolicy(getQueuePath())); + setOrderingPolicy( + conf.getAppOrderingPolicy(getQueuePath())); usersManager.setUserLimit(conf.getUserLimit(getQueuePath())); usersManager.setUserLimitFactor(conf.getUserLimitFactor(getQueuePath())); maxAMResourcePerQueuePercent = - conf.getMaximumApplicationMasterResourcePerQueuePercent(getQueuePath()); + conf.getMaximumApplicationMasterResourcePerQueuePercent( + getQueuePath()); maxApplications = conf.getMaximumApplicationsPerQueue(getQueuePath()); if (maxApplications < 0) { int maxGlobalPerQueueApps = - csContext.getConfiguration().getGlobalMaximumApplicationsPerQueue(); + conf.getGlobalMaximumApplicationsPerQueue(); if (maxGlobalPerQueueApps > 0) { maxApplications = maxGlobalPerQueueApps; } } - priorityAcls = - conf.getPriorityAcls(getQueuePath(), csContext.getMaxClusterLevelAppPriority()); + priorityAcls = conf.getPriorityAcls(getQueuePath(), + originalConfiguration.getClusterLevelApplicationMaxPriority()); Set accessibleNodeLabels = this.queueNodeLabelsSettings.getAccessibleNodeLabels(); if (!SchedulerUtils.checkQueueLabelExpression(accessibleNodeLabels, this.queueNodeLabelsSettings.getDefaultLabelExpression(), null)) { - throw new IOException("Invalid default label expression of " + " queue=" + getQueuePath() - + " doesn't have permission to access all labels " - + "in default label expression. labelExpression of resource request=" - + getDefaultNodeLabelExpressionStr() + ". Queue labels=" + ( - getAccessibleNodeLabels() == null ? "" : - StringUtils.join(getAccessibleNodeLabels().iterator(), ','))); + throw new IOException( + "Invalid default label expression of " + " queue=" + getQueuePath() + + " doesn't have permission to access all labels " + + "in default label expression. labelExpression of resource request=" + + getDefaultNodeLabelExpressionStr() + ". Queue labels=" + ( + getAccessibleNodeLabels() == null ? + "" : + StringUtils + .join(getAccessibleNodeLabels().iterator(), ','))); } - nodeLocalityDelay = schedConf.getNodeLocalityDelay(); - rackLocalityAdditionalDelay = schedConf.getRackLocalityAdditionalDelay(); - rackLocalityFullReset = schedConf.getRackLocalityFullReset(); + nodeLocalityDelay = originalConfiguration.getNodeLocalityDelay(); + rackLocalityAdditionalDelay = originalConfiguration + .getRackLocalityAdditionalDelay(); + rackLocalityFullReset = originalConfiguration + .getRackLocalityFullReset(); // re-init this since max allocation could have changed this.minimumAllocationFactor = Resources.ratio(resourceCalculator, - Resources.subtract(queueAllocationSettings.getMaximumAllocation(), + Resources.subtract( + queueAllocationSettings.getMaximumAllocation(), queueAllocationSettings.getMinimumAllocation()), queueAllocationSettings.getMaximumAllocation()); @@ -233,8 +249,8 @@ protected void setupQueueConfigs( } } - defaultAppPriorityPerQueue = - Priority.newInstance(conf.getDefaultApplicationPriorityConfPerQueue(getQueuePath())); + defaultAppPriorityPerQueue = Priority.newInstance( + conf.getDefaultApplicationPriorityConfPerQueue(getQueuePath())); // Validate leaf queue's user's weights. float queueUserLimit = Math.min(100.0f, conf.getUserLimit(getQueuePath())); @@ -369,7 +385,8 @@ void setUserLimitFactor(float userLimitFactor) { public int getNumApplications() { readLock.lock(); try { - return getNumPendingApplications() + getNumActiveApplications() + getNumNonRunnableApps(); + return getNumPendingApplications() + getNumActiveApplications() + + getNumNonRunnableApps(); } finally { readLock.unlock(); } @@ -527,10 +544,11 @@ protected void reinitialize( } // Sanity check - if (!(newlyParsedQueue instanceof AbstractLeafQueue) || !newlyParsedQueue.getQueuePath() - .equals(getQueuePath())) { - throw new IOException("Trying to reinitialize " + getQueuePath() + " from " - + newlyParsedQueue.getQueuePath()); + if (!(newlyParsedQueue instanceof AbstractLeafQueue) || !newlyParsedQueue + .getQueuePath().equals(getQueuePath())) { + throw new IOException( + "Trying to reinitialize " + getQueuePath() + " from " + + newlyParsedQueue.getQueuePath()); } AbstractLeafQueue newlyParsedLeafQueue = (AbstractLeafQueue) newlyParsedQueue; @@ -556,9 +574,9 @@ protected void reinitialize( @Override public void reinitialize( CSQueue newlyParsedQueue, Resource clusterResource) - throws IOException { + throws IOException { reinitialize(newlyParsedQueue, clusterResource, - csContext.getConfiguration()); + queueContext.getConfiguration()); } @Override @@ -724,10 +742,10 @@ public Resource getUserAMResourceLimitPerPartition( Resource preWeighteduserAMLimit = Resources.multiplyAndNormalizeUp( - resourceCalculator, queuePartitionResource, - queueCapacities.getMaxAMResourcePercentage(nodePartition) + resourceCalculator, queuePartitionResource, + queueCapacities.getMaxAMResourcePercentage(nodePartition) * preWeightedUserLimit * usersManager.getUserLimitFactor(), - minimumAllocation); + minimumAllocation); if (getUserLimitFactor() == -1) { preWeighteduserAMLimit = Resources.multiplyAndNormalizeUp( @@ -752,7 +770,8 @@ public Resource getUserAMResourceLimitPerPartition( } - public Resource calculateAndGetAMResourceLimitPerPartition(String nodePartition) { + public Resource calculateAndGetAMResourceLimitPerPartition( + String nodePartition) { writeLock.lock(); try { /* @@ -769,12 +788,13 @@ public Resource calculateAndGetAMResourceLimitPerPartition(String nodePartition) // For non-labeled partition, we need to consider the current queue // usage limit. if (nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) { - synchronized (queueResourceLimitsInfo) { + synchronized (queueResourceLimitsInfo){ queueCurrentLimit = queueResourceLimitsInfo.getQueueCurrentLimit(); } } - float amResourcePercent = queueCapacities.getMaxAMResourcePercentage(nodePartition); + float amResourcePercent = queueCapacities.getMaxAMResourcePercentage( + nodePartition); // Current usable resource for this queue and partition is the max of // queueCurrentLimit and queuePartitionResource. @@ -782,13 +802,13 @@ public Resource calculateAndGetAMResourceLimitPerPartition(String nodePartition) // guarantee, use the guarantee as the queuePartitionUsableResource // because nothing less than the queue's guarantee should be used when // calculating the AM limit. - Resource queuePartitionUsableResource = - (Resources.fitsIn(resourceCalculator, queuePartitionResource, queueCurrentLimit)) ? - queueCurrentLimit : queuePartitionResource; + Resource queuePartitionUsableResource = (Resources.fitsIn( + resourceCalculator, queuePartitionResource, queueCurrentLimit)) ? + queueCurrentLimit : queuePartitionResource; - Resource amResouceLimit = - Resources.multiplyAndNormalizeUp(resourceCalculator, queuePartitionUsableResource, - amResourcePercent, queueAllocationSettings.getMinimumAllocation()); + Resource amResouceLimit = Resources.multiplyAndNormalizeUp( + resourceCalculator, queuePartitionUsableResource, amResourcePercent, + queueAllocationSettings.getMinimumAllocation()); usageTracker.getMetrics().setAMResouceLimit(nodePartition, amResouceLimit); usageTracker.getQueueUsage().setAMLimit(nodePartition, amResouceLimit); @@ -807,7 +827,8 @@ protected void activateApplications() { writeLock.lock(); try { // limit of allowed resource usage for application masters - Map userAmPartitionLimit = new HashMap(); + Map userAmPartitionLimit = + new HashMap(); // AM Resource Limit for accessible labels can be pre-calculated. // This will help in updating AMResourceLimit for all labels when queue @@ -816,8 +837,10 @@ protected void activateApplications() { calculateAndGetAMResourceLimitPerPartition(nodePartition); } - for (Iterator fsApp = getPendingAppsOrderingPolicy().getAssignmentIterator( - IteratorSelector.EMPTY_ITERATOR_SELECTOR); fsApp.hasNext(); ) { + for (Iterator fsApp = + getPendingAppsOrderingPolicy() + .getAssignmentIterator(IteratorSelector.EMPTY_ITERATOR_SELECTOR); + fsApp.hasNext(); ) { FiCaSchedulerApp application = fsApp.next(); ApplicationId applicationId = application.getApplicationId(); @@ -831,7 +854,8 @@ protected void activateApplications() { amLimit = calculateAndGetAMResourceLimitPerPartition(partitionName); } // Check am resource limit. - Resource amIfStarted = Resources.add(application.getAMResource(partitionName), + Resource amIfStarted = Resources.add( + application.getAMResource(partitionName), usageTracker.getQueueUsage().getAMUsed(partitionName)); if (LOG.isDebugEnabled()) { @@ -844,19 +868,18 @@ protected void activateApplications() { } if (!resourceCalculator.fitsIn(amIfStarted, amLimit)) { - if (getNumActiveApplications() < 1 || (Resources.lessThanOrEqual(resourceCalculator, - lastClusterResource, usageTracker.getQueueUsage().getAMUsed(partitionName), - Resources.none()))) { + if (getNumActiveApplications() < 1 || (Resources.lessThanOrEqual( + resourceCalculator, lastClusterResource, + usageTracker.getQueueUsage().getAMUsed(partitionName), Resources.none()))) { LOG.warn("maximum-am-resource-percent is insufficient to start a" + " single application in queue, it is likely set too low." - + " skipping enforcement to allow at least one application" + " to start"); - } else { - application.updateAMContainerDiagnostics( - SchedulerApplicationAttempt.AMState.INACTIVATED, + + " skipping enforcement to allow at least one application" + + " to start"); + } else{ + application.updateAMContainerDiagnostics(SchedulerApplicationAttempt.AMState.INACTIVATED, CSAMContainerLaunchDiagnosticsConstants.QUEUE_AM_RESOURCE_LIMIT_EXCEED); - LOG.debug( - "Not activating application {} as amIfStarted: {}" + " exceeds amLimit: {}", - applicationId, amIfStarted, amLimit); + LOG.debug("Not activating application {} as amIfStarted: {}" + + " exceeds amLimit: {}", applicationId, amIfStarted, amLimit); continue; } } @@ -867,23 +890,25 @@ protected void activateApplications() { // Verify whether we already calculated user-am-limit for this label. if (userAMLimit == null) { - userAMLimit = getUserAMResourceLimitPerPartition(partitionName, application.getUser()); + userAMLimit = getUserAMResourceLimitPerPartition(partitionName, + application.getUser()); userAmPartitionLimit.put(partitionName, userAMLimit); } - Resource userAmIfStarted = Resources.add(application.getAMResource(partitionName), + Resource userAmIfStarted = Resources.add( + application.getAMResource(partitionName), user.getConsumedAMResources(partitionName)); if (!resourceCalculator.fitsIn(userAmIfStarted, userAMLimit)) { - if (getNumActiveApplications() < 1 || (Resources.lessThanOrEqual(resourceCalculator, - lastClusterResource, usageTracker.getQueueUsage().getAMUsed(partitionName), - Resources.none()))) { + if (getNumActiveApplications() < 1 || (Resources.lessThanOrEqual( + resourceCalculator, lastClusterResource, + usageTracker.getQueueUsage().getAMUsed(partitionName), Resources.none()))) { LOG.warn("maximum-am-resource-percent is insufficient to start a" + " single application in queue for user, it is likely set too" - + " low. skipping enforcement to allow at least one application" + " to start"); - } else { - application.updateAMContainerDiagnostics( - AMState.INACTIVATED, + + " low. skipping enforcement to allow at least one application" + + " to start"); + } else{ + application.updateAMContainerDiagnostics(AMState.INACTIVATED, CSAMContainerLaunchDiagnosticsConstants.USER_AM_RESOURCE_LIMIT_EXCEED); LOG.debug("Not activating application {} for user: {} as" + " userAmIfStarted: {} exceeds userAmLimit: {}", @@ -893,17 +918,17 @@ protected void activateApplications() { } user.activateApplication(); orderingPolicy.addSchedulableEntity(application); - application.updateAMContainerDiagnostics(AMState.ACTIVATED, - null); + application.updateAMContainerDiagnostics(AMState.ACTIVATED, null); - usageTracker.getQueueUsage() - .incAMUsed(partitionName, application.getAMResource(partitionName)); - user.getResourceUsage().incAMUsed(partitionName, application.getAMResource(partitionName)); + usageTracker.getQueueUsage().incAMUsed(partitionName, + application.getAMResource(partitionName)); + user.getResourceUsage().incAMUsed(partitionName, + application.getAMResource(partitionName)); user.getResourceUsage().setAMLimit(partitionName, userAMLimit); usageTracker.getMetrics().incAMUsed(partitionName, application.getUser(), application.getAMResource(partitionName)); - usageTracker.getMetrics() - .setAMResouceLimitForUser(partitionName, application.getUser(), userAMLimit); + usageTracker.getMetrics().setAMResouceLimitForUser(partitionName, + application.getUser(), userAMLimit); fsApp.remove(); LOG.info("Application " + applicationId + " from user: " + application .getUser() + " activated in queue: " + getQueuePath()); @@ -913,10 +938,12 @@ protected void activateApplications() { } } - private void addApplicationAttempt(FiCaSchedulerApp application, User user) { + private void addApplicationAttempt(FiCaSchedulerApp application, + User user) { writeLock.lock(); try { - applicationAttemptMap.put(application.getApplicationAttemptId(), application); + applicationAttemptMap.put(application.getApplicationAttemptId(), + application); if (application.isRunnable()) { runnableApps.add(application); @@ -934,8 +961,8 @@ private void addApplicationAttempt(FiCaSchedulerApp application, User user) { getPendingAppsOrderingPolicy().addSchedulableEntity(application); // Activate applications - if (Resources.greaterThan(resourceCalculator, lastClusterResource, lastClusterResource, - Resources.none())) { + if (Resources.greaterThan(resourceCalculator, lastClusterResource, + lastClusterResource, Resources.none())) { activateApplications(); } else { application.updateAMContainerDiagnostics(AMState.INACTIVATED, @@ -978,7 +1005,8 @@ public void finishApplicationAttempt(FiCaSchedulerApp application, String queue) parent.finishApplicationAttempt(application, queue); } - private void removeApplicationAttempt(FiCaSchedulerApp application, String userName) { + private void removeApplicationAttempt( + FiCaSchedulerApp application, String userName) { writeLock.lock(); try { @@ -999,10 +1027,11 @@ private void removeApplicationAttempt(FiCaSchedulerApp application, String userN boolean wasActive = orderingPolicy.removeSchedulableEntity(application); if (!wasActive) { pendingOrderingPolicy.removeSchedulableEntity(application); - } else { - usageTracker.getQueueUsage() - .decAMUsed(partitionName, application.getAMResource(partitionName)); - user.getResourceUsage().decAMUsed(partitionName, application.getAMResource(partitionName)); + } else{ + usageTracker.getQueueUsage().decAMUsed(partitionName, + application.getAMResource(partitionName)); + user.getResourceUsage().decAMUsed(partitionName, + application.getAMResource(partitionName)); usageTracker.getMetrics().decAMUsed(partitionName, application.getUser(), application.getAMResource(partitionName)); } @@ -1028,7 +1057,8 @@ private void removeApplicationAttempt(FiCaSchedulerApp application, String userN } } - private FiCaSchedulerApp getApplication(ApplicationAttemptId applicationAttemptId) { + private FiCaSchedulerApp getApplication( + ApplicationAttemptId applicationAttemptId) { return applicationAttemptMap.get(applicationAttemptId); } @@ -1038,7 +1068,7 @@ private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) { if (!usageTracker.getQueueResourceQuotas().getEffectiveMinResource(nodePartition) .equals(Resources.none())) { limits.setIsAllowPreemption(Resources.lessThan(resourceCalculator, - csContext.getClusterResource(), usageTracker.getQueueUsage().getUsed(nodePartition), + queueContext.getClusterResource(), usageTracker.getQueueUsage().getUsed(nodePartition), usageTracker.getQueueResourceQuotas().getEffectiveMinResource(nodePartition))); return; } @@ -1049,8 +1079,8 @@ private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) { } private CSAssignment allocateFromReservedContainer(Resource clusterResource, - CandidateNodeSet candidates, ResourceLimits currentResourceLimits, - SchedulingMode schedulingMode) { + CandidateNodeSet candidates, + ResourceLimits currentResourceLimits, SchedulingMode schedulingMode) { // Irrespective of Single / Multi Node Placement, the allocate from // Reserved Container has to happen only for the single node which @@ -1062,14 +1092,15 @@ private CSAssignment allocateFromReservedContainer(Resource clusterResource, if (node != null) { RMContainer reservedContainer = node.getReservedContainer(); if (reservedContainer != null) { - FiCaSchedulerApp application = getApplication(reservedContainer.getApplicationAttemptId()); + FiCaSchedulerApp application = getApplication( + reservedContainer.getApplicationAttemptId()); if (null != application) { - ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager, node, - SystemClock.getInstance().getTime(), application); - CSAssignment assignment = - application.assignContainers(clusterResource, candidates, currentResourceLimits, - schedulingMode, reservedContainer); + ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager, + node, SystemClock.getInstance().getTime(), application); + CSAssignment assignment = application.assignContainers( + clusterResource, candidates, currentResourceLimits, + schedulingMode, reservedContainer); return assignment; } } @@ -1078,7 +1109,8 @@ private CSAssignment allocateFromReservedContainer(Resource clusterResource, return null; } - private ConcurrentMap getUserLimitCache(String partition, + private ConcurrentMap getUserLimitCache( + String partition, SchedulingMode schedulingMode) { synchronized (userLimitsCache) { long latestVersion = usersManager.getLatestVersionOfUsersState(); @@ -1088,19 +1120,20 @@ private ConcurrentMap getUserLimitCache(String partitio this.currentUserLimitCacheVersion = latestVersion; userLimitsCache.clear(); - Map> uLCByPartition = - new HashMap<>(); + Map> + uLCByPartition = new HashMap<>(); userLimitsCache.put(partition, uLCByPartition); - ConcurrentMap uLCBySchedulingMode = new ConcurrentHashMap<>(); + ConcurrentMap uLCBySchedulingMode = + new ConcurrentHashMap<>(); uLCByPartition.put(schedulingMode, uLCBySchedulingMode); return uLCBySchedulingMode; } // User limits cache does not need invalidating - Map> uLCByPartition = - userLimitsCache.get(partition); + Map> + uLCByPartition = userLimitsCache.get(partition); if (uLCByPartition == null) { uLCByPartition = new HashMap<>(); userLimitsCache.put(partition, uLCByPartition); @@ -1119,8 +1152,8 @@ private ConcurrentMap getUserLimitCache(String partitio @Override public CSAssignment assignContainers(Resource clusterResource, - CandidateNodeSet candidates, ResourceLimits currentResourceLimits, - SchedulingMode schedulingMode) { + CandidateNodeSet candidates, + ResourceLimits currentResourceLimits, SchedulingMode schedulingMode) { updateCurrentResourceLimits(currentResourceLimits, clusterResource); FiCaSchedulerNode node = CandidateNodeSetUtils.getSingleNode(candidates); @@ -1132,9 +1165,8 @@ public CSAssignment assignContainers(Resource clusterResource, setPreemptionAllowed(currentResourceLimits, candidates.getPartition()); // Check for reserved resources, try to allocate reserved container first. - CSAssignment assignment = - allocateFromReservedContainer(clusterResource, candidates, currentResourceLimits, - schedulingMode); + CSAssignment assignment = allocateFromReservedContainer(clusterResource, + candidates, currentResourceLimits, schedulingMode); if (null != assignment) { return assignment; } @@ -1169,29 +1201,33 @@ public CSAssignment assignContainers(Resource clusterResource, boolean needAssignToQueueCheck = true; IteratorSelector sel = new IteratorSelector(); sel.setPartition(candidates.getPartition()); - for (Iterator assignmentIterator = orderingPolicy.getAssignmentIterator(sel); + for (Iterator assignmentIterator = + orderingPolicy.getAssignmentIterator(sel); assignmentIterator.hasNext(); ) { FiCaSchedulerApp application = assignmentIterator.next(); - ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager, node, - SystemClock.getInstance().getTime(), application); + ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager, + node, SystemClock.getInstance().getTime(), application); // Check queue max-capacity limit Resource appReserved = application.getCurrentReservation(); if (needAssignToQueueCheck) { - if (!super.canAssignToThisQueue(clusterResource, candidates.getPartition(), - currentResourceLimits, appReserved, schedulingMode)) { - ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(activitiesManager, node, - application, application.getPriority(), + if (!super.canAssignToThisQueue(clusterResource, + candidates.getPartition(), currentResourceLimits, appReserved, + schedulingMode)) { + ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue( + activitiesManager, node, application, application.getPriority(), ActivityDiagnosticConstant.QUEUE_HIT_MAX_CAPACITY_LIMIT); - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, parent.getQueuePath(), - getQueuePath(), ActivityState.REJECTED, + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + parent.getQueuePath(), getQueuePath(), + ActivityState.REJECTED, ActivityDiagnosticConstant.QUEUE_HIT_MAX_CAPACITY_LIMIT); return CSAssignment.NULL_ASSIGNMENT; } // If there was no reservation and canAssignToThisQueue returned // true, there is no reason to check further. - if (!this.reservationsContinueLooking || appReserved.equals(Resources.none())) { + if (!this.reservationsContinueLooking + || appReserved.equals(Resources.none())) { needAssignToQueueCheck = false; } } @@ -1201,12 +1237,13 @@ public CSAssignment assignContainers(Resource clusterResource, if (cul != null) { cachedUserLimit = cul.userLimit; } - Resource userLimit = - computeUserLimitAndSetHeadroom(application, clusterResource, candidates.getPartition(), - schedulingMode, cachedUserLimit); + Resource userLimit = computeUserLimitAndSetHeadroom(application, + clusterResource, candidates.getPartition(), schedulingMode, + cachedUserLimit); if (cul == null) { cul = new CachedUserLimit(userLimit); - CachedUserLimit retVal = userLimits.putIfAbsent(application.getUser(), cul); + CachedUserLimit retVal = + userLimits.putIfAbsent(application.getUser(), cul); if (retVal != null) { // another thread updated the user limit cache before us cul = retVal; @@ -1218,9 +1255,9 @@ public CSAssignment assignContainers(Resource clusterResource, if (!cul.canAssign && Resources.fitsIn(appReserved, cul.reservation)) { userAssignable = false; } else { - userAssignable = - canAssignToUser(clusterResource, application.getUser(), userLimit, application, - candidates.getPartition(), currentResourceLimits); + userAssignable = canAssignToUser(clusterResource, application.getUser(), + userLimit, application, candidates.getPartition(), + currentResourceLimits); if (!userAssignable && Resources.fitsIn(cul.reservation, appReserved)) { cul.canAssign = false; cul.reservation = appReserved; @@ -1229,54 +1266,59 @@ public CSAssignment assignContainers(Resource clusterResource, if (!userAssignable) { application.updateAMContainerDiagnostics(AMState.ACTIVATED, "User capacity has reached its maximum limit."); - ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(activitiesManager, node, - application, application.getPriority(), + ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue( + activitiesManager, node, application, application.getPriority(), ActivityDiagnosticConstant.QUEUE_HIT_USER_MAX_CAPACITY_LIMIT); continue; } // Try to schedule - assignment = application.assignContainers(clusterResource, candidates, currentResourceLimits, - schedulingMode, null); + assignment = application.assignContainers(clusterResource, + candidates, currentResourceLimits, schedulingMode, null); if (LOG.isDebugEnabled()) { - LOG.debug( - "post-assignContainers for application " + application.getApplicationId()); + LOG.debug("post-assignContainers for application " + application + .getApplicationId()); application.showRequests(); } // Did we schedule or reserve a container? Resource assigned = assignment.getResource(); - if (Resources.greaterThan(resourceCalculator, clusterResource, assigned, Resources.none())) { - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, parent.getQueuePath(), - getQueuePath(), ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); + if (Resources.greaterThan(resourceCalculator, clusterResource, assigned, + Resources.none())) { + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + parent.getQueuePath(), getQueuePath(), + ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY); return assignment; - } else if (assignment.getSkippedType() == CSAssignment.SkippedType.OTHER) { - ActivitiesLogger.APP.finishSkippedAppAllocationRecording(activitiesManager, - application.getApplicationId(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.EMPTY); + } else if (assignment.getSkippedType() + == CSAssignment.SkippedType.OTHER) { + ActivitiesLogger.APP.finishSkippedAppAllocationRecording( + activitiesManager, application.getApplicationId(), + ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY); application.updateNodeInfoForAMDiagnostics(node); - } else if (assignment.getSkippedType() == CSAssignment.SkippedType.QUEUE_LIMIT) { - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, parent.getQueuePath(), - getQueuePath(), ActivityState.REJECTED, - () -> ActivityDiagnosticConstant.QUEUE_DO_NOT_HAVE_ENOUGH_HEADROOM + " from " - + application.getApplicationId()); + } else if (assignment.getSkippedType() + == CSAssignment.SkippedType.QUEUE_LIMIT) { + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + parent.getQueuePath(), getQueuePath(), ActivityState.REJECTED, + () -> ActivityDiagnosticConstant.QUEUE_DO_NOT_HAVE_ENOUGH_HEADROOM + + " from " + application.getApplicationId()); return assignment; - } else { + } else{ // If we don't allocate anything, and it is not skipped by application, // we will return to respect FIFO of applications - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, parent.getQueuePath(), - getQueuePath(), ActivityState.SKIPPED, + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + parent.getQueuePath(), getQueuePath(), ActivityState.SKIPPED, ActivityDiagnosticConstant.QUEUE_SKIPPED_TO_RESPECT_FIFO); - ActivitiesLogger.APP.finishSkippedAppAllocationRecording(activitiesManager, - application.getApplicationId(), ActivityState.SKIPPED, - ActivityDiagnosticConstant.EMPTY); + ActivitiesLogger.APP.finishSkippedAppAllocationRecording( + activitiesManager, application.getApplicationId(), + ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY); return CSAssignment.NULL_ASSIGNMENT; } } - ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, parent.getQueuePath(), - getQueuePath(), ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY); + ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node, + parent.getQueuePath(), getQueuePath(), ActivityState.SKIPPED, + ActivityDiagnosticConstant.EMPTY); return CSAssignment.NULL_ASSIGNMENT; } @@ -1293,13 +1335,14 @@ public boolean accept(Resource cluster, if (allocation.getAllocateFromReservedContainer() == null) { readLock.lock(); try { - FiCaSchedulerApp app = schedulerContainer.getSchedulerApplicationAttempt(); + FiCaSchedulerApp app = + schedulerContainer.getSchedulerApplicationAttempt(); String username = app.getUser(); String p = schedulerContainer.getNodePartition(); // check user-limit - Resource userLimit = - computeUserLimitAndSetHeadroom(app, cluster, p, allocation.getSchedulingMode(), null); + Resource userLimit = computeUserLimitAndSetHeadroom(app, cluster, p, + allocation.getSchedulingMode(), null); // Deduct resources that we can release User user = getUser(username); @@ -1308,11 +1351,13 @@ public boolean accept(Resource cluster, return false; } Resource usedResource = Resources.clone(user.getUsed(p)); - Resources.subtractFrom(usedResource, request.getTotalReleasedResource()); + Resources.subtractFrom(usedResource, + request.getTotalReleasedResource()); - if (Resources.greaterThan(resourceCalculator, cluster, usedResource, userLimit)) { - LOG.debug("Used resource={} exceeded user-limit={}", usedResource, - userLimit); + if (Resources.greaterThan(resourceCalculator, cluster, usedResource, + userLimit)) { + LOG.debug("Used resource={} exceeded user-limit={}", + usedResource, userLimit); return false; } } finally { @@ -1335,34 +1380,42 @@ private void internalReleaseContainer(Resource clusterResource, if (rmContainer.getState() == RMContainerState.RESERVED) { // For other reserved containers // This is a reservation exchange, complete previous reserved container - completedContainer(clusterResource, schedulerContainer.getSchedulerApplicationAttempt(), - schedulerContainer.getSchedulerNode(), rmContainer, - SchedulerUtils.createAbnormalContainerStatus(rmContainer.getContainerId(), - SchedulerUtils.UNRESERVED_CONTAINER), RMContainerEventType.RELEASED, null, false); + completedContainer(clusterResource, + schedulerContainer.getSchedulerApplicationAttempt(), + schedulerContainer.getSchedulerNode(), rmContainer, SchedulerUtils + .createAbnormalContainerStatus(rmContainer.getContainerId(), + SchedulerUtils.UNRESERVED_CONTAINER), + RMContainerEventType.RELEASED, null, false); } - } else { + } else{ // When trying to preempt containers from different queue -- this // is for lazy preemption feature (kill preemption candidate in scheduling // cycle). targetLeafQueue.completedContainer(clusterResource, schedulerContainer.getSchedulerApplicationAttempt(), - schedulerContainer.getSchedulerNode(), schedulerContainer.getRmContainer(), - SchedulerUtils.createPreemptedContainerStatus(rmContainer.getContainerId(), - SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL, null, false); + schedulerContainer.getSchedulerNode(), + schedulerContainer.getRmContainer(), SchedulerUtils + .createPreemptedContainerStatus(rmContainer.getContainerId(), + SchedulerUtils.PREEMPTED_CONTAINER), + RMContainerEventType.KILL, null, false); } } private void releaseContainers(Resource clusterResource, ResourceCommitRequest request) { - for (SchedulerContainer c : request.getContainersToRelease()) { + for (SchedulerContainer c : request + .getContainersToRelease()) { internalReleaseContainer(clusterResource, c); } // Handle container reservation looking, or lazy preemption case: - if (null != request.getContainersToAllocate() && !request.getContainersToAllocate().isEmpty()) { - for (ContainerAllocationProposal context : request.getContainersToAllocate()) { + if (null != request.getContainersToAllocate() && !request + .getContainersToAllocate().isEmpty()) { + for (ContainerAllocationProposal context : request + .getContainersToAllocate()) { if (null != context.getToRelease()) { - for (SchedulerContainer c : context.getToRelease()) { + for (SchedulerContainer c : context + .getToRelease()) { internalReleaseContainer(clusterResource, c); } } @@ -1380,10 +1433,10 @@ public void apply(Resource cluster, writeLock.lock(); try { if (request.anythingAllocatedOrReserved()) { - ContainerAllocationProposal allocation = - request.getFirstAllocatedOrReservedContainer(); - SchedulerContainer schedulerContainer = - allocation.getAllocatedOrReservedContainer(); + ContainerAllocationProposal + allocation = request.getFirstAllocatedOrReservedContainer(); + SchedulerContainer + schedulerContainer = allocation.getAllocatedOrReservedContainer(); // Do not modify queue when allocation from reserved container if (allocation.getAllocateFromReservedContainer() == null) { @@ -1392,16 +1445,19 @@ public void apply(Resource cluster, applyToParentQueue = true; // Book-keeping // Note: Update headroom to account for current allocation too... - allocateResource(cluster, schedulerContainer.getSchedulerApplicationAttempt(), - allocation.getAllocatedOrReservedResource(), schedulerContainer.getNodePartition(), + allocateResource(cluster, + schedulerContainer.getSchedulerApplicationAttempt(), + allocation.getAllocatedOrReservedResource(), + schedulerContainer.getNodePartition(), schedulerContainer.getRmContainer()); - orderingPolicy.containerAllocated(schedulerContainer.getSchedulerApplicationAttempt(), + orderingPolicy.containerAllocated( + schedulerContainer.getSchedulerApplicationAttempt(), schedulerContainer.getRmContainer()); } // Update reserved resource - if (Resources.greaterThan(resourceCalculator, cluster, request.getTotalReservedResource(), - Resources.none())) { + if (Resources.greaterThan(resourceCalculator, cluster, + request.getTotalReservedResource(), Resources.none())) { incReservedResource(schedulerContainer.getNodePartition(), request.getTotalReservedResource()); } @@ -1415,6 +1471,7 @@ public void apply(Resource cluster, } } + protected Resource getHeadroom(User user, Resource queueCurrentLimit, Resource clusterResource, FiCaSchedulerApp application) { return getHeadroom(user, queueCurrentLimit, clusterResource, application, @@ -1425,8 +1482,9 @@ protected Resource getHeadroom(User user, Resource queueCurrentLimit, Resource clusterResource, FiCaSchedulerApp application, String partition) { return getHeadroom(user, queueCurrentLimit, clusterResource, - getResourceLimitForActiveUsers(application.getUser(), clusterResource, partition, - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), partition); + getResourceLimitForActiveUsers(application.getUser(), clusterResource, + partition, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), + partition); } private Resource getHeadroom(User user, @@ -1455,30 +1513,36 @@ private Resource getHeadroom(User user, * headroom = min (unused resourcelimit of a label, calculated headroom ) */ currentPartitionResourceLimit = - partition.equals(RMNodeLabelsManager.NO_LABEL) ? currentPartitionResourceLimit : - getQueueMaxResource(partition); + partition.equals(RMNodeLabelsManager.NO_LABEL) + ? currentPartitionResourceLimit + : getQueueMaxResource(partition); Resource headroom = Resources.componentwiseMin( - Resources.subtractNonNegative(userLimitResource, user.getUsed(partition)), + Resources.subtractNonNegative(userLimitResource, + user.getUsed(partition)), Resources.subtractNonNegative(currentPartitionResourceLimit, usageTracker.getQueueUsage().getUsed(partition))); // Normalize it before return - headroom = Resources.roundDown(resourceCalculator, headroom, - queueAllocationSettings.getMinimumAllocation()); + headroom = + Resources.roundDown(resourceCalculator, headroom, + queueAllocationSettings.getMinimumAllocation()); //headroom = min (unused resourcelimit of a label, calculated headroom ) - Resource clusterPartitionResource = labelManager.getResourceByLabel(partition, clusterResource); - Resource clusterFreePartitionResource = Resources.subtract(clusterPartitionResource, - csContext.getClusterResourceUsage().getUsed(partition)); - headroom = - Resources.min(resourceCalculator, clusterPartitionResource, clusterFreePartitionResource, - headroom); + Resource clusterPartitionResource = + labelManager.getResourceByLabel(partition, clusterResource); + Resource clusterFreePartitionResource = + Resources.subtract(clusterPartitionResource, + queueContext.getClusterResourceUsage().getUsed(partition)); + headroom = Resources.min(resourceCalculator, clusterPartitionResource, + clusterFreePartitionResource, headroom); return headroom; } - private void setQueueResourceLimitsInfo(Resource clusterResource) { + private void setQueueResourceLimitsInfo( + Resource clusterResource) { synchronized (queueResourceLimitsInfo) { - queueResourceLimitsInfo.setQueueCurrentLimit(cachedResourceLimitsForHeadroom.getLimit()); + queueResourceLimitsInfo.setQueueCurrentLimit(cachedResourceLimitsForHeadroom + .getLimit()); queueResourceLimitsInfo.setClusterResource(clusterResource); } } @@ -1486,8 +1550,8 @@ private void setQueueResourceLimitsInfo(Resource clusterResource) { // It doesn't necessarily to hold application's lock here. @Lock({AbstractLeafQueue.class}) Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application, - Resource clusterResource, String nodePartition, SchedulingMode schedulingMode, - Resource userLimit) { + Resource clusterResource, String nodePartition, + SchedulingMode schedulingMode, Resource userLimit) { String user = application.getUser(); User queueUser = getUser(user); if (queueUser == null) { @@ -1498,25 +1562,26 @@ Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application, // Compute user limit respect requested labels, // TODO, need consider headroom respect labels also if (userLimit == null) { - userLimit = - getResourceLimitForActiveUsers(application.getUser(), clusterResource, nodePartition, - schedulingMode); + userLimit = getResourceLimitForActiveUsers(application.getUser(), + clusterResource, nodePartition, schedulingMode); } setQueueResourceLimitsInfo(clusterResource); - Resource headroom = usageTracker.getMetrics().getUserMetrics(user) == null ? Resources.none() : - getHeadroom(queueUser, cachedResourceLimitsForHeadroom.getLimit(), clusterResource, - userLimit, nodePartition); + Resource headroom = + usageTracker.getMetrics().getUserMetrics(user) == null ? Resources.none() : + getHeadroom(queueUser, cachedResourceLimitsForHeadroom.getLimit(), + clusterResource, userLimit, nodePartition); if (LOG.isDebugEnabled()) { - LOG.debug( - "Headroom calculation for user " + user + ": " + " userLimit=" + userLimit - + " queueMaxAvailRes=" + cachedResourceLimitsForHeadroom.getLimit() + " consumed=" - + queueUser.getUsed() + " partition=" + nodePartition); + LOG.debug("Headroom calculation for user " + user + ": " + " userLimit=" + + userLimit + " queueMaxAvailRes=" + + cachedResourceLimitsForHeadroom.getLimit() + " consumed=" + + queueUser.getUsed() + " partition=" + + nodePartition); } - CapacityHeadroomProvider headroomProvider = - new CapacityHeadroomProvider(queueUser, this, application, queueResourceLimitsInfo); + CapacityHeadroomProvider headroomProvider = new CapacityHeadroomProvider( + queueUser, this, application, queueResourceLimitsInfo); application.setHeadroomProvider(headroomProvider); @@ -1553,10 +1618,11 @@ public boolean getRackLocalityFullReset() { * RESPECT_PARTITION_EXCLUSIVITY/IGNORE_PARTITION_EXCLUSIVITY * @return Computed User Limit */ - public Resource getResourceLimitForActiveUsers(String userName, Resource clusterResource, - String nodePartition, SchedulingMode schedulingMode) { - return usersManager.getComputedResourceLimitForActiveUsers(userName, clusterResource, - nodePartition, schedulingMode); + public Resource getResourceLimitForActiveUsers(String userName, + Resource clusterResource, String nodePartition, + SchedulingMode schedulingMode) { + return usersManager.getComputedResourceLimitForActiveUsers(userName, + clusterResource, nodePartition, schedulingMode); } /** @@ -1572,10 +1638,11 @@ public Resource getResourceLimitForActiveUsers(String userName, Resource cluster * RESPECT_PARTITION_EXCLUSIVITY/IGNORE_PARTITION_EXCLUSIVITY * @return Computed User Limit */ - public Resource getResourceLimitForAllUsers(String userName, Resource clusterResource, - String nodePartition, SchedulingMode schedulingMode) { - return usersManager.getComputedResourceLimitForAllUsers(userName, clusterResource, - nodePartition, schedulingMode); + public Resource getResourceLimitForAllUsers(String userName, + Resource clusterResource, String nodePartition, + SchedulingMode schedulingMode) { + return usersManager.getComputedResourceLimitForAllUsers(userName, + clusterResource, nodePartition, schedulingMode); } @Private @@ -1595,24 +1662,27 @@ protected boolean canAssignToUser(Resource clusterResource, // 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(nodePartition), - limit)) { + if (Resources.greaterThan(resourceCalculator, clusterResource, + 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) { if (Resources.lessThanOrEqual(resourceCalculator, clusterResource, - Resources.subtract(user.getUsed(), application.getCurrentReservation()), limit)) { + Resources.subtract(user.getUsed(), + application.getCurrentReservation()), limit)) { if (LOG.isDebugEnabled()) { LOG.debug("User " + userName + " in queue " + getQueuePath() - + " will exceed limit based on reservations - " + " consumed: " + user.getUsed() - + " reserved: " + application.getCurrentReservation() + " limit: " + limit); + + " will exceed limit based on reservations - " + + " consumed: " + user.getUsed() + " reserved: " + application + .getCurrentReservation() + " limit: " + limit); } - Resource amountNeededToUnreserve = - Resources.subtract(user.getUsed(nodePartition), limit); + Resource amountNeededToUnreserve = Resources.subtract( + user.getUsed(nodePartition), limit); // we can only acquire a new container if we unreserve first to // respect user-limit - currentResourceLimits.setAmountNeededUnreserve(amountNeededToUnreserve); + currentResourceLimits.setAmountNeededUnreserve( + amountNeededToUnreserve); return true; } } @@ -1639,10 +1709,10 @@ protected void setDynamicQueueProperties(CapacitySchedulerConfiguration configur super.setDynamicQueueProperties(configuration); } - private void updateSchedulerHealthForCompletedContainer(RMContainer rmContainer, - ContainerStatus containerStatus) { + private void updateSchedulerHealthForCompletedContainer( + RMContainer rmContainer, ContainerStatus containerStatus) { // Update SchedulerHealth for released / preempted container - SchedulerHealth schedulerHealth = csContext.getSchedulerHealth(); + SchedulerHealth schedulerHealth = queueContext.getSchedulerHealth(); if (null == schedulerHealth) { // Only do update if we have schedulerHealth return; @@ -1653,8 +1723,9 @@ private void updateSchedulerHealthForCompletedContainer(RMContainer rmContainer, rmContainer.getContainerId(), getQueuePath()); schedulerHealth.updateSchedulerPreemptionCounts(1); } else { - schedulerHealth.updateRelease(csContext.getLastNodeUpdateTime(), - rmContainer.getAllocatedNode(), rmContainer.getContainerId(), getQueuePath()); + schedulerHealth.updateRelease(queueContext.getLastNodeUpdateTime(), + rmContainer.getAllocatedNode(), rmContainer.getContainerId(), + getQueuePath()); } } @@ -1666,13 +1737,15 @@ private void updateSchedulerHealthForCompletedContainer(RMContainer rmContainer, * @param nodePartition * Partition */ - public void recalculateQueueUsageRatio(Resource clusterResource, String nodePartition) { + public void recalculateQueueUsageRatio(Resource clusterResource, + String nodePartition) { writeLock.lock(); try { ResourceUsage queueResourceUsage = getQueueResourceUsage(); if (nodePartition == null) { - for (String partition : Sets.union(getQueueCapacities().getNodePartitionsSet(), + for (String partition : Sets.union( + getQueueCapacities().getNodePartitionsSet(), queueResourceUsage.getNodePartitionsSet())) { usersManager.updateUsageRatio(partition, clusterResource); } @@ -1685,9 +1758,10 @@ public void recalculateQueueUsageRatio(Resource clusterResource, String nodePart } @Override - public void completedContainer(Resource clusterResource, FiCaSchedulerApp application, - FiCaSchedulerNode node, RMContainer rmContainer, ContainerStatus containerStatus, - RMContainerEventType event, CSQueue childQueue, boolean sortQueues) { + public void completedContainer(Resource clusterResource, + FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer, + ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue, + boolean sortQueues) { // Update SchedulerHealth for released / preempted container updateSchedulerHealthForCompletedContainer(rmContainer, containerStatus); @@ -1704,10 +1778,11 @@ public void completedContainer(Resource clusterResource, FiCaSchedulerApp applic // happen under scheduler's lock... // So, this is, in effect, a transaction across application & node if (rmContainer.getState() == RMContainerState.RESERVED) { - removed = application.unreserve(rmContainer.getReservedSchedulerKey(), node, rmContainer); - } else { - removed = application.containerCompleted(rmContainer, containerStatus, event, - node.getPartition()); + removed = application.unreserve(rmContainer.getReservedSchedulerKey(), + node, rmContainer); + } else{ + removed = application.containerCompleted(rmContainer, containerStatus, + event, node.getPartition()); node.releaseContainer(rmContainer.getContainerId(), false); } @@ -1725,16 +1800,20 @@ public void completedContainer(Resource clusterResource, FiCaSchedulerApp applic writeLock.unlock(); } + if (removed) { // Inform the parent queue _outside_ of the leaf-queue lock - parent.completedContainer(clusterResource, application, node, rmContainer, null, event, - this, sortQueues); + parent.completedContainer(clusterResource, application, node, + rmContainer, null, event, this, sortQueues); } } // Notify PreemptionManager - csContext.getPreemptionManager().removeKillableContainer( - new KillableContainer(rmContainer, node.getPartition(), getQueuePath())); + queueContext.getPreemptionManager().removeKillableContainer( + new KillableContainer( + rmContainer, + node.getPartition(), + getQueuePath())); // Update preemption metrics if exit status is PREEMPTED if (containerStatus != null @@ -1743,20 +1822,23 @@ public void completedContainer(Resource clusterResource, FiCaSchedulerApp applic } } - void allocateResource(Resource clusterResource, SchedulerApplicationAttempt application, - Resource resource, String nodePartition, RMContainer rmContainer) { + void allocateResource(Resource clusterResource, + SchedulerApplicationAttempt application, Resource resource, + String nodePartition, RMContainer rmContainer) { writeLock.lock(); try { super.allocateResource(clusterResource, resource, nodePartition); // handle ignore exclusivity container - if (null != rmContainer && rmContainer.getNodeLabelExpression() - .equals(RMNodeLabelsManager.NO_LABEL) && !nodePartition.equals( + if (null != rmContainer && rmContainer.getNodeLabelExpression().equals( + RMNodeLabelsManager.NO_LABEL) && !nodePartition.equals( RMNodeLabelsManager.NO_LABEL)) { TreeSet rmContainers = null; - if (null == (rmContainers = ignorePartitionExclusivityRMContainers.get(nodePartition))) { + if (null == (rmContainers = ignorePartitionExclusivityRMContainers.get( + nodePartition))) { rmContainers = new TreeSet<>(); - ignorePartitionExclusivityRMContainers.put(nodePartition, rmContainers); + ignorePartitionExclusivityRMContainers.put(nodePartition, + rmContainers); } rmContainers.add(rmContainer); } @@ -1766,17 +1848,18 @@ void allocateResource(Resource clusterResource, SchedulerApplicationAttempt appl // Increment user's resource usage. User user = usersManager.updateUserResourceUsage(userName, resource, - nodePartition, true); + queueContext.getClusterResource(), nodePartition, true); Resource partitionHeadroom = Resources.createResource(0, 0); if (usageTracker.getMetrics().getUserMetrics(userName) != null) { - partitionHeadroom = - getHeadroom(user, cachedResourceLimitsForHeadroom.getLimit(), clusterResource, - getResourceLimitForActiveUsers(userName, clusterResource, nodePartition, - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodePartition); + partitionHeadroom = getHeadroom(user, + cachedResourceLimitsForHeadroom.getLimit(), clusterResource, + getResourceLimitForActiveUsers(userName, clusterResource, + nodePartition, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), + nodePartition); } - usageTracker.getMetrics() - .setAvailableResourcesToUser(nodePartition, userName, partitionHeadroom); + usageTracker.getMetrics().setAvailableResourcesToUser(nodePartition, userName, + partitionHeadroom); if (LOG.isDebugEnabled()) { LOG.debug(getQueuePath() + " user=" + userName + " used=" @@ -1789,18 +1872,20 @@ void allocateResource(Resource clusterResource, SchedulerApplicationAttempt appl } } - void releaseResource(Resource clusterResource, FiCaSchedulerApp application, Resource resource, - String nodePartition, RMContainer rmContainer) { + void releaseResource(Resource clusterResource, + FiCaSchedulerApp application, Resource resource, String nodePartition, + RMContainer rmContainer) { writeLock.lock(); try { super.releaseResource(clusterResource, resource, nodePartition); // handle ignore exclusivity container - if (null != rmContainer && rmContainer.getNodeLabelExpression() - .equals(RMNodeLabelsManager.NO_LABEL) && !nodePartition.equals( + if (null != rmContainer && rmContainer.getNodeLabelExpression().equals( + RMNodeLabelsManager.NO_LABEL) && !nodePartition.equals( RMNodeLabelsManager.NO_LABEL)) { if (ignorePartitionExclusivityRMContainers.containsKey(nodePartition)) { - Set rmContainers = ignorePartitionExclusivityRMContainers.get(nodePartition); + Set rmContainers = + ignorePartitionExclusivityRMContainers.get(nodePartition); rmContainers.remove(rmContainer); if (rmContainers.isEmpty()) { ignorePartitionExclusivityRMContainers.remove(nodePartition); @@ -1811,17 +1896,18 @@ void releaseResource(Resource clusterResource, FiCaSchedulerApp application, Res // Update user metrics String userName = application.getUser(); User user = usersManager.updateUserResourceUsage(userName, resource, - nodePartition, false); + queueContext.getClusterResource(), nodePartition, false); Resource partitionHeadroom = Resources.createResource(0, 0); if (usageTracker.getMetrics().getUserMetrics(userName) != null) { - partitionHeadroom = - getHeadroom(user, cachedResourceLimitsForHeadroom.getLimit(), clusterResource, - getResourceLimitForActiveUsers(userName, clusterResource, nodePartition, - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodePartition); + partitionHeadroom = getHeadroom(user, + cachedResourceLimitsForHeadroom.getLimit(), clusterResource, + getResourceLimitForActiveUsers(userName, clusterResource, + nodePartition, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), + nodePartition); } - usageTracker.getMetrics() - .setAvailableResourcesToUser(nodePartition, userName, partitionHeadroom); + usageTracker.getMetrics().setAvailableResourcesToUser(nodePartition, userName, + partitionHeadroom); if (LOG.isDebugEnabled()) { LOG.debug( @@ -1834,19 +1920,20 @@ void releaseResource(Resource clusterResource, FiCaSchedulerApp application, Res } } - private void updateCurrentResourceLimits(ResourceLimits currentResourceLimits, - Resource clusterResource) { + private void updateCurrentResourceLimits( + ResourceLimits currentResourceLimits, Resource clusterResource) { // TODO: need consider non-empty node labels when resource limits supports // node labels // Even if ParentQueue will set limits respect child's max queue capacity, // but when allocating reserved container, CapacityScheduler doesn't do // this. So need cap limits by queue's max capacity here. - this.cachedResourceLimitsForHeadroom = new ResourceLimits(currentResourceLimits.getLimit()); - Resource queueMaxResource = getEffectiveMaxCapacityDown(RMNodeLabelsManager.NO_LABEL, - queueAllocationSettings.getMinimumAllocation()); - this.cachedResourceLimitsForHeadroom.setLimit( - Resources.min(resourceCalculator, clusterResource, queueMaxResource, - currentResourceLimits.getLimit())); + this.cachedResourceLimitsForHeadroom = + new ResourceLimits(currentResourceLimits.getLimit()); + Resource queueMaxResource = getEffectiveMaxCapacityDown( + RMNodeLabelsManager.NO_LABEL, queueAllocationSettings.getMinimumAllocation()); + this.cachedResourceLimitsForHeadroom.setLimit(Resources.min( + resourceCalculator, clusterResource, queueMaxResource, + currentResourceLimits.getLimit())); } @Override @@ -1861,7 +1948,7 @@ public void updateClusterResource(Resource clusterResource, super.updateEffectiveResources(clusterResource); // Update maximum applications for the queue and for users - updateMaximumApplications(csContext.getConfiguration()); + updateMaximumApplications(queueContext.getConfiguration()); updateCurrentResourceLimits(currentResourceLimits, clusterResource); @@ -1874,12 +1961,12 @@ public void updateClusterResource(Resource clusterResource, recalculateQueueUsageRatio(clusterResource, null); // Update metrics - CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, this, labelManager, - null); + CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, + this, labelManager, null); // Update configured capacity/max-capacity for default partition only CSQueueUtils.updateConfiguredCapacityMetrics(resourceCalculator, - labelManager.getResourceByLabel(null, clusterResource), RMNodeLabelsManager.NO_LABEL, - this); + labelManager.getResourceByLabel(null, clusterResource), + RMNodeLabelsManager.NO_LABEL, this); // queue metrics are updated, more resource may be available // activate the pending applications if possible @@ -1890,8 +1977,10 @@ public void updateClusterResource(Resource clusterResource, usersManager.userLimitNeedsRecompute(); // Update application properties - for (FiCaSchedulerApp application : orderingPolicy.getSchedulableEntities()) { - computeUserLimitAndSetHeadroom(application, clusterResource, RMNodeLabelsManager.NO_LABEL, + for (FiCaSchedulerApp application : orderingPolicy + .getSchedulableEntities()) { + computeUserLimitAndSetHeadroom(application, clusterResource, + RMNodeLabelsManager.NO_LABEL, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY, null); } } finally { @@ -1902,14 +1991,16 @@ public void updateClusterResource(Resource clusterResource, @Override public void incUsedResource(String nodeLabel, Resource resourceToInc, SchedulerApplicationAttempt application) { - usersManager.updateUserResourceUsage(application.getUser(), resourceToInc, nodeLabel, true); + usersManager.updateUserResourceUsage(application.getUser(), resourceToInc, + queueContext.getClusterResource(), nodeLabel, true); super.incUsedResource(nodeLabel, resourceToInc, application); } @Override public void decUsedResource(String nodeLabel, Resource resourceToDec, SchedulerApplicationAttempt application) { - usersManager.updateUserResourceUsage(application.getUser(), resourceToDec, nodeLabel, false); + usersManager.updateUserResourceUsage(application.getUser(), resourceToDec, + queueContext.getClusterResource(), nodeLabel, false); super.decUsedResource(nodeLabel, resourceToDec, application); } @@ -1920,7 +2011,8 @@ public void incAMUsedResource(String nodeLabel, Resource resourceToInc, return; } - user.getResourceUsage().incAMUsed(nodeLabel, resourceToInc); + user.getResourceUsage().incAMUsed(nodeLabel, + resourceToInc); // ResourceUsage has its own lock, no addition lock needs here. usageTracker.getQueueUsage().incAMUsed(nodeLabel, resourceToInc); } @@ -1932,7 +2024,8 @@ public void decAMUsedResource(String nodeLabel, Resource resourceToDec, return; } - user.getResourceUsage().decAMUsed(nodeLabel, resourceToDec); + user.getResourceUsage().decAMUsed(nodeLabel, + resourceToDec); // ResourceUsage has its own lock, no addition lock needs here. usageTracker.getQueueUsage().decAMUsed(nodeLabel, resourceToDec); } @@ -1949,9 +2042,11 @@ public void recoverContainer(Resource clusterResource, // Careful! Locking order is important! writeLock.lock(); try { - FiCaSchedulerNode node = csContext.getNode(rmContainer.getContainer().getNodeId()); - allocateResource(clusterResource, attempt, rmContainer.getContainer().getResource(), - node.getPartition(), rmContainer); + FiCaSchedulerNode node = queueContext.getNode( + rmContainer.getContainer().getNodeId()); + allocateResource(clusterResource, attempt, + rmContainer.getContainer().getResource(), node.getPartition(), + rmContainer); } finally { writeLock.unlock(); } @@ -1963,22 +2058,24 @@ public void recoverContainer(Resource clusterResource, * Obtain (read-only) collection of pending applications. */ public Collection getPendingApplications() { - return Collections.unmodifiableCollection(pendingOrderingPolicy.getSchedulableEntities()); + return Collections.unmodifiableCollection(pendingOrderingPolicy + .getSchedulableEntities()); } /** * Obtain (read-only) collection of active applications. */ public Collection getApplications() { - return Collections.unmodifiableCollection(orderingPolicy.getSchedulableEntities()); + return Collections.unmodifiableCollection(orderingPolicy + .getSchedulableEntities()); } /** * Obtain (read-only) collection of all applications. */ public Collection getAllApplications() { - Collection apps = - new HashSet(pendingOrderingPolicy.getSchedulableEntities()); + Collection apps = new HashSet( + pendingOrderingPolicy.getSchedulableEntities()); apps.addAll(orderingPolicy.getSchedulableEntities()); return Collections.unmodifiableCollection(apps); @@ -1992,6 +2089,7 @@ public Collection getAllApplications() { * Total pending for the queue = * sum(for each user(min((user's headroom), sum(user's pending requests)))) * NOTE: + * @param clusterResources clusterResource * @param partition node partition * @param deductReservedFromPending When a container is reserved in CS, @@ -2071,7 +2169,7 @@ public void attachContainer(Resource clusterResource, if (application != null && rmContainer != null && rmContainer.getExecutionType() == ExecutionType.GUARANTEED) { FiCaSchedulerNode node = - csContext.getNode(rmContainer.getContainer().getNodeId()); + queueContext.getNode(rmContainer.getContainer().getNodeId()); allocateResource(clusterResource, application, rmContainer.getContainer() .getResource(), node.getPartition(), rmContainer); LOG.info("movedContainer" + " container=" + rmContainer.getContainer() @@ -2089,9 +2187,9 @@ public void attachContainer(Resource clusterResource, public void detachContainer(Resource clusterResource, FiCaSchedulerApp application, RMContainer rmContainer) { if (application != null && rmContainer != null - && rmContainer.getExecutionType() == ExecutionType.GUARANTEED) { + && rmContainer.getExecutionType() == ExecutionType.GUARANTEED) { FiCaSchedulerNode node = - csContext.getNode(rmContainer.getContainer().getNodeId()); + queueContext.getNode(rmContainer.getContainer().getNodeId()); releaseResource(clusterResource, application, rmContainer.getContainer() .getResource(), node.getPartition(), rmContainer); LOG.info("movedContainer" + " container=" + rmContainer.getContainer() @@ -2110,7 +2208,7 @@ public void detachContainer(Resource clusterResource, * this will be used by preemption policy. */ public Map> - getIgnoreExclusivityRMContainers() { + getIgnoreExclusivityRMContainers() { Map> clonedMap = new HashMap<>(); readLock.lock(); @@ -2156,8 +2254,7 @@ public void setMaxAMResourcePerQueuePercent( this.maxAMResourcePerQueuePercent = maxAMResourcePerQueuePercent; } - public OrderingPolicy - getOrderingPolicy() { + public OrderingPolicy getOrderingPolicy() { return orderingPolicy; } @@ -2202,8 +2299,7 @@ public void updateApplicationPriority(SchedulerApplication app } } - public OrderingPolicy - getPendingAppsOrderingPolicy() { + public OrderingPolicy getPendingAppsOrderingPolicy() { return pendingOrderingPolicy; } @@ -2376,7 +2472,7 @@ List getCopyOfNonRunnableAppSchedulables() { @Override public boolean isEligibleForAutoDeletion() { return isDynamicQueue() && getNumApplications() == 0 - && csContext.getConfiguration(). + && queueContext.getConfiguration(). isAutoExpiredDeletionEnabled(this.getQueuePath()); } } 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/AbstractManagedParentQueue.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/AbstractManagedParentQueue.java index 7d149761cb0db..6d272184100ba 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/AbstractManagedParentQueue.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/AbstractManagedParentQueue.java @@ -44,9 +44,9 @@ public abstract class AbstractManagedParentQueue extends ParentQueue { protected AutoCreatedLeafQueueConfig leafQueueTemplate; protected AutoCreatedQueueManagementPolicy queueManagementPolicy = null; - public AbstractManagedParentQueue(CapacitySchedulerContext cs, + public AbstractManagedParentQueue(CapacitySchedulerQueueContext queueContext, String queueName, CSQueue parent, CSQueue old) throws IOException { - super(cs, queueName, parent, old); + super(queueContext, queueName, parent, old); } @Override @@ -55,7 +55,7 @@ public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) writeLock.lock(); try { // Set new configs - setupQueueConfigs(clusterResource, csContext.getConfiguration()); + setupQueueConfigs(clusterResource, queueContext.getConfiguration()); } finally { writeLock.unlock(); @@ -121,8 +121,7 @@ public CSQueue removeChildQueue(String childQueueName) CSQueue childQueue; writeLock.lock(); try { - childQueue = this.csContext.getCapacitySchedulerQueueManager().getQueue( - childQueueName); + childQueue = queueContext.getQueueManager().getQueue(childQueueName); if (childQueue != null) { removeChildQueue(childQueue); } else { @@ -176,14 +175,14 @@ protected CapacitySchedulerConfiguration initializeLeafQueueConfigs(String CapacitySchedulerConfiguration leafQueueConfigs = new CapacitySchedulerConfiguration(new Configuration(false), false); - Map rtProps = csContext + Map rtProps = queueContext .getConfiguration().getConfigurationProperties() .getPropertiesWithPrefix(YarnConfiguration.RESOURCE_TYPES + ".", true); for (Map.Entry entry : rtProps.entrySet()) { leafQueueConfigs.set(entry.getKey(), entry.getValue()); } - Map templateConfigs = csContext + Map templateConfigs = queueContext .getConfiguration().getConfigurationProperties() .getPropertiesWithPrefix(configPrefix, true); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueue.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/AutoCreatedLeafQueue.java index 57050b193abd6..2d818cd1556a5 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/AutoCreatedLeafQueue.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/AutoCreatedLeafQueue.java @@ -39,16 +39,11 @@ public class AutoCreatedLeafQueue extends AbstractAutoCreatedLeafQueue { private static final Logger LOG = LoggerFactory .getLogger(AutoCreatedLeafQueue.class); - public AutoCreatedLeafQueue(CapacitySchedulerContext cs, String queueName, + public AutoCreatedLeafQueue(CapacitySchedulerQueueContext queueContext, String queueName, ManagedParentQueue parent) throws IOException { - // TODO once YARN-10907 is merged the duplicated collection of - // leafQueueConfigs won't be necessary - super(cs, parent.getLeafQueueConfigs(queueName), - queueName, - parent, null); - super.setupQueueConfigs(cs.getClusterResource(), parent.getLeafQueueConfigs(queueName)); - - LOG.debug("Initialized AutoCreatedLeafQueue: name={}, fullname={}", queueName, getQueuePath()); + super(queueContext, queueName, parent, null); + super.setupQueueConfigs(queueContext.getClusterResource(), parent.getLeafQueueConfigs(queueName)); + updateCapacitiesToZero(); } @@ -74,8 +69,7 @@ public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) } } - public void reinitializeFromTemplate(AutoCreatedLeafQueueConfig - leafQueueTemplate) throws SchedulerDynamicEditException, IOException { + public void reinitializeFromTemplate(AutoCreatedLeafQueueConfig leafQueueTemplate) { writeLock.lock(); try { @@ -105,7 +99,7 @@ public void mergeCapacities(QueueCapacities capacities) { .getAbsoluteMaximumCapacity(nodeLabel)); Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel, - csContext.getClusterResource()); + queueContext.getClusterResource()); getQueueResourceQuotas().setEffectiveMinResource(nodeLabel, Resources.multiply(resourceByLabel, queueCapacities.getAbsoluteCapacity(nodeLabel))); @@ -133,12 +127,12 @@ protected void setDynamicQueueProperties( String parentTemplate = String.format("%s.%s", getParent().getQueuePath(), CapacitySchedulerConfiguration .AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX); - Set parentNodeLabels = csContext - .getCapacitySchedulerQueueManager().getConfiguredNodeLabels() + Set parentNodeLabels = queueContext + .getQueueManager().getConfiguredNodeLabelsForAllQueues() .getLabelsByQueue(parentTemplate); if (parentNodeLabels != null && parentNodeLabels.size() > 1) { - csContext.getCapacitySchedulerQueueManager().getConfiguredNodeLabels() + queueContext.getQueueManager().getConfiguredNodeLabelsForAllQueues() .setLabelsByQueue(getQueuePath(), new HashSet<>(parentNodeLabels)); } 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/AutoCreatedQueueManagementPolicy.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/AutoCreatedQueueManagementPolicy.java index 388e9d6233bd4..bf99c3f30c3ec 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/AutoCreatedQueueManagementPolicy.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/AutoCreatedQueueManagementPolicy.java @@ -27,17 +27,15 @@ public interface AutoCreatedQueueManagementPolicy { /** * Initialize policy - * @param schedulerContext Capacity Scheduler context + * @param parentQueue parent queue */ - void init(CapacitySchedulerContext schedulerContext, ParentQueue - parentQueue) throws IOException; + void init(ParentQueue parentQueue) throws IOException; /** * Reinitialize policy state ( if required ) - * @param schedulerContext Capacity Scheduler context + * @param parentQueue parent queue */ - void reinitialize(CapacitySchedulerContext schedulerContext, - ParentQueue parentQueue) throws IOException; + void reinitialize(ParentQueue parentQueue) throws IOException; /** * Get initial template for the specified leaf queue 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/CSQueuePreemptionSettings.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/CSQueuePreemptionSettings.java index 2cfd5a4310f82..56874888870b8 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/CSQueuePreemptionSettings.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/CSQueuePreemptionSettings.java @@ -26,11 +26,13 @@ public class CSQueuePreemptionSettings { public CSQueuePreemptionSettings( CSQueue queue, - CapacitySchedulerContext csContext, - CapacitySchedulerConfiguration configuration) { - this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(queue, csContext, configuration); + CapacitySchedulerConfiguration configuration, + CapacitySchedulerConfiguration originalSchedulerConfiguration) { + this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(queue, configuration, + originalSchedulerConfiguration); this.intraQueuePreemptionDisabledInHierarchy = - isIntraQueueHierarchyPreemptionDisabled(queue, csContext, configuration); + isIntraQueueHierarchyPreemptionDisabled(queue, configuration, + originalSchedulerConfiguration); } /** @@ -40,14 +42,14 @@ public CSQueuePreemptionSettings( * NOTE: Cross-queue preemptability is inherited from a queue's parent. * * @param q queue to check preemption state - * @param csContext * @param configuration capacity scheduler config * @return true if queue has cross-queue preemption disabled, false otherwise */ private boolean isQueueHierarchyPreemptionDisabled(CSQueue q, - CapacitySchedulerContext csContext, CapacitySchedulerConfiguration configuration) { + CapacitySchedulerConfiguration configuration, + CapacitySchedulerConfiguration originalSchedulerConfiguration) { boolean systemWidePreemption = - csContext.getConfiguration() + originalSchedulerConfiguration .getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS); CSQueue parentQ = q.getParent(); @@ -79,14 +81,14 @@ private boolean isQueueHierarchyPreemptionDisabled(CSQueue q, * NOTE: Intra-queue preemptability is inherited from a queue's parent. * * @param q queue to check intra-queue preemption state - * @param csContext * @param configuration capacity scheduler config * @return true if queue has intra-queue preemption disabled, false otherwise */ private boolean isIntraQueueHierarchyPreemptionDisabled(CSQueue q, - CapacitySchedulerContext csContext, CapacitySchedulerConfiguration configuration) { + CapacitySchedulerConfiguration configuration, + CapacitySchedulerConfiguration originalSchedulerConfiguration) { boolean systemWideIntraQueuePreemption = - csContext.getConfiguration().getBoolean( + originalSchedulerConfiguration.getBoolean( CapacitySchedulerConfiguration.INTRAQUEUE_PREEMPTION_ENABLED, CapacitySchedulerConfiguration .DEFAULT_INTRAQUEUE_PREEMPTION_ENABLED); @@ -109,7 +111,7 @@ private boolean isIntraQueueHierarchyPreemptionDisabled(CSQueue q, parentQ.getIntraQueuePreemptionDisabledInHierarchy()); } - public boolean getIntraQueuePreemptionDisabled() { + public boolean isIntraQueuePreemptionDisabled() { return intraQueuePreemptionDisabledInHierarchy || preemptionDisabled; } 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 befb82a70e22e..abd40a8062b53 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 @@ -183,6 +183,8 @@ public class CapacityScheduler extends private CapacitySchedulerQueueManager queueManager; + private CapacitySchedulerQueueContext queueContext; + private WorkflowPriorityMappingsManager workflowPriorityMappingsMgr; // timeout to join when we stop this service @@ -267,6 +269,11 @@ public CapacitySchedulerConfiguration getConfiguration() { return conf; } + @Override + public CapacitySchedulerQueueContext getQueueContext() { + return queueContext; + } + @Override public RMContainerTokenSecretManager getContainerTokenSecretManager() { return this.rmContext.getContainerTokenSecretManager(); @@ -319,6 +326,7 @@ void initScheduler(Configuration configuration) throws this.workflowPriorityMappingsMgr = new WorkflowPriorityMappingsManager(); this.activitiesManager = new ActivitiesManager(rmContext); activitiesManager.init(conf); + this.queueContext = new CapacitySchedulerQueueContext(this); initializeQueues(this.conf); this.isLazyPreemptionEnabled = conf.getLazyPreemptionEnabled(); this.assignMultipleEnabled = this.conf.getAssignMultipleEnabled(); @@ -844,6 +852,7 @@ private void initializeQueues(CapacitySchedulerConfiguration conf) @Lock(CapacityScheduler.class) private void reinitializeQueues(CapacitySchedulerConfiguration newConf) throws IOException { + queueContext.reinitialize(); this.queueManager.reinitializeQueues(newConf); updatePlacementRules(); 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 e88f83a44cbc2..2716ddebbdc6d 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 @@ -1658,6 +1658,12 @@ public Map> getConfiguredNodeLabelsByQueue() { return labelsByQueue; } + public Priority getClusterLevelApplicationMaxPriority() { + return Priority.newInstance(getInt( + YarnConfiguration.MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY, + YarnConfiguration.DEFAULT_CLUSTER_LEVEL_APPLICATION_PRIORITY)); + } + public Integer getDefaultApplicationPriorityConfPerQueue(String queue) { Integer defaultPriority = getInt(getQueuePrefix(queue) + DEFAULT_APPLICATION_PRIORITY, 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java index ae74989a7265f..1d0600f66807e 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java @@ -41,6 +41,8 @@ */ public interface CapacitySchedulerContext { CapacitySchedulerConfiguration getConfiguration(); + + CapacitySchedulerQueueContext getQueueContext(); Resource getMinimumResourceCapability(); 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/CapacitySchedulerQueueContext.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/CapacitySchedulerQueueContext.java new file mode 100644 index 0000000000000..e9ec3a0e49064 --- /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/CapacitySchedulerQueueContext.java @@ -0,0 +1,132 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; + +/** + * Class to store common queue related information, like instances + * to necessary manager classes or the global CapacityScheduler + * configuration. + */ +public class CapacitySchedulerQueueContext { + + // Manager classes + private final CapacitySchedulerContext csContext; + private final CapacitySchedulerQueueManager queueManager; + private final RMNodeLabelsManager labelManager; + private final PreemptionManager preemptionManager; + private final ActivitiesManager activitiesManager; + private final ResourceCalculator resourceCalculator; + + // CapacityScheduler configuration + private CapacitySchedulerConfiguration configuration; + + private Resource minimumAllocation; + + public CapacitySchedulerQueueContext(CapacitySchedulerContext csContext) { + this.csContext = csContext; + this.queueManager = csContext.getCapacitySchedulerQueueManager(); + this.labelManager = csContext.getRMContext().getNodeLabelManager(); + this.preemptionManager = csContext.getPreemptionManager(); + this.activitiesManager = csContext.getActivitiesManager(); + this.resourceCalculator = csContext.getResourceCalculator(); + + this.configuration = new CapacitySchedulerConfiguration(csContext.getConfiguration()); + this.minimumAllocation = csContext.getMinimumResourceCapability(); + } + + public void reinitialize() { + // When csConfProvider.loadConfiguration is called, the useLocalConfigurationProvider is + // correctly set to load the config entries from the capacity-scheduler.xml. + // For this reason there is no need to reload from it again. + this.configuration = new CapacitySchedulerConfiguration(csContext.getConfiguration(), false); + this.minimumAllocation = csContext.getMinimumResourceCapability(); + } + + public CapacitySchedulerQueueManager getQueueManager() { + return queueManager; + } + + public RMNodeLabelsManager getLabelManager() { + return labelManager; + } + + public PreemptionManager getPreemptionManager() { + return preemptionManager; + } + + public ActivitiesManager getActivitiesManager() { + return activitiesManager; + } + + public ResourceCalculator getResourceCalculator() { + return resourceCalculator; + } + + public CapacitySchedulerConfiguration getConfiguration() { + return configuration; + } + + public Resource getMinimumAllocation() { + return minimumAllocation; + } + + public Resource getClusterResource() { + return csContext.getClusterResource(); + } + + public ResourceUsage getClusterResourceUsage() { + return queueManager.getRootQueue().getQueueResourceUsage(); + } + + public SchedulerHealth getSchedulerHealth() { + return csContext.getSchedulerHealth(); + } + + public long getLastNodeUpdateTime() { + return csContext.getLastNodeUpdateTime(); + } + + public FiCaSchedulerNode getNode(NodeId nodeId) { + return csContext.getNode(nodeId); + } + + public FiCaSchedulerApp getApplicationAttempt( + ApplicationAttemptId applicationAttemptId) { + return csContext.getApplicationAttempt(applicationAttemptId); + } + + // TODO this is used in GuaranteedOrZeroCapacityOverTimePolicy, refactor the comparator there + public RMApp getRMApp(ApplicationId applicationId) { + return csContext.getRMContext().getRMApps().get(applicationId); + } +} 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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java index 407383d3bc0fc..c1669d0c763b2 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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java @@ -106,6 +106,11 @@ public CSQueue getRootQueue() { return this.root; } + @VisibleForTesting + protected void setRootQueue(CSQueue rootQueue) { + this.root = rootQueue; + } + @Override public Map getQueues() { return queues.getFullNameQueues(); @@ -167,7 +172,7 @@ public void setCapacitySchedulerContext( public void initializeQueues(CapacitySchedulerConfiguration conf) throws IOException { configuredNodeLabels = new ConfiguredNodeLabels(conf); - root = parseQueue(this.csContext, conf, null, + root = parseQueue(this.csContext.getQueueContext(), conf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, NOOP); setQueueAcls(authorizer, appPriorityACLManager, queues); labelManager.reinitializeQueueLabels(getQueueToLabels()); @@ -183,7 +188,7 @@ public void reinitializeQueues(CapacitySchedulerConfiguration newConf) // Parse new queues CSQueueStore newQueues = new CSQueueStore(); configuredNodeLabels = new ConfiguredNodeLabels(newConf); - CSQueue newRoot = parseQueue(this.csContext, newConf, null, + CSQueue newRoot = parseQueue(this.csContext.getQueueContext(), newConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues, NOOP); // When failing over, if using configuration store, don't validate queue @@ -215,7 +220,7 @@ public void reinitializeQueues(CapacitySchedulerConfiguration newConf) /** * Parse the queue from the configuration. - * @param csContext the CapacitySchedulerContext + * @param queueContext the CapacitySchedulerQueueContext * @param conf the CapacitySchedulerConfiguration * @param parent the parent queue * @param queueName the queue name @@ -226,7 +231,7 @@ public void reinitializeQueues(CapacitySchedulerConfiguration newConf) * @throws IOException */ static CSQueue parseQueue( - CapacitySchedulerContext csContext, + CapacitySchedulerQueueContext queueContext, CapacitySchedulerConfiguration conf, CSQueue parent, String queueName, CSQueueStore newQueues, @@ -265,7 +270,7 @@ static CSQueue parseQueue( // Check if the queue will be dynamically managed by the Reservation // system if (isReservableQueue) { - queue = new PlanQueue(csContext, queueName, parent, + queue = new PlanQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); //initializing the "internal" default queue, for SLS compatibility @@ -273,7 +278,7 @@ static CSQueue parseQueue( queueName + ReservationConstants.DEFAULT_QUEUE_SUFFIX; List childQueues = new ArrayList<>(); - ReservationQueue resQueue = new ReservationQueue(csContext, + ReservationQueue resQueue = new ReservationQueue(queueContext, defReservationId, (PlanQueue) queue); try { resQueue.setEntitlement(new QueueEntitlement(1.0f, 1.0f)); @@ -285,11 +290,11 @@ static CSQueue parseQueue( newQueues.add(resQueue); } else if (isAutoCreateEnabled) { - queue = new ManagedParentQueue(csContext, queueName, parent, + queue = new ManagedParentQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); } else{ - queue = new LeafQueue(csContext, queueName, parent, + queue = new LeafQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); // Used only for unit tests queue = hook.hook(queue); @@ -302,10 +307,10 @@ static CSQueue parseQueue( ParentQueue parentQueue; if (isAutoCreateEnabled) { - parentQueue = new ManagedParentQueue(csContext, queueName, parent, + parentQueue = new ManagedParentQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); } else{ - parentQueue = new ParentQueue(csContext, queueName, parent, + parentQueue = new ParentQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); } @@ -314,7 +319,7 @@ static CSQueue parseQueue( List childQueues = new ArrayList<>(); for (String childQueueName : childQueueNames) { - CSQueue childQueue = parseQueue(csContext, conf, queue, childQueueName, + CSQueue childQueue = parseQueue(queueContext, conf, queue, childQueueName, newQueues, oldQueues, hook); childQueues.add(childQueue); } @@ -633,7 +638,7 @@ public List determineMissingParents( * for all queues. * @return configured node labels */ - public ConfiguredNodeLabels getConfiguredNodeLabels() { + public ConfiguredNodeLabels getConfiguredNodeLabelsForAllQueues() { return configuredNodeLabels; } @@ -676,7 +681,7 @@ private AbstractLeafQueue createLegacyAutoQueue(QueuePath queue) (ManagedParentQueue) parentQueue; AutoCreatedLeafQueue autoCreatedLeafQueue = new AutoCreatedLeafQueue( - csContext, queue.getLeafName(), autoCreateEnabledParentQueue); + csContext.getQueueContext(), queue.getLeafName(), autoCreateEnabledParentQueue); addLegacyDynamicQueue(autoCreatedLeafQueue); return autoCreatedLeafQueue; 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 b9fa932f14141..ee53c14f8b0b0 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 @@ -31,26 +31,16 @@ public class LeafQueue extends AbstractLeafQueue { private static final Logger LOG = LoggerFactory.getLogger(LeafQueue.class); - @SuppressWarnings({ "unchecked", "rawtypes" }) - public LeafQueue(CapacitySchedulerContext cs, + public LeafQueue(CapacitySchedulerQueueContext queueContext, String queueName, CSQueue parent, CSQueue old) throws IOException { - this(cs, cs.getConfiguration(), queueName, parent, old, false); + this(queueContext, queueName, parent, old, false); } - public LeafQueue(CapacitySchedulerContext cs, - CapacitySchedulerConfiguration configuration, - String queueName, CSQueue parent, CSQueue old) throws IOException { - this(cs, configuration, queueName, parent, old, false); - } - - public LeafQueue(CapacitySchedulerContext cs, - CapacitySchedulerConfiguration configuration, + public LeafQueue(CapacitySchedulerQueueContext queueContext, String queueName, CSQueue parent, CSQueue old, boolean isDynamic) throws IOException { - super(cs, configuration, queueName, parent, old, isDynamic); - - setupQueueConfigs(cs.getClusterResource(), configuration); + super(queueContext, queueName, parent, old, isDynamic); - LOG.debug("LeafQueue: name={}, fullname={}", queueName, getQueuePath()); + setupQueueConfigs(queueContext.getClusterResource(), queueContext.getConfiguration()); } } 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/ManagedParentQueue.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/ManagedParentQueue.java index ddfb24bf6fce6..0aab2e412f396 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/ManagedParentQueue.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/ManagedParentQueue.java @@ -53,23 +53,18 @@ public class ManagedParentQueue extends AbstractManagedParentQueue { private static final Logger LOG = LoggerFactory.getLogger( ManagedParentQueue.class); - public ManagedParentQueue(final CapacitySchedulerContext cs, + public ManagedParentQueue(final CapacitySchedulerQueueContext queueContext, final String queueName, final CSQueue parent, final CSQueue old) throws IOException { - super(cs, queueName, parent, old); + super(queueContext, queueName, parent, old); shouldFailAutoCreationWhenGuaranteedCapacityExceeded = - csContext.getConfiguration() + queueContext.getConfiguration() .getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded( getQueuePath()); leafQueueTemplate = initializeLeafQueueConfigs().build(); - LOG.info( - "Created Managed Parent Queue: [{}] with capacity: [{}]" - + " with max capacity: [{}]", - queueName, super.getCapacity(), super.getMaximumCapacity()); - initializeQueueManagementPolicy(); } @@ -82,7 +77,7 @@ public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) validate(newlyParsedQueue); shouldFailAutoCreationWhenGuaranteedCapacityExceeded = - csContext.getConfiguration() + queueContext.getConfiguration() .getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded( getQueuePath()); @@ -133,23 +128,23 @@ public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) private void initializeQueueManagementPolicy() throws IOException { queueManagementPolicy = - csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass( + queueContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass( getQueuePath()); - queueManagementPolicy.init(csContext, this); + queueManagementPolicy.init(this); } private void reinitializeQueueManagementPolicy() throws IOException { AutoCreatedQueueManagementPolicy managementPolicy = - csContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass( + queueContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass( getQueuePath()); if (!(managementPolicy.getClass().equals( this.queueManagementPolicy.getClass()))) { queueManagementPolicy = managementPolicy; - queueManagementPolicy.init(csContext, this); + queueManagementPolicy.init(this); } else{ - queueManagementPolicy.reinitialize(csContext, this); + queueManagementPolicy.reinitialize(this); } } @@ -158,21 +153,25 @@ protected AutoCreatedLeafQueueConfig.Builder initializeLeafQueueConfigs() throws AutoCreatedLeafQueueConfig.Builder builder = new AutoCreatedLeafQueueConfig.Builder(); + CapacitySchedulerConfiguration configuration = + queueContext.getConfiguration(); + + // TODO load configs into CapacitySchedulerConfiguration instead of duplicating them String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix( - csContext.getConfiguration()); - //Load template configuration - CapacitySchedulerConfiguration conf = + configuration); + //Load template configuration into CapacitySchedulerConfiguration + CapacitySchedulerConfiguration autoCreatedTemplateConfig = super.initializeLeafQueueConfigs(leafQueueTemplateConfPrefix); - builder.configuration(conf); - QueuePath templateQueuePath = csContext.getConfiguration() + builder.configuration(autoCreatedTemplateConfig); + QueuePath templateQueuePath = configuration .getAutoCreatedQueueObjectTemplateConfPrefix(getQueuePath()); - Set templateConfiguredNodeLabels = csContext - .getCapacitySchedulerQueueManager().getConfiguredNodeLabels() + Set templateConfiguredNodeLabels = queueContext + .getQueueManager().getConfiguredNodeLabelsForAllQueues() .getLabelsByQueue(templateQueuePath.getFullPath()); for (String nodeLabel : templateConfiguredNodeLabels) { - Resource templateMinResource = conf.getMinimumResourceRequirement( - nodeLabel, csContext.getConfiguration() + Resource templateMinResource = autoCreatedTemplateConfig.getMinimumResourceRequirement( + nodeLabel, configuration .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()), resourceTypes); @@ -187,7 +186,7 @@ protected AutoCreatedLeafQueueConfig.Builder initializeLeafQueueConfigs() throws QueueCapacities queueCapacities = new QueueCapacities(false); CSQueueUtils.loadCapacitiesByLabelsFromConf(templateQueuePath, queueCapacities, - csContext.getConfiguration(), + configuration, templateConfiguredNodeLabels); @@ -205,35 +204,38 @@ protected AutoCreatedLeafQueueConfig.Builder initializeLeafQueueConfigs() throws } private void updateQueueCapacities(QueueCapacities queueCapacities) { + CapacitySchedulerConfiguration configuration = + queueContext.getConfiguration(); + for (String label : queueCapacities.getExistingNodeLabels()) { queueCapacities.setCapacity(label, - this.csContext.getResourceCalculator().divide( - this.csContext.getClusterResource(), - this.csContext.getConfiguration().getMinimumResourceRequirement( + resourceCalculator.divide( + queueContext.getClusterResource(), + configuration.getMinimumResourceRequirement( label, - this.csContext.getConfiguration() + configuration .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()), resourceTypes), getQueueResourceQuotas().getConfiguredMinResource(label))); - Resource childMaxResource = this.csContext.getConfiguration() + Resource childMaxResource = configuration .getMaximumResourceRequirement(label, - this.csContext.getConfiguration() + configuration .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()), resourceTypes); Resource parentMaxRes = getQueueResourceQuotas() .getConfiguredMaxResource(label); Resource effMaxResource = Resources.min( - this.csContext.getResourceCalculator(), - this.csContext.getClusterResource(), + resourceCalculator, + queueContext.getClusterResource(), childMaxResource.equals(Resources.none()) ? parentMaxRes : childMaxResource, parentMaxRes); queueCapacities.setMaximumCapacity( - label, this.csContext.getResourceCalculator().divide( - this.csContext.getClusterResource(), + label, resourceCalculator.divide( + queueContext.getClusterResource(), effMaxResource, getQueueResourceQuotas().getConfiguredMaxResource(label))); @@ -268,7 +270,7 @@ public void addChildQueue(CSQueue childQueue) "Expected child queue to be an instance of AutoCreatedLeafQueue"); } - CapacitySchedulerConfiguration conf = csContext.getConfiguration(); + CapacitySchedulerConfiguration conf = queueContext.getConfiguration(); ManagedParentQueue parentQueue = (ManagedParentQueue) childQueue.getParent(); @@ -322,8 +324,8 @@ public void addChildQueue(CSQueue childQueue) // Do one update cluster resource call to make sure all absolute resources // effective resources are updated. - updateClusterResource(this.csContext.getClusterResource(), - new ResourceLimits(this.csContext.getClusterResource())); + updateClusterResource(queueContext.getClusterResource(), + new ResourceLimits(queueContext.getClusterResource())); } finally { writeLock.unlock(); } @@ -427,12 +429,11 @@ public void validateQueueManagementChanges( + " Ignoring update " + queueManagementChanges); } - switch (queueManagementChange.getQueueAction()){ - case UPDATE_QUEUE: + if (queueManagementChange.getQueueAction() == + QueueManagementChange.QueueAction.UPDATE_QUEUE) { AutoCreatedLeafQueueConfig template = queueManagementChange.getUpdatedQueueTemplate(); ((AutoCreatedLeafQueue) childQueue).validateConfigurations(template); - break; } } @@ -442,14 +443,13 @@ private void applyQueueManagementChanges( List queueManagementChanges) throws SchedulerDynamicEditException, IOException { for (QueueManagementChange queueManagementChange : queueManagementChanges) { - switch (queueManagementChange.getQueueAction()){ - case UPDATE_QUEUE: + if (queueManagementChange.getQueueAction() == + QueueManagementChange.QueueAction.UPDATE_QUEUE) { AutoCreatedLeafQueue childQueueToBeUpdated = (AutoCreatedLeafQueue) queueManagementChange.getQueue(); //acquires write lock on leaf queue childQueueToBeUpdated.reinitializeFromTemplate( queueManagementChange.getUpdatedQueueTemplate()); - break; } } } @@ -465,7 +465,7 @@ public CapacitySchedulerConfiguration getLeafQueueConfigs( CapacitySchedulerConfiguration leafQueueConfigTemplate = new CapacitySchedulerConfiguration(new Configuration(false), false); for (final Iterator> iterator = - templateConfig.iterator(); iterator.hasNext(); ) { + templateConfig.iterator(); iterator.hasNext();) { Map.Entry confKeyValuePair = iterator.next(); final String name = confKeyValuePair.getKey().replaceFirst( CapacitySchedulerConfiguration 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 43391897cdfad..b2ff8995ff3cf 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 @@ -87,7 +87,6 @@ public class ParentQueue extends AbstractCSQueue { protected final List childQueues; private final boolean rootQueue; private volatile int numApplications; - private final CapacitySchedulerContext scheduler; private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); @@ -108,27 +107,20 @@ public class ParentQueue extends AbstractCSQueue { // after every time recalculation private volatile Map effectiveMinRatioPerResource; - public ParentQueue(CapacitySchedulerContext cs, + public ParentQueue(CapacitySchedulerQueueContext queueContext, String queueName, CSQueue parent, CSQueue old) throws IOException { - this(cs, cs.getConfiguration(), queueName, parent, old, false); - } - private ParentQueue(CapacitySchedulerContext cs, - CapacitySchedulerConfiguration csConf, String queueName, - CSQueue parent, - CSQueue old) throws IOException { - this(cs, csConf, queueName, parent, old, false); + this(queueContext, queueName, parent, old, false); } - private ParentQueue(CapacitySchedulerContext cs, - CapacitySchedulerConfiguration csConf, String queueName, CSQueue parent, - CSQueue old, boolean isDynamic) + private ParentQueue(CapacitySchedulerQueueContext queueContext, + String queueName, CSQueue parent, CSQueue old, boolean isDynamic) throws IOException { - super(cs, queueName, parent, old); + super(queueContext, queueName, parent, old); setDynamicQueue(isDynamic); - this.scheduler = cs; this.rootQueue = (parent == null); - float rawCapacity = csConf.getNonLabeledQueueCapacity(this.queuePath); + float rawCapacity = queueContext.getConfiguration() + .getNonLabeledQueueCapacity(this.queuePath); if (rootQueue && (rawCapacity != CapacitySchedulerConfiguration.MAXIMUM_CAPACITY_VALUE)) { @@ -139,13 +131,10 @@ private ParentQueue(CapacitySchedulerContext cs, this.childQueues = new ArrayList<>(); this.allowZeroCapacitySum = - cs.getConfiguration().getAllowZeroCapacitySum(getQueuePath()); - - setupQueueConfigs(cs.getClusterResource(), csConf); + queueContext.getConfiguration() + .getAllowZeroCapacitySum(getQueuePath()); - LOG.info("Initialized parent-queue " + queueName + - " name=" + queueName + - ", fullname=" + getQueuePath()); + setupQueueConfigs(queueContext.getClusterResource(), queueContext.getConfiguration()); } // returns what is configured queue ordering policy @@ -156,41 +145,42 @@ private String getQueueOrderingPolicyConfigName() { } protected void setupQueueConfigs(Resource clusterResource, - CapacitySchedulerConfiguration csConf) + CapacitySchedulerConfiguration configuration) throws IOException { writeLock.lock(); try { autoCreatedQueueTemplate = new AutoCreatedQueueTemplate( - csConf, this.queuePath); - super.setupQueueConfigs(clusterResource, csConf); + configuration, this.queuePath); + super.setupQueueConfigs(clusterResource, configuration); StringBuilder aclsString = new StringBuilder(); - for (Map.Entry e : acls.entrySet()) { - aclsString.append(e.getKey() + ":" + e.getValue().getAclString()); + for (Map.Entry e : getACLs().entrySet()) { + aclsString.append(e.getKey()).append(":") + .append(e.getValue().getAclString()); } StringBuilder labelStrBuilder = new StringBuilder(); - if (queueNodeLabelsSettings.getAccessibleNodeLabels() != null) { - for (String nodeLabel : queueNodeLabelsSettings.getAccessibleNodeLabels()) { + if (getAccessibleNodeLabels() != null) { + for (String nodeLabel : getAccessibleNodeLabels()) { labelStrBuilder.append(nodeLabel).append(","); } } // Initialize queue ordering policy - queueOrderingPolicy = csConf.getQueueOrderingPolicy( + queueOrderingPolicy = configuration.getQueueOrderingPolicy( getQueuePath(), parent == null ? null : ((ParentQueue) parent).getQueueOrderingPolicyConfigName()); queueOrderingPolicy.setQueues(childQueues); LOG.info(getQueueName() + ", " + getCapacityOrWeightString() - + ", absoluteCapacity=" + this.queueCapacities.getAbsoluteCapacity() - + ", maxCapacity=" + this.queueCapacities.getMaximumCapacity() - + ", absoluteMaxCapacity=" + this.queueCapacities - .getAbsoluteMaximumCapacity() + ", state=" + getState() + ", acls=" - + aclsString + ", labels=" + labelStrBuilder.toString() + "\n" - + ", reservationsContinueLooking=" + reservationsContinueLooking + + ", absoluteCapacity=" + getAbsoluteCapacity() + + ", maxCapacity=" + getMaximumCapacity() + + ", absoluteMaxCapacity=" + getAbsoluteMaximumCapacity() + + ", state=" + getState() + ", acls=" + + aclsString + ", labels=" + labelStrBuilder + "\n" + + ", reservationsContinueLooking=" + isReservationsContinueLooking() + ", orderingPolicy=" + getQueueOrderingPolicyConfigName() - + ", priority=" + priority + + ", priority=" + getPriority() + ", allowZeroCapacitySum=" + allowZeroCapacitySum); } finally { writeLock.unlock(); @@ -325,7 +315,7 @@ void setChildQueues(Collection childQueues) throws IOException { .getConfiguredMinResource(nodeLabel)); } Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel, - scheduler.getClusterResource()); + queueContext.getClusterResource()); Resource parentMinResource = usageTracker.getQueueResourceQuotas().getConfiguredMinResource(nodeLabel); if (!parentMinResource.equals(Resources.none()) && Resources.lessThan( @@ -488,11 +478,10 @@ private CSQueue createNewQueue(String childQueuePath, boolean isLeaf) childQueuePath.lastIndexOf(".") + 1); if (isLeaf) { - childQueue = new LeafQueue(csContext, csContext.getConfiguration(), + childQueue = new LeafQueue(queueContext, queueShortName, this, null, true); } else{ - childQueue = new ParentQueue(csContext, csContext.getConfiguration(), - queueShortName, this, null, true); + childQueue = new ParentQueue(queueContext, queueShortName, this, null, true); } childQueue.setDynamicQueue(true); // It should be sufficient now, we don't need to set more, because weights @@ -523,7 +512,7 @@ private CSQueue addDynamicChildQueue(String childQueuePath, boolean isLeaf) // should not happen, since it will be handled before calling this method) // , but we will move on. CSQueue queue = - csContext.getCapacitySchedulerQueueManager().getQueueByFullName( + queueContext.getQueueManager().getQueueByFullName( childQueuePath); if (queue != null) { LOG.warn( @@ -533,7 +522,7 @@ private CSQueue addDynamicChildQueue(String childQueuePath, boolean isLeaf) } // Check if the max queue limit is exceeded. - int maxQueues = csContext.getConfiguration(). + int maxQueues = queueContext.getConfiguration(). getAutoCreatedQueuesV2MaxChildQueuesLimit(getQueuePath()); if (childQueues.size() >= maxQueues) { throw new SchedulerDynamicEditException( @@ -564,8 +553,8 @@ private CSQueue addDynamicChildQueue(String childQueuePath, boolean isLeaf) // Call updateClusterResource. // Which will deal with all effectiveMin/MaxResource // Calculation - this.updateClusterResource(csContext.getClusterResource(), - new ResourceLimits(this.csContext.getClusterResource())); + this.updateClusterResource(queueContext.getClusterResource(), + new ResourceLimits(queueContext.getClusterResource())); return newQueue; } finally { @@ -596,14 +585,14 @@ public void removeChildQueue(CSQueue queue) // Now we can do remove and update this.childQueues.remove(queue); - this.scheduler.getCapacitySchedulerQueueManager() + queueContext.getQueueManager() .removeQueue(queue.getQueuePath()); // Call updateClusterResource, // which will deal with all effectiveMin/MaxResource // Calculation - this.updateClusterResource(csContext.getClusterResource(), - new ResourceLimits(this.csContext.getClusterResource())); + this.updateClusterResource(queueContext.getClusterResource(), + new ResourceLimits(queueContext.getClusterResource())); } finally { writeLock.unlock(); @@ -617,7 +606,7 @@ public void removeChildQueue(CSQueue queue) * false otherwise */ public boolean isEligibleForAutoQueueCreation() { - return isDynamicQueue() || csContext.getConfiguration(). + return isDynamicQueue() || queueContext.getConfiguration(). isAutoQueueCreationV2Enabled(getQueuePath()); } @@ -644,7 +633,7 @@ public void reinitialize(CSQueue newlyParsedQueue, ParentQueue newlyParsedParentQueue = (ParentQueue) newlyParsedQueue; // Set new configs - setupQueueConfigs(clusterResource, csContext.getConfiguration()); + setupQueueConfigs(clusterResource, queueContext.getConfiguration()); // Re-configure existing child queues and add new ones // The CS has already checked to ensure all existing child queues are present! @@ -685,7 +674,7 @@ public void reinitialize(CSQueue newlyParsedQueue, currentChildQueues.put(newChildQueueName, newChildQueue); // inform CapacitySchedulerQueueManager CapacitySchedulerQueueManager queueManager = - this.csContext.getCapacitySchedulerQueueManager(); + queueContext.getQueueManager(); queueManager.addQueue(newChildQueueName, newChildQueue); continue; } @@ -1399,7 +1388,7 @@ public void recoverContainer(Resource clusterResource, // Careful! Locking order is important! writeLock.lock(); try { - FiCaSchedulerNode node = scheduler.getNode( + FiCaSchedulerNode node = queueContext.getNode( rmContainer.getContainer().getNodeId()); allocateResource(clusterResource, rmContainer.getContainer().getResource(), node.getPartition()); @@ -1437,7 +1426,7 @@ public void attachContainer(Resource clusterResource, FiCaSchedulerApp application, RMContainer rmContainer) { if (application != null) { FiCaSchedulerNode node = - scheduler.getNode(rmContainer.getContainer().getNodeId()); + queueContext.getNode(rmContainer.getContainer().getNodeId()); allocateResource(clusterResource, rmContainer.getContainer() .getResource(), node.getPartition()); LOG.info("movedContainer" + " queueMoveIn=" + getQueuePath() @@ -1456,7 +1445,7 @@ public void detachContainer(Resource clusterResource, FiCaSchedulerApp application, RMContainer rmContainer) { if (application != null) { FiCaSchedulerNode node = - scheduler.getNode(rmContainer.getContainer().getNodeId()); + queueContext.getNode(rmContainer.getContainer().getNodeId()); super.releaseResource(clusterResource, rmContainer.getContainer().getResource(), node.getPartition()); @@ -1543,9 +1532,9 @@ private void killContainersToEnforceMaxQueueCapacity(String partition, while (Resources.greaterThan(resourceCalculator, partitionResource, usageTracker.getQueueUsage().getUsed(partition), maxResource)) { RMContainer toKillContainer = killableContainerIter.next(); - FiCaSchedulerApp attempt = csContext.getApplicationAttempt( + FiCaSchedulerApp attempt = queueContext.getApplicationAttempt( toKillContainer.getContainerId().getApplicationAttemptId()); - FiCaSchedulerNode node = csContext.getNode( + FiCaSchedulerNode node = queueContext.getNode( toKillContainer.getAllocatedNode()); if (null != attempt && null != node) { AbstractLeafQueue lq = attempt.getCSLeafQueue(); @@ -1656,7 +1645,7 @@ Map getEffectiveMinRatioPerResource() { @Override public boolean isEligibleForAutoDeletion() { return isDynamicQueue() && getChildQueues().size() == 0 && - csContext.getConfiguration(). + queueContext.getConfiguration(). isAutoExpiredDeletionEnabled(this.getQueuePath()); } 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/PlanQueue.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/PlanQueue.java index 3cddeac6a677d..2b182e532f4a9 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/PlanQueue.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/PlanQueue.java @@ -40,17 +40,15 @@ public class PlanQueue extends AbstractManagedParentQueue { private int maxAppsPerUserForReservation; private float userLimit; private float userLimitFactor; - protected CapacitySchedulerContext schedulerContext; private boolean showReservationsAsQueues; - public PlanQueue(CapacitySchedulerContext cs, String queueName, + public PlanQueue(CapacitySchedulerQueueContext queueContext, String queueName, CSQueue parent, CSQueue old) throws IOException { - super(cs, queueName, parent, old); + super(queueContext, queueName, parent, old); updateAbsoluteCapacities(); - this.schedulerContext = cs; // Set the reservation queue attributes for the Plan - CapacitySchedulerConfiguration conf = cs.getConfiguration(); + CapacitySchedulerConfiguration conf = queueContext.getConfiguration(); String queuePath = super.getQueuePath(); int maxAppsForReservation = conf.getMaximumApplicationsPerQueue(queuePath); showReservationsAsQueues = conf.getShowReservationAsQueues(queuePath); @@ -106,7 +104,7 @@ public void reinitialize(CSQueue newlyParsedQueue, } // Set new configs - setupQueueConfigs(clusterResource, csContext.getConfiguration()); + setupQueueConfigs(clusterResource, queueContext.getConfiguration()); updateQuotas(newlyParsedParentQueue.userLimit, newlyParsedParentQueue.userLimitFactor, 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/QueueAllocationSettings.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/QueueAllocationSettings.java index 5a19a22635d9e..730b797104fe3 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/QueueAllocationSettings.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/QueueAllocationSettings.java @@ -32,12 +32,13 @@ public class QueueAllocationSettings { private final Resource minimumAllocation; private Resource maximumAllocation; - public QueueAllocationSettings(CapacitySchedulerContext csContext) { - this.minimumAllocation = csContext.getMinimumResourceCapability(); + public QueueAllocationSettings(Resource minimumAllocation) { + this.minimumAllocation = minimumAllocation; } - void setupMaximumAllocation(CapacitySchedulerConfiguration csConf, String queuePath, - CSQueue parent, CapacitySchedulerContext csContext) { + void setupMaximumAllocation(CapacitySchedulerConfiguration configuration, + CapacitySchedulerConfiguration originalSchedulerConfiguration, String queuePath, + CSQueue parent) { /* YARN-10869: When using AutoCreatedLeafQueues, the passed configuration * object is a cloned one containing only the template configs * (see ManagedParentQueue#getLeafQueueConfigs). To ensure that the actual @@ -45,8 +46,8 @@ void setupMaximumAllocation(CapacitySchedulerConfiguration csConf, String queueP * be used. */ Resource clusterMax = ResourceUtils - .fetchMaximumAllocationFromConfig(csContext.getConfiguration()); - Resource queueMax = csConf.getQueueMaximumAllocation(queuePath); + .fetchMaximumAllocationFromConfig(originalSchedulerConfiguration); + Resource queueMax = configuration.getQueueMaximumAllocation(queuePath); maximumAllocation = Resources.clone( parent == null ? clusterMax : parent.getMaximumAllocation()); @@ -59,8 +60,8 @@ void setupMaximumAllocation(CapacitySchedulerConfiguration csConf, String queueP if (queueMax == Resources.none()) { // Handle backward compatibility - long queueMemory = csConf.getQueueMaximumAllocationMb(queuePath); - int queueVcores = csConf.getQueueMaximumAllocationVcores(queuePath); + long queueMemory = configuration.getQueueMaximumAllocationMb(queuePath); + int queueVcores = configuration.getQueueMaximumAllocationVcores(queuePath); if (queueMemory != UNDEFINED) { maximumAllocation.setMemorySize(queueMemory); } 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/QueueNodeLabelsSettings.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/QueueNodeLabelsSettings.java index 827259f1ae97a..8d64e17a5e786 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/QueueNodeLabelsSettings.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/QueueNodeLabelsSettings.java @@ -31,7 +31,6 @@ public class QueueNodeLabelsSettings { private final CSQueue parent; private final String queuePath; - private final CapacitySchedulerContext csContext; private Set accessibleLabels; private Set configuredNodeLabels; private String defaultLabelExpression; @@ -39,18 +38,18 @@ public class QueueNodeLabelsSettings { public QueueNodeLabelsSettings(CapacitySchedulerConfiguration configuration, CSQueue parent, String queuePath, - CapacitySchedulerContext csContext) throws IOException { + ConfiguredNodeLabels configuredNodeLabels) throws IOException { this.parent = parent; this.queuePath = queuePath; - this.csContext = csContext; - initializeNodeLabels(configuration); + initializeNodeLabels(configuration, configuredNodeLabels); } - private void initializeNodeLabels(CapacitySchedulerConfiguration configuration) + private void initializeNodeLabels(CapacitySchedulerConfiguration configuration, + ConfiguredNodeLabels configuredNodeLabels) throws IOException { initializeAccessibleLabels(configuration); initializeDefaultLabelExpression(configuration); - initializeConfiguredNodeLabels(); + initializeConfiguredNodeLabels(configuration, configuredNodeLabels); validateNodeLabels(); } @@ -73,19 +72,17 @@ private void initializeDefaultLabelExpression(CapacitySchedulerConfiguration con } } - private void initializeConfiguredNodeLabels() { - if (csContext.getCapacitySchedulerQueueManager() != null - && csContext.getCapacitySchedulerQueueManager().getConfiguredNodeLabels() != null) { + private void initializeConfiguredNodeLabels(CapacitySchedulerConfiguration configuration, + ConfiguredNodeLabels configuredNodeLabelsParam) { + if (configuredNodeLabelsParam != null) { if (queuePath.equals(ROOT)) { - this.configuredNodeLabels = csContext.getCapacitySchedulerQueueManager() - .getConfiguredNodeLabels().getAllConfiguredLabels(); + this.configuredNodeLabels = configuredNodeLabelsParam.getAllConfiguredLabels(); } else { - this.configuredNodeLabels = csContext.getCapacitySchedulerQueueManager() - .getConfiguredNodeLabels().getLabelsByQueue(queuePath); + this.configuredNodeLabels = configuredNodeLabelsParam.getLabelsByQueue(queuePath); } } else { // Fallback to suboptimal but correct logic - this.configuredNodeLabels = csContext.getConfiguration().getConfiguredNodeLabels(queuePath); + this.configuredNodeLabels = configuration.getConfiguredNodeLabels(queuePath); } } 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/ReservationQueue.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/ReservationQueue.java index 4208bf06a24a9..38ee4d237a6f0 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/ReservationQueue.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/ReservationQueue.java @@ -31,20 +31,17 @@ * */ public class ReservationQueue extends AbstractAutoCreatedLeafQueue { - - private static final Logger LOG = LoggerFactory - .getLogger(ReservationQueue.class); + private static final Logger LOG = + LoggerFactory.getLogger(ReservationQueue.class); private PlanQueue parent; - public ReservationQueue(CapacitySchedulerContext cs, String queueName, + public ReservationQueue(CapacitySchedulerQueueContext queueContext, String queueName, PlanQueue parent) throws IOException { - super(cs, queueName, parent, null); - super.setupQueueConfigs(cs.getClusterResource(), - cs.getConfiguration()); + super(queueContext, queueName, parent, null); + super.setupQueueConfigs(queueContext.getClusterResource(), + queueContext.getConfiguration()); - LOG.debug("Initialized ReservationQueue: name={}, fullname={}", - queueName, getQueuePath()); // the following parameters are common to all reservation in the plan updateQuotas(parent.getUserLimitForReservation(), parent.getUserLimitFactor(), 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/UsersManager.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/UsersManager.java index 73aad3c177193..e8c99408ffeec 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/UsersManager.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/UsersManager.java @@ -58,7 +58,6 @@ public class UsersManager implements AbstractUsersManager { private final AbstractLeafQueue lQueue; private final RMNodeLabelsManager labelManager; private final ResourceCalculator resourceCalculator; - private final CapacitySchedulerContext scheduler; private Map users = new ConcurrentHashMap<>(); private ResourceUsage totalResUsageForActiveUsers = new ResourceUsage(); @@ -296,17 +295,13 @@ public void setWeight(float weight) { * Leaf Queue Object * @param labelManager * Label Manager instance - * @param scheduler - * Capacity Scheduler Context * @param resourceCalculator * rc */ public UsersManager(QueueMetrics metrics, AbstractLeafQueue lQueue, - RMNodeLabelsManager labelManager, CapacitySchedulerContext scheduler, - ResourceCalculator resourceCalculator) { + RMNodeLabelsManager labelManager, ResourceCalculator resourceCalculator) { ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); this.lQueue = lQueue; - this.scheduler = scheduler; this.labelManager = labelManager; this.resourceCalculator = resourceCalculator; this.qUsageRatios = new UsageRatios(); @@ -844,10 +839,8 @@ partitionResource, getUsageRatio(nodePartition), /** * Update new usage ratio. * - * @param partition - * Node partition - * @param clusterResource - * Cluster Resource + * @param partition Node partition + * @param clusterResource cluster resource */ public void updateUsageRatio(String partition, Resource clusterResource) { writeLock.lock(); @@ -1064,6 +1057,8 @@ private ResourceUsage getTotalResourceUsagePerUser(String userName) { * Name of the user * @param resource * Resource to increment/decrement + * @param clusterResource + * Cluster resource (for testing purposes only) * @param nodePartition * Node label * @param isAllocate @@ -1071,6 +1066,7 @@ private ResourceUsage getTotalResourceUsagePerUser(String userName) { * @return user */ public User updateUserResourceUsage(String userName, Resource resource, + Resource clusterResource, String nodePartition, boolean isAllocate) { this.writeLock.lock(); try { @@ -1086,7 +1082,7 @@ public User updateUserResourceUsage(String userName, Resource resource, // Update usage ratios Resource resourceByLabel = labelManager.getResourceByLabel(nodePartition, - scheduler.getClusterResource()); + clusterResource); incQueueUsageRatio(nodePartition, user.updateUsageRatio( resourceCalculator, resourceByLabel, nodePartition)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java index b396d5761de16..669435ecba0ff 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java @@ -80,7 +80,7 @@ private boolean checkHeadroom(ResourceLimits currentResourceLimits, // require Resource resourceCouldBeUnReserved = application.getAppAttemptResourceUsage().getReserved(nodePartition); - if (!application.getCSLeafQueue().getReservationContinueLooking()) { + if (!application.getCSLeafQueue().isReservationsContinueLooking()) { // If we don't allow reservation continuous looking, // we won't allow to unreserve before allocation. resourceCouldBeUnReserved = Resources.none(); @@ -154,7 +154,7 @@ private ContainerAllocation preCheckForNodeCandidateSet(FiCaSchedulerNode node, return ContainerAllocation.PRIORITY_SKIPPED; } - if (!application.getCSLeafQueue().getReservationContinueLooking()) { + if (!application.getCSLeafQueue().isReservationsContinueLooking()) { if (!shouldAllocOrReserveNewContainer(schedulerKey, required)) { LOG.debug("doesn't need containers based on reservation algo!"); ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( @@ -551,7 +551,7 @@ private ContainerAllocation assignContainer(Resource clusterResource, RMContainer unreservedContainer = null; boolean reservationsContinueLooking = - application.getCSLeafQueue().getReservationContinueLooking(); + application.getCSLeafQueue().isReservationsContinueLooking(); // Check if we need to kill some containers to allocate this one List toKillContainers = null; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.java index 7458df904518f..14d3555e100a8 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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.java @@ -70,7 +70,6 @@ public class GuaranteedOrZeroCapacityOverTimePolicy implements AutoCreatedQueueManagementPolicy { private static final int DEFAULT_QUEUE_PRINT_SIZE_LIMIT = 25; - private CapacitySchedulerContext scheduler; private ManagedParentQueue managedParentQueue; private static final Logger LOG = @@ -263,9 +262,9 @@ private class PendingApplicationComparator @Override public int compare(FiCaSchedulerApp app1, FiCaSchedulerApp app2) { - RMApp rmApp1 = scheduler.getRMContext().getRMApps().get( + RMApp rmApp1 = managedParentQueue.getQueueContext().getRMApp( app1.getApplicationId()); - RMApp rmApp2 = scheduler.getRMContext().getRMApps().get( + RMApp rmApp2 = managedParentQueue.getQueueContext().getRMApp( app2.getApplicationId()); if (rmApp1 != null && rmApp2 != null) { return Long.compare(rmApp1.getSubmitTime(), rmApp2.getSubmitTime()); @@ -283,10 +282,7 @@ public int compare(FiCaSchedulerApp app1, FiCaSchedulerApp app2) { new PendingApplicationComparator(); @Override - public void init(final CapacitySchedulerContext schedulerContext, - final ParentQueue parentQueue) throws IOException { - this.scheduler = schedulerContext; - + public void init(final ParentQueue parentQueue) throws IOException { ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); readLock = lock.readLock(); writeLock = lock.writeLock(); @@ -372,7 +368,7 @@ public List computeQueueManagementChanges() //Populate new entitlements return leafQueueEntitlements.mapToQueueManagementChanges((leafQueueName, capacities) -> { AutoCreatedLeafQueue leafQueue = - (AutoCreatedLeafQueue) scheduler.getCapacitySchedulerQueueManager() + (AutoCreatedLeafQueue) managedParentQueue.getQueueContext().getQueueManager() .getQueue(leafQueueName); AutoCreatedLeafQueueConfig newTemplate = buildTemplate(capacities); return new QueueManagementChange.UpdateQueue(leafQueue, newTemplate); @@ -651,7 +647,8 @@ public void commitQueueManagementChanges( .mergeCapacities(updatedQueueTemplate.getQueueCapacities()); leafQueue.getQueueResourceQuotas() .setConfiguredMinResource(Resources.multiply( - this.scheduler.getClusterResource(), updatedQueueTemplate + managedParentQueue.getQueueContext().getClusterResource(), + updatedQueueTemplate .getQueueCapacities().getCapacity(nodeLabel))); deactivate(leafQueue, nodeLabel); } @@ -693,8 +690,7 @@ public boolean hasPendingApps(final AutoCreatedLeafQueue leafQueue) { } @Override - public void reinitialize(CapacitySchedulerContext schedulerContext, - final ParentQueue parentQueue) throws IOException { + public void reinitialize(final ParentQueue parentQueue) throws IOException { if (!(parentQueue instanceof ManagedParentQueue)) { throw new IllegalStateException( "Expected instance of type " + ManagedParentQueue.class + " found " diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index 4a393ccf0cb53..87147ce62c9a5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -388,7 +388,7 @@ public void testQueueSubmitWithACLsEnabledWithQueueMappingForAutoCreatedQueue() MockRM newMockRM = new MockRM(csConf); CapacityScheduler cs = ((CapacityScheduler) newMockRM.getResourceScheduler()); - ManagedParentQueue managedParentQueue = new ManagedParentQueue(cs, + ManagedParentQueue managedParentQueue = new ManagedParentQueue(cs.getQueueContext(), "managedparent", cs.getQueue("root"), null); cs.getCapacitySchedulerQueueManager().addQueue("managedparent", managedParentQueue); 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/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.java index 08462332818cf..d9051dd53e1bf 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/TestAbsoluteResourceConfiguration.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/TestAbsoluteResourceConfiguration.java @@ -215,7 +215,7 @@ public void testSimpleMinMaxResourceConfigurartionPerQueue() CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue(QUEUED); - AutoCreatedLeafQueue d1 = new AutoCreatedLeafQueue(cs, "d1", parentQueue); + AutoCreatedLeafQueue d1 = new AutoCreatedLeafQueue(cs.getQueueContext(), "d1", parentQueue); cs.addQueue(d1); /** @@ -240,7 +240,7 @@ public void testSimpleMinMaxResourceConfigurartionPerQueue() * d1 will occupy all entire resource * of Managed Parent queue. */ - AutoCreatedLeafQueue d2 = new AutoCreatedLeafQueue(cs, "d2", parentQueue); + AutoCreatedLeafQueue d2 = new AutoCreatedLeafQueue(cs.getQueueContext(), "d2", parentQueue); cs.addQueue(d2); cs.getRootQueue().updateClusterResource(cs.getClusterResource(), 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 a1252cfade79c..33134babc9f57 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 @@ -107,19 +107,11 @@ public void setUp() throws IOException { rmContext = TestUtils.getMockRMContext(); Resource clusterResource = Resources.createResource(10 * 16 * GB, 10 * 32); - CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class); - when(csContext.getConfiguration()).thenReturn(csConf); - when(csContext.getConf()).thenReturn(conf); - when(csContext.getMinimumResourceCapability()). - thenReturn(Resources.createResource(GB, 1)); - when(csContext.getMaximumResourceCapability()). - thenReturn(Resources.createResource(16*GB, 32)); - when(csContext.getClusterResource()). - thenReturn(clusterResource); - when(csContext.getResourceCalculator()). - thenReturn(resourceCalculator); + CapacitySchedulerContext csContext = createCSContext(csConf, resourceCalculator, + Resources.createResource(GB, 1), Resources.createResource(16*GB, 32), + clusterResource); when(csContext.getRMContext()).thenReturn(rmContext); - when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); + CapacitySchedulerQueueContext queueContext = new CapacitySchedulerQueueContext(csContext); RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(conf); @@ -129,13 +121,13 @@ public void setUp() throws IOException { CSQueueStore queues = new CSQueueStore(); root = CapacitySchedulerQueueManager - .parseQueue(csContext, csConf, null, "root", + .parseQueue(queueContext, csConf, null, "root", queues, queues, TestUtils.spyHook); root.updateClusterResource(clusterResource, new ResourceLimits(clusterResource)); - queue = spy(new LeafQueue(csContext, A, root, null)); + queue = spy(new LeafQueue(queueContext, A, root, null)); QueueResourceQuotas queueResourceQuotas = ((LeafQueue) queues.get(A)) .getQueueResourceQuotas(); doReturn(queueResourceQuotas).when(queue).getQueueResourceQuotas(); @@ -278,28 +270,21 @@ public void testLimitsComputation() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); setupQueueConfiguration(csConf); - YarnConfiguration conf = new YarnConfiguration(); - - CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class); - when(csContext.getConfiguration()).thenReturn(csConf); - when(csContext.getConf()).thenReturn(conf); - when(csContext.getMinimumResourceCapability()). - thenReturn(Resources.createResource(GB, 1)); - when(csContext.getMaximumResourceCapability()). - thenReturn(Resources.createResource(16*GB, 16)); - when(csContext.getResourceCalculator()).thenReturn(resourceCalculator); - when(csContext.getRMContext()).thenReturn(rmContext); - when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); - + // Say cluster has 100 nodes of 16G each Resource clusterResource = Resources.createResource(100 * 16 * GB, 100 * 16); - when(csContext.getClusterResource()).thenReturn(clusterResource); + + CapacitySchedulerContext csContext = createCSContext(csConf, resourceCalculator, Resources.createResource(GB, 1), + Resources.createResource(16*GB, 16), clusterResource); + CapacitySchedulerQueueManager queueManager = csContext.getCapacitySchedulerQueueManager(); + CapacitySchedulerQueueContext queueContext = new CapacitySchedulerQueueContext(csContext); CSQueueStore queues = new CSQueueStore(); CSQueue root = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, "root", queues, queues, TestUtils.spyHook); + queueManager.setRootQueue(root); root.updateClusterResource(clusterResource, new ResourceLimits(clusterResource)); @@ -367,12 +352,14 @@ public void testLimitsComputation() throws Exception { // Change the per-queue max AM resources percentage. csConf.setFloat(PREFIX + queue.getQueuePath() + ".maximum-am-resource-percent", 0.5f); + queueContext.reinitialize(); // Re-create queues to get new configs. queues = new CSQueueStore(); root = CapacitySchedulerQueueManager.parseQueue( - csContext, csConf, null, "root", + queueContext, csConf, null, "root", queues, queues, TestUtils.spyHook); clusterResource = Resources.createResource(100 * 16 * GB); + queueManager.setRootQueue(root); root.updateClusterResource(clusterResource, new ResourceLimits( clusterResource)); @@ -391,10 +378,11 @@ public void testLimitsComputation() throws Exception { // Change the per-queue max applications. csConf.setInt(PREFIX + queue.getQueuePath() + ".maximum-applications", 9999); + queueContext.reinitialize(); // Re-create queues to get new configs. queues = new CSQueueStore(); root = CapacitySchedulerQueueManager.parseQueue( - csContext, csConf, null, "root", + queueContext, csConf, null, "root", queues, queues, TestUtils.spyHook); root.updateClusterResource(clusterResource, new ResourceLimits( clusterResource)); @@ -587,26 +575,19 @@ public void testHeadroom() throws Exception { new CapacitySchedulerConfiguration(); csConf.setUserLimit(CapacitySchedulerConfiguration.ROOT + "." + A, 25); setupQueueConfiguration(csConf); - YarnConfiguration conf = new YarnConfiguration(); - - CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class); - when(csContext.getConfiguration()).thenReturn(csConf); - when(csContext.getConf()).thenReturn(conf); - when(csContext.getMinimumResourceCapability()). - thenReturn(Resources.createResource(GB)); - when(csContext.getMaximumResourceCapability()). - thenReturn(Resources.createResource(16*GB)); - when(csContext.getResourceCalculator()).thenReturn(resourceCalculator); - when(csContext.getRMContext()).thenReturn(rmContext); - when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); - + // Say cluster has 100 nodes of 16G each Resource clusterResource = Resources.createResource(100 * 16 * GB); - when(csContext.getClusterResource()).thenReturn(clusterResource); + + CapacitySchedulerContext csContext = createCSContext(csConf, resourceCalculator, Resources.createResource(GB), + Resources.createResource(16*GB), clusterResource); + CapacitySchedulerQueueManager queueManager = csContext.getCapacitySchedulerQueueManager(); + CapacitySchedulerQueueContext queueContext = new CapacitySchedulerQueueContext(csContext); CSQueueStore queues = new CSQueueStore(); - CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(csContext, + CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, "root", queues, queues, TestUtils.spyHook); + queueManager.setRootQueue(rootQueue); rootQueue.updateClusterResource(clusterResource, new ResourceLimits(clusterResource)); @@ -952,27 +933,17 @@ public void testAMResourceLimitWithDRCAndFullParent() throws Exception { setupQueueConfiguration(csConf); csConf.setFloat(CapacitySchedulerConfiguration. MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.3f); - YarnConfiguration conf = new YarnConfiguration(); - - CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class); - when(csContext.getConfiguration()).thenReturn(csConf); - when(csContext.getConf()).thenReturn(conf); - when(csContext.getMinimumResourceCapability()). - thenReturn(Resources.createResource(GB)); - when(csContext.getMaximumResourceCapability()). - thenReturn(Resources.createResource(16*GB)); - when(csContext.getResourceCalculator()). - thenReturn(new DominantResourceCalculator()); - when(csContext.getRMContext()).thenReturn(rmContext); - when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); // Total cluster resources. Resource clusterResource = Resources.createResource(100 * GB, 1000); - when(csContext.getClusterResource()).thenReturn(clusterResource); + + CapacitySchedulerQueueContext queueContext = new CapacitySchedulerQueueContext( + createCSContext(csConf, new DominantResourceCalculator(), Resources.createResource(GB), + Resources.createResource(16*GB), clusterResource)); // Set up queue hierarchy. CSQueueStore queues = new CSQueueStore(); - CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(csContext, + CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, "root", queues, queues, TestUtils.spyHook); rootQueue.updateClusterResource(clusterResource, new ResourceLimits(clusterResource)); @@ -1015,4 +986,32 @@ public void testAMResourceLimitWithDRCAndFullParent() throws Exception { + amLimit.getVirtualCores(), amLimit.getVirtualCores() >= expectedAmLimit.getVirtualCores()); } + + private CapacitySchedulerContext createCSContext(CapacitySchedulerConfiguration csConf, + ResourceCalculator rc, Resource minResource, Resource maxResource, Resource clusterResource) { + YarnConfiguration conf = new YarnConfiguration(); + + CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class); + when(csContext.getConfiguration()).thenReturn(csConf); + when(csContext.getConf()).thenReturn(conf); + when(csContext.getMinimumResourceCapability()). + thenReturn(minResource); + when(csContext.getMaximumResourceCapability()). + thenReturn(maxResource); + when(csContext.getResourceCalculator()). + thenReturn(rc); + CapacitySchedulerQueueManager queueManager = new CapacitySchedulerQueueManager(conf, + rmContext.getNodeLabelManager(), null); + when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); + when(csContext.getCapacitySchedulerQueueManager()).thenReturn(queueManager); + + when(csContext.getRMContext()).thenReturn(rmContext); + when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); + + // Total cluster resources. + when(csContext.getClusterResource()).thenReturn(clusterResource); + + return csContext; + } + } 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/TestApplicationLimitsByPartition.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/TestApplicationLimitsByPartition.java index a228d254d5ad8..4c2ec87e70529 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/TestApplicationLimitsByPartition.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/TestApplicationLimitsByPartition.java @@ -777,6 +777,12 @@ public void testHeadroom() throws Exception { when(spyRMContext.getNodeLabelManager()).thenReturn(mgr); when(csContext.getRMContext()).thenReturn(spyRMContext); when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); + CapacitySchedulerQueueManager queueManager = + new CapacitySchedulerQueueManager(csConf, mgr, null); + when(csContext.getCapacitySchedulerQueueManager()).thenReturn(queueManager); + + // Setup nodelabels + queueManager.reinitConfiguredNodeLabels(csConf); mgr.activateNode(NodeId.newInstance("h0", 0), Resource.newInstance(160 * GB, 16)); // default Label @@ -789,16 +795,15 @@ public void testHeadroom() throws Exception { Resource clusterResource = Resources.createResource(160 * GB); when(csContext.getClusterResource()).thenReturn(clusterResource); + CapacitySchedulerQueueContext queueContext = new CapacitySchedulerQueueContext(csContext); + CSQueueStore queues = new CSQueueStore(); - CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(csContext, + CSQueue rootQueue = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, "root", queues, queues, TestUtils.spyHook); + queueManager.setRootQueue(rootQueue); rootQueue.updateClusterResource(clusterResource, new ResourceLimits(clusterResource)); - ResourceUsage queueResUsage = rootQueue.getQueueResourceUsage(); - when(csContext.getClusterResourceUsage()) - .thenReturn(queueResUsage); - // Manipulate queue 'a' LeafQueue queue = TestLeafQueue.stubLeafQueue((LeafQueue) queues.get("b2")); queue.updateClusterResource(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/TestCSMaxRunningAppsEnforcer.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/TestCSMaxRunningAppsEnforcer.java index b560d9798e2a5..f7460de7aafd6 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/TestCSMaxRunningAppsEnforcer.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/TestCSMaxRunningAppsEnforcer.java @@ -92,9 +92,13 @@ public void setup() throws IOException { when(preemptionManager.getKillableResource(any(), anyString())) .thenReturn(Resource.newInstance(0, 0)); when(scheduler.getPreemptionManager()).thenReturn(preemptionManager); + when(scheduler.getActivitiesManager()).thenReturn(activitiesManager); queueManager = new CapacitySchedulerQueueManager(csConfig, labelManager, appPriorityACLManager); queueManager.setCapacitySchedulerContext(scheduler); + when(scheduler.getCapacitySchedulerQueueManager()).thenReturn(queueManager); + CapacitySchedulerQueueContext queueContext = new CapacitySchedulerQueueContext(scheduler); + when(scheduler.getQueueContext()).thenReturn(queueContext); queueManager.initializeQueues(csConfig); } 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/TestCSQueueStore.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/TestCSQueueStore.java index ad0843216d172..8ec8d62e74454 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/TestCSQueueStore.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/TestCSQueueStore.java @@ -39,6 +39,7 @@ public class TestCSQueueStore { private CSQueue root; private CapacitySchedulerContext csContext; + private CapacitySchedulerQueueContext queueContext; @Before public void setUp() throws IOException { @@ -62,22 +63,26 @@ public void setUp() throws IOException { when(csContext.getResourceCalculator()). thenReturn(resourceCalculator); when(csContext.getRMContext()).thenReturn(rmContext); + when(csContext.getCapacitySchedulerQueueManager()).thenReturn( + new CapacitySchedulerQueueManager(csConf, null, null)); + + queueContext = new CapacitySchedulerQueueContext(csContext); CSQueueStore queues = new CSQueueStore(); root = CapacitySchedulerQueueManager - .parseQueue(csContext, csConf, null, "root", + .parseQueue(queueContext, csConf, null, "root", queues, queues, TestUtils.spyHook); } public CSQueue createLeafQueue(String name, CSQueue parent) throws IOException { - return new LeafQueue(csContext, name, parent, null); + return new LeafQueue(queueContext, name, parent, null); } public CSQueue createParentQueue(String name, CSQueue parent) throws IOException { - return new ParentQueue(csContext, name, parent, null); + return new ParentQueue(queueContext, name, parent, null); } /** 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/TestCapacitySchedulerAutoQueueCreation.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/TestCapacitySchedulerAutoQueueCreation.java index 835d95ec87418..90c63dbd2b95f 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/TestCapacitySchedulerAutoQueueCreation.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/TestCapacitySchedulerAutoQueueCreation.java @@ -680,7 +680,7 @@ public void testAutoCreationFailsWhenParentCapacityExceeded() // Test add one auto created queue dynamically and manually modify // capacity ManagedParentQueue parentQueue = (ManagedParentQueue) newCS.getQueue("c"); - AutoCreatedLeafQueue c1 = new AutoCreatedLeafQueue(newCS, "c1", + AutoCreatedLeafQueue c1 = new AutoCreatedLeafQueue(newCS.getQueueContext(), "c1", parentQueue); newCS.addQueue(c1); c1.setCapacity(0.5f); @@ -689,13 +689,13 @@ public void testAutoCreationFailsWhenParentCapacityExceeded() setEntitlement(c1, new QueueEntitlement(0.5f, 1f)); - AutoCreatedLeafQueue c2 = new AutoCreatedLeafQueue(newCS, "c2", + AutoCreatedLeafQueue c2 = new AutoCreatedLeafQueue(newCS.getQueueContext(), "c2", parentQueue); newCS.addQueue(c2); setEntitlement(c2, new QueueEntitlement(0.5f, 1f)); try { - AutoCreatedLeafQueue c3 = new AutoCreatedLeafQueue(newCS, "c3", + AutoCreatedLeafQueue c3 = new AutoCreatedLeafQueue(newCS.getQueueContext(), "c3", parentQueue); newCS.addQueue(c3); fail("Expected exception for auto queue creation failure"); 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/TestCapacitySchedulerDynamicBehavior.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/TestCapacitySchedulerDynamicBehavior.java index 4dd537d3a0cb8..7cfd457cb76b1 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/TestCapacitySchedulerDynamicBehavior.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/TestCapacitySchedulerDynamicBehavior.java @@ -86,14 +86,14 @@ public void testRefreshQueuesWithReservations() throws Exception { // Test add one reservation dynamically and manually modify capacity ReservationQueue a1 = - new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a")); + new ReservationQueue(cs.getQueueContext(), "a1", (PlanQueue) cs.getQueue("a")); cs.addQueue(a1); a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f)); // Test add another reservation queue and use setEntitlement to modify // capacity ReservationQueue a2 = - new ReservationQueue(cs, "a2", (PlanQueue) cs.getQueue("a")); + new ReservationQueue(cs.getQueueContext(), "a2", (PlanQueue) cs.getQueue("a")); cs.addQueue(a2); cs.setEntitlement("a2", new QueueEntitlement(A2_CAPACITY / 100, 1.0f)); @@ -116,7 +116,7 @@ public void testAddQueueFailCases() throws Exception { try { // Test invalid addition (adding non-zero size queue) ReservationQueue a1 = - new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a")); + new ReservationQueue(cs.getQueueContext(), "a1", (PlanQueue) cs.getQueue("a")); a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f)); cs.addQueue(a1); fail(); @@ -126,7 +126,7 @@ public void testAddQueueFailCases() throws Exception { // Test add one reservation dynamically and manually modify capacity ReservationQueue a1 = - new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a")); + new ReservationQueue(cs.getQueueContext(), "a1", (PlanQueue) cs.getQueue("a")); cs.addQueue(a1); //set default queue capacity to zero ((ReservationQueue) cs @@ -138,7 +138,7 @@ public void testAddQueueFailCases() throws Exception { // Test add another reservation queue and use setEntitlement to modify // capacity ReservationQueue a2 = - new ReservationQueue(cs, "a2", (PlanQueue) cs.getQueue("a")); + new ReservationQueue(cs.getQueueContext(), "a2", (PlanQueue) cs.getQueue("a")); cs.addQueue(a2); @@ -165,7 +165,7 @@ public void testRemoveQueue() throws Exception { // Test add one reservation dynamically and manually modify capacity ReservationQueue a1 = - new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a")); + new ReservationQueue(cs.getQueueContext(), "a1", (PlanQueue) cs.getQueue("a")); cs.addQueue(a1); a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f)); @@ -249,7 +249,7 @@ public void testMoveAppToPlanQueue() throws Exception { // create the default reservation queue String defQName = "a" + ReservationConstants.DEFAULT_QUEUE_SUFFIX; ReservationQueue defQ = - new ReservationQueue(scheduler, defQName, + new ReservationQueue(scheduler.getQueueContext(), defQName, (PlanQueue) scheduler.getQueue("a")); scheduler.addQueue(defQ); defQ.setEntitlement(new QueueEntitlement(1f, 1f)); 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/TestCapacitySchedulerPerf.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/TestCapacitySchedulerPerf.java index b71fe063927ac..6c84c8eab5ef1 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/TestCapacitySchedulerPerf.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/TestCapacitySchedulerPerf.java @@ -177,7 +177,7 @@ private void testUserLimitThroughputWithNumberOfResourceTypes( LeafQueue qb = (LeafQueue)cs.getQueue(queueName); // For now make user limit large so we can activate all applications qb.setUserLimitFactor((float)100.0); - qb.setupConfigurableCapacities(); + qb.setupConfigurableCapacities(cs.getConfiguration()); lqs[i] = qb; } 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 31ad107de94c3..1af3563c52729 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 @@ -74,6 +74,7 @@ public class TestChildQueueOrder { YarnConfiguration conf; CapacitySchedulerConfiguration csConf; CapacitySchedulerContext csContext; + CapacitySchedulerQueueContext queueContext; final static int GB = 1024; final static String DEFAULT_RACK = "/default"; @@ -100,6 +101,10 @@ public void setUp() throws Exception { thenReturn(resourceComparator); when(csContext.getRMContext()).thenReturn(rmContext); when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); + when(csContext.getCapacitySchedulerQueueManager()).thenReturn( + new CapacitySchedulerQueueManager(csConf, rmContext.getNodeLabelManager(), null)); + + queueContext = new CapacitySchedulerQueueContext(csContext); } private FiCaSchedulerApp getMockApplication(int appId, String user) { @@ -219,9 +224,10 @@ private void setupSortedQueues(CapacitySchedulerConfiguration conf) { public void testSortedQueues() throws Exception { // Setup queue configs setupSortedQueues(csConf); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); CSQueue root = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); 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 d3545bd41f57a..1da7ce18ee01c 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 @@ -143,6 +143,7 @@ public class TestLeafQueue { CapacityScheduler cs; CapacitySchedulerConfiguration csConf; CapacitySchedulerContext csContext; + CapacitySchedulerQueueContext queueContext; private RMApp rmApp; CSQueue root; @@ -203,6 +204,7 @@ private void setUpInternal(ResourceCalculator rC, boolean withNodeLabels) csConf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true); csConf.setBoolean(CapacitySchedulerConfiguration.RESERVE_CONT_LOOK_ALL_NODES, false); + csConf.setResourceComparator(rC.getClass()); final String newRoot = "root" + System.currentTimeMillis(); setupQueueConfiguration(csConf, newRoot, withNodeLabels); YarnConfiguration conf = new YarnConfiguration(); @@ -228,12 +230,20 @@ private void setUpInternal(ResourceCalculator rC, boolean withNodeLabels) containerTokenSecretManager.rollMasterKey(); when(csContext.getContainerTokenSecretManager()).thenReturn( containerTokenSecretManager); + CapacitySchedulerQueueManager queueManager = + new CapacitySchedulerQueueManager(csConf, null, null); + when(csContext.getCapacitySchedulerQueueManager()).thenReturn(queueManager); + + queueManager.reinitConfiguredNodeLabels(csConf); + + queueContext = new CapacitySchedulerQueueContext(csContext); root = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, ROOT, queues, queues, TestUtils.spyHook); + queueManager.setRootQueue(root); root.updateClusterResource(Resources.createResource(100 * 16 * GB, 100 * 32), new ResourceLimits(Resources.createResource(100 * 16 * GB, 100 * 32))); @@ -242,8 +252,8 @@ private void setUpInternal(ResourceCalculator rC, boolean withNodeLabels) .thenReturn(queueResUsage); cs.setRMContext(spyRMContext); - cs.init(csConf); cs.setResourceCalculator(rC); + cs.init(csConf); when(spyRMContext.getScheduler()).thenReturn(cs); when(spyRMContext.getYarnConfiguration()) @@ -1087,11 +1097,12 @@ public void testUserLimitCache() throws Exception { csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 100); csConf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 100); + queueContext.reinitialize(); // reinitialize queues CSQueueStore newQueues = new CSQueueStore(); CSQueue newRoot = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues, TestUtils.spyHook); @@ -1305,11 +1316,12 @@ public void testUserLimitCacheActiveUsersChanged() throws Exception { csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 100); csConf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 100); + queueContext.reinitialize(); // reinitialize queues CSQueueStore newQueues = new CSQueueStore(); CSQueue newRoot = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues, TestUtils.spyHook); @@ -1920,6 +1932,7 @@ public void testUserSpecificUserLimits() throws Exception { + CapacitySchedulerConfiguration.USER_WEIGHT, 0.7f); csConf.reinitializeConfigurationProperties(); + queueContext.reinitialize(); when(csContext.getClusterResource()) .thenReturn(Resources.createResource(16 * GB, 32)); @@ -3216,10 +3229,12 @@ public void testRackLocalityDelayScheduling() throws Exception { csConf.setInt(CapacitySchedulerConfiguration.NODE_LOCALITY_DELAY, 2); csConf.setInt( CapacitySchedulerConfiguration.RACK_LOCALITY_ADDITIONAL_DELAY, 1); + queueContext.reinitialize(); CSQueueStore newQueues = new CSQueueStore(); - CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext, + CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, ROOT, newQueues, queues, TestUtils.spyHook); + csContext.getCapacitySchedulerQueueManager().setRootQueue(newRoot); root.reinitialize(newRoot, cs.getClusterResource()); // Manipulate queue 'b' @@ -3652,9 +3667,10 @@ public void testActivateApplicationAfterQueueRefresh() throws Exception { CapacitySchedulerConfiguration.MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT * 2); + queueContext.reinitialize(); CSQueueStore newQueues = new CSQueueStore(); CSQueue newRoot = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, ROOT, newQueues, queues, TestUtils.spyHook); @@ -3683,12 +3699,14 @@ public void testLocalityDelaysAfterQueueRefresh() throws Exception { csConf.setInt(CapacitySchedulerConfiguration.NODE_LOCALITY_DELAY, 60); csConf.setInt( CapacitySchedulerConfiguration.RACK_LOCALITY_ADDITIONAL_DELAY, 600); + queueContext.reinitialize(); CSQueueStore newQueues = new CSQueueStore(); CSQueue newRoot = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, ROOT, newQueues, queues, TestUtils.spyHook); + csContext.getCapacitySchedulerQueueManager().setRootQueue(newRoot); root.reinitialize(newRoot, cs.getClusterResource()); // after reinitialization @@ -4043,8 +4061,14 @@ public void testMaxAMResourcePerQueuePercentAfterQueueRefresh() CapacitySchedulerConfiguration.MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.1f); + CapacitySchedulerQueueManager queueManager = new CapacitySchedulerQueueManager(csConf, + rmContext.getNodeLabelManager(), null); + when(csContext.getCapacitySchedulerQueueManager()).thenReturn(queueManager); + + CapacitySchedulerQueueContext newQueueContext = new CapacitySchedulerQueueContext(csContext); + CSQueue root; - root = CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + root = CapacitySchedulerQueueManager.parseQueue(newQueueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); root.updateClusterResource(clusterResource, new ResourceLimits(clusterResource)); @@ -4060,9 +4084,10 @@ public void testMaxAMResourcePerQueuePercentAfterQueueRefresh() csConf.setFloat( CapacitySchedulerConfiguration.MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.2f); + newQueueContext.reinitialize(); clusterResource = Resources.createResource(100 * 20 * GB, 100 * 32); CSQueueStore newQueues = new CSQueueStore(); - CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext, + CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(newQueueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues, TestUtils.spyHook); root.reinitialize(newRoot, clusterResource); @@ -5112,15 +5137,15 @@ public void testSetupQueueConfigsWithSpecifiedConfiguration() assertEquals(0, conf.size()); conf.setNodeLocalityDelay(60); - conf.setCapacity(ROOT + DOT + leafQueueName, 10); - conf.setMaximumCapacity(ROOT + DOT + leafQueueName, 100); - conf.setUserLimitFactor(ROOT + DOT +leafQueueName, 0.1f); + csConf.setCapacity(ROOT + DOT + leafQueueName, 10); + csConf.setMaximumCapacity(ROOT + DOT + leafQueueName, 100); + csConf.setUserLimitFactor(ROOT + DOT +leafQueueName, 0.1f); csConf.setNodeLocalityDelay(30); csConf.setGlobalMaximumApplicationsPerQueue(20); + queueContext.reinitialize(); - LeafQueue leafQueue = new LeafQueue(csContext, conf, - leafQueueName, cs.getRootQueue(), + LeafQueue leafQueue = new LeafQueue(queueContext, leafQueueName, cs.getRootQueue(), null); leafQueue.updateClusterResource(Resource.newInstance(0, 0), @@ -5148,6 +5173,7 @@ public void testSetupQueueConfigsWithSpecifiedConfiguration() // limit maximum apps by max system apps csConf.setMaximumSystemApplications(15); + queueContext.reinitialize(); leafQueue.updateClusterResource(Resource.newInstance(0, 0), new ResourceLimits(Resource.newInstance(0, 0))); 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 31ece4f5f0f39..476abc638fba7 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 @@ -79,7 +79,8 @@ public class TestParentQueue { YarnConfiguration conf; CapacitySchedulerConfiguration csConf; CapacitySchedulerContext csContext; - + CapacitySchedulerQueueContext queueContext; + final static int GB = 1024; final static String DEFAULT_RACK = "/default"; @@ -105,6 +106,10 @@ public void setUp() throws Exception { when(csContext.getResourceCalculator()). thenReturn(resourceComparator); when(csContext.getRMContext()).thenReturn(rmContext); + when(csContext.getCapacitySchedulerQueueManager()).thenReturn( + new CapacitySchedulerQueueManager(csConf, rmContext.getNodeLabelManager(), null)); + + queueContext = new CapacitySchedulerQueueContext(csContext); } private static final String A = "a"; @@ -121,6 +126,8 @@ private void setupSingleLevelQueues(CapacitySchedulerConfiguration conf) { conf.setCapacity(Q_A, 30); conf.setCapacity(Q_B, 70); + + queueContext.reinitialize(); LOG.info("Setup top-level queues a and b"); } @@ -137,6 +144,8 @@ private void setupSingleLevelQueuesWithAbsoluteResource( conf.setMinimumResourceRequirement("", new QueuePath(Q_B), QUEUE_B_RESOURCE); + queueContext.reinitialize(); + LOG.info("Setup top-level queues a and b with absolute resource"); } @@ -253,7 +262,7 @@ public void testSingleLevelQueues() throws Exception { CSQueueStore queues = new CSQueueStore(); CSQueue root = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); @@ -371,11 +380,12 @@ public void testSingleLevelQueuesPrecision() throws Exception { setupSingleLevelQueues(csConf); csConf.setCapacity(Q_A, 30); csConf.setCapacity(Q_B, 70.5F); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); boolean exceptionOccurred = false; try { - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } catch (IOException ie) { @@ -386,10 +396,11 @@ public void testSingleLevelQueuesPrecision() throws Exception { } csConf.setCapacity(Q_A, 30); csConf.setCapacity(Q_B, 70); + queueContext.reinitialize(); exceptionOccurred = false; queues.clear(); try { - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } catch (IllegalArgumentException ie) { @@ -400,10 +411,11 @@ public void testSingleLevelQueuesPrecision() throws Exception { } csConf.setCapacity(Q_A, 30); csConf.setCapacity(Q_B, 70.005F); + queueContext.reinitialize(); exceptionOccurred = false; queues.clear(); try { - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } catch (IllegalArgumentException ie) { @@ -470,6 +482,7 @@ private void setupMultiLevelQueues(CapacitySchedulerConfiguration conf) { conf.setQueues(Q_C111, new String[] {C1111}); final String Q_C1111= Q_C111 + "." + C1111; conf.setCapacity(Q_C1111, 100); + queueContext.reinitialize(); } @Test @@ -495,7 +508,7 @@ public void testMultiLevelQueues() throws Exception { CSQueueStore queues = new CSQueueStore(); CSQueue root = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); @@ -657,9 +670,10 @@ public void testQueueCapacitySettingChildZero() throws Exception { csConf.setCapacity(Q_B + "." + B1, 0); csConf.setCapacity(Q_B + "." + B2, 0); csConf.setCapacity(Q_B + "." + B3, 0); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } @@ -672,9 +686,10 @@ public void testQueueCapacitySettingParentZero() throws Exception { // set parent capacity to 0 when child not 0 csConf.setCapacity(Q_B, 0); csConf.setCapacity(Q_A, 60); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } @@ -690,8 +705,9 @@ public void testQueueCapacitySettingParentZeroChildren100pctZeroSumAllowed() csConf.setCapacity(Q_B, 0); csConf.setCapacity(Q_A, 60); csConf.setAllowZeroCapacitySum(Q_B, true); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } @@ -710,8 +726,9 @@ public void testQueueCapacitySettingParentZeroChildren50pctZeroSumAllowed() csConf.setCapacity(Q_B + "." + B2, 20); csConf.setCapacity(Q_B + "." + B3, 20); csConf.setAllowZeroCapacitySum(Q_B, true); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } @@ -730,8 +747,9 @@ public void testQueueCapacitySettingParentNonZeroChildrenZeroSumAllowed() csConf.setCapacity(Q_B + "." + B2, 0); csConf.setCapacity(Q_B + "." + B3, 0); csConf.setAllowZeroCapacitySum(Q_B, true); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } @@ -746,12 +764,12 @@ public void testQueueCapacityZero() throws Exception { csConf.setCapacity(Q_B + "." + B1, 0); csConf.setCapacity(Q_B + "." + B2, 0); csConf.setCapacity(Q_B + "." + B3, 0); - csConf.setCapacity(Q_A, 60); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); try { - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); } catch (IllegalArgumentException e) { @@ -767,7 +785,7 @@ public void testOffSwitchScheduling() throws Exception { CSQueueStore queues = new CSQueueStore(); CSQueue root = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); @@ -849,7 +867,7 @@ public void testOffSwitchSchedulingMultiLevelQueues() throws Exception { //B3 CSQueueStore queues = new CSQueueStore(); CSQueue root = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); @@ -948,10 +966,11 @@ public void testQueueAcl() throws Exception { csConf.setAcl(Q_C, QueueACL.ADMINISTER_QUEUE, "*"); final String Q_C11= Q_C + "." + C1 + "." + C11; csConf.setAcl(Q_C11, QueueACL.SUBMIT_APPLICATIONS, "*"); + queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); CSQueue root = - CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, + CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); YarnAuthorizationProvider authorizer = @@ -1014,7 +1033,7 @@ public void testAbsoluteResourceWithChangeInClusterResource() setupSingleLevelQueuesWithAbsoluteResource(csConf); CSQueueStore queues = new CSQueueStore(); - CSQueue root = CapacitySchedulerQueueManager.parseQueue(csContext, csConf, + CSQueue root = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); @@ -1085,7 +1104,7 @@ public void testDeriveCapacityFromAbsoluteConfigurations() throws Exception { setupSingleLevelQueuesWithAbsoluteResource(csConf); CSQueueStore queues = new CSQueueStore(); - CSQueue root = CapacitySchedulerQueueManager.parseQueue(csContext, csConf, + CSQueue root = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); @@ -1138,6 +1157,7 @@ public void testDeriveCapacityFromAbsoluteConfigurations() throws Exception { // Set GlobalMaximumApplicationsPerQueue in csConf csConf.setGlobalMaximumApplicationsPerQueue(8000); + queueContext.reinitialize(); root.updateClusterResource(clusterResource, new ResourceLimits(clusterResource)); @@ -1155,6 +1175,7 @@ public void testDeriveCapacityFromAbsoluteConfigurations() throws Exception { Integer.toString(queueAMaxApplications)); csConf.set("yarn.scheduler.capacity." + Q_B + ".maximum-applications", Integer.toString(queueBMaxApplications)); + queueContext.reinitialize(); root.updateClusterResource(clusterResource, new ResourceLimits(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/TestReservationQueue.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/TestReservationQueue.java index 8407922b6331d..94d800732fb51 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/TestReservationQueue.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/TestReservationQueue.java @@ -46,6 +46,7 @@ public class TestReservationQueue { private CapacitySchedulerConfiguration csConf; private CapacitySchedulerContext csContext; + private CapacitySchedulerQueueContext queueContext; final static int DEF_MAX_APPS = 10000; final static int GB = 1024; private final ResourceCalculator resourceCalculator = @@ -63,7 +64,7 @@ public void setup() throws IOException, SchedulerDynamicEditException { CapacitySchedulerQueueManager csQm = mock( CapacitySchedulerQueueManager.class); ConfiguredNodeLabels labels = new ConfiguredNodeLabels(csConf); - when(csQm.getConfiguredNodeLabels()).thenReturn(labels); + when(csQm.getConfiguredNodeLabelsForAllQueues()).thenReturn(labels); when(csContext.getConfiguration()).thenReturn(csConf); when(csContext.getCapacitySchedulerQueueManager()).thenReturn(csQm); when(csContext.getConf()).thenReturn(conf); @@ -78,9 +79,11 @@ public void setup() throws IOException, SchedulerDynamicEditException { RMContext mockRMContext = TestUtils.getMockRMContext(); when(csContext.getRMContext()).thenReturn(mockRMContext); + queueContext = new CapacitySchedulerQueueContext(csContext); + // create a queue - planQueue = new PlanQueue(csContext, "root", null, null); - autoCreatedLeafQueue = new ReservationQueue(csContext, "a", planQueue); + planQueue = new PlanQueue(queueContext, "root", null, null); + autoCreatedLeafQueue = new ReservationQueue(queueContext, "a", planQueue); planQueue.addChildQueue(autoCreatedLeafQueue); } 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 1168f648024b1..5662df4c51037 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 @@ -90,6 +90,7 @@ public class TestReservations { CapacityScheduler cs; // CapacitySchedulerConfiguration csConf; CapacitySchedulerContext csContext; + CapacitySchedulerQueueContext queueContext; private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator(); @@ -135,7 +136,10 @@ private void setup(CapacitySchedulerConfiguration csConf, when(csContext.getClusterResource()).thenReturn( Resources.createResource(100 * 16 * GB, 100 * 12)); when(csContext.getResourceCalculator()).thenReturn(resourceCalculator); + CapacitySchedulerQueueManager queueManager = new CapacitySchedulerQueueManager(conf, + rmContext.getNodeLabelManager(), null); when(csContext.getPreemptionManager()).thenReturn(new PreemptionManager()); + when(csContext.getCapacitySchedulerQueueManager()).thenReturn(queueManager); when(csContext.getRMContext()).thenReturn(rmContext); RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager( conf); @@ -143,12 +147,11 @@ private void setup(CapacitySchedulerConfiguration csConf, when(csContext.getContainerTokenSecretManager()).thenReturn( containerTokenSecretManager); - root = CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null, - CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); + queueContext = new CapacitySchedulerQueueContext(csContext); - ResourceUsage queueResUsage = root.getQueueResourceUsage(); - when(csContext.getClusterResourceUsage()) - .thenReturn(queueResUsage); + root = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, + CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook); + queueManager.setRootQueue(root); spyRMContext = spy(rmContext); when(spyRMContext.getScheduler()).thenReturn(cs); @@ -1181,23 +1184,24 @@ public void testAssignToQueue() throws Exception { public void refreshQueuesTurnOffReservationsContLook(LeafQueue a, CapacitySchedulerConfiguration csConf) throws Exception { // before reinitialization - assertEquals(true, a.getReservationContinueLooking()); + assertEquals(true, a.isReservationsContinueLooking()); assertEquals(true, - ((ParentQueue) a.getParent()).getReservationContinueLooking()); + ((ParentQueue) a.getParent()).isReservationsContinueLooking()); csConf.setBoolean( CapacitySchedulerConfiguration.RESERVE_CONT_LOOK_ALL_NODES, false); CSQueueStore newQueues = new CSQueueStore(); - CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext, + queueContext.reinitialize(); + CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues, TestUtils.spyHook); queues = newQueues; root.reinitialize(newRoot, cs.getClusterResource()); // after reinitialization - assertEquals(false, a.getReservationContinueLooking()); + assertEquals(false, a.isReservationsContinueLooking()); assertEquals(false, - ((ParentQueue) a.getParent()).getReservationContinueLooking()); + ((ParentQueue) a.getParent()).isReservationsContinueLooking()); } @Test 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/TestUsersManager.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/TestUsersManager.java index 5b79ee2e255c1..c71d862a32a02 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/TestUsersManager.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/TestUsersManager.java @@ -58,15 +58,11 @@ public class TestUsersManager { @Mock private QueueMetrics metrics; - @Mock - private CapacitySchedulerContext context; - @Before public void setup() { usersManager = new UsersManager(metrics, lQueue, labelMgr, - context, new DefaultResourceCalculator()); when(lQueue.getMinimumAllocation()).thenReturn(MINIMUM_ALLOCATION);