diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestSchedConfCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestSchedConfCLI.java index 830fb8d5669fa..4df20da68ccca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestSchedConfCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestSchedConfCLI.java @@ -45,7 +45,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStore.LogMutation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfigurationProvider; @@ -181,9 +180,9 @@ public String getName() { private static void setupQueueConfiguration( CapacitySchedulerConfiguration config) { - config.setQueues(new QueuePath(CapacitySchedulerConfiguration.ROOT), + config.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"testqueue"}); - QueuePath a = new QueuePath(CapacitySchedulerConfiguration.ROOT + ".testqueue"); + String a = CapacitySchedulerConfiguration.ROOT + ".testqueue"; config.setCapacity(a, 100f); config.setMaximumCapacity(a, 100f); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/test/java/org/apache/hadoop/yarn/server/globalpolicygenerator/policygenerator/TestPolicyGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/test/java/org/apache/hadoop/yarn/server/globalpolicygenerator/policygenerator/TestPolicyGenerator.java index 86523d839276f..446eeee2cd922 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/test/java/org/apache/hadoop/yarn/server/globalpolicygenerator/policygenerator/TestPolicyGenerator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/test/java/org/apache/hadoop/yarn/server/globalpolicygenerator/policygenerator/TestPolicyGenerator.java @@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo; @@ -251,13 +250,13 @@ public void testCallRM() { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - final QueuePath a = new QueuePath(CapacitySchedulerConfiguration.ROOT + ".a"); - final QueuePath b = new QueuePath(CapacitySchedulerConfiguration.ROOT + ".b"); - final QueuePath a1 = new QueuePath(a + ".a1"); - final QueuePath a2 = new QueuePath(a + ".a2"); - final QueuePath b1 = new QueuePath(b + ".b1"); - final QueuePath b2 = new QueuePath(b + ".b2"); - final QueuePath b3 = new QueuePath(b + ".b3"); + final String a = CapacitySchedulerConfiguration.ROOT + ".a"; + final String b = CapacitySchedulerConfiguration.ROOT + ".b"; + final String a1 = a + ".a1"; + final String a2 = a + ".a2"; + final String b1 = b + ".b1"; + final String b2 = b + ".b2"; + final String b3 = b + ".b3"; float aCapacity = 10.5f; float bCapacity = 89.5f; float a1Capacity = 30; @@ -267,7 +266,7 @@ public void testCallRM() { float b3Capacity = 20; // Define top-level queues - csConf.setQueues(new QueuePath(CapacitySchedulerConfiguration.ROOT), + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); csConf.setCapacity(a, aCapacity); 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/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java index b0b41f9697dba..d9e4be9e5225b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java @@ -49,7 +49,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.security.CapacityReservationsACLsManager; import org.apache.hadoop.yarn.server.resourcemanager.security.FairReservationsACLsManager; @@ -431,7 +430,7 @@ protected Plan initializePlan(String planQueueName) throws YarnException { Plan plan = new InMemoryPlan(getRootQueueMetrics(), adPolicy, getAgent(planQueuePath), totCap, planStepSize, rescCalc, minAllocation, maxAllocation, planQueueName, getReplanner(planQueuePath), - getReservationSchedulerConfiguration().getMoveOnExpiry(new QueuePath(planQueuePath)), + getReservationSchedulerConfiguration().getMoveOnExpiry(planQueuePath), maxPeriodicity, rmContext); LOG.info("Initialized plan {} based on reservable queue {}", plan.toString(), planQueueName); @@ -441,7 +440,7 @@ maxAllocation, planQueueName, getReplanner(planQueuePath), protected Planner getReplanner(String planQueueName) { ReservationSchedulerConfiguration reservationConfig = getReservationSchedulerConfiguration(); - String plannerClassName = reservationConfig.getReplanner(new QueuePath(planQueueName)); + String plannerClassName = reservationConfig.getReplanner(planQueueName); LOG.info("Using Replanner: " + plannerClassName + " for queue: " + planQueueName); try { @@ -464,7 +463,7 @@ protected Planner getReplanner(String planQueueName) { protected ReservationAgent getAgent(String queueName) { ReservationSchedulerConfiguration reservationConfig = getReservationSchedulerConfiguration(); - String agentClassName = reservationConfig.getReservationAgent(new QueuePath(queueName)); + String agentClassName = reservationConfig.getReservationAgent(queueName); LOG.info("Using Agent: " + agentClassName + " for queue: " + queueName); try { Class agentClazz = conf.getClassByName(agentClassName); @@ -488,7 +487,7 @@ protected SharingPolicy getAdmissionPolicy(String queueName) { ReservationSchedulerConfiguration reservationConfig = getReservationSchedulerConfiguration(); String admissionPolicyClassName = - reservationConfig.getReservationAdmissionPolicy(new QueuePath(queueName)); + reservationConfig.getReservationAdmissionPolicy(queueName); LOG.info("Using AdmissionPolicy: " + admissionPolicyClassName + " for queue: " + queueName); try { 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/CapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java index db9794d6f6885..1f3f9bc4ba6bc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java @@ -23,7 +23,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResourceAllocation.RLEOperator; import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException; import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningQuotaException; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.util.resource.Resources; import java.util.Map; @@ -58,10 +57,9 @@ public class CapacityOverTimePolicy extends NoOverCommitPolicy { private float maxAvg; @Override - public void init(String reservationQueue, + public void init(String reservationQueuePath, ReservationSchedulerConfiguration conf) { this.conf = conf; - QueuePath reservationQueuePath = new QueuePath(reservationQueue); validWindow = this.conf.getReservationWindow(reservationQueuePath); maxInst = this.conf.getInstantaneousMaxCapacity(reservationQueuePath) / 100; maxAvg = this.conf.getAverageCapacity(reservationQueuePath) / 100; 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/ReservationSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSchedulerConfiguration.java index d6bf1ce813363..740b88c40c458 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSchedulerConfiguration.java @@ -23,7 +23,6 @@ import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.yarn.api.records.ReservationACL; import org.apache.hadoop.yarn.api.records.ReservationDefinition; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import java.util.Map; @@ -70,7 +69,7 @@ public ReservationSchedulerConfiguration( * @param queue name of the queue * @return true if the queue participates in reservation based scheduling */ - public abstract boolean isReservable(QueuePath queue); + public abstract boolean isReservable(String queue); /** * Gets a map containing the {@link AccessControlList} of users for each @@ -81,7 +80,7 @@ public ReservationSchedulerConfiguration( * which contains a list of users that have the specified permission level. */ public abstract Map getReservationAcls( - QueuePath queue); + String queue); /** * Gets the length of time in milliseconds for which the {@link SharingPolicy} @@ -90,7 +89,7 @@ public abstract Map getReservationAcls( * @return length in time in milliseconds for which to check the * {@link SharingPolicy} */ - public long getReservationWindow(QueuePath queue) { + public long getReservationWindow(String queue) { return DEFAULT_RESERVATION_WINDOW; } @@ -101,7 +100,7 @@ public long getReservationWindow(QueuePath queue) { * @param queue name of the queue * @return average capacity allowed by the {@link SharingPolicy} */ - public float getAverageCapacity(QueuePath queue) { + public float getAverageCapacity(String queue) { return DEFAULT_CAPACITY_OVER_TIME_MULTIPLIER; } @@ -110,7 +109,7 @@ public float getAverageCapacity(QueuePath queue) { * @param queue name of the queue * @return maximum allowed capacity at any time */ - public float getInstantaneousMaxCapacity(QueuePath queue) { + public float getInstantaneousMaxCapacity(String queue) { return DEFAULT_CAPACITY_OVER_TIME_MULTIPLIER; } @@ -119,7 +118,7 @@ public float getInstantaneousMaxCapacity(QueuePath queue) { * @param queue name of the queue * @return the class name of the {@link SharingPolicy} */ - public String getReservationAdmissionPolicy(QueuePath queue) { + public String getReservationAdmissionPolicy(String queue) { return DEFAULT_RESERVATION_ADMISSION_POLICY; } @@ -129,7 +128,7 @@ public String getReservationAdmissionPolicy(QueuePath queue) { * @param queue name of the queue * @return the class name of the {@code ReservationAgent} */ - public String getReservationAgent(QueuePath queue) { + public String getReservationAgent(String queue) { return DEFAULT_RESERVATION_AGENT_NAME; } @@ -138,7 +137,7 @@ public String getReservationAgent(QueuePath queue) { * @param queuePath name of the queue * @return true if reservation queues should be visible */ - public boolean getShowReservationAsQueues(QueuePath queuePath) { + public boolean getShowReservationAsQueues(String queuePath) { return DEFAULT_SHOW_RESERVATIONS_AS_QUEUES; } @@ -148,7 +147,7 @@ public boolean getShowReservationAsQueues(QueuePath queuePath) { * @param queue name of the queue * @return the class name of the {@code Planner} */ - public String getReplanner(QueuePath queue) { + public String getReplanner(String queue) { return DEFAULT_RESERVATION_PLANNER_NAME; } @@ -159,7 +158,7 @@ public String getReplanner(QueuePath queue) { * @return true if application should be moved, false if they need to be * killed */ - public boolean getMoveOnExpiry(QueuePath queue) { + public boolean getMoveOnExpiry(String queue) { return DEFAULT_RESERVATION_MOVE_ON_EXPIRY; } @@ -169,7 +168,7 @@ public boolean getMoveOnExpiry(QueuePath queue) { * @param queue name of the queue * @return the time in milliseconds for which to check constraints */ - public long getEnforcementWindow(QueuePath queue) { + public long getEnforcementWindow(String queue) { return DEFAULT_RESERVATION_ENFORCEMENT_WINDOW; } } 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/planning/SimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/SimpleCapacityReplanner.java index f14a87ed8cd1d..38880aeed524b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/SimpleCapacityReplanner.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/SimpleCapacityReplanner.java @@ -23,7 +23,6 @@ import java.util.Set; import java.util.TreeSet; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.yarn.api.records.ReservationDefinition; @@ -73,7 +72,7 @@ public SimpleCapacityReplanner() { @Override public void init(String planQueueName, ReservationSchedulerConfiguration conf) { - this.lengthOfCheckZone = conf.getEnforcementWindow(new QueuePath(planQueueName)); + this.lengthOfCheckZone = conf.getEnforcementWindow(planQueueName); } @Override 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 c056396fe1722..112ca7b4fd48a 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 @@ -61,26 +61,26 @@ public void setEntitlement(QueueEntitlement entitlement) } @Override - protected Resource getMinimumAbsoluteResource(QueuePath queuePath, + protected Resource getMinimumAbsoluteResource(String queuePath, String label) { - return super.getMinimumAbsoluteResource(QueuePrefixes - .getAutoCreatedQueueObjectTemplateConfPrefix(this.getParent().getQueuePathObject()), + return super.getMinimumAbsoluteResource(queueContext.getConfiguration() + .getAutoCreatedQueueTemplateConfPrefix(this.getParent().getQueuePath()), label); } @Override - protected Resource getMaximumAbsoluteResource(QueuePath queuePath, + protected Resource getMaximumAbsoluteResource(String queuePath, String label) { - return super.getMaximumAbsoluteResource(QueuePrefixes - .getAutoCreatedQueueObjectTemplateConfPrefix(this.getParent().getQueuePathObject()), + return super.getMaximumAbsoluteResource(queueContext.getConfiguration() + .getAutoCreatedQueueTemplateConfPrefix(this.getParent().getQueuePath()), label); } @Override - protected boolean checkConfigTypeIsAbsoluteResource(QueuePath queuePath, + protected boolean checkConfigTypeIsAbsoluteResource(String queuePath, String label) { - return super.checkConfigTypeIsAbsoluteResource(QueuePrefixes - .getAutoCreatedQueueObjectTemplateConfPrefix(this.getParent().getQueuePathObject()), + return super.checkConfigTypeIsAbsoluteResource(queueContext.getConfiguration() + .getAutoCreatedQueueTemplateConfPrefix(this.getParent().getQueuePath()), label); } 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 1bdaebb3d89e5..a84a68ac409a0 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 @@ -346,7 +346,7 @@ protected void setupQueueConfigs(Resource clusterResource) throws writeLock.lock(); try { CapacitySchedulerConfiguration configuration = queueContext.getConfiguration(); - this.acls = configuration.getAcls(getQueuePathObject()); + this.acls = configuration.getAcls(getQueuePath()); if (isDynamicQueue() || this instanceof AbstractAutoCreatedLeafQueue) { parseAndSetDynamicTemplates(); @@ -367,7 +367,7 @@ protected void setupQueueConfigs(Resource clusterResource) throws // Setup queue's maximumAllocation respecting the global // and the queue settings - this.queueAllocationSettings.setupMaximumAllocation(configuration, getQueuePathObject(), + this.queueAllocationSettings.setupMaximumAllocation(configuration, getQueuePath(), parent); // Initialize the queue state based on previous state, configured state @@ -382,10 +382,10 @@ protected void setupQueueConfigs(Resource clusterResource) throws configuration.getReservationContinueLook(); this.configuredCapacityVectors = configuration - .parseConfiguredResourceVector(queuePath, + .parseConfiguredResourceVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels()); this.configuredMaxCapacityVectors = configuration - .parseConfiguredMaximumCapacityVector(queuePath, + .parseConfiguredMaximumCapacityVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels(), QueueCapacityVector.newInstance()); @@ -420,11 +420,11 @@ protected void setupQueueConfigs(Resource clusterResource) throws // Store preemption settings this.preemptionSettings = new CSQueuePreemptionSettings(this, configuration); this.priority = configuration.getQueuePriority( - getQueuePathObject()); + getQueuePath()); // Update multi-node sorting algorithm for scheduling as configured. setMultiNodeSortingPolicyName( - configuration.getMultiNodesSortingAlgorithmPolicy(getQueuePathObject())); + configuration.getMultiNodesSortingAlgorithmPolicy(getQueuePath())); // Setup application related limits this.queueAppLifetimeSettings = new QueueAppLifetimeAndLimitSettings(configuration, @@ -440,7 +440,7 @@ protected void setupQueueConfigs(Resource clusterResource) throws protected void parseAndSetDynamicTemplates() { // Set the template properties from the parent to the queuepath of the child ((AbstractParentQueue) parent).getAutoCreatedQueueTemplate() - .setTemplateEntriesForChild(queueContext.getConfiguration(), getQueuePathObject(), + .setTemplateEntriesForChild(queueContext.getConfiguration(), getQueuePath(), this instanceof AbstractLeafQueue); String parentTemplate = String.format("%s.%s", parent.getQueuePath(), @@ -488,21 +488,21 @@ private UserWeights getUserWeightsFromHierarchy() { // Insert this queue's userWeights, overriding parent's userWeights if // there is an overlap. unionInheritedWeights.addFrom( - queueContext.getConfiguration().getAllUserWeightsForQueue(getQueuePathObject())); + queueContext.getConfiguration().getAllUserWeightsForQueue(getQueuePath())); return unionInheritedWeights; } - protected Resource getMinimumAbsoluteResource(QueuePath queuePath, String label) { + protected Resource getMinimumAbsoluteResource(String queuePath, String label) { return queueContext.getConfiguration() .getMinimumResourceRequirement(label, queuePath, resourceTypes); } - protected Resource getMaximumAbsoluteResource(QueuePath queuePath, String label) { + protected Resource getMaximumAbsoluteResource(String queuePath, String label) { return queueContext.getConfiguration() .getMaximumResourceRequirement(label, queuePath, resourceTypes); } - protected boolean checkConfigTypeIsAbsoluteResource(QueuePath queuePath, + protected boolean checkConfigTypeIsAbsoluteResource(String queuePath, String label) { return queueContext.getConfiguration().checkConfigTypeIsAbsoluteResource(label, queuePath, resourceTypes); @@ -518,7 +518,7 @@ protected void updateCapacityConfigType() { if (queueContext.getConfiguration().isLegacyQueueMode()) { localType = checkConfigTypeIsAbsoluteResource( - getQueuePathObject(), label) ? CapacityConfigType.ABSOLUTE_RESOURCE + getQueuePath(), label) ? CapacityConfigType.ABSOLUTE_RESOURCE : CapacityConfigType.PERCENTAGE; } else { // TODO: revisit this later @@ -556,8 +556,8 @@ protected void updateCapacityConfigType() { */ protected void updateConfigurableResourceLimits(Resource clusterResource) { for (String label : queueNodeLabelsSettings.getConfiguredNodeLabels()) { - final Resource minResource = getMinimumAbsoluteResource(getQueuePathObject(), label); - Resource maxResource = getMaximumAbsoluteResource(getQueuePathObject(), label); + final Resource minResource = getMinimumAbsoluteResource(getQueuePath(), label); + Resource maxResource = getMaximumAbsoluteResource(getQueuePath(), label); if (parent != null) { final Resource parentMax = parent.getQueueResourceQuotas() 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 e40b8aaeb0bb4..280d3d182fb27 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 @@ -193,16 +193,16 @@ protected void setupQueueConfigs(Resource clusterResource) throws setQueueResourceLimitsInfo(clusterResource); setOrderingPolicy( - configuration.getAppOrderingPolicy(getQueuePathObject())); + configuration.getAppOrderingPolicy(getQueuePath())); - usersManager.setUserLimit(configuration.getUserLimit(getQueuePathObject())); - usersManager.setUserLimitFactor(configuration.getUserLimitFactor(getQueuePathObject())); + usersManager.setUserLimit(configuration.getUserLimit(getQueuePath())); + usersManager.setUserLimitFactor(configuration.getUserLimitFactor(getQueuePath())); maxAMResourcePerQueuePercent = configuration.getMaximumApplicationMasterResourcePerQueuePercent( - getQueuePathObject()); + getQueuePath()); - maxApplications = configuration.getMaximumApplicationsPerQueue(getQueuePathObject()); + maxApplications = configuration.getMaximumApplicationsPerQueue(getQueuePath()); if (maxApplications < 0) { int maxGlobalPerQueueApps = configuration.getGlobalMaximumApplicationsPerQueue(); @@ -211,7 +211,7 @@ protected void setupQueueConfigs(Resource clusterResource) throws } } - priorityAcls = configuration.getPriorityAcls(getQueuePathObject(), + priorityAcls = configuration.getPriorityAcls(getQueuePath(), configuration.getClusterLevelApplicationMaxPriority()); Set accessibleNodeLabels = this.queueNodeLabelsSettings.getAccessibleNodeLabels(); @@ -254,10 +254,10 @@ protected void setupQueueConfigs(Resource clusterResource) throws } defaultAppPriorityPerQueue = Priority.newInstance( - configuration.getDefaultApplicationPriorityConfPerQueue(getQueuePathObject())); + configuration.getDefaultApplicationPriorityConfPerQueue(getQueuePath())); // Validate leaf queue's user's weights. - float queueUserLimit = Math.min(100.0f, configuration.getUserLimit(getQueuePathObject())); + float queueUserLimit = Math.min(100.0f, configuration.getUserLimit(getQueuePath())); getUserWeights().validateForLeafQueue(queueUserLimit, getQueuePath()); usersManager.updateUserWeights(); @@ -1703,10 +1703,10 @@ protected boolean canAssignToUser(Resource clusterResource, @Override protected void parseAndSetDynamicTemplates() { // set to -1, to disable it - queueContext.getConfiguration().setUserLimitFactor(getQueuePathObject(), -1); + queueContext.getConfiguration().setUserLimitFactor(getQueuePath(), -1); // Set Max AM percentage to a higher value queueContext.getConfiguration().setMaximumApplicationMasterResourcePerQueuePercent( - getQueuePathObject(), 1f); + getQueuePath(), 1f); super.parseAndSetDynamicTemplates(); } @@ -1716,7 +1716,7 @@ protected void setDynamicQueueACLProperties() { if (parent instanceof AbstractManagedParentQueue) { acls.putAll(queueContext.getConfiguration().getACLsForLegacyAutoCreatedLeafQueue( - parent.getQueuePathObject())); + parent.getQueuePath())); } else if (parent instanceof ParentQueue) { acls.putAll(getACLsForFlexibleAutoCreatedLeafQueue( ((ParentQueue) parent).getAutoCreatedQueueTemplate())); @@ -2432,7 +2432,7 @@ public void stopQueue() { void updateMaximumApplications(boolean absoluteCapacityIsReadyForUse) { CapacitySchedulerConfiguration configuration = queueContext.getConfiguration(); - int maxAppsForQueue = configuration.getMaximumApplicationsPerQueue(getQueuePathObject()); + int maxAppsForQueue = configuration.getMaximumApplicationsPerQueue(getQueuePath()); int maxDefaultPerQueueApps = configuration.getGlobalMaximumApplicationsPerQueue(); int maxSystemApps = configuration.getMaximumSystemApplications(); @@ -2566,6 +2566,6 @@ List getCopyOfNonRunnableAppSchedulables() { public boolean isEligibleForAutoDeletion() { return isDynamicQueue() && getNumApplications() == 0 && queueContext.getConfiguration(). - isAutoExpiredDeletionEnabled(this.getQueuePathObject()); + 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/AbstractParentQueue.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/AbstractParentQueue.java index 33f5cd73c6716..50516dd2bc5fa 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/AbstractParentQueue.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/AbstractParentQueue.java @@ -134,7 +134,7 @@ public AbstractParentQueue(CapacitySchedulerQueueContext queueContext, this.childQueues = new ArrayList<>(); this.allowZeroCapacitySum = queueContext.getConfiguration() - .getAllowZeroCapacitySum(getQueuePathObject()); + .getAllowZeroCapacitySum(getQueuePath()); } @@ -168,7 +168,7 @@ protected void setupQueueConfigs(Resource clusterResource) // Initialize queue ordering policy queueOrderingPolicy = configuration.getQueueOrderingPolicy( - getQueuePathObject(), parent == null ? + getQueuePath(), parent == null ? null : ((AbstractParentQueue) parent).getQueueOrderingPolicyConfigName()); queueOrderingPolicy.setQueues(childQueues); @@ -550,7 +550,7 @@ public void removeChildQueue(CSQueue queue) */ public boolean isEligibleForAutoQueueCreation() { return isDynamicQueue() || queueContext.getConfiguration(). - isAutoQueueCreationV2Enabled(getQueuePathObject()); + isAutoQueueCreationV2Enabled(getQueuePath()); } @Override @@ -1618,7 +1618,7 @@ Map getEffectiveMinRatio(String label) { public boolean isEligibleForAutoDeletion() { return isDynamicQueue() && getChildQueues().size() == 0 && queueContext.getConfiguration(). - isAutoExpiredDeletionEnabled(this.getQueuePathObject()); + isAutoExpiredDeletionEnabled(this.getQueuePath()); } public AutoCreatedQueueTemplate getAutoCreatedQueueTemplate() { 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/AutoCreatedQueueTemplate.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/AutoCreatedQueueTemplate.java index d31f4307cc73d..bbeca4b072450 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/AutoCreatedQueueTemplate.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/AutoCreatedQueueTemplate.java @@ -25,8 +25,10 @@ import java.util.Map; import java.util.Set; +import org.apache.hadoop.util.Lists; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AUTO_QUEUE_CREATION_V2_PREFIX; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getQueuePrefix; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueuePrefix; /** * A handler for storing and setting auto created queue template settings. @@ -39,7 +41,7 @@ public class AutoCreatedQueueTemplate { public static final String AUTO_QUEUE_PARENT_TEMPLATE_PREFIX = AUTO_QUEUE_CREATION_V2_PREFIX + "parent-template."; - public static final String WILDCARD_QUEUE = "*"; + private static final String WILDCARD_QUEUE = "*"; private final Map templateProperties = new HashMap<>(); private final Map leafOnlyProperties = new HashMap<>(); @@ -51,8 +53,9 @@ public AutoCreatedQueueTemplate(CapacitySchedulerConfiguration configuration, } @VisibleForTesting - public static String getAutoQueueTemplatePrefix(QueuePath queuePath) { - return getQueuePrefix(queuePath) + AUTO_QUEUE_TEMPLATE_PREFIX; + public static String getAutoQueueTemplatePrefix(String queue) { + return CapacitySchedulerConfiguration.getQueuePrefix(queue) + + AUTO_QUEUE_TEMPLATE_PREFIX; } /** @@ -86,7 +89,7 @@ public Map getParentOnlyProperties() { * @param childQueuePath child queue path used for prefixing the properties */ public void setTemplateEntriesForChild(CapacitySchedulerConfiguration conf, - QueuePath childQueuePath) { + String childQueuePath) { setTemplateEntriesForChild(conf, childQueuePath, false); } @@ -100,9 +103,9 @@ public void setTemplateEntriesForChild(CapacitySchedulerConfiguration conf, * @param childQueuePath child queue path used for prefixing the properties */ public void setTemplateEntriesForChild(CapacitySchedulerConfiguration conf, - QueuePath childQueuePath, + String childQueuePath, boolean isLeaf) { - if (childQueuePath.isRoot()) { + if (childQueuePath.equals(ROOT)) { return; } @@ -111,7 +114,8 @@ public void setTemplateEntriesForChild(CapacitySchedulerConfiguration conf, // Get all properties that are explicitly set Set alreadySetProps = configurationProperties - .getPropertiesWithPrefix(getQueuePrefix(childQueuePath)).keySet(); + .getPropertiesWithPrefix(CapacitySchedulerConfiguration + .getQueuePrefix(childQueuePath)).keySet(); // Check template properties only set for leaf or parent queues Map queueTypeSpecificTemplates = parentOnlyProperties; @@ -125,7 +129,8 @@ public void setTemplateEntriesForChild(CapacitySchedulerConfiguration conf, if (alreadySetProps.contains(entry.getKey())) { continue; } - conf.set(getQueuePrefix(childQueuePath) + entry.getKey(), entry.getValue()); + conf.set(CapacitySchedulerConfiguration.getQueuePrefix( + childQueuePath) + entry.getKey(), entry.getValue()); } for (Map.Entry entry : templateProperties.entrySet()) { @@ -135,7 +140,8 @@ public void setTemplateEntriesForChild(CapacitySchedulerConfiguration conf, || queueTypeSpecificTemplates.containsKey(entry.getKey())) { continue; } - conf.set(getQueuePrefix(childQueuePath) + entry.getKey(), entry.getValue()); + conf.set(CapacitySchedulerConfiguration.getQueuePrefix( + childQueuePath) + entry.getKey(), entry.getValue()); } } @@ -148,26 +154,46 @@ public void setTemplateEntriesForChild(CapacitySchedulerConfiguration conf, */ private void setTemplateConfigEntries(CapacitySchedulerConfiguration configuration, QueuePath queuePath) { - if (!queuePath.isInvalid()) { - ConfigurationProperties configurationProperties = - configuration.getConfigurationProperties(); - - int maxAutoCreatedQueueDepth = configuration - .getMaximumAutoCreatedQueueDepth(queuePath); - List wildcardedQueuePaths = - queuePath.getWildcardedQueuePaths(maxAutoCreatedQueueDepth); - - for (QueuePath templateQueuePath: wildcardedQueuePaths) { - // Get all configuration entries with - // yarn.scheduler.capacity. prefix - Map queueProps = configurationProperties - .getPropertiesWithPrefix(getQueuePrefix(templateQueuePath)); - - // Store template, parent-template and leaf-template properties - for (Map.Entry entry : queueProps.entrySet()) { - storeConfiguredTemplates(entry.getKey(), entry.getValue()); - } + ConfigurationProperties configurationProperties = + configuration.getConfigurationProperties(); + + List queuePathParts = Lists.newArrayList(queuePath.iterator()); + + if (queuePathParts.size() <= 1 && !queuePath.isRoot()) { + // This is an invalid queue path + return; + } + int queuePathMaxIndex = queuePathParts.size() - 1; + + // start with the most explicit format (without wildcard) + int wildcardLevel = 0; + // root can not be wildcarded + int supportedWildcardLevel = Math.min(queuePathMaxIndex, + configuration.getMaximumAutoCreatedQueueDepth(queuePath.getFullPath())); + // Allow root to have template properties + if (queuePath.isRoot()) { + supportedWildcardLevel = 0; + } + + // Collect all template entries + while (wildcardLevel <= supportedWildcardLevel) { + String templateQueuePath = String.join(".", queuePathParts); + // Get all configuration entries with + // yarn.scheduler.capacity. prefix + Map queueProps = configurationProperties + .getPropertiesWithPrefix(getQueuePrefix(templateQueuePath)); + + // Store template, parent-template and leaf-template properties + for (Map.Entry entry : queueProps.entrySet()) { + storeConfiguredTemplates(entry.getKey(), entry.getValue()); } + + // Replace a queue part with a wildcard based on the wildcard level + // eg. root.a -> root.* + int queuePartToWildcard = queuePathMaxIndex - wildcardLevel; + queuePathParts.set(queuePartToWildcard, WILDCARD_QUEUE); + + ++wildcardLevel; } } 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 d429b208233b9..dc254747dfec5 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 @@ -58,14 +58,14 @@ private boolean isQueueHierarchyPreemptionDisabled(CSQueue q, // on, then q does not have preemption disabled (default=false, below) // unless the preemption_disabled property is explicitly set. if (parentQ == null) { - return configuration.getPreemptionDisabled(q.getQueuePathObject(), false); + return configuration.getPreemptionDisabled(q.getQueuePath(), false); } // If this is not the root queue, inherit the default value for the // preemption_disabled property from the parent. Preemptability will be // inherited from the parent's hierarchy unless explicitly overridden at // this level. - return configuration.getPreemptionDisabled(q.getQueuePathObject(), + return configuration.getPreemptionDisabled(q.getQueuePath(), parentQ.getPreemptionDisabled()); } @@ -96,13 +96,13 @@ private boolean isIntraQueueHierarchyPreemptionDisabled(CSQueue q, CSQueue parentQ = q.getParent(); if (parentQ == null) { return configuration - .getIntraQueuePreemptionDisabled(q.getQueuePathObject(), false); + .getIntraQueuePreemptionDisabled(q.getQueuePath(), false); } // At this point, the master preemption switch is enabled down to this // queue's level. Determine whether intra-queue preemption is enabled // down to this queue's level and return that value. - return configuration.getIntraQueuePreemptionDisabled(q.getQueuePathObject(), + return configuration.getIntraQueuePreemptionDisabled(q.getQueuePath(), parentQ.getIntraQueuePreemptionDisabledInHierarchy()); } 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/CSQueueUtils.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/CSQueueUtils.java index 76aab3179d759..9e912d3dc4c5c 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/CSQueueUtils.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/CSQueueUtils.java @@ -80,7 +80,7 @@ public static void loadCapacitiesByLabelsFromConf( label, csConf.getMaximumAMResourcePercentPerPartition(queuePath, label)); queueCapacities.setWeight(label, - csConf.getNonLabeledQueueWeight(queuePath)); + csConf.getNonLabeledQueueWeight(queuePath.getFullPath())); } else{ queueCapacities.setCapacity(label, csConf.getLabeledQueueCapacity(queuePath, label) / 100); 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 ec5b074915ef0..6ebadcba1c916 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 @@ -2537,7 +2537,7 @@ private String resolveReservationQueueName(String queueName, if (queue == null) { // reservation has terminated during failover if (isRecovering && conf.getMoveOnExpiry( - getQueue(queueName).getQueuePathObject())) { + getQueue(queueName).getQueuePath())) { // move to the default child queue of the plan return getDefaultReservationQueueName(queueName); } 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/CapacitySchedulerConfigValidator.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/CapacitySchedulerConfigValidator.java index e2b61df9eb3fe..e40e978a80022 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/CapacitySchedulerConfigValidator.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/CapacitySchedulerConfigValidator.java @@ -136,8 +136,8 @@ public static void validateQueueHierarchy( } final String queuePath = oldQueue.getQueuePath(); - final String configPrefix = QueuePrefixes.getQueuePrefix( - oldQueue.getQueuePathObject()); + final String configPrefix = CapacitySchedulerConfiguration.getQueuePrefix( + oldQueue.getQueuePath()); final QueueState newQueueState = createQueueState(newConf.get(configPrefix + "state"), queuePath); final CSQueue newQueue = newQueues.get(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/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 8d9cf20793014..5ab237d282a6a 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 @@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.security.AccessType; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.placement.QueueMapping; @@ -77,10 +78,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getAutoCreatedQueueObjectTemplateConfPrefix; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getNodeLabelPrefix; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getQueuePrefix; - public class CapacitySchedulerConfiguration extends ReservationSchedulerConfiguration { private static final Logger LOG = @@ -441,13 +438,13 @@ public static QueueCapacityConfigParser getQueueCapacityConfigParser() { return queueCapacityConfigParser; } - public int getMaximumAutoCreatedQueueDepth(QueuePath queuePath) { + public int getMaximumAutoCreatedQueueDepth(String queuePath) { return getInt(getQueuePrefix(queuePath) + MAXIMUM_QUEUE_DEPTH, getInt(PREFIX + MAXIMUM_QUEUE_DEPTH, DEFAULT_MAXIMUM_QUEUE_DEPTH)); } - public void setMaximumAutoCreatedQueueDepth(QueuePath queue, int value) { - setInt(getQueuePrefix(queue) + MAXIMUM_QUEUE_DEPTH, value); + public void setMaximumAutoCreatedQueueDepth(String queuePath, int value) { + setInt(getQueuePrefix(queuePath) + MAXIMUM_QUEUE_DEPTH, value); } public void setMaximumAutoCreatedQueueDepth(int value) { @@ -479,10 +476,27 @@ public CapacitySchedulerConfiguration(Configuration configuration, } } + public static String getQueuePrefix(String queue) { + String queueName = PREFIX + queue + DOT; + return queueName; + } + + static String getQueueOrderingPolicyPrefix(String queue) { + String queueName = PREFIX + queue + DOT + ORDERING_POLICY + DOT; + return queueName; + } + static String getUserPrefix(String user) { return PREFIX + "user." + user + DOT; } + public static String getNodeLabelPrefix(String queue, String label) { + if (label.equals(CommonNodeLabelsManager.NO_LABEL)) { + return getQueuePrefix(queue); + } + return getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS + DOT + label + DOT; + } + public void setMaximumSystemApplications(int numMaxApps) { setInt(MAXIMUM_SYSTEM_APPLICATIONS, numMaxApps); } @@ -505,10 +519,10 @@ public float getMaximumApplicationMasterResourcePercent() { /** * Get the maximum applications per queue setting. - * @param queue path of the queue + * @param queue name of the queue * @return setting specified or -1 if not set */ - public int getMaximumApplicationsPerQueue(QueuePath queue) { + public int getMaximumApplicationsPerQueue(String queue) { int maxApplicationsPerQueue = getInt(getQueuePrefix(queue) + MAXIMUM_APPLICATIONS_SUFFIX, (int)UNDEFINED); @@ -516,7 +530,7 @@ public int getMaximumApplicationsPerQueue(QueuePath queue) { } @VisibleForTesting - public void setMaximumApplicationsPerQueue(QueuePath queue, + public void setMaximumApplicationsPerQueue(String queue, int numMaxApps) { setInt(getQueuePrefix(queue) + MAXIMUM_APPLICATIONS_SUFFIX, numMaxApps); @@ -524,54 +538,54 @@ public void setMaximumApplicationsPerQueue(QueuePath queue, /** * Get the maximum am resource percent per queue setting. - * @param queue path of the queue + * @param queue name of the queue * @return per queue setting or defaults to the global am-resource-percent * setting if per queue setting not present */ - public float getMaximumApplicationMasterResourcePerQueuePercent(QueuePath queue) { + public float getMaximumApplicationMasterResourcePerQueuePercent(String queue) { return getFloat(getQueuePrefix(queue) + MAXIMUM_AM_RESOURCE_SUFFIX, getMaximumApplicationMasterResourcePercent()); } - public void setMaximumApplicationMasterResourcePerQueuePercent(QueuePath queue, + public void setMaximumApplicationMasterResourcePerQueuePercent(String queue, float percent) { setFloat(getQueuePrefix(queue) + MAXIMUM_AM_RESOURCE_SUFFIX, percent); } - private void throwExceptionForUnexpectedWeight(float weight, QueuePath queue, + private void throwExceptionForUnexpectedWeight(float weight, String queue, String label) { if ((weight < -1e-6 && Math.abs(weight + 1) > 1e-6) || weight > 10000) { throw new IllegalArgumentException( - "Illegal " + "weight=" + weight + " for queue=" + queue.getFullPath() + "label=" + "Illegal " + "weight=" + weight + " for queue=" + queue + "label=" + label + ". Acceptable values: [0, 10000], -1 is same as not set"); } } - public float getNonLabeledQueueWeight(QueuePath queue) { + public float getNonLabeledQueueWeight(String queue) { String configuredValue = get(getQueuePrefix(queue) + CAPACITY); float weight = extractFloatValueFromWeightConfig(configuredValue); throwExceptionForUnexpectedWeight(weight, queue, ""); return weight; } - public void setNonLabeledQueueWeight(QueuePath queue, float weight) { + public void setNonLabeledQueueWeight(String queue, float weight) { set(getQueuePrefix(queue) + CAPACITY, weight + WEIGHT_SUFFIX); } - public void setLabeledQueueWeight(QueuePath queue, String label, float weight) { + public void setLabeledQueueWeight(String queue, String label, float weight) { set(getNodeLabelPrefix(queue, label) + CAPACITY, weight + WEIGHT_SUFFIX); } public float getLabeledQueueWeight(QueuePath queue, String label) { - String configuredValue = get(getNodeLabelPrefix(queue, label) + CAPACITY); + String configuredValue = get(getNodeLabelPrefix(queue.getFullPath(), label) + CAPACITY); float weight = extractFloatValueFromWeightConfig(configuredValue); - throwExceptionForUnexpectedWeight(weight, queue, label); + throwExceptionForUnexpectedWeight(weight, queue.getFullPath(), label); return weight; } public float getNonLabeledQueueCapacity(QueuePath queue) { - String configuredCapacity = get(getQueuePrefix(queue) + CAPACITY); + String configuredCapacity = get(getQueuePrefix(queue.getFullPath()) + CAPACITY); boolean absoluteResourceConfigured = (configuredCapacity != null) && RESOURCE_PATTERN.matcher(configuredCapacity).find(); boolean isCapacityVectorFormat = queueCapacityConfigParser @@ -593,16 +607,16 @@ public float getNonLabeledQueueCapacity(QueuePath queue) { if (capacity < MINIMUM_CAPACITY_VALUE || capacity > MAXIMUM_CAPACITY_VALUE) { throw new IllegalArgumentException( - "Illegal " + "capacity of " + capacity + " for queue " + queue.getFullPath()); + "Illegal " + "capacity of " + capacity + " for queue " + queue); } LOG.debug("CSConf - getCapacity: queuePrefix={}, capacity={}", - getQueuePrefix(queue), capacity); + getQueuePrefix(queue.getFullPath()), capacity); return capacity; } - public void setCapacity(QueuePath queue, float capacity) { - if (queue.isRoot()) { + public void setCapacity(String queue, float capacity) { + if (queue.equals("root")) { throw new IllegalArgumentException( "Cannot set capacity, root queue has a fixed capacity of 100.0f"); } @@ -613,8 +627,8 @@ public void setCapacity(QueuePath queue, float capacity) { } @VisibleForTesting - public void setCapacity(QueuePath queue, String absoluteResourceCapacity) { - if (queue.isRoot()) { + public void setCapacity(String queue, String absoluteResourceCapacity) { + if (queue.equals("root")) { throw new IllegalArgumentException( "Cannot set capacity, root queue has a fixed capacity"); } @@ -625,7 +639,7 @@ public void setCapacity(QueuePath queue, String absoluteResourceCapacity) { } public float getNonLabeledQueueMaximumCapacity(QueuePath queue) { - String configuredCapacity = get(getQueuePrefix(queue) + MAXIMUM_CAPACITY); + String configuredCapacity = get(getQueuePrefix(queue.getFullPath()) + MAXIMUM_CAPACITY); boolean matcher = (configuredCapacity != null) && RESOURCE_PATTERN.matcher(configuredCapacity).find() || queueCapacityConfigParser.isCapacityVectorFormat(configuredCapacity); @@ -646,38 +660,38 @@ public float getNonLabeledQueueMaximumCapacity(QueuePath queue) { return maxCapacity; } - public void setMaximumCapacity(QueuePath queue, float maxCapacity) { + public void setMaximumCapacity(String queue, float maxCapacity) { if (maxCapacity > MAXIMUM_CAPACITY_VALUE) { throw new IllegalArgumentException("Illegal " + - "maximum-capacity of " + maxCapacity + " for queue " + queue.getFullPath()); + "maximum-capacity of " + maxCapacity + " for queue " + queue); } setFloat(getQueuePrefix(queue) + MAXIMUM_CAPACITY, maxCapacity); LOG.debug("CSConf - setMaxCapacity: queuePrefix={}, maxCapacity={}", getQueuePrefix(queue), maxCapacity); } - public void setCapacityByLabel(QueuePath queue, String label, float capacity) { + public void setCapacityByLabel(String queue, String label, float capacity) { setFloat(getNodeLabelPrefix(queue, label) + CAPACITY, capacity); } @VisibleForTesting - public void setCapacityByLabel(QueuePath queue, String label, + public void setCapacityByLabel(String queue, String label, String absoluteResourceCapacity) { set(getNodeLabelPrefix(queue, label) + CAPACITY, absoluteResourceCapacity); } - public void setMaximumCapacityByLabel(QueuePath queue, String label, + public void setMaximumCapacityByLabel(String queue, String label, float capacity) { setFloat(getNodeLabelPrefix(queue, label) + MAXIMUM_CAPACITY, capacity); } - public void setMaximumCapacityByLabel(QueuePath queue, String label, + public void setMaximumCapacityByLabel(String queue, String label, String absoluteResourceCapacity) { set(getNodeLabelPrefix(queue, label) + MAXIMUM_CAPACITY, absoluteResourceCapacity); } - public float getUserLimit(QueuePath queue) { + public float getUserLimit(String queue) { float defaultUserLimit = getFloat(PREFIX + USER_LIMIT, DEFAULT_USER_LIMIT); float userLimit = getFloat(getQueuePrefix(queue) + USER_LIMIT, defaultUserLimit); @@ -689,7 +703,7 @@ public float getUserLimit(QueuePath queue) { // if possible. @SuppressWarnings("unchecked") public OrderingPolicy getAppOrderingPolicy( - QueuePath queue) { + String queue) { String policyType = get(getQueuePrefix(queue) + ORDERING_POLICY, DEFAULT_APP_ORDERING_POLICY); @@ -728,7 +742,7 @@ public OrderingPolicy getAppOrderingPolicy( return orderingPolicy; } - public void setUserLimit(QueuePath queue, float userLimit) { + public void setUserLimit(String queue, float userLimit) { setFloat(getQueuePrefix(queue) + USER_LIMIT, userLimit); LOG.debug("here setUserLimit: queuePrefix={}, userLimit={}", getQueuePrefix(queue), getUserLimit(queue)); @@ -739,7 +753,7 @@ public void setDefaultUserLimit(float defaultUserLimit) { setFloat(PREFIX + USER_LIMIT, defaultUserLimit); } - public float getUserLimitFactor(QueuePath queue) { + public float getUserLimitFactor(String queue) { float defaultUserLimitFactor = getFloat(PREFIX + USER_LIMIT_FACTOR, DEFAULT_USER_LIMIT_FACTOR); float userLimitFactor = getFloat(getQueuePrefix(queue) + USER_LIMIT_FACTOR, @@ -747,8 +761,8 @@ public float getUserLimitFactor(QueuePath queue) { return userLimitFactor; } - public void setUserLimitFactor(QueuePath queuePath, float userLimitFactor) { - setFloat(getQueuePrefix(queuePath) + USER_LIMIT_FACTOR, userLimitFactor); + public void setUserLimitFactor(String queue, float userLimitFactor) { + setFloat(getQueuePrefix(queue) + USER_LIMIT_FACTOR, userLimitFactor); } @VisibleForTesting @@ -756,7 +770,7 @@ public void setDefaultUserLimitFactor(float defaultUserLimitFactor) { setFloat(PREFIX + USER_LIMIT_FACTOR, defaultUserLimitFactor); } - public QueueState getConfiguredState(QueuePath queue) { + public QueueState getConfiguredState(String queue) { String state = get(getQueuePrefix(queue) + STATE); if (state == null) { return null; @@ -765,18 +779,18 @@ public QueueState getConfiguredState(QueuePath queue) { } } - public QueueState getState(QueuePath queue) { + public QueueState getState(String queue) { QueueState state = getConfiguredState(queue); return (state == null) ? QueueState.RUNNING : state; } @Private @VisibleForTesting - public void setState(QueuePath queue, QueueState state) { + public void setState(String queue, QueueState state) { set(getQueuePrefix(queue) + STATE, state.name()); } - public void setAccessibleNodeLabels(QueuePath queue, Set labels) { + public void setAccessibleNodeLabels(String queue, Set labels) { if (labels == null) { return; } @@ -784,27 +798,27 @@ public void setAccessibleNodeLabels(QueuePath queue, Set labels) { set(getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS, str); } - public Set getAccessibleNodeLabels(QueuePath queue) { + public Set getAccessibleNodeLabels(String queue) { String accessibleLabelStr = get(getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS); // When accessible-label is null, if (accessibleLabelStr == null) { // Only return null when queue is not ROOT - if (!queue.isRoot()) { + if (!queue.equals(ROOT)) { return null; } } else { // print a warning when accessibleNodeLabel specified in config and queue // is ROOT - if (queue.isRoot()) { + if (queue.equals(ROOT)) { LOG.warn("Accessible node labels for root queue will be ignored," + " it will be automatically set to \"*\"."); } } // always return ANY for queue root - if (queue.isRoot()) { + if (queue.equals(ROOT)) { return ImmutableSet.of(RMNodeLabelsManager.ANY); } @@ -824,12 +838,12 @@ public Set getAccessibleNodeLabels(QueuePath queue) { return Collections.unmodifiableSet(set); } - public void setCapacityVector(QueuePath queuePath, String label, String capacityVector) { + public void setCapacityVector(String queuePath, String label, String capacityVector) { String capacityPropertyName = getNodeLabelPrefix(queuePath, label) + CAPACITY; set(capacityPropertyName, capacityVector); } - public void setMaximumCapacityVector(QueuePath queuePath, String label, String capacityVector) { + public void setMaximumCapacityVector(String queuePath, String label, String capacityVector) { String capacityPropertyName = getNodeLabelPrefix(queuePath, label) + MAXIMUM_CAPACITY; set(capacityPropertyName, capacityVector); } @@ -852,7 +866,7 @@ private float extractFloatValueFromWeightConfig(String configureValue) { private float internalGetLabeledQueueCapacity(QueuePath queue, String label, String suffix, float defaultValue) { - String capacityPropertyName = getNodeLabelPrefix(queue, label) + suffix; + String capacityPropertyName = getNodeLabelPrefix(queue.getFullPath(), label) + suffix; String configuredCapacity = get(capacityPropertyName); boolean absoluteResourceConfigured = (configuredCapacity != null) && RESOURCE_PATTERN.matcher( @@ -877,7 +891,7 @@ private float internalGetLabeledQueueCapacity(QueuePath queue, String label, } if (LOG.isDebugEnabled()) { LOG.debug( - "CSConf - getCapacityOfLabel: prefix=" + getNodeLabelPrefix(queue, + "CSConf - getCapacityOfLabel: prefix=" + getNodeLabelPrefix(queue.getFullPath(), label) + ", capacity=" + capacity); } return capacity; @@ -891,7 +905,7 @@ public float getLabeledQueueMaximumCapacity(QueuePath queue, String label) { return internalGetLabeledQueueCapacity(queue, label, MAXIMUM_CAPACITY, 100f); } - public String getDefaultNodeLabelExpression(QueuePath queue) { + public String getDefaultNodeLabelExpression(String queue) { String defaultLabelExpression = get(getQueuePrefix(queue) + DEFAULT_NODE_LABEL_EXPRESSION); if (defaultLabelExpression == null) { @@ -900,7 +914,7 @@ public String getDefaultNodeLabelExpression(QueuePath queue) { return defaultLabelExpression.trim(); } - public void setDefaultNodeLabelExpression(QueuePath queue, String exp) { + public void setDefaultNodeLabelExpression(String queue, String exp) { set(getQueuePrefix(queue) + DEFAULT_NODE_LABEL_EXPRESSION, exp); } @@ -908,12 +922,12 @@ public float getMaximumAMResourcePercentPerPartition(QueuePath queue, String label) { // If per-partition max-am-resource-percent is not configured, // use default value as max-am-resource-percent for this queue. - return getFloat(getNodeLabelPrefix(queue, label) + return getFloat(getNodeLabelPrefix(queue.getFullPath(), label) + MAXIMUM_AM_RESOURCE_SUFFIX, - getMaximumApplicationMasterResourcePerQueuePercent(queue)); + getMaximumApplicationMasterResourcePerQueuePercent(queue.getFullPath())); } - public void setMaximumAMResourcePercentPerPartition(QueuePath queue, + public void setMaximumAMResourcePercentPerPartition(String queue, String label, float percent) { setFloat(getNodeLabelPrefix(queue, label) + MAXIMUM_AM_RESOURCE_SUFFIX, percent); @@ -939,16 +953,16 @@ private static String getAclKey(QueueACL acl) { return "acl_" + StringUtils.toLowerCase(acl.toString()); } - public AccessControlList getAcl(QueuePath queue, QueueACL acl) { + public AccessControlList getAcl(String queue, QueueACL acl) { String queuePrefix = getQueuePrefix(queue); // The root queue defaults to all access if not defined // Sub queues inherit access if not defined - String defaultAcl = queue.isRoot() ? ALL_ACL : NONE_ACL; + String defaultAcl = queue.equals(ROOT) ? ALL_ACL : NONE_ACL; String aclString = get(queuePrefix + getAclKey(acl), defaultAcl); return new AccessControlList(aclString); } - public void setAcl(QueuePath queue, QueueACL acl, String aclString) { + public void setAcl(String queue, QueueACL acl, String aclString) { String queuePrefix = getQueuePrefix(queue); set(queuePrefix + getAclKey(acl), aclString); } @@ -968,9 +982,9 @@ private static String getAclKey(AccessType acl) { * @return A mapping of the queue ACLs. */ public Map getACLsForLegacyAutoCreatedLeafQueue( - QueuePath parentQueuePath) { + String parentQueuePath) { final String prefix = - getQueuePrefix(getAutoCreatedQueueObjectTemplateConfPrefix( + getQueuePrefix(getAutoCreatedQueueTemplateConfPrefix( parentQueuePath)); Map properties = new HashMap<>(); @@ -1046,7 +1060,7 @@ private static Map getACLsFromProperties( } @Override - public Map getReservationAcls(QueuePath + public Map getReservationAcls(String queue) { Map resAcls = new HashMap<>(); for (ReservationACL acl : ReservationACL.values()) { @@ -1055,7 +1069,7 @@ public Map getReservationAcls(QueuePath return resAcls; } - private AccessControlList getReservationAcl(QueuePath queue, ReservationACL + private AccessControlList getReservationAcl(String queue, ReservationACL acl) { String queuePrefix = getQueuePrefix(queue); // The root queue defaults to all access if not defined @@ -1065,17 +1079,17 @@ private AccessControlList getReservationAcl(QueuePath queue, ReservationACL return new AccessControlList(aclString); } - private void setAcl(QueuePath queue, ReservationACL acl, String aclString) { + private void setAcl(String queue, ReservationACL acl, String aclString) { String queuePrefix = getQueuePrefix(queue); set(queuePrefix + getAclKey(acl), aclString); } - private void setAcl(QueuePath queue, AccessType acl, String aclString) { + private void setAcl(String queue, AccessType acl, String aclString) { String queuePrefix = getQueuePrefix(queue); set(queuePrefix + getAclKey(acl), aclString); } - public Map getAcls(QueuePath queue) { + public Map getAcls(String queue) { Map acls = new HashMap(); for (QueueACL acl : QueueACL.values()) { @@ -1084,14 +1098,14 @@ public Map getAcls(QueuePath queue) { return acls; } - public void setAcls(QueuePath queue, Map acls) { + public void setAcls(String queue, Map acls) { for (Map.Entry e : acls.entrySet()) { setAcl(queue, e.getKey(), e.getValue().getAclString()); } } @VisibleForTesting - public void setReservationAcls(QueuePath queue, + public void setReservationAcls(String queue, Map acls) { for (Map.Entry e : acls.entrySet()) { setAcl(queue, e.getKey(), e.getValue().getAclString()); @@ -1099,7 +1113,7 @@ public void setReservationAcls(QueuePath queue, } @VisibleForTesting - public void setPriorityAcls(QueuePath queue, Priority priority, + public void setPriorityAcls(String queue, Priority priority, Priority defaultPriority, String[] acls) { StringBuilder aclString = new StringBuilder(); @@ -1116,7 +1130,7 @@ public void setPriorityAcls(QueuePath queue, Priority priority, setAcl(queue, AccessType.APPLICATION_MAX_PRIORITY, aclString.toString()); } - public List getPriorityAcls(QueuePath queue, + public List getPriorityAcls(String queue, Priority clusterMaxPriority) { String queuePrefix = getQueuePrefix(queue); String defaultAcl = ALL_ACL; @@ -1127,7 +1141,7 @@ public List getPriorityAcls(QueuePath queue, return priorityACLConfig.getPriorityAcl(clusterMaxPriority, aclString); } - public List getQueues(QueuePath queue) { + public String[] getQueues(String queue) { LOG.debug("CSConf - getQueues called for: queuePrefix={}", getQueuePrefix(queue)); String[] queues = getStrings(getQueuePrefix(queue) + QUEUES); @@ -1136,16 +1150,17 @@ public List getQueues(QueuePath queue) { for (String s : queues) { trimmedQueueNames.add(s.trim()); } + queues = trimmedQueueNames.toArray(new String[0]); } LOG.debug("CSConf - getQueues: queuePrefix={}, queues={}", getQueuePrefix(queue), ((queues == null) ? "" : StringUtils.arrayToString(queues))); - return trimmedQueueNames; + return queues; } - public void setQueues(QueuePath queue, String[] subQueues) { + public void setQueues(String queue, String[] subQueues) { set(getQueuePrefix(queue) + QUEUES, StringUtils.arrayToString(subQueues)); LOG.debug("CSConf - setQueues: qPrefix={}, queues={}", getQueuePrefix(queue), StringUtils.arrayToString(subQueues)); @@ -1162,7 +1177,7 @@ public Resource getMinimumAllocation() { } @Private - public Priority getQueuePriority(QueuePath queue) { + public Priority getQueuePriority(String queue) { String queuePolicyPrefix = getQueuePrefix(queue); Priority pri = Priority.newInstance( getInt(queuePolicyPrefix + "priority", 0)); @@ -1170,7 +1185,7 @@ public Priority getQueuePriority(QueuePath queue) { } @Private - public void setQueuePriority(QueuePath queue, int priority) { + public void setQueuePriority(String queue, int priority) { String queuePolicyPrefix = getQueuePrefix(queue); setInt(queuePolicyPrefix + "priority", priority); } @@ -1183,7 +1198,7 @@ public void setQueuePriority(QueuePath queue, int priority) { * name of the queue * @return Resource object or Resource.none if not set */ - public Resource getQueueMaximumAllocation(QueuePath queue) { + public Resource getQueueMaximumAllocation(String queue) { String queuePrefix = getQueuePrefix(queue); String rawQueueMaxAllocation = get(queuePrefix + MAXIMUM_ALLOCATION, null); if (Strings.isNullOrEmpty(rawQueueMaxAllocation)) { @@ -1194,7 +1209,7 @@ public Resource getQueueMaximumAllocation(QueuePath queue) { } } - public void setQueueMaximumAllocation(QueuePath queue, String maximumAllocation) { + public void setQueueMaximumAllocation(String queue, String maximumAllocation) { String queuePrefix = getQueuePrefix(queue); set(queuePrefix + MAXIMUM_ALLOCATION, maximumAllocation); } @@ -1222,22 +1237,22 @@ public void reinitializeConfigurationProperties() { configurationProperties = new ConfigurationProperties(props); } - public void setQueueMaximumAllocationMb(QueuePath queue, int value) { + public void setQueueMaximumAllocationMb(String queue, int value) { String queuePrefix = getQueuePrefix(queue); setInt(queuePrefix + MAXIMUM_ALLOCATION_MB, value); } - public void setQueueMaximumAllocationVcores(QueuePath queue, int value) { + public void setQueueMaximumAllocationVcores(String queue, int value) { String queuePrefix = getQueuePrefix(queue); setInt(queuePrefix + MAXIMUM_ALLOCATION_VCORES, value); } - public long getQueueMaximumAllocationMb(QueuePath queue) { + public long getQueueMaximumAllocationMb(String queue) { String queuePrefix = getQueuePrefix(queue); return getInt(queuePrefix + MAXIMUM_ALLOCATION_MB, (int)UNDEFINED); } - public int getQueueMaximumAllocationVcores(QueuePath queue) { + public int getQueueMaximumAllocationVcores(String queue) { String queuePrefix = getQueuePrefix(queue); return getInt(queuePrefix + MAXIMUM_ALLOCATION_VCORES, (int)UNDEFINED); } @@ -1580,20 +1595,20 @@ void setWorkflowPriorityMappings( .getWorkflowPriorityMappingStr(workflowPriorityMappings)); } - public boolean isReservable(QueuePath queue) { + public boolean isReservable(String queue) { boolean isReservable = getBoolean(getQueuePrefix(queue) + IS_RESERVABLE, false); return isReservable; } - public void setReservable(QueuePath queue, boolean isReservable) { + public void setReservable(String queue, boolean isReservable) { setBoolean(getQueuePrefix(queue) + IS_RESERVABLE, isReservable); LOG.debug("here setReservableQueue: queuePrefix={}, isReservableQueue={}", getQueuePrefix(queue), isReservable(queue)); } @Override - public long getReservationWindow(QueuePath queue) { + public long getReservationWindow(String queue) { long reservationWindow = getLong(getQueuePrefix(queue) + RESERVATION_WINDOW, DEFAULT_RESERVATION_WINDOW); @@ -1601,7 +1616,7 @@ public long getReservationWindow(QueuePath queue) { } @Override - public float getAverageCapacity(QueuePath queue) { + public float getAverageCapacity(String queue) { float avgCapacity = getFloat(getQueuePrefix(queue) + AVERAGE_CAPACITY, MAXIMUM_CAPACITY_VALUE); @@ -1609,53 +1624,53 @@ public float getAverageCapacity(QueuePath queue) { } @Override - public float getInstantaneousMaxCapacity(QueuePath queue) { + public float getInstantaneousMaxCapacity(String queue) { float instMaxCapacity = getFloat(getQueuePrefix(queue) + INSTANTANEOUS_MAX_CAPACITY, MAXIMUM_CAPACITY_VALUE); return instMaxCapacity; } - public void setInstantaneousMaxCapacity(QueuePath queue, float instMaxCapacity) { + public void setInstantaneousMaxCapacity(String queue, float instMaxCapacity) { setFloat(getQueuePrefix(queue) + INSTANTANEOUS_MAX_CAPACITY, instMaxCapacity); } - public void setReservationWindow(QueuePath queue, long reservationWindow) { + public void setReservationWindow(String queue, long reservationWindow) { setLong(getQueuePrefix(queue) + RESERVATION_WINDOW, reservationWindow); } - public void setAverageCapacity(QueuePath queue, float avgCapacity) { + public void setAverageCapacity(String queue, float avgCapacity) { setFloat(getQueuePrefix(queue) + AVERAGE_CAPACITY, avgCapacity); } @Override - public String getReservationAdmissionPolicy(QueuePath queue) { + public String getReservationAdmissionPolicy(String queue) { String reservationPolicy = get(getQueuePrefix(queue) + RESERVATION_ADMISSION_POLICY, DEFAULT_RESERVATION_ADMISSION_POLICY); return reservationPolicy; } - public void setReservationAdmissionPolicy(QueuePath queue, + public void setReservationAdmissionPolicy(String queue, String reservationPolicy) { set(getQueuePrefix(queue) + RESERVATION_ADMISSION_POLICY, reservationPolicy); } @Override - public String getReservationAgent(QueuePath queue) { + public String getReservationAgent(String queue) { String reservationAgent = get(getQueuePrefix(queue) + RESERVATION_AGENT_NAME, DEFAULT_RESERVATION_AGENT_NAME); return reservationAgent; } - public void setReservationAgent(QueuePath queue, String reservationPolicy) { + public void setReservationAgent(String queue, String reservationPolicy) { set(getQueuePrefix(queue) + RESERVATION_AGENT_NAME, reservationPolicy); } @Override - public boolean getShowReservationAsQueues(QueuePath queuePath) { + public boolean getShowReservationAsQueues(String queuePath) { boolean showReservationAsQueues = getBoolean(getQueuePrefix(queuePath) + RESERVATION_SHOW_RESERVATION_AS_QUEUE, @@ -1664,7 +1679,7 @@ public boolean getShowReservationAsQueues(QueuePath queuePath) { } @Override - public String getReplanner(QueuePath queue) { + public String getReplanner(String queue) { String replanner = get(getQueuePrefix(queue) + RESERVATION_PLANNER_NAME, DEFAULT_RESERVATION_PLANNER_NAME); @@ -1672,7 +1687,7 @@ public String getReplanner(QueuePath queue) { } @Override - public boolean getMoveOnExpiry(QueuePath queue) { + public boolean getMoveOnExpiry(String queue) { boolean killOnExpiry = getBoolean(getQueuePrefix(queue) + RESERVATION_MOVE_ON_EXPIRY, DEFAULT_RESERVATION_MOVE_ON_EXPIRY); @@ -1680,7 +1695,7 @@ public boolean getMoveOnExpiry(QueuePath queue) { } @Override - public long getEnforcementWindow(QueuePath queue) { + public long getEnforcementWindow(String queue) { long enforcementWindow = getLong(getQueuePrefix(queue) + RESERVATION_ENFORCEMENT_WINDOW, DEFAULT_RESERVATION_ENFORCEMENT_WINDOW); @@ -1695,7 +1710,7 @@ public long getEnforcementWindow(QueuePath queue) { * @param queue queue path * @param preemptionDisabled true if preemption is disabled on queue */ - public void setPreemptionDisabled(QueuePath queue, boolean preemptionDisabled) { + public void setPreemptionDisabled(String queue, boolean preemptionDisabled) { setBoolean(getQueuePrefix(queue) + QUEUE_PREEMPTION_DISABLED, preemptionDisabled); } @@ -1708,7 +1723,7 @@ public void setPreemptionDisabled(QueuePath queue, boolean preemptionDisabled) { * is not set in the configuration * @return true if preemption is disabled on queue, false otherwise */ - public boolean getPreemptionDisabled(QueuePath queue, boolean defaultVal) { + public boolean getPreemptionDisabled(String queue, boolean defaultVal) { boolean preemptionDisabled = getBoolean(getQueuePrefix(queue) + QUEUE_PREEMPTION_DISABLED, defaultVal); @@ -1723,7 +1738,7 @@ public boolean getPreemptionDisabled(QueuePath queue, boolean defaultVal) { * configuration * @return true if preemption is disabled on queue, false otherwise */ - public boolean getIntraQueuePreemptionDisabled(QueuePath queue, + public boolean getIntraQueuePreemptionDisabled(String queue, boolean defaultVal) { return getBoolean(getQueuePrefix(queue) + INTRA_QUEUE_PREEMPTION_CONFIG_PREFIX @@ -1744,7 +1759,7 @@ public boolean getPreemptionObserveOnly() { * @param queuePath queue path. * @return configured node labels. */ - public Set getConfiguredNodeLabels(QueuePath queuePath) { + public Set getConfiguredNodeLabels(String queuePath) { Set configuredNodeLabels = new HashSet(); Entry e = null; @@ -1815,7 +1830,7 @@ public Priority getClusterLevelApplicationMaxPriority() { YarnConfiguration.DEFAULT_CLUSTER_LEVEL_APPLICATION_PRIORITY)); } - public Integer getDefaultApplicationPriorityConfPerQueue(QueuePath queue) { + public Integer getDefaultApplicationPriorityConfPerQueue(String queue) { Integer defaultPriority = getInt(getQueuePrefix(queue) + DEFAULT_APPLICATION_PRIORITY, DEFAULT_CONFIGURATION_APPLICATION_PRIORITY); @@ -1823,12 +1838,12 @@ public Integer getDefaultApplicationPriorityConfPerQueue(QueuePath queue) { } @VisibleForTesting - public void setOrderingPolicy(QueuePath queue, String policy) { + public void setOrderingPolicy(String queue, String policy) { set(getQueuePrefix(queue) + ORDERING_POLICY, policy); } @VisibleForTesting - public void setOrderingPolicyParameter(QueuePath queue, + public void setOrderingPolicyParameter(String queue, String parameterKey, String parameterValue) { set(getQueuePrefix(queue) + ORDERING_POLICY + "." + parameterKey, parameterValue); @@ -1873,11 +1888,11 @@ public Integer getMaxParallelAppsForUser(String user) { : getDefaultMaxParallelAppsPerUser(); } - public void setMaxParallelAppsForQueue(QueuePath queue, String value) { + public void setMaxParallelAppsForQueue(String queue, String value) { set(getQueuePrefix(queue) + MAX_PARALLEL_APPLICATIONS, value); } - public Integer getMaxParallelAppsForQueue(QueuePath queue) { + public Integer getMaxParallelAppsForQueue(String queue) { String maxParallelAppsForQueue = get(getQueuePrefix(queue) + MAX_PARALLEL_APPLICATIONS); @@ -1886,12 +1901,12 @@ public Integer getMaxParallelAppsForQueue(QueuePath queue) { : getDefaultMaxParallelApps(); } - public boolean getAllowZeroCapacitySum(QueuePath queue) { + public boolean getAllowZeroCapacitySum(String queue) { return getBoolean(getQueuePrefix(queue) + ALLOW_ZERO_CAPACITY_SUM, DEFAULT_ALLOW_ZERO_CAPACITY_SUM); } - public void setAllowZeroCapacitySum(QueuePath queue, boolean value) { + public void setAllowZeroCapacitySum(String queue, boolean value) { setBoolean(getQueuePrefix(queue) + ALLOW_ZERO_CAPACITY_SUM, value); } @@ -2089,12 +2104,12 @@ public void setGlobalMaximumApplicationsPerQueue(int val) { @Private - public void setQueueOrderingPolicy(QueuePath queue, String policy) { + public void setQueueOrderingPolicy(String queue, String policy) { set(getQueuePrefix(queue) + ORDERING_POLICY, policy); } @Private - public QueueOrderingPolicy getQueueOrderingPolicy(QueuePath queue, + public QueueOrderingPolicy getQueueOrderingPolicy(String queue, String parentPolicy) { String defaultPolicy = parentPolicy; if (null == defaultPolicy) { @@ -2116,7 +2131,7 @@ public QueueOrderingPolicy getQueueOrderingPolicy(QueuePath queue, qop = (QueueOrderingPolicy) Class.forName(policyType).newInstance(); } catch (Exception e) { String message = "Unable to construct queue ordering policy=" - + policyType + " queue=" + queue.getFullPath(); + + policyType + " queue=" + queue; throw new YarnRuntimeException(message, e); } } @@ -2235,7 +2250,7 @@ public void setPUOrderingPolicyUnderUtilizedPreemptionMoveReservation( * @param queuePath full queue path * @return map of user weights, if they exist. Otherwise, return empty map. */ - public UserWeights getAllUserWeightsForQueue(QueuePath queuePath) { + public UserWeights getAllUserWeightsForQueue(String queuePath) { return UserWeights.createByConfig(this, getConfigurationProperties(), queuePath); } @@ -2253,23 +2268,23 @@ public int getMaxAssignPerHeartbeat() { public static final String DEFAULT_LIFETIME_SUFFIX = "default-application-lifetime"; - public long getMaximumLifetimePerQueue(QueuePath queue) { + public long getMaximumLifetimePerQueue(String queue) { long maximumLifetimePerQueue = getLong( getQueuePrefix(queue) + MAXIMUM_LIFETIME_SUFFIX, (long) UNDEFINED); return maximumLifetimePerQueue; } - public void setMaximumLifetimePerQueue(QueuePath queue, long maximumLifetime) { + public void setMaximumLifetimePerQueue(String queue, long maximumLifetime) { setLong(getQueuePrefix(queue) + MAXIMUM_LIFETIME_SUFFIX, maximumLifetime); } - public long getDefaultLifetimePerQueue(QueuePath queue) { + public long getDefaultLifetimePerQueue(String queue) { long maximumLifetimePerQueue = getLong( getQueuePrefix(queue) + DEFAULT_LIFETIME_SUFFIX, (long) UNDEFINED); return maximumLifetimePerQueue; } - public void setDefaultLifetimePerQueue(QueuePath queue, long defaultLifetime) { + public void setDefaultLifetimePerQueue(String queue, long defaultLifetime) { setLong(getQueuePrefix(queue) + DEFAULT_LIFETIME_SUFFIX, defaultLifetime); } @@ -2330,7 +2345,7 @@ public void setDefaultLifetimePerQueue(QueuePath queue, long defaultLifetime) { * is false */ @Private - public boolean isAutoCreateChildQueueEnabled(QueuePath queuePath) { + public boolean isAutoCreateChildQueueEnabled(String queuePath) { boolean isAutoCreateEnabled = getBoolean( getQueuePrefix(queuePath) + AUTO_CREATE_CHILD_QUEUE_ENABLED, DEFAULT_AUTO_CREATE_CHILD_QUEUE_ENABLED); @@ -2339,27 +2354,44 @@ public boolean isAutoCreateChildQueueEnabled(QueuePath queuePath) { @Private @VisibleForTesting - public void setAutoCreateChildQueueEnabled(QueuePath queuePath, + public void setAutoCreateChildQueueEnabled(String queuePath, boolean autoCreationEnabled) { setBoolean(getQueuePrefix(queuePath) + AUTO_CREATE_CHILD_QUEUE_ENABLED, autoCreationEnabled); } - public void setAutoQueueCreationV2Enabled(QueuePath queuePath, + public void setAutoQueueCreationV2Enabled(String queuePath, boolean autoQueueCreation) { setBoolean( getQueuePrefix(queuePath) + AUTO_QUEUE_CREATION_V2_ENABLED, autoQueueCreation); } - public boolean isAutoQueueCreationV2Enabled(QueuePath queuePath) { + public boolean isAutoQueueCreationV2Enabled(String queuePath) { boolean isAutoQueueCreation = getBoolean( getQueuePrefix(queuePath) + AUTO_QUEUE_CREATION_V2_ENABLED, DEFAULT_AUTO_QUEUE_CREATION_ENABLED); return isAutoQueueCreation; } + /** + * Get the auto created leaf queue's template configuration prefix + * Leaf queue's template capacities are configured at the parent queue + * + * @param queuePath parent queue's path + * @return Config prefix for leaf queue template configurations + */ + @Private + public String getAutoCreatedQueueTemplateConfPrefix(String queuePath) { + return queuePath + DOT + AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX; + } + + @Private + public QueuePath getAutoCreatedQueueObjectTemplateConfPrefix(String queuePath) { + return new QueuePath(queuePath, AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX); + } + @Private public static final String FAIL_AUTO_CREATION_ON_EXCEEDING_CAPACITY = "auto-create-child-queue.fail-on-exceeding-parent-capacity"; @@ -2377,7 +2409,7 @@ public boolean isAutoQueueCreationV2Enabled(QueuePath queuePath) { */ @Private public boolean getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded( - QueuePath queuePath) { + String queuePath) { boolean shouldFailAutoQueueCreationOnExceedingGuaranteedCapacity = getBoolean(getQueuePrefix(queuePath) + FAIL_AUTO_CREATION_ON_EXCEEDING_CAPACITY, @@ -2388,7 +2420,7 @@ public boolean getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded( @VisibleForTesting @Private public void setShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded( - QueuePath queuePath, boolean autoCreationEnabled) { + String queuePath, boolean autoCreationEnabled) { setBoolean( getQueuePrefix(queuePath) + FAIL_AUTO_CREATION_ON_EXCEEDING_CAPACITY, @@ -2403,7 +2435,7 @@ public void setShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded( * @return the max number of leaf queues allowed to be auto created */ @Private - public int getAutoCreatedQueuesMaxChildQueuesLimit(QueuePath queuePath) { + public int getAutoCreatedQueuesMaxChildQueuesLimit(String queuePath) { return getInt(getQueuePrefix(queuePath) + AUTO_CREATE_QUEUE_MAX_QUEUES, DEFAULT_AUTO_CREATE_QUEUE_MAX_QUEUES); @@ -2418,14 +2450,14 @@ public int getAutoCreatedQueuesMaxChildQueuesLimit(QueuePath queuePath) { * in new auto created. */ @Private - public int getAutoCreatedQueuesV2MaxChildQueuesLimit(QueuePath queuePath) { + public int getAutoCreatedQueuesV2MaxChildQueuesLimit(String queuePath) { return getInt(getQueuePrefix(queuePath) + AUTO_QUEUE_CREATION_V2_MAX_QUEUES, DEFAULT_AUTO_QUEUE_CREATION_V2_MAX_QUEUES); } @VisibleForTesting - public void setAutoCreatedQueuesV2MaxChildQueuesLimit(QueuePath queuePath, + public void setAutoCreatedQueuesV2MaxChildQueuesLimit(String queuePath, int maxQueues) { setInt(getQueuePrefix(queuePath) + AUTO_QUEUE_CREATION_V2_MAX_QUEUES, maxQueues); @@ -2482,7 +2514,7 @@ public void setAutoCreatedQueuesV2MaxChildQueuesLimit(QueuePath queuePath, * is true. */ @Private - public boolean isAutoExpiredDeletionEnabled(QueuePath queuePath) { + public boolean isAutoExpiredDeletionEnabled(String queuePath) { boolean isAutoExpiredDeletionEnabled = getBoolean( getQueuePrefix(queuePath) + AUTO_CREATE_CHILD_QUEUE_AUTO_REMOVAL_ENABLE, @@ -2492,7 +2524,7 @@ public boolean isAutoExpiredDeletionEnabled(QueuePath queuePath) { @Private @VisibleForTesting - public void setAutoExpiredDeletionEnabled(QueuePath queuePath, + public void setAutoExpiredDeletionEnabled(String queuePath, boolean autoRemovalEnable) { setBoolean(getQueuePrefix(queuePath) + AUTO_CREATE_CHILD_QUEUE_AUTO_REMOVAL_ENABLE, @@ -2530,7 +2562,7 @@ public long getAutoExpiredDeletionTime() { * @return Configured policy class name */ @Private - public String getAutoCreatedQueueManagementPolicy(QueuePath queue) { + public String getAutoCreatedQueueManagementPolicy(String queue) { String autoCreatedQueueManagementPolicy = get(getQueuePrefix(queue) + AUTO_CREATED_QUEUE_MANAGEMENT_POLICY, DEFAULT_AUTO_CREATED_QUEUE_MANAGEMENT_POLICY); @@ -2541,19 +2573,19 @@ public String getAutoCreatedQueueManagementPolicy(QueuePath queue) { * Get The policy class configured to manage capacities for auto created leaf * queues under the specified parent * - * @param queue The parent queue's path + * @param queueName The parent queue's name * @return The policy class configured to manage capacities for auto created * leaf queues under the specified parent queue */ @Private protected AutoCreatedQueueManagementPolicy getAutoCreatedQueueManagementPolicyClass( - QueuePath queue) { + String queueName) { String queueManagementPolicyClassName = - getAutoCreatedQueueManagementPolicy(queue); + getAutoCreatedQueueManagementPolicy(queueName); LOG.info("Using Auto Created Queue Management Policy: " - + queueManagementPolicyClassName + " for queue: " + queue.getFullPath()); + + queueManagementPolicyClassName + " for queue: " + queueName); try { Class queueManagementPolicyClazz = getClassByName( queueManagementPolicyClassName); @@ -2569,36 +2601,36 @@ public String getAutoCreatedQueueManagementPolicy(QueuePath queue) { } catch (ClassNotFoundException e) { throw new YarnRuntimeException( "Could not instantiate " + "AutoCreatedQueueManagementPolicy: " - + queueManagementPolicyClassName + " for queue: " + queue.getFullPath(), + + queueManagementPolicyClassName + " for queue: " + queueName, e); } } @VisibleForTesting @Private - public void setAutoCreatedLeafQueueConfigCapacity(QueuePath queuePath, + public void setAutoCreatedLeafQueueConfigCapacity(String queuePath, float val) { - QueuePath leafQueueConfPrefix = getAutoCreatedQueueObjectTemplateConfPrefix( + String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix( queuePath); setCapacity(leafQueueConfPrefix, val); } @VisibleForTesting @Private - public void setAutoCreatedLeafQueueTemplateCapacityByLabel(QueuePath queuePath, + public void setAutoCreatedLeafQueueTemplateCapacityByLabel(String queuePath, String label, float val) { - QueuePath leafQueueConfPrefix = - getAutoCreatedQueueObjectTemplateConfPrefix(queuePath); + String leafQueueConfPrefix = + getAutoCreatedQueueTemplateConfPrefix(queuePath); setCapacityByLabel(leafQueueConfPrefix, label, val); } @VisibleForTesting @Private - public void setAutoCreatedLeafQueueTemplateCapacityByLabel(QueuePath queuePath, + public void setAutoCreatedLeafQueueTemplateCapacityByLabel(String queuePath, String label, Resource resource) { - QueuePath leafQueueConfPrefix = - getAutoCreatedQueueObjectTemplateConfPrefix(queuePath); + String leafQueueConfPrefix = + getAutoCreatedQueueTemplateConfPrefix(queuePath); StringBuilder resourceString = new StringBuilder(); @@ -2615,27 +2647,27 @@ public void setAutoCreatedLeafQueueTemplateCapacityByLabel(QueuePath queuePath, @Private @VisibleForTesting - public void setAutoCreatedLeafQueueConfigMaxCapacity(QueuePath queuePath, + public void setAutoCreatedLeafQueueConfigMaxCapacity(String queuePath, float val) { - QueuePath leafQueueConfPrefix = getAutoCreatedQueueObjectTemplateConfPrefix( + String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix( queuePath); setMaximumCapacity(leafQueueConfPrefix, val); } @Private @VisibleForTesting - public void setAutoCreatedLeafQueueTemplateMaxCapacity(QueuePath queuePath, + public void setAutoCreatedLeafQueueTemplateMaxCapacity(String queuePath, String label, float val) { - QueuePath leafQueueConfPrefix = getAutoCreatedQueueObjectTemplateConfPrefix( + String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix( queuePath); setMaximumCapacityByLabel(leafQueueConfPrefix, label, val); } @Private @VisibleForTesting - public void setAutoCreatedLeafQueueTemplateMaxCapacity(QueuePath queuePath, + public void setAutoCreatedLeafQueueTemplateMaxCapacity(String queuePath, String label, Resource resource) { - QueuePath leafQueueConfPrefix = getAutoCreatedQueueObjectTemplateConfPrefix( + String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix( queuePath); StringBuilder resourceString = new StringBuilder(); @@ -2653,37 +2685,37 @@ public void setAutoCreatedLeafQueueTemplateMaxCapacity(QueuePath queuePath, @VisibleForTesting @Private - public void setAutoCreatedLeafQueueConfigUserLimit(QueuePath queuePath, + public void setAutoCreatedLeafQueueConfigUserLimit(String queuePath, int val) { - QueuePath leafQueueConfPrefix = getAutoCreatedQueueObjectTemplateConfPrefix( + String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix( queuePath); setUserLimit(leafQueueConfPrefix, val); } @VisibleForTesting @Private - public void setAutoCreatedLeafQueueConfigUserLimitFactor(QueuePath queuePath, + public void setAutoCreatedLeafQueueConfigUserLimitFactor(String queuePath, float val) { - QueuePath leafQueueConfPrefix = getAutoCreatedQueueObjectTemplateConfPrefix( + String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix( queuePath); setUserLimitFactor(leafQueueConfPrefix, val); } @Private @VisibleForTesting - public void setAutoCreatedLeafQueueConfigDefaultNodeLabelExpression(QueuePath + public void setAutoCreatedLeafQueueConfigDefaultNodeLabelExpression(String queuePath, String expression) { - QueuePath leafQueueConfPrefix = getAutoCreatedQueueObjectTemplateConfPrefix( + String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix( queuePath); setDefaultNodeLabelExpression(leafQueueConfPrefix, expression); } @Private @VisibleForTesting - public void setAutoCreatedLeafQueueConfigMaximumAllocation(QueuePath + public void setAutoCreatedLeafQueueConfigMaximumAllocation(String queuePath, String expression) { - QueuePath leafQueueConfPrefix = getAutoCreatedQueueObjectTemplateConfPrefix( + String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix( queuePath); setQueueMaximumAllocation(leafQueueConfPrefix, expression); } @@ -2712,7 +2744,7 @@ public static String getUnits(String resourceValue) { * Resource types * @return ResourceInformation */ - public Resource getMinimumResourceRequirement(String label, QueuePath queue, + public Resource getMinimumResourceRequirement(String label, String queue, Set resourceTypes) { return internalGetLabeledResourceRequirementForQueue(queue, label, resourceTypes, CAPACITY); @@ -2729,7 +2761,7 @@ public Resource getMinimumResourceRequirement(String label, QueuePath queue, * Resource types * @return Resource */ - public Resource getMaximumResourceRequirement(String label, QueuePath queue, + public Resource getMaximumResourceRequirement(String label, String queue, Set resourceTypes) { return internalGetLabeledResourceRequirementForQueue(queue, label, resourceTypes, MAXIMUM_CAPACITY); @@ -2748,10 +2780,10 @@ public void setMaximumResourceRequirement(String label, QueuePath queue, } public Map parseConfiguredResourceVector( - QueuePath queuePath, Set labels) { + String queuePath, Set labels) { Map queueResourceVectors = new HashMap<>(); for (String label : labels) { - String propertyName = QueuePrefixes.getNodeLabelPrefix( + String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( queuePath, label) + CapacitySchedulerConfiguration.CAPACITY; String capacityString = get(propertyName); queueResourceVectors.put(label, queueCapacityConfigParser.parse(capacityString, queuePath)); @@ -2761,10 +2793,10 @@ public Map parseConfiguredResourceVector( } public Map parseConfiguredMaximumCapacityVector( - QueuePath queuePath, Set labels, QueueCapacityVector defaultVector) { + String queuePath, Set labels, QueueCapacityVector defaultVector) { Map queueResourceVectors = new HashMap<>(); for (String label : labels) { - String propertyName = QueuePrefixes.getNodeLabelPrefix( + String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( queuePath, label) + CapacitySchedulerConfiguration.MAXIMUM_CAPACITY; String capacityString = get(propertyName); QueueCapacityVector capacityVector = queueCapacityConfigParser.parse(capacityString, @@ -2795,15 +2827,15 @@ private void updateMinMaxResourceToConf(String label, QueuePath queue, + ResourceUtils. getCustomResourcesStrings(resource) + "]"); - String prefix = getQueuePrefix(queue) + type; + String prefix = getQueuePrefix(queue.getFullPath()) + type; if (!label.isEmpty()) { - prefix = getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS + DOT + label + prefix = getQueuePrefix(queue.getFullPath()) + ACCESSIBLE_NODE_LABELS + DOT + label + DOT + type; } set(prefix, resourceString.toString()); } - public boolean checkConfigTypeIsAbsoluteResource(String label, QueuePath queue, + public boolean checkConfigTypeIsAbsoluteResource(String label, String queue, Set resourceTypes) { String propertyName = getNodeLabelPrefix(queue, label) + CAPACITY; String resourceString = get(propertyName); @@ -2818,7 +2850,7 @@ public boolean checkConfigTypeIsAbsoluteResource(String label, QueuePath queue, return false; } - private Resource internalGetLabeledResourceRequirementForQueue(QueuePath queue, + private Resource internalGetLabeledResourceRequirementForQueue(String queue, String label, Set resourceTypes, String suffix) { String propertyName = getNodeLabelPrefix(queue, label) + suffix; String resourceString = get(propertyName); @@ -2937,7 +2969,7 @@ private void updateResourceValuesFromConfig(Set resourceTypes, public static final boolean DEFAULT_MULTI_NODE_PLACEMENT_ENABLED = false; public String getMultiNodesSortingAlgorithmPolicy( - QueuePath queue) { + String queue) { String policyName = get( getQueuePrefix(queue) + "multi-node-sorting.policy"); 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 3d7f31d897481..9e4dc55708d55 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 @@ -20,6 +20,8 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -237,47 +239,46 @@ static CSQueue parseQueue( CSQueue parent, String queueName, CSQueueStore newQueues, CSQueueStore oldQueues, QueueHook hook) throws IOException { CSQueue queue; - QueuePath fullQueuePath = (parent == null) ? new QueuePath(queueName) : - (QueuePath.createFromQueues(parent.getQueuePath(), queueName)); - List childQueueNames = conf.getQueues(fullQueuePath); - CSQueue oldQueue = oldQueues.get(fullQueuePath.getFullPath()); - - boolean isReservableQueue = conf.isReservable(fullQueuePath); - boolean isAutoCreateEnabled = conf.isAutoCreateChildQueueEnabled(fullQueuePath); + String fullQueueName = (parent == null) ? queueName : + (QueuePath.createFromQueues(parent.getQueuePath(), queueName).getFullPath()); + String[] staticChildQueueNames = conf.getQueues(fullQueueName); + List childQueueNames = staticChildQueueNames != null ? + Arrays.asList(staticChildQueueNames) : Collections.emptyList(); + CSQueue oldQueue = oldQueues.get(fullQueueName); + + boolean isReservableQueue = conf.isReservable(fullQueueName); + boolean isAutoCreateEnabled = conf.isAutoCreateChildQueueEnabled(fullQueueName); // if a queue is eligible for auto queue creation v2 it must be a ParentQueue // (even if it is empty) final boolean isDynamicParent = oldQueue instanceof AbstractParentQueue && oldQueue.isDynamicQueue(); boolean isAutoQueueCreationEnabledParent = isDynamicParent || conf.isAutoQueueCreationV2Enabled( - fullQueuePath) || isAutoCreateEnabled; + fullQueueName) || isAutoCreateEnabled; if (childQueueNames.size() == 0 && !isAutoQueueCreationEnabledParent) { validateParent(parent, queueName); // Check if the queue will be dynamically managed by the Reservation system if (isReservableQueue) { - queue = new PlanQueue(queueContext, queueName, parent, - oldQueues.get(fullQueuePath.getFullPath())); + queue = new PlanQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); ReservationQueue defaultResQueue = ((PlanQueue) queue).initializeDefaultInternalQueue(); newQueues.add(defaultResQueue); } else { - queue = new LeafQueue(queueContext, queueName, parent, - oldQueues.get(fullQueuePath.getFullPath())); + queue = new LeafQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); } queue = hook.hook(queue); } else { if (isReservableQueue) { - throw new IllegalStateException("Only Leaf Queues can be reservable for " + - fullQueuePath.getFullPath()); + throw new IllegalStateException("Only Leaf Queues can be reservable for " + fullQueueName); } AbstractParentQueue parentQueue; if (isAutoCreateEnabled) { parentQueue = new ManagedParentQueue(queueContext, queueName, parent, oldQueues.get( - fullQueuePath.getFullPath())); + fullQueueName)); } else { parentQueue = new ParentQueue(queueContext, queueName, parent, oldQueues.get( - fullQueuePath.getFullPath())); + fullQueueName)); } queue = hook.hook(parentQueue); @@ -296,7 +297,7 @@ static CSQueue parseQueue( newQueues.add(queue); - LOG.info("Initialized queue: " + fullQueuePath.getFullPath()); + LOG.info("Initialized queue: " + fullQueueName); return queue; } @@ -322,7 +323,7 @@ private void updateQueues(CSQueueStore existingQueues, boolean isRemovable = isDanglingDynamicQueue || !isDynamicQueue(queue) && newQueues.get(queue.getQueuePath()) == null && !(queue instanceof AutoCreatedLeafQueue && - conf.isAutoCreateChildQueueEnabled(queue.getParent().getQueuePathObject())); + conf.isAutoCreateChildQueueEnabled(queue.getParent().getQueuePath())); if (isRemovable) { existingQueues.remove(queue); @@ -512,7 +513,7 @@ public AbstractLeafQueue createQueue(QueuePath queue) CSQueue parentQueue = getQueue(parentQueueName); if (parentQueue != null && csContext.getConfiguration() - .isAutoCreateChildQueueEnabled(parentQueue.getQueuePathObject())) { + .isAutoCreateChildQueueEnabled(parentQueue.getQueuePath())) { return createLegacyAutoQueue(queue); } else { return createAutoQueue(queue); @@ -583,7 +584,7 @@ public List determineMissingParents( } int maximumDepthOfStaticParent = csContext.getConfiguration().getMaximumAutoCreatedQueueDepth( - firstExistingStaticParent.getQueuePathObject()); + firstExistingStaticParent.getQueuePath()); if (firstStaticParentDistance > maximumDepthOfStaticParent) { throw new SchedulerDynamicEditException( "Could not auto create queue " + queue.getFullPath() @@ -633,7 +634,7 @@ public List getPermissionsForDynamicQueue( if (parentQueue instanceof AbstractManagedParentQueue) { // An AbstractManagedParentQueue must have been found for Legacy AQC permissions.add(new Permission(privilegedEntity, - csConf.getACLsForLegacyAutoCreatedLeafQueue(queuePath.getParentObject()))); + csConf.getACLsForLegacyAutoCreatedLeafQueue(queuePath.getParent()))); } else { // Every other case must be a Flexible Leaf Queue permissions.add(new Permission(privilegedEntity, 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 fa749f077964e..66dd316be7447 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 @@ -39,6 +39,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.CAPACITY; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.MAXIMUM_CAPACITY; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueueCapacityConfigParser; @@ -65,7 +66,7 @@ public ManagedParentQueue(final CapacitySchedulerQueueContext queueContext, shouldFailAutoCreationWhenGuaranteedCapacityExceeded = queueContext.getConfiguration() .getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded( - getQueuePathObject()); + getQueuePath()); leafQueueTemplate = initializeLeafQueueConfigs().build(); @@ -83,7 +84,7 @@ public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) shouldFailAutoCreationWhenGuaranteedCapacityExceeded = queueContext.getConfiguration() .getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded( - getQueuePathObject()); + getQueuePath()); //validate if capacity is exceeded for child queues if (shouldFailAutoCreationWhenGuaranteedCapacityExceeded) { @@ -133,7 +134,7 @@ public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) private void initializeQueueManagementPolicy() throws IOException { queueManagementPolicy = queueContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass( - getQueuePathObject()); + getQueuePath()); queueManagementPolicy.init(this); } @@ -141,7 +142,7 @@ private void initializeQueueManagementPolicy() throws IOException { private void reinitializeQueueManagementPolicy() throws IOException { AutoCreatedQueueManagementPolicy managementPolicy = queueContext.getConfiguration().getAutoCreatedQueueManagementPolicyClass( - getQueuePathObject()); + getQueuePath()); if (!(managementPolicy.getClass().equals( this.queueManagementPolicy.getClass()))) { @@ -161,7 +162,8 @@ protected AutoCreatedLeafQueueConfig.Builder initializeLeafQueueConfigs() throws queueContext.getConfiguration(); // TODO load configs into CapacitySchedulerConfiguration instead of duplicating them - String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(); + String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix( + configuration); //Load template configuration into CapacitySchedulerConfiguration CapacitySchedulerConfiguration autoCreatedTemplateConfig = super.initializeLeafQueueConfigs(leafQueueTemplateConfPrefix); @@ -169,8 +171,8 @@ protected AutoCreatedLeafQueueConfig.Builder initializeLeafQueueConfigs() throws QueueResourceQuotas queueResourceQuotas = new QueueResourceQuotas(); setAbsoluteResourceTemplates(configuration, queueResourceQuotas); - QueuePath templateQueuePath = QueuePrefixes - .getAutoCreatedQueueObjectTemplateConfPrefix(getQueuePathObject()); + QueuePath templateQueuePath = configuration + .getAutoCreatedQueueObjectTemplateConfPrefix(getQueuePath()); Set templateConfiguredNodeLabels = queueContext .getQueueManager().getConfiguredNodeLabelsForAllQueues() .getLabelsByQueue(templateQueuePath.getFullPath()); @@ -197,15 +199,15 @@ protected AutoCreatedLeafQueueConfig.Builder initializeLeafQueueConfigs() throws private void setAbsoluteResourceTemplates(CapacitySchedulerConfiguration configuration, QueueResourceQuotas queueResourceQuotas) throws IOException { - QueuePath templateQueuePath = QueuePrefixes - .getAutoCreatedQueueObjectTemplateConfPrefix(getQueuePathObject()); + QueuePath templateQueuePath = configuration + .getAutoCreatedQueueObjectTemplateConfPrefix(getQueuePath()); Set templateConfiguredNodeLabels = queueContext .getQueueManager().getConfiguredNodeLabelsForAllQueues() .getLabelsByQueue(templateQueuePath.getFullPath()); for (String nodeLabel : templateConfiguredNodeLabels) { Resource templateMinResource = configuration.getMinimumResourceRequirement( - nodeLabel, templateQueuePath, resourceTypes); + nodeLabel, templateQueuePath.getFullPath(), resourceTypes); queueResourceQuotas.setConfiguredMinResource(nodeLabel, templateMinResource); if (this.capacityConfigType.equals(CapacityConfigType.PERCENTAGE) @@ -226,15 +228,15 @@ private void updateQueueCapacities(QueueCapacities queueCapacities) { queueContext.getClusterResource(), configuration.getMinimumResourceRequirement( label, - QueuePrefixes - .getAutoCreatedQueueObjectTemplateConfPrefix(getQueuePathObject()), + configuration + .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()), resourceTypes), getQueueResourceQuotas().getConfiguredMinResource(label))); Resource childMaxResource = configuration .getMaximumResourceRequirement(label, - QueuePrefixes - .getAutoCreatedQueueObjectTemplateConfPrefix(getQueuePathObject()), + configuration + .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()), resourceTypes); Resource parentMaxRes = getQueueResourceQuotas() .getConfiguredMaxResource(label); @@ -294,7 +296,7 @@ public void addChildQueue(CSQueue childQueue) String leafQueuePath = childQueue.getQueuePath(); int maxQueues = conf.getAutoCreatedQueuesMaxChildQueuesLimit( - parentQueue.getQueuePathObject()); + parentQueue.getQueuePath()); if (parentQueue.getChildQueues().size() >= maxQueues) { throw new SchedulerDynamicEditException( @@ -353,15 +355,14 @@ private void setLeafQueuesCapacityVector(AutoCreatedLeafQueue leafQueue) { .getLabelsByQueue(queuePath.getFullPath()); for (String label : templateConfiguredNodeLabels) { final String leafConfigPath = - QueuePrefixes.getNodeLabelPrefix( - QueuePrefixes.getAutoCreatedQueueObjectTemplateConfPrefix(getQueuePathObject()), - label); + CapacitySchedulerConfiguration.getNodeLabelPrefix( + getQueuePath() + DOT + AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX, label); String capacityString = leafConfig.get(leafConfigPath + CAPACITY, "0"); leafQueue.setConfiguredMinCapacityVector(label, - getQueueCapacityConfigParser().parse(capacityString, leafQueue.getQueuePathObject())); + getQueueCapacityConfigParser().parse(capacityString, leafQueue.getQueuePath())); String maxCapacityString = leafConfig.get(leafConfigPath + MAXIMUM_CAPACITY, "100"); leafQueue.setConfiguredMaxCapacityVector(label, - getQueueCapacityConfigParser().parse(maxCapacityString, leafQueue.getQueuePathObject())); + getQueueCapacityConfigParser().parse(maxCapacityString, leafQueue.getQueuePath())); } } @@ -404,9 +405,9 @@ public List getAllApplications() { } } - public String getLeafQueueConfigPrefix() { - return CapacitySchedulerConfiguration.PREFIX + QueuePrefixes - .getAutoCreatedQueueTemplateConfPrefix(getQueuePathObject()); + public String getLeafQueueConfigPrefix(CapacitySchedulerConfiguration conf) { + return CapacitySchedulerConfiguration.PREFIX + conf + .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()); } public boolean shouldFailAutoCreationWhenGuaranteedCapacityExceeded() { 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 aac901d6d0912..0b9f36779cfbc 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 @@ -75,7 +75,7 @@ private CSQueue addDynamicChildQueue(String childQueuePath, boolean isLeaf) // Check if the max queue limit is exceeded. int maxQueues = queueContext.getConfiguration(). - getAutoCreatedQueuesV2MaxChildQueuesLimit(getQueuePathObject()); + getAutoCreatedQueuesV2MaxChildQueuesLimit(getQueuePath()); if (childQueues.size() >= maxQueues) { throw new SchedulerDynamicEditException( "Cannot auto create queue " + childQueuePath + ". Max Child " 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 847c90e54589a..903539fcf1850 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 @@ -52,7 +52,7 @@ public PlanQueue(CapacitySchedulerQueueContext queueContext, String queueName, // Set the reservation queue attributes for the Plan CapacitySchedulerConfiguration conf = queueContext.getConfiguration(); - QueuePath queuePath = super.getQueuePathObject(); + String queuePath = super.getQueuePath(); int maxAppsForReservation = conf.getMaximumApplicationsPerQueue(queuePath); showReservationsAsQueues = conf.getShowReservationAsQueues(queuePath); if (maxAppsForReservation < 0) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/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 7028423e8c227..101c8076fdc04 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 @@ -36,7 +36,7 @@ public QueueAllocationSettings(Resource minimumAllocation) { this.minimumAllocation = minimumAllocation; } - void setupMaximumAllocation(CapacitySchedulerConfiguration configuration, QueuePath queuePath, + void setupMaximumAllocation(CapacitySchedulerConfiguration configuration, String queuePath, CSQueue parent) { Resource clusterMax = ResourceUtils .fetchMaximumAllocationFromConfig(configuration); 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/QueueAppLifetimeAndLimitSettings.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/QueueAppLifetimeAndLimitSettings.java index 9218c877e71b8..2e71d8eef3edb 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/QueueAppLifetimeAndLimitSettings.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/QueueAppLifetimeAndLimitSettings.java @@ -37,7 +37,7 @@ public class QueueAppLifetimeAndLimitSettings { public QueueAppLifetimeAndLimitSettings(CapacitySchedulerConfiguration configuration, AbstractCSQueue q, QueuePath queuePath) { // Store max parallel apps property - this.maxParallelApps = configuration.getMaxParallelAppsForQueue(queuePath); + this.maxParallelApps = configuration.getMaxParallelAppsForQueue(queuePath.getFullPath()); this.maxApplicationLifetime = getInheritedMaxAppLifetime(q, configuration); this.defaultApplicationLifetime = setupInheritedDefaultAppLifetime(q, queuePath, configuration, maxApplicationLifetime); @@ -45,7 +45,7 @@ public QueueAppLifetimeAndLimitSettings(CapacitySchedulerConfiguration configura private long getInheritedMaxAppLifetime(CSQueue q, CapacitySchedulerConfiguration conf) { CSQueue parentQ = q.getParent(); - long maxAppLifetime = conf.getMaximumLifetimePerQueue(q.getQueuePathObject()); + long maxAppLifetime = conf.getMaximumLifetimePerQueue(q.getQueuePath()); // If q is the root queue, then get max app lifetime from conf. if (q.getQueuePathObject().isRoot()) { @@ -64,7 +64,7 @@ private long getInheritedMaxAppLifetime(CSQueue q, CapacitySchedulerConfiguratio private long setupInheritedDefaultAppLifetime(CSQueue q, QueuePath queuePath, CapacitySchedulerConfiguration conf, long myMaxAppLifetime) { CSQueue parentQ = q.getParent(); - long defaultAppLifetime = conf.getDefaultLifetimePerQueue(queuePath); + long defaultAppLifetime = conf.getDefaultLifetimePerQueue(queuePath.getFullPath()); defaultAppLifetimeWasSpecifiedInConfig = (defaultAppLifetime >= 0 || (!queuePath.isRoot() && 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 adf5f6e0ffcb4..c431d2bb45543 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 @@ -53,7 +53,7 @@ private void initializeNodeLabels(CapacitySchedulerConfiguration configuration, } private void initializeAccessibleLabels(CapacitySchedulerConfiguration configuration) { - this.accessibleLabels = configuration.getAccessibleNodeLabels(queuePath); + this.accessibleLabels = configuration.getAccessibleNodeLabels(queuePath.getFullPath()); // Inherit labels from parent if not set if (this.accessibleLabels == null && parent != null) { this.accessibleLabels = parent.getAccessibleNodeLabels(); @@ -62,7 +62,7 @@ private void initializeAccessibleLabels(CapacitySchedulerConfiguration configura private void initializeDefaultLabelExpression(CapacitySchedulerConfiguration configuration) { this.defaultLabelExpression = configuration.getDefaultNodeLabelExpression( - queuePath); + queuePath.getFullPath()); // If the accessible labels is not null and the queue has a parent with a // similar set of labels copy the defaultNodeLabelExpression from the parent if (this.accessibleLabels != null && parent != null @@ -83,7 +83,7 @@ private void initializeConfiguredNodeLabels(CapacitySchedulerConfiguration confi } } else { // Fallback to suboptimal but correct logic - this.configuredNodeLabels = configuration.getConfiguredNodeLabels(queuePath); + this.configuredNodeLabels = configuration.getConfiguredNodeLabels(queuePath.getFullPath()); } } 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/QueuePath.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/QueuePath.java index 4e34c3a6ee934..692be1132eac6 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/QueuePath.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/QueuePath.java @@ -18,14 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; -import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.WILDCARD_QUEUE; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; @@ -113,14 +110,6 @@ public boolean hasEmptyPart() { return false; } - /** - * Simple helper method to determine if the queue path is invalid or not. - * @return true if the queue path is invalid. - */ - public boolean isInvalid() { - return getPathComponents().length <= 1 && !isRoot(); - } - /** * Getter for the parent part of the path. * @return Parent path of the queue, null if there is no parent. @@ -129,14 +118,6 @@ public String getParent() { return parent; } - /** - * Getter for the parent object of the path. - * @return Parent QueuePath object of the queue, null if there is no parent. - */ - public QueuePath getParentObject() { - return hasParent() ? new QueuePath(parent) : null; - } - /** * Getter for the leaf part of the path. * @return The leaf queue name @@ -186,7 +167,7 @@ public QueuePath createNewLeaf(String childQueue) { */ @Override public Iterator iterator() { - return Arrays.asList(getPathComponents()).iterator(); + return Arrays.asList(getFullPath().split(QUEUE_REGEX_DELIMITER)).iterator(); } /** @@ -223,54 +204,6 @@ public String next() { }; } - /** - * Returns the list of wildcarded queue paths based on the autoCreatedQueueDepth config value. - * An example template precedence hierarchy for root.a ParentQueue from highest to lowest: - * yarn.scheduler.capacity.root.a.auto-queue-creation-v2.template.capacity - * yarn.scheduler.capacity.root.*.auto-queue-creation-v2.template.capacity - * @param maxAutoCreatedQueueDepth the maximum depth of auto-created queues stored in the - * configuration - * @return list of wildcarded QueuePath objects - */ - public List getWildcardedQueuePaths(int maxAutoCreatedQueueDepth) { - List wildcardedPaths = new ArrayList<>(); - // Start with the most explicit format (without wildcard) - wildcardedPaths.add(this); - - String[] pathComponents = getPathComponents(); - int supportedWildcardLevel = getSupportedWildcardLevel(maxAutoCreatedQueueDepth); - - // Collect all template entries - for (int wildcardLevel = 1; wildcardLevel <= supportedWildcardLevel; wildcardLevel++) { - int wildcardedComponentIndex = pathComponents.length - wildcardLevel; - pathComponents[wildcardedComponentIndex] = WILDCARD_QUEUE; - QueuePath wildcardedPath = createFromQueues(pathComponents); - wildcardedPaths.add(wildcardedPath); - } - - return wildcardedPaths; - } - - /** - * Returns the supported wildcard level for this queue path. - * @param maxAutoCreatedQueueDepth the maximum depth of auto-created queues stored in the - * configuration - * @return int value of the supported wildcard level - */ - private int getSupportedWildcardLevel(int maxAutoCreatedQueueDepth) { - int queuePathMaxIndex = getPathComponents().length - 1; - // Allow root to have template properties - return isRoot() ? 0 : Math.min(queuePathMaxIndex, maxAutoCreatedQueueDepth); - } - - /** - * Returns queue path components. - * @return String array containing the queue names. - */ - public String[] getPathComponents() { - return getFullPath().split(QUEUE_REGEX_DELIMITER); - } - @Override public String toString() { return getFullPath(); 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/QueuePrefixes.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/QueuePrefixes.java deleted file mode 100644 index 75b7f77a53394..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueuePrefixes.java +++ /dev/null @@ -1,57 +0,0 @@ -/** - * 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.nodelabels.CommonNodeLabelsManager; - -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ACCESSIBLE_NODE_LABELS; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; - -public final class QueuePrefixes { - - private QueuePrefixes() { - } - - public static String getQueuePrefix(QueuePath queuePath) { - return PREFIX + queuePath.getFullPath() + DOT; - } - - public static String getNodeLabelPrefix(QueuePath queuePath, String label) { - if (label.equals(CommonNodeLabelsManager.NO_LABEL)) { - return getQueuePrefix(queuePath); - } - return getQueuePrefix(queuePath) + ACCESSIBLE_NODE_LABELS + DOT + label + DOT; - } - - /** - * Get the auto created leaf queue's template configuration prefix. - * Leaf queue's template capacities are configured at the parent queue. - * - * @param queuePath parent queue's path - * @return Config prefix for leaf queue template configurations - */ - public static String getAutoCreatedQueueTemplateConfPrefix(QueuePath queuePath) { - return queuePath.getFullPath() + DOT + AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX; - } - - public static QueuePath getAutoCreatedQueueObjectTemplateConfPrefix(QueuePath queuePath) { - return new QueuePath(getAutoCreatedQueueTemplateConfPrefix(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/QueueStateHelper.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/QueueStateHelper.java index 7e561c6fe0741..5ec7d01bce9b1 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/QueueStateHelper.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/QueueStateHelper.java @@ -42,7 +42,7 @@ private QueueStateHelper() {} public static void setQueueState(AbstractCSQueue queue) { QueueState previousState = queue.getState(); QueueState configuredState = queue.getQueueContext().getConfiguration().getConfiguredState( - queue.getQueuePathObject()); + queue.getQueuePath()); QueueState parentState = (queue.getParent() == null) ? null : queue.getParent().getState(); // verify that we can not any value for State other than RUNNING/STOPPED 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/UserWeights.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/UserWeights.java index 6535de1c61d8c..a9d755227ffca 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/UserWeights.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/UserWeights.java @@ -24,7 +24,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.USER_SETTINGS; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.USER_WEIGHT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.USER_WEIGHT_PATTERN; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getQueuePrefix; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueuePrefix; public final class UserWeights { public static final float DEFAULT_WEIGHT = 1.0F; @@ -43,7 +43,7 @@ public static UserWeights createEmpty() { public static UserWeights createByConfig( CapacitySchedulerConfiguration conf, ConfigurationProperties configurationProperties, - QueuePath queuePath) { + String queuePath) { String queuePathPlusPrefix = getQueuePrefix(queuePath) + USER_SETTINGS; Map props = configurationProperties .getPropertiesWithPrefix(queuePathPlusPrefix); 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/conf/ConfigurationUpdateAssembler.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/conf/ConfigurationUpdateAssembler.java index 6213edb6a76f0..88c93019680cb 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/conf/ConfigurationUpdateAssembler.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/conf/ConfigurationUpdateAssembler.java @@ -21,17 +21,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes; import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo; import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ORDERING_POLICY; + public final class ConfigurationUpdateAssembler { private ConfigurationUpdateAssembler() { @@ -64,34 +65,38 @@ private static void removeQueue( if (queueToRemove == null) { return; } - QueuePath queuePath = new QueuePath(queueToRemove); - if (queuePath.isRoot() || queuePath.isInvalid()) { - throw new IOException("Can't remove queue " + queuePath.getFullPath()); + if (queueToRemove.lastIndexOf('.') == -1) { + throw new IOException("Can't remove queue " + queueToRemove); } - String queueName = queuePath.getLeafName(); - List siblingQueues = getSiblingQueues(queuePath, + String queueName = queueToRemove.substring( + queueToRemove.lastIndexOf('.') + 1); + List siblingQueues = getSiblingQueues(queueToRemove, proposedConf); if (!siblingQueues.contains(queueName)) { - throw new IOException("Queue " + queuePath.getFullPath() + " not found"); + throw new IOException("Queue " + queueToRemove + " not found"); } siblingQueues.remove(queueName); - - QueuePath parentPath = queuePath.getParentObject(); - proposedConf.setQueues(parentPath, siblingQueues.toArray( + String parentQueuePath = queueToRemove.substring(0, queueToRemove + .lastIndexOf('.')); + proposedConf.setQueues(parentQueuePath, siblingQueues.toArray( new String[0])); - String queuesConfig = getQueuesConfig(parentPath); + String queuesConfig = CapacitySchedulerConfiguration.PREFIX + + parentQueuePath + CapacitySchedulerConfiguration.DOT + + CapacitySchedulerConfiguration.QUEUES; if (siblingQueues.isEmpty()) { confUpdate.put(queuesConfig, null); // Unset Ordering Policy of Leaf Queue converted from // Parent Queue after removeQueue - String queueOrderingPolicy = getOrderingPolicyConfig(parentPath); + String queueOrderingPolicy = CapacitySchedulerConfiguration.PREFIX + + parentQueuePath + CapacitySchedulerConfiguration.DOT + + ORDERING_POLICY; proposedConf.unset(queueOrderingPolicy); confUpdate.put(queueOrderingPolicy, null); } else { confUpdate.put(queuesConfig, Joiner.on(',').join(siblingQueues)); } for (Map.Entry confRemove : proposedConf.getValByRegex( - ".*" + queuePath.getFullPath() + "\\..*") + ".*" + queueToRemove + "\\..*") .entrySet()) { proposedConf.unset(confRemove.getKey()); confUpdate.put(confRemove.getKey(), null); @@ -104,23 +109,27 @@ private static void addQueue( if (addInfo == null) { return; } - QueuePath queuePath = new QueuePath(addInfo.getQueue()); - String queueName = queuePath.getLeafName(); - if (queuePath.isRoot() || queuePath.isInvalid()) { + String queuePath = addInfo.getQueue(); + String queueName = queuePath.substring(queuePath.lastIndexOf('.') + 1); + if (queuePath.lastIndexOf('.') == -1) { throw new IOException("Can't add invalid queue " + queuePath); } else if (getSiblingQueues(queuePath, proposedConf).contains( queueName)) { throw new IOException("Can't add existing queue " + queuePath); } - - QueuePath parentPath = queuePath.getParentObject(); - List siblingQueues = proposedConf.getQueues(parentPath); - siblingQueues.add(queueName); - proposedConf.setQueues(parentPath, + String parentQueue = queuePath.substring(0, queuePath.lastIndexOf('.')); + String[] siblings = proposedConf.getQueues(parentQueue); + List siblingQueues = siblings == null ? new ArrayList<>() : + new ArrayList<>(Arrays.asList(siblings)); + siblingQueues.add(queuePath.substring(queuePath.lastIndexOf('.') + 1)); + proposedConf.setQueues(parentQueue, siblingQueues.toArray(new String[0])); - confUpdate.put(getQueuesConfig(parentPath), + confUpdate.put(CapacitySchedulerConfiguration.PREFIX + + parentQueue + CapacitySchedulerConfiguration.DOT + + CapacitySchedulerConfiguration.QUEUES, Joiner.on(',').join(siblingQueues)); - String keyPrefix = QueuePrefixes.getQueuePrefix(queuePath); + String keyPrefix = CapacitySchedulerConfiguration.PREFIX + + queuePath + CapacitySchedulerConfiguration.DOT; for (Map.Entry kv : addInfo.getParams().entrySet()) { String keyValue = kv.getValue(); if (keyValue == null || keyValue.isEmpty()) { @@ -133,7 +142,8 @@ private static void addQueue( } // Unset Ordering Policy of Parent Queue converted from // Leaf Queue after addQueue - String queueOrderingPolicy = getOrderingPolicyConfig(parentPath); + String queueOrderingPolicy = CapacitySchedulerConfiguration.PREFIX + + parentQueue + CapacitySchedulerConfiguration.DOT + ORDERING_POLICY; if (siblingQueues.size() == 1) { proposedConf.unset(queueOrderingPolicy); confUpdate.put(queueOrderingPolicy, null); @@ -146,8 +156,9 @@ private static void updateQueue(QueueConfigInfo updateInfo, if (updateInfo == null) { return; } - QueuePath queuePath = new QueuePath(updateInfo.getQueue()); - String keyPrefix = QueuePrefixes.getQueuePrefix(queuePath); + String queuePath = updateInfo.getQueue(); + String keyPrefix = CapacitySchedulerConfiguration.PREFIX + + queuePath + CapacitySchedulerConfiguration.DOT; for (Map.Entry kv : updateInfo.getParams().entrySet()) { String keyValue = kv.getValue(); if (keyValue == null || keyValue.isEmpty()) { @@ -160,16 +171,11 @@ private static void updateQueue(QueueConfigInfo updateInfo, } } - private static List getSiblingQueues(QueuePath queuePath, Configuration conf) { - String childQueuesKey = getQueuesConfig(queuePath.getParentObject()); + private static List getSiblingQueues(String queuePath, Configuration conf) { + String parentQueue = queuePath.substring(0, queuePath.lastIndexOf('.')); + String childQueuesKey = CapacitySchedulerConfiguration.PREFIX + + parentQueue + CapacitySchedulerConfiguration.DOT + + CapacitySchedulerConfiguration.QUEUES; return new ArrayList<>(conf.getTrimmedStringCollection(childQueuesKey)); } - - private static String getQueuesConfig(QueuePath queuePath) { - return QueuePrefixes.getQueuePrefix(queuePath) + CapacitySchedulerConfiguration.QUEUES; - } - - private static String getOrderingPolicyConfig(QueuePath queuePath) { - return QueuePrefixes.getQueuePrefix(queuePath) + CapacitySchedulerConfiguration.ORDERING_POLICY; - } } 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/conf/QueueCapacityConfigParser.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/conf/QueueCapacityConfigParser.java index 5ebaa5c867f3c..644938e33e775 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/conf/QueueCapacityConfigParser.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/conf/QueueCapacityConfigParser.java @@ -18,9 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.util.UnitsConversionUtil; import java.util.ArrayList; @@ -65,9 +65,9 @@ public QueueCapacityConfigParser() { * @param queuePath queue for which the capacity property is parsed * @return a parsed capacity vector */ - public QueueCapacityVector parse(String capacityString, QueuePath queuePath) { + public QueueCapacityVector parse(String capacityString, String queuePath) { - if (queuePath.isRoot()) { + if (queuePath.equals(CapacitySchedulerConfiguration.ROOT)) { return QueueCapacityVector.of(100f, ResourceUnitCapacityType.PERCENTAGE); } 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/fair/AllocationConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java index 870f7b1d7a1a4..b3705c600575a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java @@ -28,7 +28,6 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.security.AccessType; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocation.AllocationFileParser; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocation.QueueProperties; import org.apache.hadoop.yarn.util.resource.Resources; @@ -195,9 +194,9 @@ public Map> getQueueAcls() { * Get the map of reservation ACLs to {@link AccessControlList} for the * specified queue. */ - public Map getReservationAcls(QueuePath + public Map getReservationAcls(String queue) { - return this.resAcls.get(queue.getFullPath()); + return this.resAcls.get(queue); } /** @@ -342,52 +341,52 @@ public Map> getConfiguredQueues() { } @Override - public boolean isReservable(QueuePath queue) { - return reservableQueues.contains(queue.getFullPath()); + public boolean isReservable(String queue) { + return reservableQueues.contains(queue); } @Override - public long getReservationWindow(QueuePath queue) { + public long getReservationWindow(String queue) { return globalReservationQueueConfig.getReservationWindowMsec(); } @Override - public float getAverageCapacity(QueuePath queue) { + public float getAverageCapacity(String queue) { return globalReservationQueueConfig.getAvgOverTimeMultiplier() * 100; } @Override - public float getInstantaneousMaxCapacity(QueuePath queue) { + public float getInstantaneousMaxCapacity(String queue) { return globalReservationQueueConfig.getMaxOverTimeMultiplier() * 100; } @Override - public String getReservationAdmissionPolicy(QueuePath queue) { + public String getReservationAdmissionPolicy(String queue) { return globalReservationQueueConfig.getReservationAdmissionPolicy(); } @Override - public String getReservationAgent(QueuePath queue) { + public String getReservationAgent(String queue) { return globalReservationQueueConfig.getReservationAgent(); } @Override - public boolean getShowReservationAsQueues(QueuePath queue) { + public boolean getShowReservationAsQueues(String queue) { return globalReservationQueueConfig.shouldShowReservationAsQueues(); } @Override - public String getReplanner(QueuePath queue) { + public String getReplanner(String queue) { return globalReservationQueueConfig.getPlanner(); } @Override - public boolean getMoveOnExpiry(QueuePath queue) { + public boolean getMoveOnExpiry(String queue) { return globalReservationQueueConfig.shouldMoveOnExpiry(); } @Override - public long getEnforcementWindow(QueuePath queue) { + public long getEnforcementWindow(String queue) { return globalReservationQueueConfig.getEnforcementWindowMsec(); } 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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java index dc4e842518c6f..32a3b7be4be68 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java @@ -76,7 +76,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils.MaxResourceValidationResult; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; @@ -1364,11 +1363,7 @@ private String resolveReservationQueueName(String queueName, readLock.lock(); try { FSQueue queue = queueMgr.getQueue(queueName); - if (queue == null) { - return queueName; - } - QueuePath queuePath = new QueuePath(queue.getQueueName()); - if (!allocConf.isReservable(queuePath)) { + if ((queue == null) || !allocConf.isReservable(queue.getQueueName())) { return queueName; } // Use fully specified name from now on (including root. prefix) @@ -1378,7 +1373,7 @@ private String resolveReservationQueueName(String queueName, queue = queueMgr.getQueue(resQName); if (queue == null) { // reservation has terminated during failover - if (isRecovering && allocConf.getMoveOnExpiry(queuePath)) { + if (isRecovering && allocConf.getMoveOnExpiry(queueName)) { // move to the default child queue of the plan return getDefaultQueueForPlanQueue(queueName); } @@ -1977,7 +1972,7 @@ public Set getPlanQueues() throws YarnException { Set planQueues = new HashSet(); for (FSQueue fsQueue : queueMgr.getQueues()) { String queueName = fsQueue.getName(); - if (allocConf.isReservable(new QueuePath(queueName))) { + if (allocConf.isReservable(queueName)) { planQueues.add(queueName); } } @@ -2018,7 +2013,7 @@ public void removeQueue(String queueName) throws YarnException { private String handleMoveToPlanQueue(String targetQueueName) { FSQueue dest = queueMgr.getQueue(targetQueueName); - if (dest != null && allocConf.isReservable(new QueuePath(dest.getQueueName()))) { + if (dest != null && allocConf.isReservable(dest.getQueueName())) { // use the default child reservation queue of the plan targetQueueName = getDefaultQueueForPlanQueue(targetQueueName); } 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/fair/converter/FSConfigToCSConfigConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigToCSConfigConverter.java index ff47b606c7403..6c8fed8182b80 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigToCSConfigConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigToCSConfigConverter.java @@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.MappingRulesDescription; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationConfigurationException; @@ -423,13 +422,13 @@ private void generateQueueAcl(String queue, if (!submitAcls.getGroups().isEmpty() || !submitAcls.getUsers().isEmpty() || submitAcls.isAllAllowed()) { - capacitySchedulerConfig.setAcl(new QueuePath(queue), QueueACL.SUBMIT_APPLICATIONS, + capacitySchedulerConfig.setAcl(queue, QueueACL.SUBMIT_APPLICATIONS, submitAcls.getAclString()); } if (!adminAcls.getGroups().isEmpty() || !adminAcls.getUsers().isEmpty() || adminAcls.isAllAllowed()) { - capacitySchedulerConfig.setAcl(new QueuePath(queue), QueueACL.ADMINISTER_QUEUE, + capacitySchedulerConfig.setAcl(queue, QueueACL.ADMINISTER_QUEUE, adminAcls.getAclString()); } } 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/fair/converter/FSQueueConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSQueueConverter.java index 510705ce14cd2..1bbf056c148d0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSQueueConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSQueueConverter.java @@ -23,7 +23,6 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.ConfigurableResource; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; @@ -105,8 +104,7 @@ private void emitChildQueues(String queueName, List children) { List childQueues = children.stream() .map(child -> getQueueShortName(child.getName())) .collect(Collectors.toList()); - capacitySchedulerConfig.setQueues(new QueuePath(queueName), - childQueues.toArray(new String[0])); + capacitySchedulerConfig.setQueues(queueName, childQueues.toArray(new String[0])); } } @@ -124,13 +122,13 @@ private void emitMaxAMShare(String queueName, FSQueue queue) { && queueMaxAmShare != queueMaxAMShareDefault && queueMaxAmShare != QUEUE_MAX_AM_SHARE_DISABLED) { capacitySchedulerConfig.setMaximumApplicationMasterResourcePerQueuePercent( - new QueuePath(queueName), queueMaxAmShare); + queueName, queueMaxAmShare); } if (queueMaxAmShare == QUEUE_MAX_AM_SHARE_DISABLED && queueMaxAmShare != queueMaxAMShareDefault) { capacitySchedulerConfig.setMaximumApplicationMasterResourcePerQueuePercent( - new QueuePath(queueName), 1.0f); + queueName, 1.0f); } } @@ -143,7 +141,7 @@ private void emitMaxAMShare(String queueName, FSQueue queue) { private void emitMaxParallelApps(String queueName, FSQueue queue) { if (queue.getMaxRunningApps() != MAX_RUNNING_APPS_UNSET && queue.getMaxRunningApps() != queueMaxAppsDefault) { - capacitySchedulerConfig.setMaxParallelAppsForQueue(new QueuePath(queueName), + capacitySchedulerConfig.setMaxParallelAppsForQueue(queueName, String.valueOf(queue.getMaxRunningApps())); } } @@ -163,7 +161,7 @@ private void emitMaximumCapacity(String queueName, FSQueue queue) { ruleHandler.handleMaxResources(); } - capacitySchedulerConfig.setMaximumCapacity(new QueuePath(queueName), + capacitySchedulerConfig.setMaximumCapacity(queueName, 100.0f); } @@ -196,10 +194,10 @@ private void emitMaxAllocations(String queueName, FSQueue queue) { // only emit max allocation if it differs from the parent's setting if (maxVcores != parentMaxVcores || maxMemory != parentMaxMemory) { capacitySchedulerConfig.setQueueMaximumAllocationMb( - new QueuePath(queueName), (int) maxMemory); + queueName, (int) maxMemory); capacitySchedulerConfig.setQueueMaximumAllocationVcores( - new QueuePath(queueName), maxVcores); + queueName, maxVcores); } } } @@ -212,14 +210,14 @@ private void emitMaxAllocations(String queueName, FSQueue queue) { */ private void emitPreemptionDisabled(String queueName, FSQueue queue) { if (preemptionEnabled && !queue.isPreemptable()) { - capacitySchedulerConfig.setPreemptionDisabled(new QueuePath(queueName), true); + capacitySchedulerConfig.setPreemptionDisabled(queueName, true); } } public void emitDefaultUserLimitFactor(String queueName, List children) { if (children.isEmpty() && - !capacitySchedulerConfig.isAutoQueueCreationV2Enabled(new QueuePath(queueName))) { - capacitySchedulerConfig.setUserLimitFactor(new QueuePath(queueName), -1.0f); + !capacitySchedulerConfig.isAutoQueueCreationV2Enabled(queueName)) { + capacitySchedulerConfig.setUserLimitFactor(queueName, -1.0f); } } @@ -248,16 +246,16 @@ private void emitOrderingPolicy(String queueName, FSQueue queue) { switch (policy) { case DominantResourceFairnessPolicy.NAME: - capacitySchedulerConfig.setOrderingPolicy(new QueuePath(queueName), FAIR_POLICY); + capacitySchedulerConfig.setOrderingPolicy(queueName, FAIR_POLICY); break; case FairSharePolicy.NAME: - capacitySchedulerConfig.setOrderingPolicy(new QueuePath(queueName), FAIR_POLICY); + capacitySchedulerConfig.setOrderingPolicy(queueName, FAIR_POLICY); if (drfUsed) { ruleHandler.handleFairAsDrf(queueName); } break; case FifoPolicy.NAME: - capacitySchedulerConfig.setOrderingPolicy(new QueuePath(queueName), FIFO_POLICY); + capacitySchedulerConfig.setOrderingPolicy(queueName, FIFO_POLICY); break; default: String msg = String.format("Unexpected ordering policy " + 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/fair/converter/QueuePlacementConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/QueuePlacementConverter.java index 10777a61bd58c..6ff19e21a6e71 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/QueuePlacementConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/QueuePlacementConverter.java @@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.SpecifiedPlacementRule; import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.MappingRulesDescription; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.Rule; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.Rule.FallbackResult; @@ -122,7 +121,7 @@ private void handleNestedRule(List rules, PlacementRule parentRule = userRule.getParentRule(); boolean parentCreate = ((FSPlacementRule) parentRule).getCreateFlag(); Policy policy; - QueuePath queueName = null; + String queueName = null; if (parentRule instanceof PrimaryGroupPlacementRule) { policy = Policy.PRIMARY_GROUP_USER; @@ -131,7 +130,7 @@ private void handleNestedRule(List rules, } else if (parentRule instanceof DefaultPlacementRule) { DefaultPlacementRule defaultRule = (DefaultPlacementRule) parentRule; policy = Policy.USER; - queueName = new QueuePath(defaultRule.defaultQueueName); + queueName = defaultRule.defaultQueueName; } else { throw new IllegalArgumentException( "Unsupported parent nested rule: " @@ -188,7 +187,7 @@ private Rule createNestedRule(Policy policy, boolean create, FSConfigToCSConfigRuleHandler ruleHandler, boolean fsParentCreate, - QueuePath parentQueue, + String parentQueue, CapacitySchedulerConfiguration csConf, boolean usePercentages) { @@ -197,7 +196,7 @@ private Rule createNestedRule(Policy policy, // "parent" is already set to "root" at this point, // so we override it if necessary if (parentQueue != null) { - rule.setParentQueue(parentQueue.getFullPath()); + rule.setParentQueue(parentQueue); } if (usePercentages) { @@ -208,13 +207,13 @@ private Rule createNestedRule(Policy policy, } else if (policy == Policy.SECONDARY_GROUP_USER) { ruleHandler.handleFSParentCreateFlag("root."); } else { - ruleHandler.handleFSParentCreateFlag(parentQueue.getFullPath()); + ruleHandler.handleFSParentCreateFlag(parentQueue); } } // check if parent conflicts with existing static queues if (create && policy == Policy.USER) { - ruleHandler.handleRuleAutoCreateFlag(parentQueue.getFullPath()); + ruleHandler.handleRuleAutoCreateFlag(parentQueue); checkStaticDynamicConflict(parentQueue, csConf, ruleHandler); } } else { @@ -231,15 +230,15 @@ private Rule createNestedRule(Policy policy, return rule; } - private void checkStaticDynamicConflict(QueuePath parentPath, + private void checkStaticDynamicConflict(String parentPath, CapacitySchedulerConfiguration csConf, FSConfigToCSConfigRuleHandler ruleHandler) { - List childQueues = csConf.getQueues(parentPath); + String[] childQueues = csConf.getQueues(parentPath); // User must be warned: static + dynamic queues are under the // same parent - if (childQueues != null && childQueues.size() > 0) { - ruleHandler.handleChildStaticDynamicConflict(parentPath.getFullPath()); + if (childQueues != null && childQueues.length > 0) { + ruleHandler.handleChildStaticDynamicConflict(parentPath); } } } 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/fair/converter/weightconversion/WeightToPercentConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/WeightToPercentConverter.java index 0e437c112e074..600b92626db9a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/WeightToPercentConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/WeightToPercentConverter.java @@ -29,7 +29,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; public class WeightToPercentConverter @@ -51,11 +50,11 @@ public void convertWeightsForChildQueues(FSQueue queue, boolean shouldAllowZeroSumCapacity = result.getRight(); capacities - .forEach((key, value) -> csConfig.setCapacity(new QueuePath(key), value.toString())); + .forEach((key, value) -> csConfig.setCapacity(key, value.toString())); if (shouldAllowZeroSumCapacity) { String queueName = queue.getName(); - csConfig.setAllowZeroCapacitySum(new QueuePath(queueName), true); + csConfig.setAllowZeroCapacitySum(queueName, 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/fair/converter/weightconversion/WeightToWeightConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/WeightToWeightConverter.java index bd13e7a9a7115..75554fc4d2e6d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/WeightToWeightConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/WeightToWeightConverter.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSParentQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; @@ -35,14 +34,13 @@ public void convertWeightsForChildQueues(FSQueue queue, List children = queue.getChildQueues(); if (queue instanceof FSParentQueue || !children.isEmpty()) { - QueuePath queuePath = new QueuePath(queue.getName()); if (queue.getName().equals(ROOT_QUEUE)) { - csConfig.setNonLabeledQueueWeight(queuePath, queue.getWeight()); + csConfig.setNonLabeledQueueWeight(queue.getName(), queue.getWeight()); } children.forEach(fsQueue -> csConfig.setNonLabeledQueueWeight( - new QueuePath(fsQueue.getName()), fsQueue.getWeight())); - csConfig.setAutoQueueCreationV2Enabled(queuePath, true); + fsQueue.getName(), fsQueue.getWeight())); + csConfig.setAutoQueueCreationV2Enabled(queue.getName(), 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/security/CapacityReservationsACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/CapacityReservationsACLsManager.java index 418f822187fa8..531d2a315331b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/CapacityReservationsACLsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/CapacityReservationsACLsManager.java @@ -39,7 +39,7 @@ public CapacityReservationsACLsManager(ResourceScheduler scheduler, for (String planQueue : scheduler.getPlanQueues()) { CSQueue queue = ((CapacityScheduler) scheduler).getQueue(planQueue); reservationAcls.put(planQueue, - csConf.getReservationAcls(queue.getQueuePathObject())); + csConf.getReservationAcls(queue.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/security/FairReservationsACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/FairReservationsACLsManager.java index a2d05de4e39b8..09f147f89ea14 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/FairReservationsACLsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/FairReservationsACLsManager.java @@ -20,7 +20,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; @@ -36,7 +35,7 @@ public FairReservationsACLsManager(ResourceScheduler scheduler, AllocationConfiguration aConf = ((FairScheduler) scheduler) .getAllocationConfiguration(); for (String planQueue : scheduler.getPlanQueues()) { - reservationAcls.put(planQueue, aConf.getReservationAcls(new QueuePath(planQueue))); + reservationAcls.put(planQueue, aConf.getReservationAcls(planQueue)); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java index dbf57cfa05dfe..5d5b3f26cc1f0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java @@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper.CapacitySchedulerInfoHelper; import java.util.ArrayList; @@ -101,7 +100,7 @@ public CapacitySchedulerInfo(CSQueue parent, CapacityScheduler cs) { CapacitySchedulerConfiguration conf = cs.getConfiguration(); queueAcls = new QueueAclsInfo(); - queueAcls.addAll(getSortedQueueAclInfoList(parent, new QueuePath(queuePath), conf)); + queueAcls.addAll(getSortedQueueAclInfoList(parent, queueName, conf)); queuePriority = parent.getPriority().getPriority(); if (parent instanceof AbstractParentQueue) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java index 318adee1caac4..1b5a3714d68c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java @@ -40,9 +40,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper.CapacitySchedulerInfoHelper; @XmlRootElement @@ -102,7 +99,6 @@ public class CapacitySchedulerQueueInfo { CapacitySchedulerQueueInfo(CapacityScheduler cs, CSQueue q) { queuePath = q.getQueuePath(); - QueuePath queuePathObject = new QueuePath(queuePath); capacity = q.getCapacity() * 100; usedCapacity = q.getUsedCapacity() * 100; @@ -157,7 +153,7 @@ public class CapacitySchedulerQueueInfo { CapacitySchedulerConfiguration conf = cs.getConfiguration(); queueAcls = new QueueAclsInfo(); - queueAcls.addAll(getSortedQueueAclInfoList(q, queuePathObject, conf)); + queueAcls.addAll(getSortedQueueAclInfoList(q, queuePath, conf)); queuePriority = q.getPriority().getPriority(); if (q instanceof AbstractParentQueue) { @@ -179,12 +175,12 @@ public class CapacitySchedulerQueueInfo { AbstractCSQueue.CapacityConfigType.ABSOLUTE_RESOURCE; autoCreateChildQueueEnabled = conf. - isAutoCreateChildQueueEnabled(queuePathObject); - leafQueueTemplate = new LeafQueueTemplateInfo(conf, queuePathObject); + isAutoCreateChildQueueEnabled(queuePath); + leafQueueTemplate = new LeafQueueTemplateInfo(conf, queuePath); } public static ArrayList getSortedQueueAclInfoList( - CSQueue queue, QueuePath queuePath, CapacitySchedulerConfiguration conf) { + CSQueue queue, String queuePath, CapacitySchedulerConfiguration conf) { ArrayList queueAclsInfo = new ArrayList<>(); for (Map.Entry e : ((AbstractCSQueue) queue).getACLs().entrySet()) { @@ -195,7 +191,7 @@ public static ArrayList getSortedQueueAclInfoList( String aclApplicationMaxPriority = "acl_" + StringUtils.toLowerCase(AccessType.APPLICATION_MAX_PRIORITY.toString()); - String priorityAcls = conf.get(QueuePrefixes + String priorityAcls = conf.get(CapacitySchedulerConfiguration .getQueuePrefix(queuePath) + aclApplicationMaxPriority, CapacitySchedulerConfiguration.ALL_ACL); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java index 65e5a700d6be8..446b7928bf93b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java @@ -27,8 +27,6 @@ import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlSeeAlso; import javax.xml.bind.annotation.XmlTransient; - -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; @@ -113,9 +111,8 @@ public FairSchedulerQueueInfo(FSQueue queue, FairScheduler scheduler) { reservedContainers = queue.getMetrics().getReservedContainers(); pendingContainers = queue.getMetrics().getPendingContainers(); - QueuePath queuePath = new QueuePath(queueName); - if (allocConf.isReservable(queuePath) && - !allocConf.getShowReservationAsQueues(queuePath)) { + if (allocConf.isReservable(queueName) && + !allocConf.getShowReservationAsQueues(queueName)) { return; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LeafQueueTemplateInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LeafQueueTemplateInfo.java index c4a20494bc38b..f528a47449b0e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LeafQueueTemplateInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LeafQueueTemplateInfo.java @@ -18,8 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import javax.xml.bind.annotation.XmlAccessType; import javax.xml.bind.annotation.XmlAccessorType; @@ -41,8 +40,8 @@ public class LeafQueueTemplateInfo { public LeafQueueTemplateInfo() { } // JAXB needs this - public LeafQueueTemplateInfo(Configuration conf, QueuePath queuePath) { - String configPrefix = QueuePrefixes. + public LeafQueueTemplateInfo(Configuration conf, String queuePath) { + String configPrefix = CapacitySchedulerConfiguration. getQueuePrefix(queuePath) + AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX + DOT; conf.forEach(entry -> { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ACLsTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ACLsTestBase.java index 26e3ee2920d0a..a011c8a8b35ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ACLsTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ACLsTestBase.java @@ -22,8 +22,6 @@ import java.net.InetSocketAddress; import java.security.PrivilegedExceptionAction; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -49,13 +47,6 @@ public abstract class ACLsTestBase { protected static final String QUEUEA = "queueA"; protected static final String QUEUEB = "queueB"; protected static final String QUEUEC = "queueC"; - protected static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - protected static final QueuePath A_QUEUE_PATH = new QueuePath( - CapacitySchedulerConfiguration.ROOT + "." + QUEUEA); - protected static final QueuePath B_QUEUE_PATH = new QueuePath( - CapacitySchedulerConfiguration.ROOT + "." + QUEUEB); - protected static final QueuePath C_QUEUE_PATH = new QueuePath( - CapacitySchedulerConfiguration.ROOT + "." + QUEUEC); protected static final Logger LOG = LoggerFactory.getLogger(TestApplicationACLs.class); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java index a0fe708c77e20..af4fe7d6c6e5a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java @@ -499,15 +499,19 @@ private boolean checkCapacity(Collection plans) { private static Configuration createCapacitySchedulerConfiguration() { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] { + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { QUEUEA, QUEUEB, QUEUEC }); - csConf.setCapacity(A_QUEUE_PATH, 50f); - csConf.setCapacity(B_QUEUE_PATH, 20f); - csConf.setCapacity(C_QUEUE_PATH, 30f); - csConf.setReservable(A_QUEUE_PATH, true); - csConf.setReservable(B_QUEUE_PATH, true); - csConf.setReservable(C_QUEUE_PATH, true); + String absoluteQueueA = CapacitySchedulerConfiguration.ROOT + "." + QUEUEA; + String absoluteQueueB = CapacitySchedulerConfiguration.ROOT + "." + QUEUEB; + String absoluteQueueC = CapacitySchedulerConfiguration.ROOT + "." + QUEUEC; + + csConf.setCapacity(absoluteQueueA, 50f); + csConf.setCapacity(absoluteQueueB, 20f); + csConf.setCapacity(absoluteQueueC, 30f); + csConf.setReservable(absoluteQueueA, true); + csConf.setReservable(absoluteQueueB, true); + csConf.setReservable(absoluteQueueC, true); // Set up ACLs on Queue A Map reservationAclsOnQueueA = @@ -524,7 +528,7 @@ private static Configuration createCapacitySchedulerConfiguration() { reservationAclsOnQueueA.put(ReservationACL.LIST_RESERVATIONS, listACLonQueueA); - csConf.setReservationAcls(A_QUEUE_PATH, reservationAclsOnQueueA); + csConf.setReservationAcls(absoluteQueueA, reservationAclsOnQueueA); // Set up ACLs on Queue B Map reservationAclsOnQueueB = @@ -541,7 +545,7 @@ private static Configuration createCapacitySchedulerConfiguration() { reservationAclsOnQueueB.put(ReservationACL.LIST_RESERVATIONS, listACLonQueueB); - csConf.setReservationAcls(B_QUEUE_PATH, reservationAclsOnQueueB); + csConf.setReservationAcls(absoluteQueueB, reservationAclsOnQueueB); csConf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true); csConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true); 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 50fc8fd67deac..0369feb130c48 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 @@ -116,8 +116,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.AUTO_QUEUE_PARENT_TEMPLATE_PREFIX; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getAutoCreatedQueueTemplateConfPrefix; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getQueuePrefix; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueuePrefix; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; @@ -150,12 +149,7 @@ public class TestAppManager extends AppManagerTestBase{ private ResourceScheduler scheduler; private static final String USER_ID_PREFIX = "userid="; - private static final String ROOT_PARENT_PATH = PREFIX + "root.parent."; - private static final QueuePath ROOT = new QueuePath("root"); - private static final QueuePath DEFAULT = new QueuePath("root.default"); - private static final QueuePath TEST = new QueuePath("root.test"); - private static final QueuePath PARENT = new QueuePath("root.parent"); - private static final QueuePath MANAGED_PARENT = new QueuePath("root.managedparent"); + private static final String ROOT_PARENT = PREFIX + "root.parent."; public synchronized RMAppEventType getAppEventType() { return appEventType; @@ -327,11 +321,11 @@ public void testQueueSubmitWithACLsEnabledWithQueueMapping() CapacitySchedulerConfiguration(conf, false); csConf.set(PREFIX + "root.queues", "default,test"); - csConf.setCapacity(DEFAULT, 50.0f); - csConf.setMaximumCapacity(DEFAULT, 100.0f); + csConf.setCapacity("root.default", 50.0f); + csConf.setMaximumCapacity("root.default", 100.0f); - csConf.setCapacity(TEST, 50.0f); - csConf.setMaximumCapacity(TEST, 100.0f); + csConf.setCapacity("root.test", 50.0f); + csConf.setMaximumCapacity("root.test", 100.0f); csConf.set(PREFIX + "root.acl_submit_applications", " "); csConf.set(PREFIX + "root.acl_administer_queue", " "); @@ -369,11 +363,11 @@ public void testQueueSubmitWithLeafQueueName() CapacitySchedulerConfiguration(conf, false); csConf.set(PREFIX + "root.queues", "default,test"); - csConf.setCapacity(DEFAULT, 50.0f); - csConf.setMaximumCapacity(DEFAULT, 100.0f); + csConf.setCapacity("root.default", 50.0f); + csConf.setMaximumCapacity("root.default", 100.0f); - csConf.setCapacity(TEST, 50.0f); - csConf.setMaximumCapacity(TEST, 100.0f); + csConf.setCapacity("root.test", 50.0f); + csConf.setMaximumCapacity("root.test", 100.0f); MockRM newMockRM = new MockRM(csConf); RMContext newMockRMContext = newMockRM.getRMContext(); @@ -396,11 +390,11 @@ public void testQueueSubmitWithACLsEnabledWithQueueMappingForLegacyAutoCreatedQu conf, false); csConf.set(PREFIX + "root.queues", "default,managedparent"); - csConf.setCapacity(DEFAULT, 50.0f); - csConf.setMaximumCapacity(DEFAULT, 100.0f); + csConf.setCapacity("root.default", 50.0f); + csConf.setMaximumCapacity("root.default", 100.0f); - csConf.setCapacity(MANAGED_PARENT, 50.0f); - csConf.setMaximumCapacity(MANAGED_PARENT, 100.0f); + csConf.setCapacity("root.managedparent", 50.0f); + csConf.setMaximumCapacity("root.managedparent", 100.0f); csConf.set(PREFIX + "root.acl_submit_applications", " "); csConf.set(PREFIX + "root.acl_administer_queue", " "); @@ -411,9 +405,9 @@ public void testQueueSubmitWithACLsEnabledWithQueueMappingForLegacyAutoCreatedQu csConf.set(PREFIX + "root.managedparent.acl_administer_queue", "admin"); csConf.set(PREFIX + "root.managedparent.acl_submit_applications", "user1"); - csConf.setAutoCreateChildQueueEnabled(MANAGED_PARENT, true); - csConf.setAutoCreatedLeafQueueConfigCapacity(MANAGED_PARENT, 30f); - csConf.setAutoCreatedLeafQueueConfigMaxCapacity(MANAGED_PARENT, 100f); + csConf.setAutoCreateChildQueueEnabled("root.managedparent", true); + csConf.setAutoCreatedLeafQueueConfigCapacity("root.managedparent", 30f); + csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.managedparent", 100f); MockRM newMockRM = new MockRM(csConf); CapacityScheduler cs = @@ -447,27 +441,21 @@ public void testLegacyAutoCreatedQueuesWithACLTemplates() YarnConfiguration conf = createYarnACLEnabledConfiguration(); CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration( conf, false); - - QueuePath parentQueuePath = new QueuePath("root.parent"); - QueuePath user1QueuePath = new QueuePath("root.parent.user1"); - QueuePath user2QueuePath = new QueuePath("root.parent.user2"); - csConf.set(PREFIX + "root.queues", "parent"); csConf.set(PREFIX + "root.acl_submit_applications", " "); csConf.set(PREFIX + "root.acl_administer_queue", " "); - csConf.setCapacity(parentQueuePath, 100.0f); + csConf.setCapacity("root.parent", 100.0f); csConf.set(PREFIX + "root.parent.acl_administer_queue", "user1,user4"); csConf.set(PREFIX + "root.parent.acl_submit_applications", "user1,user4"); - csConf.setAutoCreateChildQueueEnabled(parentQueuePath, true); - csConf.setAutoCreatedLeafQueueConfigCapacity(parentQueuePath, 50f); - csConf.setAutoCreatedLeafQueueConfigMaxCapacity(parentQueuePath, 100f); - String autoCreatedQueuePrefix = - getAutoCreatedQueueTemplateConfPrefix(parentQueuePath); - QueuePath autoCreatedQueuePath = new QueuePath(autoCreatedQueuePrefix); - csConf.set(getQueuePrefix(autoCreatedQueuePath) + "acl_administer_queue", "user2,user4"); - csConf.set(getQueuePrefix(autoCreatedQueuePath) + "acl_submit_applications", "user2,user4"); + csConf.setAutoCreateChildQueueEnabled("root.parent", true); + csConf.setAutoCreatedLeafQueueConfigCapacity("root.parent", 50f); + csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.parent", 100f); + csConf.set(getQueuePrefix(csConf.getAutoCreatedQueueTemplateConfPrefix("root.parent")) + + "acl_administer_queue", "user2,user4"); + csConf.set(getQueuePrefix(csConf.getAutoCreatedQueueTemplateConfPrefix("root.parent")) + + "acl_submit_applications", "user2,user4"); MockRM newMockRM = new MockRM(csConf); @@ -476,22 +464,22 @@ public void testLegacyAutoCreatedQueuesWithACLTemplates() // user1 has permission on root.parent so a queue would be created newMockRMContext.setQueuePlacementManager(createMockPlacementManager( - "user1", "user1", parentQueuePath.getFullPath())); + "user1", "user1", "root.parent")); verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)), newAppMonitor, newMockRMContext, "user1", - user1QueuePath.getFullPath()); + "root.parent.user1"); newMockRMContext.setQueuePlacementManager(createMockPlacementManager( - "user1|user2|user3|user4", "user2", parentQueuePath.getFullPath())); + "user1|user2|user3|user4", "user2", "root.parent")); // user2 has permission (due to ACL templates) verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)), newAppMonitor, newMockRMContext, "user2", - user2QueuePath.getFullPath()); + "root.parent.user2"); // user3 doesn't have permission verifyAppSubmissionFailure(newAppMonitor, @@ -503,12 +491,12 @@ public void testLegacyAutoCreatedQueuesWithACLTemplates() newAppMonitor, newMockRMContext, "user4", - user2QueuePath.getFullPath()); + "root.parent.user2"); // create the root.parent.user2 manually CapacityScheduler cs = ((CapacityScheduler) newMockRM.getResourceScheduler()); - cs.getCapacitySchedulerQueueManager().createQueue(user2QueuePath); + cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2")); AutoCreatedLeafQueue autoCreatedLeafQueue = (AutoCreatedLeafQueue) cs.getQueue("user2"); Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue("parent"); @@ -521,7 +509,7 @@ public void testLegacyAutoCreatedQueuesWithACLTemplates() newAppMonitor, newMockRMContext, "user2", - user2QueuePath.getFullPath()); + "root.parent.user2"); // user3 doesn't have permission for root.parent.user2 queue verifyAppSubmissionFailure(newAppMonitor, @@ -533,7 +521,7 @@ public void testLegacyAutoCreatedQueuesWithACLTemplates() newAppMonitor, newMockRMContext, "user1", - user2QueuePath.getFullPath()); + "root.parent.user2"); } @Test @@ -542,19 +530,19 @@ public void testFlexibleAutoCreatedQueuesWithSpecializedACLTemplatesAndDynamicPa YarnConfiguration conf = createYarnACLEnabledConfiguration(); CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity", + csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity", "1w"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_administer_queue", + csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_administer_queue", "user2"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_submit_applications", + csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_submit_applications", "user2"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", "1w"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + "user3"); + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications", "user3"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX - + "acl_submit_applications", "user3"); MockRM newMockRM = new MockRM(csConf); @@ -633,19 +621,19 @@ public void testFlexibleAutoCreatedQueuesWithMixedCommonLeafACLTemplatesAndDynam YarnConfiguration conf = createYarnACLEnabledConfiguration(); CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", "1w"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", "user2"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", "user2"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", "1w"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + "user3"); + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications", "user3"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX - + "acl_submit_applications", "user3"); testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario(conf, csConf); } @@ -656,18 +644,18 @@ public void testFlexibleAutoCreatedQueuesWithMixedCommonCommonACLTemplatesAndDyn YarnConfiguration conf = createYarnACLEnabledConfiguration(); CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", "1w"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", "user2"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", "user2"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", "1w"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", "user3"); - csConf.set(ROOT_PARENT_PATH + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", "user3"); testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario(conf, csConf); @@ -756,11 +744,11 @@ public void testFlexibleAutoCreatedQueuesWithACLTemplatesALeafOnly() YarnConfiguration conf = createYarnACLEnabledConfiguration(); CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", "1w"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", "user2"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", "user2"); testFlexibleAQCLeafOnly(conf, csConf); @@ -772,11 +760,11 @@ public void testFlexibleAutoCreatedQueuesWithSpecialisedACLTemplatesALeafOnly() YarnConfiguration conf = createYarnACLEnabledConfiguration(); CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", + csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", "1w"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", "user2"); - csConf.set(ROOT_PARENT_PATH + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications", + csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications", "user2"); testFlexibleAQCLeafOnly(conf, csConf); @@ -856,7 +844,7 @@ public void testFlexibleAutoCreatedQueuesWithSpecializedACLTemplatesAndDynamicRo csConf.set(PREFIX + "root.acl_submit_applications", "user1"); csConf.set(PREFIX + "root.acl_administer_queue", "admin1"); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); + csConf.setAutoQueueCreationV2Enabled("root", true); csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity", "1w"); @@ -955,7 +943,7 @@ public void testFlexibleAutoCreatedQueuesMultiLevelDynamicParentACL() csConf.set(PREFIX + "root.acl_submit_applications", "user1"); csConf.set(PREFIX + "root.acl_administer_queue", "admin1"); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); + csConf.setAutoQueueCreationV2Enabled("root", true); csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity", "1w"); @@ -1033,11 +1021,11 @@ private CapacitySchedulerConfiguration createFlexibleAQCBaseACLConfiguration( csConf.set(PREFIX + "root.acl_submit_applications", " "); csConf.set(PREFIX + "root.acl_administer_queue", " "); - csConf.setCapacity(PARENT, "1w"); + csConf.setCapacity("root.parent", "1w"); csConf.set(PREFIX + "root.parent.acl_administer_queue", "user1"); csConf.set(PREFIX + "root.parent.acl_submit_applications", "user1"); - csConf.setAutoQueueCreationV2Enabled(PARENT, true); + csConf.setAutoQueueCreationV2Enabled("root.parent", true); return csConf; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java index ab68515911651..80f84c34503f9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java @@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -261,26 +260,23 @@ private Configuration getConfigurationWithQueueLabels(Configuration config) { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(config); // Define top-level queues - final QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - conf.setQueues(root, new String[] {"a", "b"}); - conf.setCapacityByLabel(root, "x", 100); - conf.setCapacityByLabel(root, "y", 100); - - final String aPath = CapacitySchedulerConfiguration.ROOT + ".a"; - final QueuePath a = new QueuePath(aPath); - conf.setCapacity(a, 50); - conf.setMaximumCapacity(a, 100); - conf.setAccessibleNodeLabels(a, toSet("x")); - conf.setDefaultNodeLabelExpression(a, "x"); - conf.setCapacityByLabel(a, "x", 100); - - final String bPath = CapacitySchedulerConfiguration.ROOT + ".b"; - final QueuePath b = new QueuePath(bPath); - conf.setCapacity(b, 50); - conf.setMaximumCapacity(b, 100); - conf.setAccessibleNodeLabels(b, toSet("y")); - conf.setDefaultNodeLabelExpression(b, "y"); - conf.setCapacityByLabel(b, "y", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + conf.setCapacity(A, 50); + conf.setMaximumCapacity(A, 100); + conf.setAccessibleNodeLabels(A, toSet("x")); + conf.setDefaultNodeLabelExpression(A, "x"); + conf.setCapacityByLabel(A, "x", 100); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + conf.setCapacity(B, 50); + conf.setMaximumCapacity(B, 100); + conf.setAccessibleNodeLabels(B, toSet("y")); + conf.setDefaultNodeLabelExpression(B, "y"); + conf.setCapacityByLabel(B, "y", 100); return conf; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java index 76cf0db9a02b5..ce4a7a84fdcd3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java @@ -70,7 +70,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity .TestCapacitySchedulerAutoCreatedQueueBase; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSAppAttempt; @@ -573,48 +572,46 @@ private void checkFSQueue(ResourceManager rm, private static final String QUEUE_DOESNT_EXIST = "NoSuchQueue"; private static final String USER_1 = "user1"; private static final String USER_2 = "user2"; - private static final String Q_R_PATH = CapacitySchedulerConfiguration.ROOT + "." + R; - private static final String Q_A_PATH = Q_R_PATH + "." + A; - private static final String Q_B_PATH = Q_R_PATH + "." + B; - private static final String Q_B1_PATH = Q_B_PATH + "." + B1; - private static final String Q_B2_PATH = Q_B_PATH + "." + B2; - - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath R_QUEUE_PATH = new QueuePath(Q_R_PATH); - private static final QueuePath A_QUEUE_PATH = new QueuePath(Q_A_PATH); - private static final QueuePath B_QUEUE_PATH = new QueuePath(Q_B_PATH); - private static final QueuePath B1_QUEUE_PATH = new QueuePath(Q_B1_PATH); - private static final QueuePath B2_QUEUE_PATH = new QueuePath(Q_B2_PATH); private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) { - conf.setQueues(ROOT, new String[] {R}); - conf.setCapacity(R_QUEUE_PATH, 100); - conf.setQueues(R_QUEUE_PATH, new String[] {A, B}); - conf.setCapacity(A_QUEUE_PATH, 50); - conf.setCapacity(B_QUEUE_PATH, 50); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { R }); + final String Q_R = CapacitySchedulerConfiguration.ROOT + "." + R; + conf.setCapacity(Q_R, 100); + final String Q_A = Q_R + "." + A; + final String Q_B = Q_R + "." + B; + conf.setQueues(Q_R, new String[] {A, B}); + conf.setCapacity(Q_A, 50); + conf.setCapacity(Q_B, 50); conf.setDouble(CapacitySchedulerConfiguration .MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.5f); } private void setupQueueConfigurationOnlyA( CapacitySchedulerConfiguration conf) { - conf.setQueues(ROOT, new String[] {R}); - conf.setCapacity(R_QUEUE_PATH, 100); - conf.setQueues(R_QUEUE_PATH, new String[] {A}); - conf.setCapacity(A_QUEUE_PATH, 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { R }); + final String Q_R = CapacitySchedulerConfiguration.ROOT + "." + R; + conf.setCapacity(Q_R, 100); + final String Q_A = Q_R + "." + A; + conf.setQueues(Q_R, new String[] {A}); + conf.setCapacity(Q_A, 100); conf.setDouble(CapacitySchedulerConfiguration .MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 1.0f); } private void setupQueueConfigurationChildOfB(CapacitySchedulerConfiguration conf) { - conf.setQueues(ROOT, new String[] {R}); - conf.setCapacity(R_QUEUE_PATH, 100); - conf.setQueues(R_QUEUE_PATH, new String[] {A, B}); - conf.setCapacity(A_QUEUE_PATH, 50); - conf.setCapacity(B_QUEUE_PATH, 50); - conf.setQueues(B_QUEUE_PATH, new String[] {B1, B2}); - conf.setCapacity(B1_QUEUE_PATH, 50); - conf.setCapacity(B2_QUEUE_PATH, 50); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { R }); + final String Q_R = CapacitySchedulerConfiguration.ROOT + "." + R; + conf.setCapacity(Q_R, 100); + final String Q_A = Q_R + "." + A; + final String Q_B = Q_R + "." + B; + final String Q_B1 = Q_B + "." + B1; + final String Q_B2 = Q_B + "." + B2; + conf.setQueues(Q_R, new String[] {A, B}); + conf.setCapacity(Q_A, 50); + conf.setCapacity(Q_B, 50); + conf.setQueues(Q_B, new String[] {B1, B2}); + conf.setCapacity(Q_B1, 50); + conf.setCapacity(Q_B2, 50); conf.setDouble(CapacitySchedulerConfiguration .MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.5f); } @@ -644,9 +641,9 @@ public void testRMRestartWithRemovedQueue() throws Exception{ MockRM.finishAMAndVerifyAppState(app1, rm1, nm1, am1); CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf); - csConf.setQueues(ROOT, new String[]{QUEUE_DOESNT_EXIST}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUE_DOESNT_EXIST}); final String noQueue = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_DOESNT_EXIST; - csConf.setCapacity(new QueuePath(noQueue), 100); + csConf.setCapacity(noQueue, 100); rm2 = new MockRM(csConf, memStore); rm2.start(); @@ -1672,19 +1669,24 @@ public void testQueueRecoveryOnRMWorkPreservingRestart() throws Exception { return; } CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf); - final QueuePath defaultPath = new QueuePath(ROOT + "." + "default"); - final QueuePath joe = new QueuePath(ROOT + "." + "joe"); - final QueuePath john = new QueuePath(ROOT + "." + "john"); - - csConf.setQueues(ROOT, new String[] {"default", "joe", "john"}); - csConf.setCapacity(joe, 25); - csConf.setCapacity(john, 25); - csConf.setCapacity(defaultPath, 50); - - csConf.setQueues(joe, new String[] {"test"}); - csConf.setQueues(john, new String[] {"test"}); - csConf.setCapacity(new QueuePath(joe.getFullPath(), "test"), 100); - csConf.setCapacity(new QueuePath(john.getFullPath(), "test"), 100); + + csConf.setQueues( + CapacitySchedulerConfiguration.ROOT, new String[] {"default", "joe", "john"}); + csConf.setCapacity( + CapacitySchedulerConfiguration.ROOT + "." + "joe", 25); + csConf.setCapacity( + CapacitySchedulerConfiguration.ROOT + "." + "john", 25); + csConf.setCapacity( + CapacitySchedulerConfiguration.ROOT + "." + "default", 50); + + final String q1 = CapacitySchedulerConfiguration.ROOT + "." + "joe"; + final String q2 = CapacitySchedulerConfiguration.ROOT + "." + "john"; + csConf.setQueues(q1, new String[] {"test"}); + csConf.setQueues(q2, new String[] {"test"}); + csConf.setCapacity( + CapacitySchedulerConfiguration.ROOT + "." + "joe.test", 100); + csConf.setCapacity( + CapacitySchedulerConfiguration.ROOT + "." + "john.test", 100); csConf.set(CapacitySchedulerConfiguration.MAPPING_RULE_JSON, "{\"rules\" : [{\"type\": \"user\", \"policy\" : \"specified\", " + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java index 1bac22743b0ff..fe89a698cf2e6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java @@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.QueueOrderingPolicy; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue; @@ -128,14 +127,6 @@ public class TestProportionalCapacityPreemptionPolicy { ApplicationId.newInstance(TS, 4), 0); final ArgumentCaptor evtCaptor = ArgumentCaptor.forClass(ContainerPreemptEvent.class); - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath QUEUE_A = new QueuePath("root.queueA"); - private static final QueuePath QUEUE_A_QUEUE_B = new QueuePath("root.queueA.queueB"); - private static final QueuePath QUEUE_B = new QueuePath("root.queueB"); - private static final QueuePath QUEUE_D = new QueuePath("root.queueD"); - private static final QueuePath QUEUE_D_QUEUE_E = new QueuePath("root.queueD.queueE"); - private static final QueuePath QUEUE_A_QUEUE_C = new QueuePath("root.queueA.queueC"); - public enum priority { AMCONTAINER(0), CONTAINER(1), LABELEDCONTAINER(2); @@ -355,7 +346,7 @@ public void testPerQueueDisablePreemption() { { 3, 0, 0, 0 }, // subqueues }; - conf.setPreemptionDisabled(QUEUE_B, true); + conf.setPreemptionDisabled("root.queueB", true); ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData); policy.editSchedule(); @@ -368,7 +359,7 @@ public void testPerQueueDisablePreemption() { // event handler will count only events from the following test and not the // previous one. setup(); - conf.setPreemptionDisabled(QUEUE_B, false); + conf.setPreemptionDisabled("root.queueB", false); ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData); policy2.editSchedule(); @@ -407,7 +398,7 @@ public void testPerQueueDisablePreemptionHierarchical() { // Need to call setup() again to reset mDisp setup(); // Turn off preemption for queueB and it's children - conf.setPreemptionDisabled(QUEUE_A_QUEUE_B, true); + conf.setPreemptionDisabled("root.queueA.queueB", true); ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData); policy2.editSchedule(); ApplicationAttemptId expectedAttemptOnQueueC = @@ -454,7 +445,7 @@ public void testPerQueueDisablePreemptionBroadHierarchical() { // Need to call setup() again to reset mDisp setup(); // Turn off preemption for queueB(appA) - conf.setPreemptionDisabled(QUEUE_A_QUEUE_B, true); + conf.setPreemptionDisabled("root.queueA.queueB", true); ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData); policy2.editSchedule(); // Now that queueB(appA) is not preemptable, verify that resources come @@ -464,8 +455,8 @@ public void testPerQueueDisablePreemptionBroadHierarchical() { setup(); // Turn off preemption for two of the 3 queues with over-capacity. - conf.setPreemptionDisabled(QUEUE_D_QUEUE_E, true); - conf.setPreemptionDisabled(QUEUE_A_QUEUE_B, true); + conf.setPreemptionDisabled("root.queueD.queueE", true); + conf.setPreemptionDisabled("root.queueA.queueB", true); ProportionalCapacityPreemptionPolicy policy3 = buildPolicy(qData); policy3.editSchedule(); @@ -506,7 +497,7 @@ public void testPerQueueDisablePreemptionInheritParent() { // Turn off preemption for queueA and it's children. queueF(appC)'s request // should starve. setup(); // Call setup() to reset mDisp - conf.setPreemptionDisabled(QUEUE_A, true); + conf.setPreemptionDisabled("root.queueA", true); ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData); policy2.editSchedule(); verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA))); // queueC @@ -530,7 +521,7 @@ public void testPerQueuePreemptionNotAllUntouchable() { { -1, -1, 1, 1, 1, -1, 1, 1, 1 }, // req granularity { 2, 3, 0, 0, 0, 3, 0, 0, 0 }, // subqueues }; - conf.setPreemptionDisabled(QUEUE_A_QUEUE_C, true); + conf.setPreemptionDisabled("root.queueA.queueC", true); ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData); policy.editSchedule(); // Although queueC(appB) is way over capacity and is untouchable, @@ -554,7 +545,7 @@ public void testPerQueueDisablePreemptionRootDisablesAll() { { 3, 2, 0, 0, 2, 0, 0, 2, 0, 0 }, // subqueues }; - conf.setPreemptionDisabled(ROOT, true); + conf.setPreemptionDisabled("root", true); ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData); policy.editSchedule(); // All queues should be non-preemptable, so request should starve. @@ -581,7 +572,7 @@ public void testPerQueueDisablePreemptionOverAbsMaxCapacity() { { 2, 2, 0, 0, 2, 0, 0 }, // subqueues }; // QueueE inherits non-preemption from QueueD - conf.setPreemptionDisabled(QUEUE_D, true); + conf.setPreemptionDisabled("root.queueD", true); ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData); policy.editSchedule(); // appC is running on QueueE. QueueE is over absMaxCap, but is not @@ -1334,7 +1325,7 @@ private boolean mockPreemptionStatus(String queuePathName) { String qName = ""; while(tokenizer.hasMoreTokens()) { qName += tokenizer.nextToken(); - preemptionDisabled = conf.getPreemptionDisabled(new QueuePath(qName), preemptionDisabled); + preemptionDisabled = conf.getPreemptionDisabled(qName, preemptionDisabled); qName += "."; } return preemptionDisabled; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java index 2ddb0d8ae1c51..96e91d5945b7e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java @@ -23,7 +23,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor; import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Before; import org.junit.Test; @@ -367,7 +366,7 @@ public void testNodePartitionDisablePreemptionForSingleLevelQueue() "c\t" // app3 in c + "(1,1,n1,x,20,false)"; // 20x in n1 - conf.setPreemptionDisabled(new QueuePath("root.b"), true); + conf.setPreemptionDisabled("root.b", true); buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig); policy.editSchedule(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java index a11153a0482f5..2fb69f555728e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java @@ -20,7 +20,6 @@ import java.io.IOException; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.junit.Before; import org.junit.Test; @@ -121,15 +120,10 @@ public void testNoIntraQueuePreemptionWithPreemptionDisabledOnQueues() * purpose is to test that disabling preemption on a specific queue will * avoid intra-queue preemption. */ - QueuePath a = new QueuePath("root.a"); - QueuePath b = new QueuePath("root.b"); - QueuePath c = new QueuePath("root.c"); - QueuePath d = new QueuePath("root.d"); - - conf.setPreemptionDisabled(a, true); - conf.setPreemptionDisabled(b, true); - conf.setPreemptionDisabled(c, true); - conf.setPreemptionDisabled(d, true); + conf.setPreemptionDisabled("root.a", true); + conf.setPreemptionDisabled("root.b", true); + conf.setPreemptionDisabled("root.c", true); + conf.setPreemptionDisabled("root.d", true); String labelsConfig = "=100,true;"; String nodesConfig = // n1 has no label diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/MockQueueHierarchy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/MockQueueHierarchy.java index 44075b322d3aa..ae4ff5a663ed2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/MockQueueHierarchy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/MockQueueHierarchy.java @@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.QueueOrderingPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy; @@ -274,7 +273,7 @@ private void setupQueue(CSQueue queue, String q, String[] queueExprArray, // Setup preemption disabled when(queue.getPreemptionDisabled()).thenReturn( - conf.getPreemptionDisabled(new QueuePath(queuePath), false)); + conf.getPreemptionDisabled(queuePath, false)); // Setup other queue configurations Map otherConfigs = getOtherConfigurations( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java index affd984e5334a..0400f7dfcbf0e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java @@ -50,7 +50,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; @@ -668,14 +667,13 @@ private Configuration getConfigurationWithQueueLabels(Configuration config) { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(config); // Define top-level queues - final QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - conf.setQueues(root, new String[] {"a"}); - conf.setCapacityByLabel(root, "x", 100); - - final QueuePath a = root.createNewLeaf("a"); - conf.setCapacity(a, 100); - conf.setAccessibleNodeLabels(a, ImmutableSet.of("x")); - conf.setCapacityByLabel(a, "x", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a" }); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + conf.setCapacity(A, 100); + conf.setAccessibleNodeLabels(A, ImmutableSet.of("x")); + conf.setCapacityByLabel(A, "x", 100); return conf; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java index 7e172aee35308..bf5fb8ca1a4d5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java @@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; @@ -76,18 +75,6 @@ public class ReservationSystemTestUtil { public final static String RESERVATION_Q_SHORT = "dedicated"; public final static String reservationQ = "root." + RESERVATION_Q_SHORT; - public final static String DEDICATED_PATH = CapacitySchedulerConfiguration.ROOT - + CapacitySchedulerConfiguration.DOT + RESERVATION_Q_SHORT; - private final static String DEFAULT_PATH = CapacitySchedulerConfiguration.ROOT + ".default"; - private final static String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - private final static String A1_PATH = A_PATH + ".a1"; - private final static String A2_PATH = A_PATH + ".a2"; - private final static QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private final static QueuePath DEDICATED = new QueuePath(DEDICATED_PATH); - private final static QueuePath DEFAULT = new QueuePath(DEFAULT_PATH); - private final static QueuePath A = new QueuePath(A_PATH); - private final static QueuePath A1 = new QueuePath(A1_PATH); - private final static QueuePath A2 = new QueuePath(A2_PATH); public static ReservationId getNewReservationId() { return ReservationId.newInstance(rand.nextLong(), rand.nextLong()); @@ -100,11 +87,10 @@ public static ReservationSchedulerConfiguration createConf( ReservationSchedulerConfiguration realConf = new CapacitySchedulerConfiguration(); ReservationSchedulerConfiguration conf = spy(realConf); - QueuePath reservationQueuePath = new QueuePath(reservationQ); - when(conf.getReservationWindow(reservationQueuePath)).thenReturn(timeWindow); - when(conf.getInstantaneousMaxCapacity(reservationQueuePath)) + when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow); + when(conf.getInstantaneousMaxCapacity(reservationQ)) .thenReturn(instConstraint); - when(conf.getAverageCapacity(reservationQueuePath)).thenReturn(avgConstraint); + when(conf.getAverageCapacity(reservationQ)).thenReturn(avgConstraint); return conf; } @@ -306,19 +292,25 @@ public Resource answer(InvocationOnMock invocation) throws Throwable { public static void setupQueueConfiguration( CapacitySchedulerConfiguration conf) { // Define default queue - final String defQPath = CapacitySchedulerConfiguration.ROOT + ".default"; - final QueuePath defQ = new QueuePath(defQPath); + final String defQ = CapacitySchedulerConfiguration.ROOT + ".default"; conf.setCapacity(defQ, 10); // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"default", "a", RESERVATION_Q_SHORT}); + + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); - conf.setCapacity(DEDICATED, 80); + + final String dedicated = CapacitySchedulerConfiguration.ROOT + + CapacitySchedulerConfiguration.DOT + RESERVATION_Q_SHORT; + conf.setCapacity(dedicated, 80); // Set as reservation queue - conf.setReservable(DEDICATED, true); + conf.setReservable(dedicated, true); // Define 2nd-level queues + final String A1 = A + ".a1"; + final String A2 = A + ".a2"; conf.setQueues(A, new String[] {"a1", "a2"}); conf.setCapacity(A1, 30); conf.setCapacity(A2, 70); @@ -327,11 +319,13 @@ public static void setupQueueConfiguration( public static void setupDynamicQueueConfiguration( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {RESERVATION_Q_SHORT}); - conf.setCapacity(DEDICATED, 100); + final String dedicated = CapacitySchedulerConfiguration.ROOT + + CapacitySchedulerConfiguration.DOT + RESERVATION_Q_SHORT; + conf.setCapacity(dedicated, 100); // Set as reservation queue - conf.setReservable(DEDICATED, true); + conf.setReservable(dedicated, true); } public static String getFullReservationQueueName() { @@ -339,10 +333,6 @@ public static String getFullReservationQueueName() { + CapacitySchedulerConfiguration.DOT + RESERVATION_Q_SHORT; } - public static QueuePath getFullReservationQueuePath() { - return new QueuePath(getFullReservationQueueName()); - } - public static String getReservationQueueName() { return reservationQ; } @@ -351,23 +341,29 @@ public static void updateQueueConfiguration( CapacitySchedulerConfiguration conf, String newQ) { // Define default queue final String prefix = CapacitySchedulerConfiguration.ROOT - + CapacitySchedulerConfiguration.DOT; - conf.setCapacity(DEFAULT, 5); + + CapacitySchedulerConfiguration.DOT; + final String defQ = prefix + "default"; + conf.setCapacity(defQ, 5); // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"default", "a", RESERVATION_Q_SHORT, newQ}); + + final String A = prefix + "a"; conf.setCapacity(A, 5); - conf.setCapacity(DEDICATED, 10); + + final String dedicated = prefix + RESERVATION_Q_SHORT; + conf.setCapacity(dedicated, 10); // Set as reservation queue - conf.setReservable(DEDICATED, true); + conf.setReservable(dedicated, true); - final QueuePath newQueue = new QueuePath(prefix + newQ); - conf.setCapacity(newQueue, 80); + conf.setCapacity(prefix + newQ, 80); // Set as reservation queue - conf.setReservable(newQueue, true); + conf.setReservable(prefix + newQ, true); // Define 2nd-level queues + final String A1 = A + ".a1"; + final String A2 = A + ".a2"; conf.setQueues(A, new String[] { "a1", "a2" }); conf.setCapacity(A1, 30); conf.setCapacity(A2, 70); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java index 20aa5511f8222..b4ad8be8587f9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java @@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; @@ -129,12 +128,10 @@ private void setupPlanFollower() throws Exception { String reservationQ = ReservationSystemTestUtil.getFullReservationQueueName(); - QueuePath reservationQueuePath = - ReservationSystemTestUtil.getFullReservationQueuePath(); CapacitySchedulerConfiguration csConf = cs.getConfiguration(); - csConf.setReservationWindow(reservationQueuePath, 20L); - csConf.setMaximumCapacity(reservationQueuePath, 40); - csConf.setAverageCapacity(reservationQueuePath, 20); + csConf.setReservationWindow(reservationQ, 20L); + csConf.setMaximumCapacity(reservationQ, 40); + csConf.setAverageCapacity(reservationQ, 20); policy.init(reservationQ, csConf); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java index 1209c7e757ce9..b8521969b343c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java @@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.SharingPolicy; import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; @@ -75,7 +74,7 @@ public void testReplanningPlanCapacityLoss() throws PlanningException { RMContext context = ReservationSystemTestUtil.createMockRMContext(); ReservationSchedulerConfiguration conf = mock(ReservationSchedulerConfiguration.class); - when(conf.getEnforcementWindow(any(QueuePath.class))).thenReturn(6L); + when(conf.getEnforcementWindow(any(String.class))).thenReturn(6L); enf.init("blah", conf); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java index 569919c01c4fa..2e003151e79a2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java @@ -60,7 +60,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.util.Times; import org.slf4j.event.Level; @@ -76,10 +75,6 @@ @RunWith(Parameterized.class) public class TestApplicationLifetimeMonitor { private final long maxLifetime = 30L; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final String CQ1 = "child1"; - private static final QueuePath CQ1_QUEUE_PATH = ROOT.createNewLeaf(CQ1); - private static final QueuePath DEFAULT_QUEUE_PATH = ROOT.createNewLeaf("default"); private YarnConfiguration conf; @@ -395,6 +390,7 @@ public synchronized void updateApplicationStateInternal( } } + static final String CQ1 = "child1"; @Test(timeout = 120000) public void testInheritAppLifetimeFromParentQueue() throws Exception { YarnConfiguration yarnConf = conf; @@ -403,10 +399,13 @@ public void testInheritAppLifetimeFromParentQueue() throws Exception { if (scheduler.equals(CapacityScheduler.class)) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] {CQ1}); - csConf.setCapacity(CQ1_QUEUE_PATH, 100); - csConf.setMaximumLifetimePerQueue(ROOT, maxRootLifetime); - csConf.setDefaultLifetimePerQueue(ROOT, defaultRootLifetime); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {CQ1}); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + CQ1, 100); + csConf.setMaximumLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT, maxRootLifetime); + csConf.setDefaultLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT, defaultRootLifetime); yarnConf = new YarnConfiguration(csConf); } @@ -466,12 +465,17 @@ public void testOverrideParentQueueMaxAppLifetime() throws Exception { if (scheduler.equals(CapacityScheduler.class)) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] {CQ1}); - csConf.setCapacity(CQ1_QUEUE_PATH, 100); - csConf.setMaximumLifetimePerQueue(ROOT, maxRootLifetime); - csConf.setMaximumLifetimePerQueue(CQ1_QUEUE_PATH, maxChildLifetime); - csConf.setDefaultLifetimePerQueue(ROOT, defaultRootLifetime); - csConf.setDefaultLifetimePerQueue(CQ1_QUEUE_PATH, maxChildLifetime); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {CQ1}); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + CQ1, 100); + csConf.setMaximumLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT, maxRootLifetime); + csConf.setMaximumLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT + "." + CQ1, maxChildLifetime); + csConf.setDefaultLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT, defaultRootLifetime); + csConf.setDefaultLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT + "." + CQ1, maxChildLifetime); yarnConf = new YarnConfiguration(csConf); } @@ -525,11 +529,16 @@ public void testOverrideParentQueueDefaultAppLifetime() throws Exception { if (scheduler.equals(CapacityScheduler.class)) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] {CQ1}); - csConf.setCapacity(CQ1_QUEUE_PATH, 100); - csConf.setMaximumLifetimePerQueue(ROOT, maxRootLifetime); - csConf.setMaximumLifetimePerQueue(CQ1_QUEUE_PATH, maxChildLifetime); - csConf.setDefaultLifetimePerQueue(CQ1_QUEUE_PATH, defaultChildLifetime); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {CQ1}); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + CQ1, 100); + csConf.setMaximumLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT, maxRootLifetime); + csConf.setMaximumLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT + "." + CQ1, maxChildLifetime); + csConf.setDefaultLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT + "." + CQ1, + defaultChildLifetime); yarnConf = new YarnConfiguration(csConf); } @@ -583,11 +592,13 @@ private CapacitySchedulerConfiguration setUpCSQueue(long maxLifetime, long defaultLifetime) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"default"}); - csConf.setCapacity(DEFAULT_QUEUE_PATH, 100); - csConf.setMaximumLifetimePerQueue(DEFAULT_QUEUE_PATH, maxLifetime); - csConf.setDefaultLifetimePerQueue(DEFAULT_QUEUE_PATH, defaultLifetime); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".default", 100); + csConf.setMaximumLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT + ".default", maxLifetime); + csConf.setDefaultLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT + ".default", defaultLifetime); return csConf; } 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/CapacitySchedulerConfigGeneratorForTest.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/CapacitySchedulerConfigGeneratorForTest.java index db033a4be1de5..087b797f2d84c 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/CapacitySchedulerConfigGeneratorForTest.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/CapacitySchedulerConfigGeneratorForTest.java @@ -65,8 +65,8 @@ public static void setMaxAllocMb(Configuration conf, int maxAllocMb) { } public static void setMaxAllocMb(CapacitySchedulerConfiguration conf, - QueuePath queuePath, int maxAllocMb) { - String propName = QueuePrefixes.getQueuePrefix(queuePath) + String queueName, int maxAllocMb) { + String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName) + MAXIMUM_ALLOCATION_MB; conf.setInt(propName, maxAllocMb); } @@ -82,22 +82,22 @@ public static void setMaxAllocVcores(Configuration conf, int maxAllocVcores) { } public static void setMaxAllocVcores(CapacitySchedulerConfiguration conf, - QueuePath queuePath, int maxAllocVcores) { - String propName = QueuePrefixes.getQueuePrefix(queuePath) + String queueName, int maxAllocVcores) { + String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName) + CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_VCORES; conf.setInt(propName, maxAllocVcores); } public static void setMaxAllocation(CapacitySchedulerConfiguration conf, - QueuePath queuePath, String maxAllocation) { - String propName = QueuePrefixes.getQueuePrefix(queuePath) + String queueName, String maxAllocation) { + String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName) + MAXIMUM_ALLOCATION; conf.set(propName, maxAllocation); } public static void unsetMaxAllocation(CapacitySchedulerConfiguration conf, - QueuePath queuePath) { - String propName = QueuePrefixes.getQueuePrefix(queuePath) + String queueName) { + String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName) + MAXIMUM_ALLOCATION; conf.unset(propName); } 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/CapacitySchedulerQueueCalculationTestBase.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/CapacitySchedulerQueueCalculationTestBase.java index 0f539f8b74e63..e61412d71c9d5 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/CapacitySchedulerQueueCalculationTestBase.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/CapacitySchedulerQueueCalculationTestBase.java @@ -33,15 +33,14 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; public class CapacitySchedulerQueueCalculationTestBase { - protected static final QueuePath ROOT = new QueuePath("root"); - protected static final QueuePath A = new QueuePath("root.a"); - protected static final QueuePath A1 = new QueuePath("root.a.a1"); - protected static final QueuePath A11 = new QueuePath("root.a.a1.a11"); - protected static final QueuePath A12 = new QueuePath("root.a.a1.a12"); - protected static final QueuePath A2 = new QueuePath("root.a.a2"); - protected static final QueuePath B = new QueuePath("root.b"); - protected static final QueuePath B1 = new QueuePath("root.b.b1"); - protected static final QueuePath C = new QueuePath("root.c"); + protected static final String A = "root.a"; + protected static final String A1 = "root.a.a1"; + protected static final String A11 = "root.a.a1.a11"; + protected static final String A12 = "root.a.a1.a12"; + protected static final String A2 = "root.a.a2"; + protected static final String B = "root.b"; + protected static final String B1 = "root.b.b1"; + protected static final String C = "root.c"; private static final String CAPACITY_VECTOR_TEMPLATE = "[memory=%s, vcores=%s]"; @@ -58,14 +57,14 @@ public void setUp() throws Exception { csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); - csConf.setQueues(ROOT, new String[]{"a", "b"}); - csConf.setCapacity(A, 50f); - csConf.setCapacity(B, 50f); - csConf.setQueues(A, new String[]{"a1", "a2"}); - csConf.setCapacity(A1, 100f); - csConf.setQueues(A1, new String[]{"a11", "a12"}); - csConf.setCapacity(A11, 50f); - csConf.setCapacity(A12, 50f); + csConf.setQueues("root", new String[]{"a", "b"}); + csConf.setCapacity("root.a", 50f); + csConf.setCapacity("root.b", 50f); + csConf.setQueues("root.a", new String[]{"a1", "a2"}); + csConf.setCapacity("root.a.a1", 100f); + csConf.setQueues("root.a.a1", new String[]{"a11", "a12"}); + csConf.setCapacity("root.a.a1.a11", 50f); + csConf.setCapacity("root.a.a1.a12", 50f); mgr = new NullRMNodeLabelsManager(); mgr.init(csConf); 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/CapacitySchedulerQueueHelpers.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/CapacitySchedulerQueueHelpers.java index 757c86de305d1..1e11d3b417831 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/CapacitySchedulerQueueHelpers.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/CapacitySchedulerQueueHelpers.java @@ -27,43 +27,22 @@ public final class CapacitySchedulerQueueHelpers { - public static final String DEFAULT_PATH = CapacitySchedulerConfiguration.ROOT + ".default"; - public static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - public static final String B_PATH = CapacitySchedulerConfiguration.ROOT + ".b"; - public static final String A_CHILD_PATH = A_PATH + ".a"; - public static final String A1_PATH = A_PATH + ".a1"; - public static final String A2_PATH = A_PATH + ".a2"; - public static final String A3_PATH = A_PATH + ".a3"; - public static final String B1_PATH = B_PATH + ".b1"; - public static final String B2_PATH = B_PATH + ".b2"; - public static final String B3_PATH = B_PATH + ".b3"; - public static final String A1_B1_PATH = A1_PATH + ".b1"; - public static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - public static final QueuePath DEFAULT = new QueuePath(DEFAULT_PATH); - public static final QueuePath A = new QueuePath(A_PATH); - public static final QueuePath A_CHILD = new QueuePath(A_CHILD_PATH); - public static final QueuePath A1 = new QueuePath(A1_PATH); - public static final QueuePath A2 = new QueuePath(A2_PATH); - public static final QueuePath A3 = new QueuePath(A3_PATH); - public static final QueuePath B = new QueuePath(B_PATH); - public static final QueuePath B1 = new QueuePath(B1_PATH); - public static final QueuePath B2 = new QueuePath(B2_PATH); - public static final QueuePath B3 = new QueuePath(B3_PATH); - public static final QueuePath A1_B1 = new QueuePath(A1_B1_PATH); + public static final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + public static final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + public static final String A_CHILD = A + ".a"; + public static final String A1 = A + ".a1"; + public static final String A2 = A + ".a2"; + public static final String B1 = B + ".b1"; + public static final String B2 = B + ".b2"; + public static final String B3 = B + ".b3"; public static final float A_CAPACITY = 10.5f; public static final float B_CAPACITY = 89.5f; - public static final String P1_PATH = CapacitySchedulerConfiguration.ROOT + ".p1"; - public static final String P2_PATH = CapacitySchedulerConfiguration.ROOT + ".p2"; - public static final String X1_PATH = P1_PATH + ".x1"; - public static final String X2_PATH = P1_PATH + ".x2"; - public static final String Y1_PATH = P2_PATH + ".y1"; - public static final String Y2_PATH = P2_PATH + ".y2"; - public static final QueuePath P1 = new QueuePath(P1_PATH); - public static final QueuePath P2 = new QueuePath(P2_PATH); - public static final QueuePath X1 = new QueuePath(X1_PATH); - public static final QueuePath X2 = new QueuePath(X2_PATH); - public static final QueuePath Y1 = new QueuePath(Y1_PATH); - public static final QueuePath Y2 = new QueuePath(Y2_PATH); + public static final String P1 = CapacitySchedulerConfiguration.ROOT + ".p1"; + public static final String P2 = CapacitySchedulerConfiguration.ROOT + ".p2"; + public static final String X1 = P1 + ".x1"; + public static final String X2 = P1 + ".x2"; + public static final String Y1 = P2 + ".y1"; + public static final String Y2 = P2 + ".y2"; public static final float A1_CAPACITY = 30; public static final float A2_CAPACITY = 70; public static final float B1_CAPACITY = 79.2f; @@ -88,7 +67,7 @@ public static CapacitySchedulerConfiguration setupQueueConfiguration( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[]{"a", "b"}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); conf.setCapacity(A, A_CAPACITY); conf.setCapacity(B, B_CAPACITY); @@ -115,7 +94,7 @@ public static CapacitySchedulerConfiguration setupAdditionalQueues( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[]{"a", "b"}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); conf.setCapacity(A, A_CAPACITY); conf.setCapacity(B, B_CAPACITY); @@ -126,8 +105,8 @@ public static CapacitySchedulerConfiguration setupAdditionalQueues( conf.setUserLimitFactor(A1, 100.0f); conf.setCapacity(A2, 30.0f); conf.setUserLimitFactor(A2, 100.0f); - conf.setCapacity(A3, 40.0f); - conf.setUserLimitFactor(A3, 100.0f); + conf.setCapacity("root.a.a3", 40.0f); + conf.setUserLimitFactor("root.a.a3", 100.0f); conf.setQueues(B, new String[]{"b1", "b2", "b3"}); conf.setCapacity(B1, B1_CAPACITY); @@ -153,7 +132,7 @@ public static CapacitySchedulerConfiguration setupQueueConfAmbiguousQueue( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); conf.setCapacity(A, A_CAPACITY); @@ -182,7 +161,7 @@ public static CapacitySchedulerConfiguration setupQueueConfWithoutChildrenOfB( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); conf.setCapacity(A, A_CAPACITY); @@ -211,7 +190,7 @@ public static CapacitySchedulerConfiguration setupQueueConfigurationWithoutB1( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); conf.setCapacity(A, A_CAPACITY); @@ -248,7 +227,7 @@ public static CapacitySchedulerConfiguration setupQueueConfigurationWithB1AsPare CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); conf.setCapacity(A, A_CAPACITY); @@ -271,8 +250,7 @@ public static CapacitySchedulerConfiguration setupQueueConfigurationWithB1AsPare // Set childQueue for B1 conf.setQueues(B1, new String[]{"b11"}); - final String b11Path = B1 + ".b11"; - final QueuePath b11 = new QueuePath(b11Path); + final String b11 = B1 + ".b11"; conf.setCapacity(b11, 100.0f); conf.setUserLimitFactor(b11, 100.0f); @@ -287,7 +265,7 @@ public static CapacitySchedulerConfiguration setupQueueConfigurationWithoutB( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[]{"a"}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a"}); conf.setCapacity(A, A_CAPACITY + B_CAPACITY); @@ -305,7 +283,7 @@ public static CapacitySchedulerConfiguration setupBlockedQueueConfiguration( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); conf.setCapacity(A, 80f); @@ -321,7 +299,7 @@ public static CapacitySchedulerConfiguration setupOtherBlockedQueueConfiguration CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"p1", "p2"}); conf.setCapacity(P1, 50f); @@ -365,13 +343,13 @@ public float getAbsCapacity() { public static Map getDefaultCapacities(float capA, float capB) { Map capacities = new HashMap<>(); - capacities.put(A.getFullPath(), new ExpectedCapacities(capA, 1.0f)); - capacities.put(B.getFullPath(), new ExpectedCapacities(capB, 1.0f)); - capacities.put(A1.getFullPath(), new ExpectedCapacities((A1_CAPACITY / 100.0f), capA)); - capacities.put(A2.getFullPath(), new ExpectedCapacities((A2_CAPACITY / 100.0f), capA)); - capacities.put(B1.getFullPath(), new ExpectedCapacities((B1_CAPACITY / 100.0f), capB)); - capacities.put(B2.getFullPath(), new ExpectedCapacities((B2_CAPACITY / 100.0f), capB)); - capacities.put(B3.getFullPath(), new ExpectedCapacities((B3_CAPACITY / 100.0f), capB)); + capacities.put(A, new ExpectedCapacities(capA, 1.0f)); + capacities.put(B, new ExpectedCapacities(capB, 1.0f)); + capacities.put(A1, new ExpectedCapacities((A1_CAPACITY / 100.0f), capA)); + capacities.put(A2, new ExpectedCapacities((A2_CAPACITY / 100.0f), capA)); + capacities.put(B1, new ExpectedCapacities((B1_CAPACITY / 100.0f), capB)); + capacities.put(B2, new ExpectedCapacities((B2_CAPACITY / 100.0f), capB)); + capacities.put(B3, new ExpectedCapacities((B3_CAPACITY / 100.0f), capB)); return capacities; } 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/QueueAssertionBuilder.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/QueueAssertionBuilder.java index 329d8b1bedbd2..1c066719dd090 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/QueueAssertionBuilder.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/QueueAssertionBuilder.java @@ -64,15 +64,15 @@ class QueueAssertionBuilder { } public class QueueAssertion { - private final QueuePath queuePath; + private final String queuePath; private final List assertions = new ArrayList<>(); - QueueAssertion(QueuePath queuePath) { + QueueAssertion(String queuePath) { this.queuePath = queuePath; } - public QueueAssertion withQueue(QueuePath queuePath) { + public QueueAssertion withQueue(String queuePath) { return QueueAssertionBuilder.this.withQueue(queuePath); } @@ -139,7 +139,7 @@ public void setLabel(String label) { public void withResourceSupplier( BiFunction assertion, String messageInfo) { - CSQueue queue = cs.getQueue(queuePath.getFullPath()); + CSQueue queue = cs.getQueue(queuePath); if (queue == null) { Assert.fail("Queue " + queuePath + " is not found"); } @@ -150,7 +150,7 @@ public void withResourceSupplier( public void withCapacitySupplier( BiFunction assertion, String messageInfo) { - CSQueue queue = cs.getQueue(queuePath.getFullPath()); + CSQueue queue = cs.getQueue(queuePath); if (queue == null) { Assert.fail("Queue " + queuePath + " is not found"); } @@ -161,7 +161,7 @@ public void withCapacitySupplier( } - private final Map assertions = new LinkedHashMap<>(); + private final Map assertions = new LinkedHashMap<>(); public QueueAssertionBuilder build() { return this; @@ -172,7 +172,7 @@ public QueueAssertionBuilder build() { * @param queuePath path of the queue * @return queue assertion group */ - public QueueAssertion withQueue(QueuePath queuePath) { + public QueueAssertion withQueue(String queuePath) { assertions.putIfAbsent(queuePath, new QueueAssertion(queuePath)); return assertions.get(queuePath); } @@ -181,7 +181,7 @@ public QueueAssertion withQueue(QueuePath queuePath) { * Executes assertions created for all queues. */ public void finishAssertion() { - for (Map.Entry assertionEntry : assertions.entrySet()) { + for (Map.Entry assertionEntry : assertions.entrySet()) { for (QueueAssertion.ValueAssertion assertion : assertionEntry.getValue().assertions) { if (assertion.resourceSupplier != null) { String errorMessage = String.format(RESOURCE_ASSERTION_ERROR_MESSAGE, @@ -204,7 +204,7 @@ public void finishAssertion() { * Returns all queues that have defined assertions. * @return queue paths */ - public Set getQueues() { + public Set getQueues() { return assertions.keySet(); } } 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 2443f723653cc..09b52d3b246dc 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 @@ -48,8 +48,6 @@ public class TestAbsoluteResourceConfiguration { private static final String QUEUEA2 = "queueA2"; private static final String QUEUEB1 = "queueB1"; - private static final QueuePath ROOT = - new QueuePath(CapacitySchedulerConfiguration.ROOT); private static final QueuePath QUEUEA_FULL = new QueuePath(CapacitySchedulerConfiguration.ROOT, QUEUEA); private static final QueuePath QUEUEB_FULL = @@ -105,9 +103,9 @@ public class TestAbsoluteResourceConfiguration { private CapacitySchedulerConfiguration setupNormalizationConfiguration() { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUEA, QUEUEB}); - csConf.setQueues(QUEUEA_FULL, new String[]{QUEUEA1, QUEUEA2}); + csConf.setQueues(QUEUEA_FULL.getFullPath(), new String[]{QUEUEA1, QUEUEA2}); // 60, 28 csConf.setMinimumResourceRequirement("", QUEUEA_FULL, Resource.newInstance(50 * GB, 20)); @@ -121,23 +119,23 @@ private CapacitySchedulerConfiguration setupNormalizationConfiguration() { private CapacitySchedulerConfiguration setupSimpleQueueConfiguration( boolean isCapacityNeeded) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUEA, QUEUEB, QUEUEC, QUEUED}); // Set default capacities like normal configuration. if (isCapacityNeeded) { - csConf.setCapacity(QUEUEA_FULL, 50f); - csConf.setCapacity(QUEUEB_FULL, 25f); - csConf.setCapacity(QUEUEC_FULL, 25f); - csConf.setCapacity(QUEUED_FULL, 25f); + csConf.setCapacity(QUEUEA_FULL.getFullPath(), 50f); + csConf.setCapacity(QUEUEB_FULL.getFullPath(), 25f); + csConf.setCapacity(QUEUEC_FULL.getFullPath(), 25f); + csConf.setCapacity(QUEUED_FULL.getFullPath(), 25f); } - csConf.setAutoCreateChildQueueEnabled(QUEUED_FULL, true); + csConf.setAutoCreateChildQueueEnabled(QUEUED_FULL.getFullPath(), true); // Setup leaf queue template configs - csConf.setAutoCreatedLeafQueueTemplateCapacityByLabel(QUEUED_FULL, "", + csConf.setAutoCreatedLeafQueueTemplateCapacityByLabel(QUEUED_FULL.getFullPath(), "", QUEUE_D_TEMPL_MINRES); - csConf.setAutoCreatedLeafQueueTemplateMaxCapacity(QUEUED_FULL, "", + csConf.setAutoCreatedLeafQueueTemplateMaxCapacity(QUEUED_FULL.getFullPath(), "", QUEUE_D_TEMPL_MAXRES); return csConf; @@ -146,19 +144,19 @@ private CapacitySchedulerConfiguration setupSimpleQueueConfiguration( private CapacitySchedulerConfiguration setupComplexQueueConfiguration( boolean isCapacityNeeded) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUEA, QUEUEB, QUEUEC}); - csConf.setQueues(QUEUEA_FULL, new String[]{QUEUEA1, QUEUEA2}); - csConf.setQueues(QUEUEB_FULL, new String[]{QUEUEB1}); + csConf.setQueues(QUEUEA_FULL.getFullPath(), new String[]{QUEUEA1, QUEUEA2}); + csConf.setQueues(QUEUEB_FULL.getFullPath(), new String[]{QUEUEB1}); // Set default capacities like normal configuration. if (isCapacityNeeded) { - csConf.setCapacity(QUEUEA_FULL, 50f); - csConf.setCapacity(QUEUEB_FULL, 25f); - csConf.setCapacity(QUEUEC_FULL, 25f); - csConf.setCapacity(QUEUEA1_FULL, 50f); - csConf.setCapacity(QUEUEA2_FULL, 50f); - csConf.setCapacity(QUEUEB1_FULL, 100f); + csConf.setCapacity(QUEUEA_FULL.getFullPath(), 50f); + csConf.setCapacity(QUEUEB_FULL.getFullPath(), 25f); + csConf.setCapacity(QUEUEC_FULL.getFullPath(), 25f); + csConf.setCapacity(QUEUEA1_FULL.getFullPath(), 50f); + csConf.setCapacity(QUEUEA2_FULL.getFullPath(), 50f); + csConf.setCapacity(QUEUEB1_FULL.getFullPath(), 100f); } return csConf; @@ -227,22 +225,22 @@ public void testSimpleMinMaxResourceConfigurartionPerQueue() Assert.assertEquals("Min resource configured for QUEUEA is not correct", QUEUE_A_MINRES, - csConf.getMinimumResourceRequirement("", QUEUEA_FULL, resourceTypes)); + csConf.getMinimumResourceRequirement("", QUEUEA_FULL.getFullPath(), resourceTypes)); Assert.assertEquals("Max resource configured for QUEUEA is not correct", QUEUE_A_MAXRES, - csConf.getMaximumResourceRequirement("", QUEUEA_FULL, resourceTypes)); + csConf.getMaximumResourceRequirement("", QUEUEA_FULL.getFullPath(), resourceTypes)); Assert.assertEquals("Min resource configured for QUEUEB is not correct", QUEUE_B_MINRES, - csConf.getMinimumResourceRequirement("", QUEUEB_FULL, resourceTypes)); + csConf.getMinimumResourceRequirement("", QUEUEB_FULL.getFullPath(), resourceTypes)); Assert.assertEquals("Max resource configured for QUEUEB is not correct", QUEUE_B_MAXRES, - csConf.getMaximumResourceRequirement("", QUEUEB_FULL, resourceTypes)); + csConf.getMaximumResourceRequirement("", QUEUEB_FULL.getFullPath(), resourceTypes)); Assert.assertEquals("Min resource configured for QUEUEC is not correct", QUEUE_C_MINRES, - csConf.getMinimumResourceRequirement("", QUEUEC_FULL, resourceTypes)); + csConf.getMinimumResourceRequirement("", QUEUEC_FULL.getFullPath(), resourceTypes)); Assert.assertEquals("Max resource configured for QUEUEC is not correct", QUEUE_C_MAXRES, - csConf.getMaximumResourceRequirement("", QUEUEC_FULL, resourceTypes)); + csConf.getMaximumResourceRequirement("", QUEUEC_FULL.getFullPath(), resourceTypes)); csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); @@ -566,9 +564,9 @@ public void testComplexValidateAbsoluteResourceConfig() throws Exception { // 1. Explicitly set percentage based config for parent queues. This will // make Queue A,B and C with percentage based and A1,A2 or B1 with absolute // resource. - csConf.setCapacity(QUEUEA_FULL, 50f); - csConf.setCapacity(QUEUEB_FULL, 25f); - csConf.setCapacity(QUEUEC_FULL, 25f); + csConf.setCapacity(QUEUEA_FULL.getFullPath(), 50f); + csConf.setCapacity(QUEUEB_FULL.getFullPath(), 25f); + csConf.setCapacity(QUEUEC_FULL.getFullPath(), 25f); // Get queue object to verify min/max resource configuration. CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); @@ -620,15 +618,15 @@ public void testValidateAbsoluteResourceConfig() throws Exception { // create conf with basic queue configuration. CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {QUEUEA, QUEUEB}); - csConf.setQueues(QUEUEA_FULL, new String[] {QUEUEA1, QUEUEA2}); + csConf.setQueues(QUEUEA_FULL.getFullPath(), new String[] {QUEUEA1, QUEUEA2}); // Set default capacities like normal configuration. - csConf.setCapacity(QUEUEA_FULL, "[memory=125]"); - csConf.setCapacity(QUEUEB_FULL, "[memory=0]"); - csConf.setCapacity(QUEUEA1_FULL, "[memory=100]"); - csConf.setCapacity(QUEUEA2_FULL, "[memory=25]"); + csConf.setCapacity(QUEUEA_FULL.getFullPath(), "[memory=125]"); + csConf.setCapacity(QUEUEB_FULL.getFullPath(), "[memory=0]"); + csConf.setCapacity(QUEUEA1_FULL.getFullPath(), "[memory=100]"); + csConf.setCapacity(QUEUEA2_FULL.getFullPath(), "[memory=25]"); // Update min/max resource to queueA csConf.setMinimumResourceRequirement("", QUEUEA_FULL, QUEUE_A_MINRES); @@ -648,8 +646,8 @@ public void testValidateAbsoluteResourceConfig() throws Exception { // doesnt throw exception saying "Parent queue 'root.A' and // child queue 'root.A.A2' should use either percentage // based capacityconfiguration or absolute resource together for label" - csConf.setCapacity(QUEUEA1_FULL, "[memory=125]"); - csConf.setCapacity(QUEUEA2_FULL, "[memory=0]"); + csConf.setCapacity(QUEUEA1_FULL.getFullPath(), "[memory=125]"); + csConf.setCapacity(QUEUEA2_FULL.getFullPath(), "[memory=0]"); // Get queue object to verify min/max resource configuration. CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); 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/TestAbsoluteResourceWithAutoQueue.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/TestAbsoluteResourceWithAutoQueue.java index 66ab96308f982..54fb966ded84e 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/TestAbsoluteResourceWithAutoQueue.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/TestAbsoluteResourceWithAutoQueue.java @@ -71,13 +71,6 @@ public class TestAbsoluteResourceWithAutoQueue private static final String QUEUED_FULL = CapacitySchedulerConfiguration.ROOT + "." + QUEUED; - private static final QueuePath ROOT_QUEUE_PATH = - new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A_QUEUE_PATH = new QueuePath(QUEUEA_FULL); - private static final QueuePath B_QUEUE_PATH = new QueuePath(QUEUEB_FULL); - private static final QueuePath C_QUEUE_PATH = new QueuePath(QUEUEC_FULL); - private static final QueuePath D_QUEUE_PATH = new QueuePath(QUEUED_FULL); - private static final Resource QUEUE_A_MINRES = Resource.newInstance(100 * GB, 10); private static final Resource QUEUE_A_MAXRES = @@ -111,15 +104,15 @@ public void tearDown() { private CapacitySchedulerConfiguration setupMinMaxResourceConfiguration( CapacitySchedulerConfiguration csConf) { // Update min/max resource to queueA/B/C - csConf.setMinimumResourceRequirement("", A_QUEUE_PATH, QUEUE_A_MINRES); - csConf.setMinimumResourceRequirement("", B_QUEUE_PATH, QUEUE_B_MINRES); - csConf.setMinimumResourceRequirement("", C_QUEUE_PATH, QUEUE_C_MINRES); - csConf.setMinimumResourceRequirement("", D_QUEUE_PATH, QUEUE_D_MINRES); + csConf.setMinimumResourceRequirement("", new QueuePath(QUEUEA_FULL), QUEUE_A_MINRES); + csConf.setMinimumResourceRequirement("", new QueuePath(QUEUEB_FULL), QUEUE_B_MINRES); + csConf.setMinimumResourceRequirement("", new QueuePath(QUEUEC_FULL), QUEUE_C_MINRES); + csConf.setMinimumResourceRequirement("", new QueuePath(QUEUED_FULL), QUEUE_D_MINRES); - csConf.setMaximumResourceRequirement("", A_QUEUE_PATH, QUEUE_A_MAXRES); - csConf.setMaximumResourceRequirement("", B_QUEUE_PATH, QUEUE_B_MAXRES); - csConf.setMaximumResourceRequirement("", C_QUEUE_PATH, QUEUE_C_MAXRES); - csConf.setMaximumResourceRequirement("", D_QUEUE_PATH, QUEUE_D_MAXRES); + csConf.setMaximumResourceRequirement("", new QueuePath(QUEUEA_FULL), QUEUE_A_MAXRES); + csConf.setMaximumResourceRequirement("", new QueuePath(QUEUEB_FULL), QUEUE_B_MAXRES); + csConf.setMaximumResourceRequirement("", new QueuePath(QUEUEC_FULL), QUEUE_C_MAXRES); + csConf.setMaximumResourceRequirement("", new QueuePath(QUEUED_FULL), QUEUE_D_MAXRES); return csConf; } @@ -134,30 +127,30 @@ private CapacitySchedulerConfiguration setupSimpleQueueConfiguration( boolean isCapacityNeeded) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT_QUEUE_PATH, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { QUEUEA, QUEUEB, QUEUEC, QUEUED }); // Set default capacities like normal configuration. if (isCapacityNeeded) { - csConf.setCapacity(A_QUEUE_PATH, 50f); - csConf.setCapacity(B_QUEUE_PATH, 25f); - csConf.setCapacity(C_QUEUE_PATH, 25f); - csConf.setCapacity(D_QUEUE_PATH, 25f); + csConf.setCapacity(QUEUEA_FULL, 50f); + csConf.setCapacity(QUEUEB_FULL, 25f); + csConf.setCapacity(QUEUEC_FULL, 25f); + csConf.setCapacity(QUEUED_FULL, 25f); } - csConf.setAutoCreateChildQueueEnabled(C_QUEUE_PATH, true); + csConf.setAutoCreateChildQueueEnabled(QUEUEC_FULL, true); - csConf.setAutoCreatedLeafQueueTemplateCapacityByLabel(C_QUEUE_PATH, "", + csConf.setAutoCreatedLeafQueueTemplateCapacityByLabel(QUEUEC_FULL, "", QUEUE_C_MINRES); - csConf.setAutoCreatedLeafQueueTemplateMaxCapacity(C_QUEUE_PATH, "", + csConf.setAutoCreatedLeafQueueTemplateMaxCapacity(QUEUEC_FULL, "", QUEUE_C_MAXRES); - csConf.setAutoCreateChildQueueEnabled(D_QUEUE_PATH, true); + csConf.setAutoCreateChildQueueEnabled(QUEUED_FULL, true); // Setup leaf queue template configs - csConf.setAutoCreatedLeafQueueTemplateCapacityByLabel(D_QUEUE_PATH, "", + csConf.setAutoCreatedLeafQueueTemplateCapacityByLabel(QUEUED_FULL, "", Resource.newInstance(10 * GB, 2)); - csConf.setAutoCreatedLeafQueueTemplateMaxCapacity(D_QUEUE_PATH, "", + csConf.setAutoCreatedLeafQueueTemplateMaxCapacity(QUEUED_FULL, "", QUEUE_D_MAXRES); return csConf; 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 ed5ff571e028a..0df032062580d 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 @@ -145,33 +145,22 @@ public void setUp() throws IOException { doReturn(100).when(queue).getMaxApplications(); doReturn(25).when(queue).getMaxApplicationsPerUser(); } - + private static final String A = "a"; private static final String B = "b"; - private static final String C = "c"; - private static final String D = "d"; - private static final String AA1 = "a1"; - private static final String AA2 = "a2"; - private static final String AA3 = "a3"; - private static final QueuePath ROOT_QUEUE_PATH = - new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A_QUEUE_PATH = ROOT_QUEUE_PATH.createNewLeaf(A); - private static final QueuePath B_QUEUE_PATH = ROOT_QUEUE_PATH.createNewLeaf(B); - private static final QueuePath C_QUEUE_PATH = ROOT_QUEUE_PATH.createNewLeaf(C); - private static final QueuePath D_QUEUE_PATH = ROOT_QUEUE_PATH.createNewLeaf(D); - private static final QueuePath AA1_QUEUE_PATH = A_QUEUE_PATH.createNewLeaf(AA1); - private static final QueuePath AA2_QUEUE_PATH = A_QUEUE_PATH.createNewLeaf(AA2); - private static final QueuePath AA3_QUEUE_PATH = A_QUEUE_PATH.createNewLeaf(AA3); private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT_QUEUE_PATH, new String[] {A, B}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {A, B}); - conf.setCapacity(A_QUEUE_PATH, 10); - conf.setCapacity(B_QUEUE_PATH, 90); + final String Q_A = CapacitySchedulerConfiguration.ROOT + "." + A; + conf.setCapacity(Q_A, 10); + + final String Q_B = CapacitySchedulerConfiguration.ROOT + "." + B; + conf.setCapacity(Q_B, 90); - conf.setUserLimit(A_QUEUE_PATH, 50); - conf.setUserLimitFactor(A_QUEUE_PATH, 5.0f); + conf.setUserLimit(CapacitySchedulerConfiguration.ROOT + "." + A, 50); + conf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + "." + A, 5.0f); LOG.info("Setup top-level queues a and b"); } @@ -347,7 +336,7 @@ public void testLimitsComputation() throws Exception { // should return -1 if per queue setting not set assertEquals( (int)CapacitySchedulerConfiguration.UNDEFINED, - csConf.getMaximumApplicationsPerQueue(queue.getQueuePathObject())); + csConf.getMaximumApplicationsPerQueue(queue.getQueuePath())); int expectedMaxApps = (int) (CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS * @@ -362,7 +351,7 @@ public void testLimitsComputation() throws Exception { // should default to global setting if per queue setting not set assertEquals(CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT, csConf.getMaximumApplicationMasterResourcePerQueuePercent( - queue.getQueuePathObject()), epsilon); + queue.getQueuePath()), epsilon); // Change the per-queue max AM resources percentage. csConf.setFloat(PREFIX + queue.getQueuePath() @@ -382,7 +371,7 @@ public void testLimitsComputation() throws Exception { assertEquals(0.5f, csConf.getMaximumApplicationMasterResourcePerQueuePercent( - queue.getQueuePathObject()), epsilon); + queue.getQueuePath()), epsilon); assertThat(queue.calculateAndGetAMResourceLimit()).isEqualTo( Resource.newInstance(800 * GB, 1)); @@ -402,8 +391,7 @@ public void testLimitsComputation() throws Exception { clusterResource)); queue = (LeafQueue)queues.get(A); - assertEquals(9999, (int)csConf.getMaximumApplicationsPerQueue( - queue.getQueuePathObject())); + assertEquals(9999, (int)csConf.getMaximumApplicationsPerQueue(queue.getQueuePath())); assertEquals(9999, queue.getMaxApplications()); expectedMaxAppsPerUser = Math.min(9999, (int)(9999 * @@ -588,7 +576,7 @@ public void testActiveLimitsWithKilledApps() throws Exception { public void testHeadroom() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setUserLimit(A_QUEUE_PATH, 25); + csConf.setUserLimit(CapacitySchedulerConfiguration.ROOT + "." + A, 25); setupQueueConfiguration(csConf); // Say cluster has 100 nodes of 16G each @@ -733,46 +721,53 @@ private Configuration getConfigurationWithQueueLabels(Configuration config) { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(config); // Define top-level - conf.setQueues(ROOT_QUEUE_PATH, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b", "c", "d"}); - conf.setCapacityByLabel(ROOT_QUEUE_PATH, "x", 100); - conf.setCapacityByLabel(ROOT_QUEUE_PATH, "y", 100); - conf.setCapacityByLabel(ROOT_QUEUE_PATH, "z", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100); conf.setInt(CapacitySchedulerConfiguration.QUEUE_GLOBAL_MAX_APPLICATION, 20); conf.setInt("yarn.scheduler.capacity.root.a.a1.maximum-applications", 1); conf.setFloat("yarn.scheduler.capacity.root.d.user-limit-factor", 0.1f); conf.setInt("yarn.scheduler.capacity.maximum-applications", 4); - - conf.setQueues(A_QUEUE_PATH, new String[]{"a1", "a2", "a3"}); - conf.setCapacity(A_QUEUE_PATH, 50); - conf.setCapacity(B_QUEUE_PATH, 50); - conf.setCapacity(C_QUEUE_PATH, 0); - conf.setCapacity(D_QUEUE_PATH, 0); - conf.setCapacity(AA1_QUEUE_PATH, 50); - conf.setCapacity(AA2_QUEUE_PATH, 50); - conf.setCapacity(AA3_QUEUE_PATH, 0); - - conf.setCapacityByLabel(A_QUEUE_PATH, "y", 25); - conf.setCapacityByLabel(B_QUEUE_PATH, "y", 50); - conf.setCapacityByLabel(C_QUEUE_PATH, "y", 25); - conf.setCapacityByLabel(D_QUEUE_PATH, "y", 0); - - conf.setCapacityByLabel(A_QUEUE_PATH, "x", 50); - conf.setCapacityByLabel(B_QUEUE_PATH, "x", 50); - - conf.setCapacityByLabel(A_QUEUE_PATH, "z", 50); - conf.setCapacityByLabel(B_QUEUE_PATH, "z", 50); - - conf.setCapacityByLabel(AA1_QUEUE_PATH, "x", 100); - conf.setCapacityByLabel(AA2_QUEUE_PATH, "x", 0); - - conf.setCapacityByLabel(AA1_QUEUE_PATH, "y", 25); - conf.setCapacityByLabel(AA2_QUEUE_PATH, "y", 75); - - conf.setCapacityByLabel(AA2_QUEUE_PATH, "z", 75); - conf.setCapacityByLabel(AA3_QUEUE_PATH, "z", 25); + final String a = CapacitySchedulerConfiguration.ROOT + ".a"; + final String b = CapacitySchedulerConfiguration.ROOT + ".b"; + final String c = CapacitySchedulerConfiguration.ROOT + ".c"; + final String d = CapacitySchedulerConfiguration.ROOT + ".d"; + final String aa1 = a + ".a1"; + final String aa2 = a + ".a2"; + final String aa3 = a + ".a3"; + + conf.setQueues(a, new String[]{"a1", "a2", "a3"}); + conf.setCapacity(a, 50); + conf.setCapacity(b, 50); + conf.setCapacity(c, 0); + conf.setCapacity(d, 0); + conf.setCapacity(aa1, 50); + conf.setCapacity(aa2, 50); + conf.setCapacity(aa3, 0); + + conf.setCapacityByLabel(a, "y", 25); + conf.setCapacityByLabel(b, "y", 50); + conf.setCapacityByLabel(c, "y", 25); + conf.setCapacityByLabel(d, "y", 0); + + conf.setCapacityByLabel(a, "x", 50); + conf.setCapacityByLabel(b, "x", 50); + + conf.setCapacityByLabel(a, "z", 50); + conf.setCapacityByLabel(b, "z", 50); + + conf.setCapacityByLabel(aa1, "x", 100); + conf.setCapacityByLabel(aa2, "x", 0); + + conf.setCapacityByLabel(aa1, "y", 25); + conf.setCapacityByLabel(aa2, "y", 75); + + conf.setCapacityByLabel(aa2, "z", 75); + conf.setCapacityByLabel(aa3, "z", 25); return conf; } 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 6e8c0b02f9473..ef50e5271e718 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 @@ -77,15 +77,6 @@ public class TestApplicationLimitsByPartition { final static int GB = 1024; - final static String A1_PATH = CapacitySchedulerConfiguration.ROOT + ".a" + ".a1"; - final static String B1_PATH = CapacitySchedulerConfiguration.ROOT + ".b" + ".b1"; - final static String B2_PATH = CapacitySchedulerConfiguration.ROOT + ".b" + ".b2"; - final static String C1_PATH = CapacitySchedulerConfiguration.ROOT + ".c" + ".c1"; - final static QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - final static QueuePath A1 = new QueuePath(A1_PATH); - final static QueuePath B1 = new QueuePath(B1_PATH); - final static QueuePath B2 = new QueuePath(B2_PATH); - final static QueuePath C1 = new QueuePath(C1_PATH); LeafQueue queue; RMNodeLabelsManager mgr; @@ -145,6 +136,8 @@ public void testAMResourceLimitWithLabels() throws Exception { // After getting queue conf, configure AM resource percent for Queue A1 // as 0.2 (Label X) and for Queue C1 as 0.2 (Empty Label) + final String A1 = CapacitySchedulerConfiguration.ROOT + ".a" + ".a1"; + final String C1 = CapacitySchedulerConfiguration.ROOT + ".c" + ".c1"; config.setMaximumAMResourcePercentPerPartition(A1, "x", 0.2f); config.setMaximumApplicationMasterResourcePerQueuePercent(C1, 0.2f); @@ -299,6 +292,8 @@ public void testAtleastOneAMRunPerPartition() throws Exception { // After getting queue conf, configure AM resource percent for Queue A1 // as 0.15 (Label X) and for Queue C1 as 0.15 (Empty Label) + final String A1 = CapacitySchedulerConfiguration.ROOT + ".a" + ".a1"; + final String C1 = CapacitySchedulerConfiguration.ROOT + ".c" + ".c1"; config.setMaximumAMResourcePercentPerPartition(A1, "x", 0.15f); config.setMaximumApplicationMasterResourcePerQueuePercent(C1, 0.15f); // inject node label manager @@ -406,6 +401,7 @@ public void testDefaultAMLimitFromQueueForPartition() throws Exception { // After getting queue conf, configure AM resource percent for Queue A1 // as 0.2 (not for partition, rather in queue level) + final String A1 = CapacitySchedulerConfiguration.ROOT + ".a" + ".a1"; config.setMaximumApplicationMasterResourcePerQueuePercent(A1, 0.2f); // inject node label manager MockRM rm1 = new MockRM(config) { @@ -487,6 +483,7 @@ public void testUserAMResourceLimitWithLabels() throws Exception { // as 0.4 (Label X). Also set userlimit as 50% for this queue. So when we // have two users submitting applications, each user will get 50% of AM // resource which is available in this partition. + final String A1 = CapacitySchedulerConfiguration.ROOT + ".a" + ".a1"; config.setMaximumAMResourcePercentPerPartition(A1, "x", 0.4f); config.setUserLimit(A1, 50); @@ -628,6 +625,8 @@ public void testAMResourceLimitForMultipleApplications() throws Exception { * A1 : 0.25 * B2 : 0.15 */ + final String A1 = CapacitySchedulerConfiguration.ROOT + ".a" + ".a1"; + final String B1 = CapacitySchedulerConfiguration.ROOT + ".b" + ".b1"; config.setMaximumAMResourcePercentPerPartition(A1, "y", 0.25f); config.setMaximumApplicationMasterResourcePerQueuePercent(B1, 0.15f); @@ -757,6 +756,8 @@ public void testHeadroom() throws Exception { CapacitySchedulerConfiguration csConf = (CapacitySchedulerConfiguration) TestUtils .getComplexConfigurationWithQueueLabels(conf); + final String A1 = CapacitySchedulerConfiguration.ROOT + ".a" + ".a1"; + final String B2 = CapacitySchedulerConfiguration.ROOT + ".b" + ".b2"; csConf.setUserLimit(A1, 25); csConf.setUserLimit(B2, 25); @@ -940,9 +941,9 @@ public void testAMLimitByAllResources() throws Exception { new CapacitySchedulerConfiguration(); csconf.setResourceComparator(DominantResourceCalculator.class); String queueName = "a1"; - csconf.setQueues(ROOT, + csconf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {queueName}); - csconf.setCapacity(new QueuePath("root.a1"), 100); + csconf.setCapacity("root." + queueName, 100); ResourceInformation res0 = ResourceInformation.newInstance("memory-mb", ResourceInformation.MEMORY_MB.getUnits(), GB, Long.MAX_VALUE); 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/TestApplicationPriorityACLConfiguration.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/TestApplicationPriorityACLConfiguration.java index e2011ff9cdb98..598bd49b0e7bf 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/TestApplicationPriorityACLConfiguration.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/TestApplicationPriorityACLConfiguration.java @@ -40,36 +40,28 @@ public class TestApplicationPriorityACLConfiguration { private static final String QUEUEB = "queueB"; private static final String QUEUEC = "queueC"; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A_QUEUE_PATH = new QueuePath( - CapacitySchedulerConfiguration.ROOT + "." + QUEUEA); - private static final QueuePath B_QUEUE_PATH = new QueuePath( - CapacitySchedulerConfiguration.ROOT + "." + QUEUEB); - private static final QueuePath C_QUEUE_PATH = new QueuePath( - CapacitySchedulerConfiguration.ROOT + "." + QUEUEC); - - @Test public void testSimpleACLConfiguration() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUEA, QUEUEB, QUEUEC}); - csConf.setCapacity(A_QUEUE_PATH, 50f); - csConf.setCapacity(B_QUEUE_PATH, 25f); - csConf.setCapacity(C_QUEUE_PATH, 25f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 25f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEC, 25f); // Success case: Configure one user/group level priority acl for queue A. String[] aclsForA = new String[2]; aclsForA[0] = QUEUE_A_USER; aclsForA[1] = QUEUE_A_GROUP; - csConf.setPriorityAcls(A_QUEUE_PATH, + csConf.setPriorityAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, Priority.newInstance(maxPriorityQueueA), Priority.newInstance(defaultPriorityQueueA), aclsForA); // Try to get the ACL configs and make sure there are errors/exceptions List pGroupA = csConf.getPriorityAcls( - A_QUEUE_PATH, Priority.newInstance(clusterMaxPriority)); + CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, + Priority.newInstance(clusterMaxPriority)); // Validate! verifyACLs(pGroupA, QUEUE_A_USER, QUEUE_A_GROUP, maxPriorityQueueA, @@ -79,32 +71,34 @@ public void testSimpleACLConfiguration() throws Exception { @Test public void testACLConfigurationForInvalidCases() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUEA, QUEUEB, QUEUEC}); - csConf.setCapacity(A_QUEUE_PATH, 50f); - csConf.setCapacity(B_QUEUE_PATH, 25f); - csConf.setCapacity(C_QUEUE_PATH, 25f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 25f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEC, 25f); // Success case: Configure one user/group level priority acl for queue A. String[] aclsForA = new String[2]; aclsForA[0] = QUEUE_A_USER; aclsForA[1] = QUEUE_A_GROUP; - csConf.setPriorityAcls(A_QUEUE_PATH, + csConf.setPriorityAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, Priority.newInstance(maxPriorityQueueA), Priority.newInstance(defaultPriorityQueueA), aclsForA); String[] aclsForB = new String[1]; aclsForB[0] = QUEUE_B_USER; - csConf.setPriorityAcls(B_QUEUE_PATH, + csConf.setPriorityAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, Priority.newInstance(maxPriorityQueueB), Priority.newInstance(defaultPriorityQueueB), aclsForB); // Try to get the ACL configs and make sure there are errors/exceptions List pGroupA = csConf.getPriorityAcls( - A_QUEUE_PATH, Priority.newInstance(clusterMaxPriority)); + CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, + Priority.newInstance(clusterMaxPriority)); List pGroupB = csConf.getPriorityAcls( - B_QUEUE_PATH, Priority.newInstance(clusterMaxPriority)); + CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, + Priority.newInstance(clusterMaxPriority)); // Validate stored ACL values with configured ones. verifyACLs(pGroupA, QUEUE_A_USER, QUEUE_A_GROUP, maxPriorityQueueA, 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/TestApplicationPriorityACLs.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/TestApplicationPriorityACLs.java index 674bb5c8c1eba..d3193537046a7 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/TestApplicationPriorityACLs.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/TestApplicationPriorityACLs.java @@ -177,24 +177,24 @@ private void verifyAppPriorityIsAccepted(String submitter, @Override protected Configuration createConfiguration() { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUEA, QUEUEB, QUEUEC}); - csConf.setCapacity(A_QUEUE_PATH, 50f); - csConf.setCapacity(B_QUEUE_PATH, 25f); - csConf.setCapacity(C_QUEUE_PATH, 25f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 25f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEC, 25f); String[] aclsForA = new String[2]; aclsForA[0] = QUEUE_A_USER; aclsForA[1] = QUEUE_A_GROUP; - csConf.setPriorityAcls(A_QUEUE_PATH, + csConf.setPriorityAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, Priority.newInstance(maxPriorityQueueA), Priority.newInstance(defaultPriorityQueueA), aclsForA); String[] aclsForB = new String[2]; aclsForB[0] = QUEUE_B_USER; aclsForB[1] = QUEUE_B_GROUP; - csConf.setPriorityAcls(B_QUEUE_PATH, + csConf.setPriorityAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, Priority.newInstance(maxPriorityQueueB), Priority.newInstance(defaultPriorityQueueB), aclsForB); 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/TestAutoCreatedQueueDeletionPolicy.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/TestAutoCreatedQueueDeletionPolicy.java index 6ed76b6341c82..502b6a40ef0ac 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/TestAutoCreatedQueueDeletionPolicy.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/TestAutoCreatedQueueDeletionPolicy.java @@ -48,11 +48,6 @@ public class TestAutoCreatedQueueDeletionPolicy private CapacitySchedulerQueueManager autoQueueHandler; - public static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - public static final QueuePath ROOT_A = new QueuePath("root", "a"); - public static final QueuePath ROOT_A_A1 = QueuePath.createFromQueues("root", "a", "a1"); - public static final QueuePath ROOT_B = new QueuePath("root", "b"); - /* Create the following structure: root @@ -68,15 +63,15 @@ public void setUp() throws Exception { ResourceScheduler.class); // By default, set 3 queues, a/b, and a.a1 - csConf.setQueues(ROOT, new String[]{"a", "b"}); - csConf.setNonLabeledQueueWeight(ROOT, 1f); - csConf.setNonLabeledQueueWeight(ROOT_A, 1f); - csConf.setNonLabeledQueueWeight(ROOT_B, 1f); - csConf.setQueues(ROOT_A, new String[]{"a1"}); - csConf.setNonLabeledQueueWeight(ROOT_A_A1, 1f); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); - csConf.setAutoQueueCreationV2Enabled(ROOT_A, true); - csConf.setAutoQueueCreationV2Enabled(PARENT_QUEUE_PATH, true); + csConf.setQueues("root", new String[]{"a", "b"}); + csConf.setNonLabeledQueueWeight("root", 1f); + csConf.setNonLabeledQueueWeight("root.a", 1f); + csConf.setNonLabeledQueueWeight("root.b", 1f); + csConf.setQueues("root.a", new String[]{"a1"}); + csConf.setNonLabeledQueueWeight("root.a.a1", 1f); + csConf.setAutoQueueCreationV2Enabled("root", true); + csConf.setAutoQueueCreationV2Enabled("root.a", true); + csConf.setAutoQueueCreationV2Enabled(PARENT_QUEUE, true); // Test for auto deletion when expired csConf.setAutoExpiredDeletionTime(1); } 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/TestAutoCreatedQueueTemplate.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/TestAutoCreatedQueueTemplate.java index 5b3acf11c4fdd..201214ec2674b 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/TestAutoCreatedQueueTemplate.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/TestAutoCreatedQueueTemplate.java @@ -29,79 +29,75 @@ public class TestAutoCreatedQueueTemplate { private static final QueuePath TEST_QUEUE_AB = new QueuePath("root.a.b"); private static final QueuePath TEST_QUEUE_A = new QueuePath("root.a"); private static final QueuePath TEST_QUEUE_B = new QueuePath("root.b"); - private static final QueuePath TEST_QUEUE_WILDCARD = new QueuePath("*"); - private static final QueuePath TEST_QUEUE_ROOT_WILDCARD = new QueuePath("root.*"); - private static final QueuePath TEST_QUEUE_TWO_LEVEL_WILDCARDS = new QueuePath("root.*.*"); - private static final QueuePath TEST_QUEUE_A_WILDCARD = new QueuePath("root.a.*"); - private static final QueuePath ROOT = new QueuePath("root"); + private static final String ROOT = "root"; private CapacitySchedulerConfiguration conf; @Before public void setUp() throws Exception { conf = new CapacitySchedulerConfiguration(); - conf.setQueues(ROOT, new String[]{"a"}); - conf.setQueues(TEST_QUEUE_A, new String[]{"b"}); - conf.setQueues(TEST_QUEUE_B, new String[]{"c"}); + conf.setQueues("root", new String[]{"a"}); + conf.setQueues("a", new String[]{"b"}); + conf.setQueues("b", new String[]{"c"}); } @Test public void testNonWildCardTemplate() { - conf.set(getTemplateKey(TEST_QUEUE_AB, "capacity"), "6w"); + conf.set(getTemplateKey(TEST_QUEUE_AB.getFullPath(), "capacity"), "6w"); AutoCreatedQueueTemplate template = new AutoCreatedQueueTemplate(conf, TEST_QUEUE_AB); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_ABC); + template.setTemplateEntriesForChild(conf, TEST_QUEUE_ABC.getFullPath()); Assert.assertEquals("weight is not set", 6f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC), 10e-6); + conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC.getFullPath()), 10e-6); } @Test public void testOneLevelWildcardTemplate() { - conf.set(getTemplateKey(TEST_QUEUE_A_WILDCARD, "capacity"), "6w"); + conf.set(getTemplateKey("root.a.*", "capacity"), "6w"); AutoCreatedQueueTemplate template = new AutoCreatedQueueTemplate(conf, TEST_QUEUE_AB); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_ABC); + template.setTemplateEntriesForChild(conf, TEST_QUEUE_ABC.getFullPath()); Assert.assertEquals("weight is not set", 6f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC), 10e-6); + conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC.getFullPath()), 10e-6); } @Test public void testTwoLevelWildcardTemplate() { - conf.set(getTemplateKey(TEST_QUEUE_ROOT_WILDCARD, "capacity"), "6w"); - conf.set(getTemplateKey(TEST_QUEUE_TWO_LEVEL_WILDCARDS, "capacity"), "5w"); + conf.set(getTemplateKey("root.*", "capacity"), "6w"); + conf.set(getTemplateKey("root.*.*", "capacity"), "5w"); new AutoCreatedQueueTemplate(conf, TEST_QUEUE_A) - .setTemplateEntriesForChild(conf, TEST_QUEUE_AB); + .setTemplateEntriesForChild(conf, TEST_QUEUE_AB.getFullPath()); new AutoCreatedQueueTemplate(conf, TEST_QUEUE_AB) - .setTemplateEntriesForChild(conf, TEST_QUEUE_ABC); + .setTemplateEntriesForChild(conf, TEST_QUEUE_ABC.getFullPath()); Assert.assertEquals("weight is not set", 6f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_AB), 10e-6); + conf.getNonLabeledQueueWeight(TEST_QUEUE_AB.getFullPath()), 10e-6); Assert.assertEquals("weight is not set", 5f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC), 10e-6); + conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC.getFullPath()), 10e-6); } @Test public void testIgnoredWhenRootWildcarded() { - conf.set(getTemplateKey(TEST_QUEUE_WILDCARD, "capacity"), "6w"); + conf.set(getTemplateKey("*", "capacity"), "6w"); AutoCreatedQueueTemplate template = - new AutoCreatedQueueTemplate(conf, ROOT); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_A); + new AutoCreatedQueueTemplate(conf, new QueuePath(ROOT)); + template.setTemplateEntriesForChild(conf, TEST_QUEUE_A.getFullPath()); Assert.assertEquals("weight is set", -1f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_A), 10e-6); + conf.getNonLabeledQueueWeight(TEST_QUEUE_A.getFullPath()), 10e-6); } @Test public void testIgnoredWhenNoParent() { - conf.set(getTemplateKey(ROOT, "capacity"), "6w"); + conf.set(getTemplateKey("root", "capacity"), "6w"); AutoCreatedQueueTemplate template = - new AutoCreatedQueueTemplate(conf, ROOT); + new AutoCreatedQueueTemplate(conf, new QueuePath(ROOT)); template.setTemplateEntriesForChild(conf, ROOT); Assert.assertEquals("weight is set", -1f, @@ -110,10 +106,10 @@ public void testIgnoredWhenNoParent() { @Test public void testWildcardAfterRoot() { - conf.set(getTemplateKey(TEST_QUEUE_ROOT_WILDCARD, "acl_submit_applications"), "user"); + conf.set(getTemplateKey("root.*", "acl_submit_applications"), "user"); AutoCreatedQueueTemplate template = - new AutoCreatedQueueTemplate(conf, TEST_QUEUE_A); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_A); + new AutoCreatedQueueTemplate(conf, new QueuePath("root.a")); + template.setTemplateEntriesForChild(conf, "root.a"); Assert.assertEquals("acl_submit_applications is set", "user", template.getTemplateProperties().get("acl_submit_applications")); @@ -121,122 +117,80 @@ public void testWildcardAfterRoot() { @Test public void testTemplatePrecedence() { - conf.set(getTemplateKey(TEST_QUEUE_AB, "capacity"), "6w"); - conf.set(getTemplateKey(TEST_QUEUE_A_WILDCARD, "capacity"), "4w"); - conf.set(getTemplateKey(TEST_QUEUE_TWO_LEVEL_WILDCARDS, "capacity"), "2w"); + conf.set(getTemplateKey("root.a.b", "capacity"), "6w"); + conf.set(getTemplateKey("root.a.*", "capacity"), "4w"); + conf.set(getTemplateKey("root.*.*", "capacity"), "2w"); AutoCreatedQueueTemplate template = new AutoCreatedQueueTemplate(conf, TEST_QUEUE_AB); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_ABC); + template.setTemplateEntriesForChild(conf, TEST_QUEUE_ABC.getFullPath()); Assert.assertEquals( "explicit template does not have the highest precedence", 6f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC), 10e-6); + conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC.getFullPath()), 10e-6); CapacitySchedulerConfiguration newConf = new CapacitySchedulerConfiguration(); - newConf.set(getTemplateKey(TEST_QUEUE_A_WILDCARD, "capacity"), "4w"); + newConf.set(getTemplateKey("root.a.*", "capacity"), "4w"); template = new AutoCreatedQueueTemplate(newConf, TEST_QUEUE_AB); - template.setTemplateEntriesForChild(newConf, TEST_QUEUE_ABC); + template.setTemplateEntriesForChild(newConf, TEST_QUEUE_ABC.getFullPath()); Assert.assertEquals("precedence is invalid", 4f, - newConf.getNonLabeledQueueWeight(TEST_QUEUE_ABC), 10e-6); + newConf.getNonLabeledQueueWeight(TEST_QUEUE_ABC.getFullPath()), 10e-6); } @Test public void testRootTemplate() { - conf.set(getTemplateKey(ROOT, "capacity"), "2w"); + conf.set(getTemplateKey("root", "capacity"), "2w"); AutoCreatedQueueTemplate template = - new AutoCreatedQueueTemplate(conf, ROOT); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_A); + new AutoCreatedQueueTemplate(conf, new QueuePath(ROOT)); + template.setTemplateEntriesForChild(conf, TEST_QUEUE_A.getFullPath()); Assert.assertEquals("root property is not set", 2f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_A), 10e-6); + conf.getNonLabeledQueueWeight(TEST_QUEUE_A.getFullPath()), 10e-6); } @Test public void testQueueSpecificTemplates() { - conf.set(getTemplateKey(ROOT, "capacity"), "2w"); - conf.set(getLeafTemplateKey(ROOT, + conf.set(getTemplateKey("root", "capacity"), "2w"); + conf.set(getLeafTemplateKey("root", "default-node-label-expression"), "test"); - conf.set(getLeafTemplateKey(ROOT, "capacity"), "10w"); + conf.set(getLeafTemplateKey("root", "capacity"), "10w"); conf.setBoolean(getParentTemplateKey( - ROOT, AUTO_CREATE_CHILD_QUEUE_AUTO_REMOVAL_ENABLE), false); + "root", AUTO_CREATE_CHILD_QUEUE_AUTO_REMOVAL_ENABLE), false); AutoCreatedQueueTemplate template = - new AutoCreatedQueueTemplate(conf, ROOT); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_A); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_B, true); + new AutoCreatedQueueTemplate(conf, new QueuePath(ROOT)); + template.setTemplateEntriesForChild(conf, TEST_QUEUE_A.getFullPath()); + template.setTemplateEntriesForChild(conf, TEST_QUEUE_B.getFullPath(), true); Assert.assertNull("default-node-label-expression is set for parent", - conf.getDefaultNodeLabelExpression(TEST_QUEUE_A)); + conf.getDefaultNodeLabelExpression(TEST_QUEUE_A.getFullPath())); Assert.assertEquals("default-node-label-expression is not set for leaf", - "test", conf.getDefaultNodeLabelExpression(TEST_QUEUE_B)); + "test", conf.getDefaultNodeLabelExpression(TEST_QUEUE_B.getFullPath())); Assert.assertFalse("auto queue removal is not disabled for parent", - conf.isAutoExpiredDeletionEnabled(TEST_QUEUE_A)); + conf.isAutoExpiredDeletionEnabled(TEST_QUEUE_A.getFullPath())); Assert.assertEquals("weight should not be overridden when set by " + "queue type specific template", - 10f, conf.getNonLabeledQueueWeight(TEST_QUEUE_B), 10e-6); + 10f, conf.getNonLabeledQueueWeight(TEST_QUEUE_B.getFullPath()), 10e-6); Assert.assertEquals("weight should be set by common template", - 2f, conf.getNonLabeledQueueWeight(TEST_QUEUE_A), 10e-6); + 2f, conf.getNonLabeledQueueWeight(TEST_QUEUE_A.getFullPath()), 10e-6); } - @Test - public void testWildcardTemplateWithLimitedAutoCreatedQueueDepth() { - conf.set(getTemplateKey(TEST_QUEUE_ROOT_WILDCARD, "capacity"), "6w"); - conf.set(getTemplateKey(TEST_QUEUE_A_WILDCARD, "capacity"), "5w"); - conf.setMaximumAutoCreatedQueueDepth(TEST_QUEUE_A, 1); - conf.setMaximumAutoCreatedQueueDepth(TEST_QUEUE_AB, 1); - - new AutoCreatedQueueTemplate(conf, TEST_QUEUE_A) - .setTemplateEntriesForChild(conf, TEST_QUEUE_AB); - new AutoCreatedQueueTemplate(conf, TEST_QUEUE_AB) - .setTemplateEntriesForChild(conf, TEST_QUEUE_ABC); - - Assert.assertEquals("weight is not set", 6f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_AB), 10e-6); - Assert.assertEquals("weight is not set", 5f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC), 10e-6); - } - - @Test - public void testIgnoredTemplateWithLimitedAutoCreatedQueueDepth() { - conf.set(getTemplateKey(TEST_QUEUE_TWO_LEVEL_WILDCARDS, "capacity"), "5w"); - conf.setMaximumAutoCreatedQueueDepth(TEST_QUEUE_AB, 1); - - new AutoCreatedQueueTemplate(conf, TEST_QUEUE_AB) - .setTemplateEntriesForChild(conf, TEST_QUEUE_ABC); - - Assert.assertEquals("weight is set incorrectly", -1f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_ABC), 10e-6); - } - - @Test - public void testIgnoredTemplateWhenQueuePathIsInvalid() { - QueuePath invalidPath = new QueuePath("a"); - conf.set(getTemplateKey(invalidPath, "capacity"), "6w"); - AutoCreatedQueueTemplate template = - new AutoCreatedQueueTemplate(conf, invalidPath); - template.setTemplateEntriesForChild(conf, TEST_QUEUE_AB); - - Assert.assertEquals("weight is set using invalid queue path", -1f, - conf.getNonLabeledQueueWeight(TEST_QUEUE_AB), 10e-6); - } - - private String getTemplateKey(QueuePath queuePath, String entryKey) { - return QueuePrefixes.getQueuePrefix(queuePath) + private String getTemplateKey(String queuePath, String entryKey) { + return CapacitySchedulerConfiguration.getQueuePrefix(queuePath) + AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX + entryKey; } - private String getParentTemplateKey(QueuePath queuePath, String entryKey) { - return QueuePrefixes.getQueuePrefix(queuePath) + private String getParentTemplateKey(String queuePath, String entryKey) { + return CapacitySchedulerConfiguration.getQueuePrefix(queuePath) + AutoCreatedQueueTemplate.AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + entryKey; } - private String getLeafTemplateKey(QueuePath queuePath, String entryKey) { - return QueuePrefixes.getQueuePrefix(queuePath) + private String getLeafTemplateKey(String queuePath, String entryKey) { + return CapacitySchedulerConfiguration.getQueuePrefix(queuePath) + AutoCreatedQueueTemplate.AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + entryKey; } } \ No newline at end of file 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/TestCSAllocateCustomResource.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/TestCSAllocateCustomResource.java index 5c0f5d1d318f6..04e2f8e298cae 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/TestCSAllocateCustomResource.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/TestCSAllocateCustomResource.java @@ -71,9 +71,6 @@ * */ public class TestCSAllocateCustomResource { - private static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A = new QueuePath(A_PATH); private YarnConfiguration conf; private RMNodeLabelsManager mgr; @@ -120,7 +117,7 @@ public void testCapacitySchedulerJobWhenConfigureCustomResourceType() .getConfigurationWithMultipleQueues(conf); newConf.setClass(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, DominantResourceCalculator.class, ResourceCalculator.class); - newConf.set(QueuePrefixes.getQueuePrefix(A) + newConf.set(CapacitySchedulerConfiguration.getQueuePrefix("root.a") + MAXIMUM_ALLOCATION_MB, "4096"); // We must set this to false to avoid MockRM init configuration with // resource-types.xml by ResourceUtils.resetResourceTypes(conf); @@ -301,7 +298,7 @@ public void testCapacitySchedulerAbsoluteConfWithCustomResourceType() Resource.newInstance(1000, 10, valuesMax); // Define top-level queues - newConf.setQueues(ROOT, + newConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"}); newConf.setMinimumResourceRequirement("", new QueuePath("root", "a"), aMINRES); @@ -319,10 +316,10 @@ public void testCapacitySchedulerAbsoluteConfWithCustomResourceType() CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); Assert.assertEquals(aMINRES, cs.getConfiguration(). - getMinimumResourceRequirement("", A, resourceTypes)); + getMinimumResourceRequirement("", "root.a", resourceTypes)); Assert.assertEquals(aMAXRES, cs.getConfiguration(). - getMaximumResourceRequirement("", A, resourceTypes)); + getMaximumResourceRequirement("", "root.a", resourceTypes)); // Check the gpu resource of queue is right. Assert.assertEquals(aMINRES, cs.getQueue("root.a"). 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 768768e95dbf9..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 @@ -103,15 +103,11 @@ public void setup() throws IOException { } private void setupQueues(CapacitySchedulerConfiguration config) { - QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - QueuePath queue1 = new QueuePath("root.queue1"); - QueuePath subqueue1 = new QueuePath("root.queue1.subqueue1"); - QueuePath subqueue2 = new QueuePath("root.queue1.subqueue2"); - - config.setQueues(root, new String[] {"queue1", "queue2"}); - config.setQueues(queue1, new String[] {"subqueue1", "subqueue2"}); - config.setQueues(subqueue1, new String[] {"leaf1"}); - config.setQueues(subqueue2, new String[] {"leaf2"}); + config.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"queue1", "queue2"}); + config.setQueues("root.queue1", new String[] {"subqueue1", "subqueue2"}); + config.setQueues("root.queue1.subqueue1", new String[] {"leaf1"}); + config.setQueues("root.queue1.subqueue2", new String[] {"leaf2"}); config.setFloat(PREFIX + "root.capacity", 100.0f); config.setFloat(PREFIX + "root.queue1.capacity", 50.0f); config.setFloat(PREFIX + "root.queue2.capacity", 50.0f); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java index e46d6b07f7590..ede33b6f38687 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java @@ -31,17 +31,12 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A1; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A2; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A3; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A_CAPACITY; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B1; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B1_CAPACITY; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B2; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B2_PATH; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B_CAPACITY; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B_PATH; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.DEFAULT; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.ROOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.GB; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.appHelper; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.checkApplicationResourceUsage; @@ -592,14 +587,15 @@ protected RMNodeLabelsManager createNodeLabelManager() { @Test public void testMaximumCapacitySetup() { float delta = 0.0000001f; + QueuePath queuePathA = new QueuePath(A); CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); assertEquals(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY_VALUE, - conf.getNonLabeledQueueMaximumCapacity(A), delta); + conf.getNonLabeledQueueMaximumCapacity(queuePathA), delta); conf.setMaximumCapacity(A, 50.0f); - assertEquals(50.0f, conf.getNonLabeledQueueMaximumCapacity(A), delta); + assertEquals(50.0f, conf.getNonLabeledQueueMaximumCapacity(queuePathA), delta); conf.setMaximumCapacity(A, -1); assertEquals(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY_VALUE, - conf.getNonLabeledQueueMaximumCapacity(A), delta); + conf.getNonLabeledQueueMaximumCapacity(queuePathA), delta); } @Test @@ -640,19 +636,18 @@ public void testQueueMaximumAllocations() { public void testParseQueueWithAbsoluteResource() { String childQueue = "testQueue"; String labelName = "testLabel"; - QueuePath childQueuePath = new QueuePath("root." + childQueue); CapacityScheduler cs = new CapacityScheduler(); cs.setConf(new YarnConfiguration()); cs.setRMContext(resourceManager.getRMContext()); CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.setQueues(ROOT, new String[] {childQueue}); - conf.setCapacity(childQueuePath, "[memory=20480,vcores=200]"); - conf.setAccessibleNodeLabels(childQueuePath, + conf.setQueues("root", new String[] {childQueue}); + conf.setCapacity("root." + childQueue, "[memory=20480,vcores=200]"); + conf.setAccessibleNodeLabels("root." + childQueue, Sets.newHashSet(labelName)); - conf.setCapacityByLabel(ROOT, labelName, "[memory=10240,vcores=100]"); - conf.setCapacityByLabel(childQueuePath, labelName, + conf.setCapacityByLabel("root", labelName, "[memory=10240,vcores=100]"); + conf.setCapacityByLabel("root." + childQueue, labelName, "[memory=4096,vcores=10]"); cs.init(conf); @@ -1243,8 +1238,8 @@ null, new RMContainerTokenSecretManager(conf), cs.reinitialize(conf, rmContext); CSQueue rootQueue = cs.getRootQueue(); - CSQueue queueB = findQueue(rootQueue, B_PATH); - CSQueue queueB2 = findQueue(queueB, B2_PATH); + CSQueue queueB = findQueue(rootQueue, B); + CSQueue queueB2 = findQueue(queueB, B2); // When preemption turned on for the whole system // (yarn.resourcemanager.scheduler.monitor.enable=true), and with no other @@ -1255,7 +1250,7 @@ null, new RMContainerTokenSecretManager(conf), // Disable preemption at the root queue level. // The preemption property should be inherited from root all the // way down so that root.b.b2 should NOT be preemptable. - conf.setPreemptionDisabled(rootQueue.getQueuePathObject(), true); + conf.setPreemptionDisabled(rootQueue.getQueuePath(), true); cs.reinitialize(conf, rmContext); assertTrue( "queue " + B2 + " should have inherited non-preemptability from root", @@ -1263,8 +1258,8 @@ null, new RMContainerTokenSecretManager(conf), // Enable preemption for root (grandparent) but disable for root.b (parent). // root.b.b2 should inherit property from parent and NOT be preemptable - conf.setPreemptionDisabled(rootQueue.getQueuePathObject(), false); - conf.setPreemptionDisabled(queueB.getQueuePathObject(), true); + conf.setPreemptionDisabled(rootQueue.getQueuePath(), false); + conf.setPreemptionDisabled(queueB.getQueuePath(), true); cs.reinitialize(conf, rmContext); assertTrue( "queue " + B2 + " should have inherited non-preemptability from parent", @@ -1272,7 +1267,7 @@ null, new RMContainerTokenSecretManager(conf), // When preemption is turned on for root.b.b2, it should be preemptable // even though preemption is disabled on root.b (parent). - conf.setPreemptionDisabled(queueB2.getQueuePathObject(), false); + conf.setPreemptionDisabled(queueB2.getQueuePath(), false); cs.reinitialize(conf, rmContext); assertFalse("queue " + B2 + " should have been preemptable", queueB2.getPreemptionDisabled()); @@ -1555,7 +1550,7 @@ public void testParentQueueMaxCapsAreRespected() throws Exception { * A1 A2 */ CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] {"a", "b"}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); csConf.setCapacity(A, 50); csConf.setMaximumCapacity(A, 50); csConf.setCapacity(B, 50); @@ -2559,11 +2554,13 @@ private CapacityScheduler setUpCSQueue(long maxLifetime, long defaultLifetime) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"default"}); - csConf.setCapacity(DEFAULT, 100); - csConf.setMaximumLifetimePerQueue(DEFAULT, maxLifetime); - csConf.setDefaultLifetimePerQueue(DEFAULT, defaultLifetime); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".default", 100); + csConf.setMaximumLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT + ".default", maxLifetime); + csConf.setDefaultLifetimePerQueue( + CapacitySchedulerConfiguration.ROOT + ".default", defaultLifetime); YarnConfiguration conf = new YarnConfiguration(csConf); CapacityScheduler cs = new CapacityScheduler(); @@ -2899,11 +2896,11 @@ ImmutableMap. builder() .getMaxAbsoluteCapacity(), DELTA); // Add child queue to a, and reinitialize. Metrics should be updated - csConf.setQueues(A, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT + ".a", new String[] {"a1", "a2", "a3"}); - csConf.setCapacity(A2, 29.5f); - csConf.setCapacity(A3, 40.5f); - csConf.setMaximumCapacity(A3, + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a.a2", 29.5f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a.a3", 40.5f); + csConf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + ".a.a3", 50.0f); cs.reinitialize(csConf, new RMContextImpl(null, null, null, null, null, @@ -2948,14 +2945,14 @@ null, new RMContainerTokenSecretManager(csConf), public void testReservedContainerLeakWhenMoveApplication() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); - csConf.setCapacity(A, 50); - csConf.setMaximumCapacity(A, 100); - csConf.setUserLimitFactor(A, 100); - csConf.setCapacity(B, 50); - csConf.setMaximumCapacity(B, 100); - csConf.setUserLimitFactor(B, 100); + csConf.setCapacity("root.a", 50); + csConf.setMaximumCapacity("root.a", 100); + csConf.setUserLimitFactor("root.a", 100); + csConf.setCapacity("root.b", 50); + csConf.setMaximumCapacity("root.b", 100); + csConf.setUserLimitFactor("root.b", 100); YarnConfiguration conf=new YarnConfiguration(csConf); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, 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/TestCapacitySchedulerAmbiguousLeafs.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/TestCapacitySchedulerAmbiguousLeafs.java index a3500943f59e1..88e6aff2537d0 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/TestCapacitySchedulerAmbiguousLeafs.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/TestCapacitySchedulerAmbiguousLeafs.java @@ -29,26 +29,14 @@ import java.io.IOException; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; + public class TestCapacitySchedulerAmbiguousLeafs { /** * Internal counter for incremental application id generation */ int appId = 0; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath DEFAULT = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".default"); - private static final QueuePath A = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".a"); - private static final QueuePath B = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".b"); - private static final QueuePath A_UNIQUE = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".a.unique"); - private static final QueuePath A_AMBI = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".a.ambi"); - private static final QueuePath B_AMBI = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".b.ambi"); - /** * Helper method to submit applications via RMClientService, to make sure * all submissions go through RMAppManager. @@ -91,19 +79,19 @@ public void testAmbiguousSubmissionWithACL() throws Exception { schedulerConf.setAcl(ROOT, QueueACL.SUBMIT_APPLICATIONS, " "); schedulerConf.setAcl(ROOT, QueueACL.ADMINISTER_QUEUE, "forbidden forbidden"); - schedulerConf.setQueues(A, new String[] {"unique", "ambi"}); - schedulerConf.setAcl(A, QueueACL.SUBMIT_APPLICATIONS, "forbidden forbidden"); - schedulerConf.setCapacity(A, 45); + schedulerConf.setQueues(ROOT + ".a", new String[] {"unique", "ambi"}); + schedulerConf.setAcl(ROOT + ".a", QueueACL.SUBMIT_APPLICATIONS, "forbidden forbidden"); + schedulerConf.setCapacity(ROOT + ".a", 45); - schedulerConf.setQueues(B, new String[] {"ambi"}); - schedulerConf.setCapacity(B, 45); - schedulerConf.setCapacity(DEFAULT, 10); + schedulerConf.setQueues(ROOT + ".b", new String[] {"ambi"}); + schedulerConf.setCapacity(ROOT + ".b", 45); + schedulerConf.setCapacity(ROOT + ".default", 10); - schedulerConf.setCapacity(A_UNIQUE, 50); - schedulerConf.setAcl(A_UNIQUE, QueueACL.SUBMIT_APPLICATIONS, "* *"); - schedulerConf.setCapacity(A_AMBI, 50); - schedulerConf.setAcl(A_AMBI, QueueACL.SUBMIT_APPLICATIONS, "* *"); - schedulerConf.setCapacity(B_AMBI, 100); + schedulerConf.setCapacity(ROOT + ".a.unique", 50); + schedulerConf.setAcl(ROOT + ".a.unique", QueueACL.SUBMIT_APPLICATIONS, "* *"); + schedulerConf.setCapacity(ROOT + ".a.ambi", 50); + schedulerConf.setAcl(ROOT + ".a.ambi", QueueACL.SUBMIT_APPLICATIONS, "* *"); + schedulerConf.setCapacity(ROOT + ".b.ambi", 100); schedulerConf.set(CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT, "json"); //Simple %specified mapping rule for all submissions with skip fallback 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/TestCapacitySchedulerApps.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/TestCapacitySchedulerApps.java index e881c3327b672..d192e7dcc6933 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/TestCapacitySchedulerApps.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/TestCapacitySchedulerApps.java @@ -97,7 +97,6 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A2; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B1; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.ROOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.setupQueueConfiguration; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.GB; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.appHelper; @@ -1043,7 +1042,7 @@ public void testMaxParallelAppsPendingQueueMetrics() throws Exception { MockRM rm = setUpMove(); ResourceScheduler scheduler = rm.getResourceScheduler(); CapacityScheduler cs = (CapacityScheduler) scheduler; - cs.getQueueContext().getConfiguration().setInt(QueuePrefixes.getQueuePrefix(A1) + cs.getQueueContext().getConfiguration().setInt(CapacitySchedulerConfiguration.getQueuePrefix(A1) + CapacitySchedulerConfiguration.MAX_PARALLEL_APPLICATIONS, MAX_PARALLEL_APPS); cs.reinitialize(cs.getQueueContext().getConfiguration(), mockContext); List attemptIds = new ArrayList<>(); @@ -1174,7 +1173,7 @@ public void testMoveAppWithActiveUsersWithOnlyPendingApps() throws Exception { new CapacitySchedulerConfiguration(conf); // Define top-level queues - newConf.setQueues(ROOT, + newConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); newConf.setCapacity(A, 50); @@ -1470,7 +1469,7 @@ public void testAppSubmission() throws Exception { ResourceScheduler.class); conf.setQueues(A, new String[]{"a1", "a2", "b"}); conf.setCapacity(A1, 20); - conf.setCapacity(new QueuePath("root.a.b"), 10); + conf.setCapacity("root.a.b", 10); MockRM rm = new MockRM(conf); rm.start(); @@ -1545,7 +1544,7 @@ private Configuration getCapacityConfiguration(Configuration config) { new CapacitySchedulerConfiguration(config); // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); conf.setCapacity(A, 50); conf.setCapacity(B, 50); 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/TestCapacitySchedulerAutoCreatedQueueBase.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/TestCapacitySchedulerAutoCreatedQueueBase.java index e03e59d2512c8..48431b6eaf0f7 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/TestCapacitySchedulerAutoCreatedQueueBase.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/TestCapacitySchedulerAutoCreatedQueueBase.java @@ -96,6 +96,8 @@ .capacity.CapacitySchedulerConfiguration.DOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler .capacity.CapacitySchedulerConfiguration.FAIR_APP_ORDERING_POLICY; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler + .capacity.CapacitySchedulerConfiguration.ROOT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -109,44 +111,23 @@ public class TestCapacitySchedulerAutoCreatedQueueBase { public static final ContainerUpdates NULL_UPDATE_REQUESTS = new ContainerUpdates(); - public static final String DEFAULT_PATH = CapacitySchedulerConfiguration.ROOT + ".default"; - public static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - public static final String B_PATH = CapacitySchedulerConfiguration.ROOT + ".b"; - public static final String C_PATH = CapacitySchedulerConfiguration.ROOT + ".c"; - public static final String D_PATH = CapacitySchedulerConfiguration.ROOT + ".d"; - public static final String E_PATH = CapacitySchedulerConfiguration.ROOT + ".e"; - - public static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - public static final QueuePath DEFAULT = new QueuePath(DEFAULT_PATH); - public static final QueuePath A = new QueuePath(A_PATH); - public static final QueuePath B = new QueuePath(B_PATH); - public static final QueuePath C = new QueuePath(C_PATH); - public static final QueuePath D = new QueuePath(D_PATH); - public static final QueuePath E = new QueuePath(E_PATH); - public static final String ESUBGROUP1_PATH = + public static final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + public static final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + public static final String C = CapacitySchedulerConfiguration.ROOT + ".c"; + public static final String D = CapacitySchedulerConfiguration.ROOT + ".d"; + public static final String E = CapacitySchedulerConfiguration.ROOT + ".e"; + public static final String ESUBGROUP1 = CapacitySchedulerConfiguration.ROOT + ".esubgroup1"; - public static final String FGROUP_PATH = + public static final String FGROUP = CapacitySchedulerConfiguration.ROOT + ".fgroup"; - public static final String A1_PATH = A_PATH + ".a1"; - public static final String A2_PATH = A_PATH + ".a2"; - public static final String B1_PATH = B_PATH + ".b1"; - public static final String B2_PATH = B_PATH + ".b2"; - public static final String B3_PATH = B_PATH + ".b3"; - public static final String B4_PATH = B_PATH + ".b4subgroup1"; - public static final String ESUBGROUP1_A_PATH = ESUBGROUP1_PATH + ".e"; - public static final String FGROUP_F_PATH = FGROUP_PATH + ".f"; - - public static final QueuePath A1 = new QueuePath(A1_PATH); - public static final QueuePath A2 = new QueuePath(A2_PATH); - public static final QueuePath B1 = new QueuePath(B1_PATH); - public static final QueuePath B2 = new QueuePath(B2_PATH); - public static final QueuePath B3 = new QueuePath(B3_PATH); - public static final QueuePath B4 = new QueuePath(B4_PATH); - public static final QueuePath E_GROUP = new QueuePath(ESUBGROUP1_PATH); - public static final QueuePath F_GROUP = new QueuePath(FGROUP_PATH); - public static final QueuePath E_SG = new QueuePath(ESUBGROUP1_A_PATH); - public static final QueuePath F_SG = new QueuePath(FGROUP_F_PATH); - + public static final String A1 = A + ".a1"; + public static final String A2 = A + ".a2"; + public static final String B1 = B + ".b1"; + public static final String B2 = B + ".b2"; + public static final String B3 = B + ".b3"; + public static final String B4 = B + ".b4subgroup1"; + public static final String ESUBGROUP1_A = ESUBGROUP1 + ".e"; + public static final String FGROUP_F = FGROUP + ".f"; public static final float A_CAPACITY = 20f; public static final float B_CAPACITY = 20f; public static final float C_CAPACITY = 20f; @@ -178,7 +159,6 @@ public class TestCapacitySchedulerAutoCreatedQueueBase { public static final String USER2 = USER + 2; public static final String USER3 = USER + 3; public static final String PARENT_QUEUE = "c"; - public static final QueuePath PARENT_QUEUE_PATH = new QueuePath(PARENT_QUEUE); public static final Set accessibleNodeLabelsOnC = new HashSet<>(); @@ -395,8 +375,8 @@ public static CapacitySchedulerConfiguration setupQueueConfiguration( conf.setCapacity(B, B_CAPACITY); conf.setCapacity(C, C_CAPACITY); conf.setCapacity(D, D_CAPACITY); - conf.setCapacity(E_GROUP, ESUBGROUP1_CAPACITY); - conf.setCapacity(F_GROUP, FGROUP_CAPACITY); + conf.setCapacity(ESUBGROUP1, ESUBGROUP1_CAPACITY); + conf.setCapacity(FGROUP, FGROUP_CAPACITY); // Define 2nd-level queues conf.setQueues(A, new String[] { "a1", "a2" }); @@ -415,12 +395,12 @@ public static CapacitySchedulerConfiguration setupQueueConfiguration( conf.setCapacity(B4, B4_CAPACITY); conf.setUserLimitFactor(B4, 100.0f); - conf.setQueues(E_GROUP, new String[] {"e"}); - conf.setCapacity(E_SG, 100f); - conf.setUserLimitFactor(E_SG, 100.0f); - conf.setQueues(F_GROUP, new String[] {"f"}); - conf.setCapacity(F_SG, 100f); - conf.setUserLimitFactor(F_SG, 100.0f); + conf.setQueues(ESUBGROUP1, new String[] {"e"}); + conf.setCapacity(ESUBGROUP1_A, 100f); + conf.setUserLimitFactor(ESUBGROUP1_A, 100.0f); + conf.setQueues(FGROUP, new String[] {"f"}); + conf.setCapacity(FGROUP_F, 100f); + conf.setUserLimitFactor(FGROUP_F, 100.0f); conf.setUserLimitFactor(C, 1.0f); conf.setAutoCreateChildQueueEnabled(C, true); @@ -491,7 +471,7 @@ public static CapacitySchedulerConfiguration setupQueueConfiguration( //setup new queues with one of them auto enabled // Define top-level queues // Set childQueue for root - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"c"}); conf.setCapacity(C, 100f); @@ -513,7 +493,7 @@ public static void setupQueueConfigurationForSingleFlexibleAutoCreatedLeafQueue( //setup new queues with one of them auto enabled // Define top-level queues // Set childQueue for root - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"c"}); conf.setCapacity(C, 100f); @@ -816,8 +796,8 @@ protected void validateEffectiveMinResource(ResourceManager rm, if (expectedQueueEntitlements.get(label).getCapacity() > EPSILON) { if (leafQueue.getCapacityConfigType().equals(ABSOLUTE_RESOURCE)) { - QueuePath templatePrefix = QueuePrefixes.getAutoCreatedQueueObjectTemplateConfPrefix( - parentQueue.getQueuePathObject()); + String templatePrefix = cs.getConfiguration().getAutoCreatedQueueTemplateConfPrefix( + parentQueue.getQueuePath()); Resource resourceTemplate = parentQueue.getLeafQueueTemplate().getLeafQueueConfigs() .getMinimumResourceRequirement(label, templatePrefix, RESOURCE_TYPES); assertEquals(resourceTemplate, leafQueue.getEffectiveCapacity(label)); 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/TestCapacitySchedulerAutoCreatedQueuePreemption.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/TestCapacitySchedulerAutoCreatedQueuePreemption.java index 09724b7d8d6ce..a025f1ed7de0f 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/TestCapacitySchedulerAutoCreatedQueuePreemption.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/TestCapacitySchedulerAutoCreatedQueuePreemption.java @@ -26,8 +26,6 @@ .capacity.TestCapacitySchedulerAutoCreatedQueueBase.D; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler .capacity.TestCapacitySchedulerAutoCreatedQueueBase.E; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler - .capacity.TestCapacitySchedulerAutoCreatedQueueBase.ROOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler .capacity.TestCapacitySchedulerAutoCreatedQueueBase.USER0; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler @@ -62,7 +60,7 @@ public void setUp() throws Exception { //setup new queues with one of them auto enabled // Define top-level queues // Set childQueue for root - conf.setQueues(ROOT, new String[] {"c"}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "c" }); conf.setCapacity(C, 100f); conf.setUserLimitFactor(C, 1.0f); @@ -93,7 +91,7 @@ public void setUp() throws Exception { //setup new queues with one of them auto enabled // Define top-level queues // Set childQueue for root - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "c", "d", "e" }); conf.setCapacity(C, 45f); conf.setCapacity(D, 45f); @@ -122,8 +120,8 @@ public void setUp() throws Exception { conf.setAutoCreatedLeafQueueConfigUserLimit(E, 100); conf.setAutoCreatedLeafQueueConfigUserLimitFactor(E, 3.0f); - conf.setQueuePriority(C, 1); - conf.setQueuePriority(D, 2); + conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".c", 1); + conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".d", 2); return conf; } 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 cd00d31bcd03e..590b0a4b7668d 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 @@ -584,7 +584,7 @@ public void testAutoQueueCreationFailsForEmptyPathWithAQCAndWeightMode() //And weight mode, to allow dynamic auto queue creation for root CapacitySchedulerConfiguration conf = setupSchedulerConfiguration(); conf.setAutoQueueCreationV2Enabled(ROOT, true); - conf.setCapacity(DEFAULT, "1w"); + conf.setCapacity("root.default", "1w"); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); @@ -1006,14 +1006,15 @@ public void testDynamicAutoQueueCreationWithTags() try { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] {"a", "b"}); - csConf.setCapacity(A, 90); - csConf.setCapacity(B, 10); - csConf.setAutoCreateChildQueueEnabled(A, true); - csConf.setAutoCreatedLeafQueueConfigCapacity(A, 50); - csConf.setAutoCreatedLeafQueueConfigMaxCapacity(A, 100); - csConf.setAcl(A, QueueACL.ADMINISTER_QUEUE, "*"); - csConf.setAcl(A, QueueACL.SUBMIT_APPLICATIONS, "*"); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a", "b"}); + csConf.setCapacity("root.a", 90); + csConf.setCapacity("root.b", 10); + csConf.setAutoCreateChildQueueEnabled("root.a", true); + csConf.setAutoCreatedLeafQueueConfigCapacity("root.a", 50); + csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.a", 100); + csConf.setAcl("root.a", QueueACL.ADMINISTER_QUEUE, "*"); + csConf.setAcl("root.a", QueueACL.SUBMIT_APPLICATIONS, "*"); csConf.setBoolean(YarnConfiguration .APPLICATION_TAG_BASED_PLACEMENT_ENABLED, true); csConf.setStrings(YarnConfiguration 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/TestCapacitySchedulerConfigValidator.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/TestCapacitySchedulerConfigValidator.java index 8c9f518fcfbeb..7303a1407236d 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/TestCapacitySchedulerConfigValidator.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/TestCapacitySchedulerConfigValidator.java @@ -699,10 +699,10 @@ private CapacitySchedulerConfiguration setupCSConfiguration(YarnConfiguration co csConf.set(YarnConfiguration.RESOURCE_TYPES, ResourceInformation.GPU_URI); } - csConf.setQueues(new QueuePath(CapacitySchedulerConfiguration.ROOT), + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{PARENT_A, PARENT_B}); - csConf.setQueues(PARENT_A_FULL_PATH, new String[]{LEAF_A}); - csConf.setQueues(PARENT_B_FULL_PATH, new String[]{LEAF_B}); + csConf.setQueues(PARENT_A_FULL_PATH.getFullPath(), new String[]{LEAF_A}); + csConf.setQueues(PARENT_B_FULL_PATH.getFullPath(), new String[]{LEAF_B}); if (useDominantRC) { setupGpuResourceValues(); 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/TestCapacitySchedulerConfiguration.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/TestCapacitySchedulerConfiguration.java index 85e0ff494eb93..2cbb11fc9bdbe 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/TestCapacitySchedulerConfiguration.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/TestCapacitySchedulerConfiguration.java @@ -24,15 +24,15 @@ import java.util.Set; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueuePrefix; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class TestCapacitySchedulerConfiguration { - private static final String ROOT_TEST_PATH = CapacitySchedulerConfiguration.ROOT + ".test"; - private static final QueuePath ROOT_TEST = new QueuePath(ROOT_TEST_PATH); - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); + private static final String ROOT_TEST = ROOT + ".test"; private static final String EMPTY_ACL = ""; private static final String SPACE_ACL = " "; private static final String USER1 = "user1"; @@ -47,29 +47,29 @@ private CapacitySchedulerConfiguration createDefaultCsConf() { return new CapacitySchedulerConfiguration(new Configuration(false), false); } - private AccessControlList getSubmitAcl(CapacitySchedulerConfiguration csConf, QueuePath queue) { + private AccessControlList getSubmitAcl(CapacitySchedulerConfiguration csConf, String queue) { return csConf.getAcl(queue, QueueACL.SUBMIT_APPLICATIONS); } - private void setSubmitAppsConfig(CapacitySchedulerConfiguration csConf, QueuePath queue, + private void setSubmitAppsConfig(CapacitySchedulerConfiguration csConf, String queue, String value) { csConf.set(getSubmitAppsConfigKey(queue), value); } - private String getSubmitAppsConfigKey(QueuePath queue) { - return QueuePrefixes.getQueuePrefix(queue) + "acl_submit_applications"; + private String getSubmitAppsConfigKey(String queue) { + return getQueuePrefix(queue) + "acl_submit_applications"; } - private void testWithGivenAclNoOneHasAccess(QueuePath queue, String aclValue) { + private void testWithGivenAclNoOneHasAccess(String queue, String aclValue) { testWithGivenAclNoOneHasAccessInternal(queue, queue, aclValue); } - private void testWithGivenAclNoOneHasAccess(QueuePath queueToSet, QueuePath queueToVerify, + private void testWithGivenAclNoOneHasAccess(String queueToSet, String queueToVerify, String aclValue) { testWithGivenAclNoOneHasAccessInternal(queueToSet, queueToVerify, aclValue); } - private void testWithGivenAclNoOneHasAccessInternal(QueuePath queueToSet, QueuePath queueToVerify, + private void testWithGivenAclNoOneHasAccessInternal(String queueToSet, String queueToVerify, String aclValue) { CapacitySchedulerConfiguration csConf = createDefaultCsConf(); setSubmitAppsConfig(csConf, queueToSet, aclValue); @@ -79,14 +79,14 @@ private void testWithGivenAclNoOneHasAccessInternal(QueuePath queueToSet, QueueP assertFalse(acl.isAllAllowed()); } - private void testWithGivenAclCorrectUserAndGroupHasAccess(QueuePath queue, String aclValue, + private void testWithGivenAclCorrectUserAndGroupHasAccess(String queue, String aclValue, Set expectedUsers, Set expectedGroups) { testWithGivenAclCorrectUserAndGroupHasAccessInternal(queue, queue, aclValue, expectedUsers, expectedGroups); } - private void testWithGivenAclCorrectUserAndGroupHasAccessInternal(QueuePath queueToSet, - QueuePath queueToVerify, String aclValue, Set expectedUsers, + private void testWithGivenAclCorrectUserAndGroupHasAccessInternal(String queueToSet, + String queueToVerify, String aclValue, Set expectedUsers, Set expectedGroups) { CapacitySchedulerConfiguration csConf = createDefaultCsConf(); setSubmitAppsConfig(csConf, queueToSet, aclValue); 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 e8577fba2c387..e8bbaec17c6fb 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 @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; - import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A1_CAPACITY; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A2_CAPACITY; @@ -31,7 +30,6 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B3; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B3_CAPACITY; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B_CAPACITY; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.ROOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.checkQueueStructureCapacities; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.getDefaultCapacities; import static org.junit.Assert.assertEquals; @@ -294,7 +292,7 @@ public void testMoveAppToPlanQueue() throws Exception { private void setupPlanQueueConfiguration(CapacitySchedulerConfiguration conf) { - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" }); conf.setCapacity(A, A_CAPACITY); 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/TestCapacitySchedulerLazyPreemption.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/TestCapacitySchedulerLazyPreemption.java index 8f07668192dff..e5c6710126d47 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/TestCapacitySchedulerLazyPreemption.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/TestCapacitySchedulerLazyPreemption.java @@ -651,8 +651,7 @@ public void testPreemptionConsidersUserLimit() * from app1, and app2 will receive the preempted container */ CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf); - QueuePath cQueuePath = new QueuePath(CapacitySchedulerConfiguration.ROOT + ".c"); - csConf.setUserLimitFactor(cQueuePath, 0.1f); + csConf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".c", 0.1f); MockRM rm1 = new MockRM(csConf); rm1.getRMContext().setNodeLabelManager(mgr); rm1.start(); 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/TestCapacitySchedulerMultiNodes.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/TestCapacitySchedulerMultiNodes.java index 484308442ff31..b08903893ffeb 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/TestCapacitySchedulerMultiNodes.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/TestCapacitySchedulerMultiNodes.java @@ -64,7 +64,6 @@ public class TestCapacitySchedulerMultiNodes { private static final Logger LOG = LoggerFactory .getLogger(TestCapacitySchedulerMultiNodes.class); - private static final QueuePath DEFAULT = new QueuePath("root.default"); private CapacitySchedulerConfiguration conf; private static final String POLICY_CLASS_NAME = "org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceUsageMultiNodeLookupPolicy"; @@ -205,7 +204,7 @@ public void testExcessReservationWillBeUnreserved() throws Exception { YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER); newConf.setInt(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME + ".resource-based.sorting-interval.ms", 0); - newConf.setMaximumApplicationMasterResourcePerQueuePercent(DEFAULT, + newConf.setMaximumApplicationMasterResourcePerQueuePercent("root.default", 1.0f); MockRM rm1 = new MockRM(newConf); @@ -291,7 +290,7 @@ public void testAllocateForReservedContainer() throws Exception { YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER); newConf.setInt(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME + ".resource-based.sorting-interval.ms", 0); - newConf.setMaximumApplicationMasterResourcePerQueuePercent(DEFAULT, + newConf.setMaximumApplicationMasterResourcePerQueuePercent("root.default", 1.0f); MockRM rm1 = new MockRM(newConf); @@ -362,7 +361,7 @@ public void testAllocateOfReservedContainerFromAnotherNode() YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER); newConf.setInt(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME + ".resource-based.sorting-interval.ms", 0); - newConf.setMaximumApplicationMasterResourcePerQueuePercent(DEFAULT, + newConf.setMaximumApplicationMasterResourcePerQueuePercent("root.default", 1.0f); MockRM rm1 = new MockRM(newConf); @@ -491,7 +490,7 @@ public void testSkipAllocationOnNodeReservedByAnotherApp() throws Exception { YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER); newConf.setInt(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME + ".resource-based.sorting-interval.ms", 0); - newConf.setMaximumApplicationMasterResourcePerQueuePercent(DEFAULT, 1.0f); + newConf.setMaximumApplicationMasterResourcePerQueuePercent("root.default", 1.0f); newConf.set(CapacitySchedulerConfiguration.SKIP_ALLOCATE_ON_NODES_WITH_RESERVED_CONTAINERS, "true"); MockRM rm1 = new MockRM(newConf); 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/TestCapacitySchedulerNewQueueAutoCreation.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/TestCapacitySchedulerNewQueueAutoCreation.java index 9e2134143b8a4..d2891379c47e0 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/TestCapacitySchedulerNewQueueAutoCreation.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/TestCapacitySchedulerNewQueueAutoCreation.java @@ -59,12 +59,6 @@ public class TestCapacitySchedulerNewQueueAutoCreation private static final Logger LOG = LoggerFactory.getLogger( org.apache.hadoop.yarn.server.resourcemanager .scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.class); - private static final QueuePath EMPTY_AUTO_PARENT = new QueuePath("root.empty-auto-parent"); - private static final QueuePath A_A2_AUTO = new QueuePath("root.a.a2-auto"); - private static final QueuePath E_AUTO = new QueuePath("root.e-auto"); - private static final QueuePath E_E1 = new QueuePath("root.e.e1"); - private static final QueuePath A_A_AUTO_A2 = new QueuePath("root.a.a-auto.a2"); - private static final QueuePath A_A1_AUTO_A2_AUTO = new QueuePath("root.a.a1-auto.a2-auto"); public static final int GB = 1024; public static final int MAX_MEMORY = 1200; private MockRM mockRM = null; @@ -97,16 +91,16 @@ public void setUp() throws Exception { ResourceScheduler.class); // By default, set 3 queues, a/b, and a.a1 - csConf.setQueues(ROOT, new String[]{"a", "b"}); - csConf.setNonLabeledQueueWeight(ROOT, 1f); - csConf.setNonLabeledQueueWeight(A, 1f); - csConf.setNonLabeledQueueWeight(B, 1f); - csConf.setQueues(A, new String[]{"a1"}); - csConf.setNonLabeledQueueWeight(A1, 1f); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); - csConf.setAutoQueueCreationV2Enabled(A, true); - csConf.setAutoQueueCreationV2Enabled(E, true); - csConf.setAutoQueueCreationV2Enabled(new QueuePath(PARENT_QUEUE), true); + csConf.setQueues("root", new String[]{"a", "b"}); + csConf.setNonLabeledQueueWeight("root", 1f); + csConf.setNonLabeledQueueWeight("root.a", 1f); + csConf.setNonLabeledQueueWeight("root.b", 1f); + csConf.setQueues("root.a", new String[]{"a1"}); + csConf.setNonLabeledQueueWeight("root.a.a1", 1f); + csConf.setAutoQueueCreationV2Enabled("root", true); + csConf.setAutoQueueCreationV2Enabled("root.a", true); + csConf.setAutoQueueCreationV2Enabled("root.e", true); + csConf.setAutoQueueCreationV2Enabled(PARENT_QUEUE, true); // Test for auto deletion when expired csConf.setAutoExpiredDeletionTime(1); } @@ -294,7 +288,7 @@ public void testAutoCreateQueueStaticParentOneLevel() throws Exception { public void testAutoCreateQueueAutoParentTwoLevelsWithSiblings() throws Exception { startScheduler(); - csConf.setAutoQueueCreationV2Enabled(A_A2_AUTO, true); + csConf.setAutoQueueCreationV2Enabled("root.a.a2-auto", true); // root.a has 1/2 abs resource -> a1 and a2-auto same weight 1/4 // -> a3-auto is alone with weight 1/4 @@ -332,9 +326,9 @@ public void testAutoCreateQueueWhenSiblingsNotInWeightMode() // create a new dynamic queue when the sibling is // not in weight mode assumeThat(csConf.isLegacyQueueMode(), is(true)); - csConf.setCapacity(A, 50f); - csConf.setCapacity(B, 50f); - csConf.setCapacity(A1, 100f); + csConf.setCapacity("root.a", 50f); + csConf.setCapacity("root.b", 50f); + csConf.setCapacity("root.a.a1", 100f); cs.reinitialize(csConf, mockRM.getRMContext()); createQueue("root.a.a2-auto"); } @@ -348,7 +342,7 @@ public void testAutoCreateMaximumQueueDepth() () -> createQueue("root.a.a3-auto.a4-auto.a5-auto")); // Set depth 3 for root.a, making it a valid scenario - csConf.setMaximumAutoCreatedQueueDepth(A, 3); + csConf.setMaximumAutoCreatedQueueDepth("root.a", 3); cs.reinitialize(csConf, mockRM.getRMContext()); try { createQueue("root.a.a3-auto.a4-auto.a5-auto"); @@ -359,7 +353,7 @@ public void testAutoCreateMaximumQueueDepth() // Set global depth to 3 csConf.setMaximumAutoCreatedQueueDepth(3); - csConf.unset(QueuePrefixes.getQueuePrefix(A) + csConf.unset(CapacitySchedulerConfiguration.getQueuePrefix("root.a") + CapacitySchedulerConfiguration.MAXIMUM_QUEUE_DEPTH); cs.reinitialize(csConf, mockRM.getRMContext()); try { @@ -370,7 +364,7 @@ public void testAutoCreateMaximumQueueDepth() } // Set depth on a dynamic queue, which has no effect on auto queue creation validation - csConf.setMaximumAutoCreatedQueueDepth(new QueuePath("root.a.a6-auto.a7-auto.a8-auto"), 10); + csConf.setMaximumAutoCreatedQueueDepth("root.a.a6-auto.a7-auto.a8-auto", 10); Assert.assertThrows(SchedulerDynamicEditException.class, () -> createQueue("root.a.a6-auto.a7-auto.a8-auto.a9-auto.a10-auto.a11-auto")); } @@ -379,7 +373,7 @@ public void testAutoCreateMaximumQueueDepth() public void testAutoCreateQueueShouldFailIfNotEnabledForParent() throws Exception { startScheduler(); - csConf.setAutoQueueCreationV2Enabled(ROOT, false); + csConf.setAutoQueueCreationV2Enabled("root", false); cs.reinitialize(csConf, mockRM.getRMContext()); createQueue("root.c-auto"); } @@ -407,7 +401,7 @@ public void testConvertDynamicToStaticQueue() throws Exception { createBasicQueueStructureAndValidate(); // Now, update root.a's weight to 6 - csConf.setNonLabeledQueueWeight(new QueuePath("root.a"), 6f); + csConf.setNonLabeledQueueWeight("root.a", 6f); cs.reinitialize(csConf, mockRM.getRMContext()); // Double confirm, after refresh, we should still see root queue has 5 @@ -425,8 +419,8 @@ public void testConvertDynamicToStaticQueue() throws Exception { Assert.assertEquals(6f, a.getQueueCapacities().getWeight(), 1e-6); // Set queue c-auto's weight to 6, and mark c-auto to be static queue - csConf.setQueues(ROOT, new String[]{"a", "b", "c-auto"}); - csConf.setNonLabeledQueueWeight(new QueuePath("root.c-auto"), 6f); + csConf.setQueues("root", new String[]{"a", "b", "c-auto"}); + csConf.setNonLabeledQueueWeight("root.c-auto", 6f); cs.reinitialize(csConf, mockRM.getRMContext()); // Get queue c @@ -443,10 +437,10 @@ public void testConvertDynamicToStaticQueue() throws Exception { createQueue("root.e-auto.e2-auto"); // Do change 2nd level queue from dynamic to static - csConf.setQueues(ROOT, new String[]{"a", "b", "c-auto", "e-auto"}); - csConf.setNonLabeledQueueWeight(new QueuePath("root.e-auto"), 6f); - csConf.setQueues(E_AUTO, new String[]{"e1-auto"}); - csConf.setNonLabeledQueueWeight(new QueuePath("root.e-auto.e1-auto"), 6f); + csConf.setQueues("root", new String[]{"a", "b", "c-auto", "e-auto"}); + csConf.setNonLabeledQueueWeight("root.e-auto", 6f); + csConf.setQueues("root.e-auto", new String[]{"e1-auto"}); + csConf.setNonLabeledQueueWeight("root.e-auto.e1-auto", 6f); cs.reinitialize(csConf, mockRM.getRMContext()); // Get queue e1 @@ -473,9 +467,9 @@ public void testConvertDynamicToStaticQueue() throws Exception { public void testConvertDynamicParentToStaticParent() throws Exception { startScheduler(); createQueue("root.d-auto.d1-auto"); - csConf.setQueues(ROOT, new String[]{"a", "b", "d-auto"}); - csConf.setNonLabeledQueueWeight(new QueuePath("root.a"), 6f); - csConf.setNonLabeledQueueWeight(new QueuePath("root.d-auto"), 1f); + csConf.setQueues("root", new String[]{"a", "b", "d-auto"}); + csConf.setNonLabeledQueueWeight("root.a", 6f); + csConf.setNonLabeledQueueWeight("root.d-auto", 1f); cs.reinitialize(csConf, mockRM.getRMContext()); CSQueue d = cs.getQueue("root.d-auto"); @@ -510,14 +504,14 @@ public void testAutoQueueCreationOnAppSubmission() throws Exception { public void testChildlessParentQueueWhenAutoQueueCreationEnabled() throws Exception { startScheduler(); - csConf.setQueues(ROOT, new String[]{"a", "b", "empty-auto-parent"}); - csConf.setNonLabeledQueueWeight(ROOT, 1f); - csConf.setNonLabeledQueueWeight(A, 1f); - csConf.setNonLabeledQueueWeight(B, 1f); - csConf.setQueues(A, new String[]{"a1"}); - csConf.setNonLabeledQueueWeight(A1, 1f); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); - csConf.setAutoQueueCreationV2Enabled(A, true); + csConf.setQueues("root", new String[]{"a", "b", "empty-auto-parent"}); + csConf.setNonLabeledQueueWeight("root", 1f); + csConf.setNonLabeledQueueWeight("root.a", 1f); + csConf.setNonLabeledQueueWeight("root.b", 1f); + csConf.setQueues("root.a", new String[]{"a1"}); + csConf.setNonLabeledQueueWeight("root.a.a1", 1f); + csConf.setAutoQueueCreationV2Enabled("root", true); + csConf.setAutoQueueCreationV2Enabled("root.a", true); cs.reinitialize(csConf, mockRM.getRMContext()); CSQueue empty = cs.getQueue("root.empty-auto-parent"); @@ -525,15 +519,15 @@ public void testChildlessParentQueueWhenAutoQueueCreationEnabled() empty instanceof LeafQueue); empty.stopQueue(); - csConf.setQueues(ROOT, new String[]{"a", "b", "empty-auto-parent"}); - csConf.setNonLabeledQueueWeight(ROOT, 1f); - csConf.setNonLabeledQueueWeight(A, 1f); - csConf.setNonLabeledQueueWeight(B, 1f); - csConf.setQueues(A, new String[]{"a1"}); - csConf.setNonLabeledQueueWeight(A1, 1f); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); - csConf.setAutoQueueCreationV2Enabled(A, true); - csConf.setAutoQueueCreationV2Enabled(EMPTY_AUTO_PARENT, true); + csConf.setQueues("root", new String[]{"a", "b", "empty-auto-parent"}); + csConf.setNonLabeledQueueWeight("root", 1f); + csConf.setNonLabeledQueueWeight("root.a", 1f); + csConf.setNonLabeledQueueWeight("root.b", 1f); + csConf.setQueues("root.a", new String[]{"a1"}); + csConf.setNonLabeledQueueWeight("root.a.a1", 1f); + csConf.setAutoQueueCreationV2Enabled("root", true); + csConf.setAutoQueueCreationV2Enabled("root.a", true); + csConf.setAutoQueueCreationV2Enabled("root.empty-auto-parent", true); cs.reinitialize(csConf, mockRM.getRMContext()); empty = cs.getQueue("root.empty-auto-parent"); @@ -650,7 +644,7 @@ public void testAutoQueueCreationMaxAppUpdate() throws Exception { Assert.assertEquals(1000, e.getMaxApplications()); // when set some queue for max apps - csConf.setMaximumApplicationsPerQueue(new QueuePath("root.e1"), 50); + csConf.setMaximumApplicationsPerQueue("root.e1", 50); createQueue("root.e1"); LeafQueue e1 = (LeafQueue)cs. getQueue("root.e1"); @@ -711,7 +705,7 @@ public void testAutoCreateQueueIfAmbiguousQueueNames() throws Exception { public void testAutoCreateQueueMaxQueuesLimit() throws Exception { startScheduler(); - csConf.setAutoCreatedQueuesV2MaxChildQueuesLimit(E, 5); + csConf.setAutoCreatedQueuesV2MaxChildQueuesLimit("root.e", 5); cs.reinitialize(csConf, mockRM.getRMContext()); for (int i = 0; i < 5; ++i) { @@ -731,13 +725,8 @@ public void testAutoCreateQueueMaxQueuesLimit() throws Exception { @Test public void testAutoCreatedQueueTemplateConfig() throws Exception { startScheduler(); - - QueuePath childQueuesOfA = new QueuePath("root.a.*"); - QueuePath aQueuePath = new QueuePath("root.a"); - QueuePath cQueuePath = new QueuePath("root.c"); - csConf.set(AutoCreatedQueueTemplate.getAutoQueueTemplatePrefix( - childQueuesOfA) + "capacity", "6w"); + "root.a.*") + "capacity", "6w"); cs.reinitialize(csConf, mockRM.getRMContext()); AbstractLeafQueue a2 = createQueue("root.a.a-auto.a2"); @@ -751,9 +740,9 @@ public void testAutoCreatedQueueTemplateConfig() throws Exception { // Set the user-limit-factor and maximum-am-resource-percent via templates to ensure their // modified defaults are indeed overridden csConf.set(AutoCreatedQueueTemplate.getAutoQueueTemplatePrefix( - childQueuesOfA) + "user-limit-factor", "10"); + "root.a.*") + "user-limit-factor", "10"); csConf.set(AutoCreatedQueueTemplate.getAutoQueueTemplatePrefix( - childQueuesOfA) + "maximum-am-resource-percent", "0.8"); + "root.a.*") + "maximum-am-resource-percent", "0.8"); cs.reinitialize(csConf, mockRM.getRMContext()); a2 = (LeafQueue) cs.getQueue("root.a.a-auto.a2"); @@ -765,13 +754,13 @@ public void testAutoCreatedQueueTemplateConfig() throws Exception { 0.8f, a2.getMaxAMResourcePerQueuePercent(), 1e-6); - csConf.setNonLabeledQueueWeight(new QueuePath("root.a.a-auto.a2"), 4f); + csConf.setNonLabeledQueueWeight("root.a.a-auto.a2", 4f); cs.reinitialize(csConf, mockRM.getRMContext()); Assert.assertEquals("weight is not explicitly set", 4f, a2.getQueueCapacities().getWeight(), 1e-6); csConf.setBoolean(AutoCreatedQueueTemplate.getAutoQueueTemplatePrefix( - aQueuePath) + CapacitySchedulerConfiguration + "root.a") + CapacitySchedulerConfiguration .AUTO_CREATE_CHILD_QUEUE_AUTO_REMOVAL_ENABLE, false); cs.reinitialize(csConf, mockRM.getRMContext()); AbstractLeafQueue a3 = createQueue("root.a.a3"); @@ -780,9 +769,9 @@ public void testAutoCreatedQueueTemplateConfig() throws Exception { // Set the capacity of label TEST csConf.set(AutoCreatedQueueTemplate.getAutoQueueTemplatePrefix( - cQueuePath) + "accessible-node-labels.TEST.capacity", "6w"); - csConf.setQueues(ROOT, new String[]{"a", "b", "c"}); - csConf.setAutoQueueCreationV2Enabled(C, true); + "root.c") + "accessible-node-labels.TEST.capacity", "6w"); + csConf.setQueues("root", new String[]{"a", "b", "c"}); + csConf.setAutoQueueCreationV2Enabled("root.c", true); cs.reinitialize(csConf, mockRM.getRMContext()); AbstractLeafQueue c1 = createQueue("root.c.c1"); Assert.assertEquals("weight is not set for label TEST", 6f, @@ -797,19 +786,19 @@ public void testAutoCreatedQueueTemplateConfig() throws Exception { public void testAutoCreatedQueueConfigChange() throws Exception { startScheduler(); AbstractLeafQueue a2 = createQueue("root.a.a-auto.a2"); - csConf.setNonLabeledQueueWeight(a2.getQueuePathObject(), 4f); + csConf.setNonLabeledQueueWeight("root.a.a-auto.a2", 4f); cs.reinitialize(csConf, mockRM.getRMContext()); Assert.assertEquals("weight is not explicitly set", 4f, a2.getQueueCapacities().getWeight(), 1e-6); a2 = (AbstractLeafQueue) cs.getQueue("root.a.a-auto.a2"); - csConf.setState(A_A_AUTO_A2, QueueState.STOPPED); + csConf.setState("root.a.a-auto.a2", QueueState.STOPPED); cs.reinitialize(csConf, mockRM.getRMContext()); Assert.assertEquals("root.a.a-auto.a2 has not been stopped", QueueState.STOPPED, a2.getState()); - csConf.setState(A_A_AUTO_A2, QueueState.RUNNING); + csConf.setState("root.a.a-auto.a2", QueueState.RUNNING); cs.reinitialize(csConf, mockRM.getRMContext()); Assert.assertEquals("root.a.a-auto.a2 is not running", QueueState.RUNNING, a2.getState()); @@ -820,9 +809,9 @@ public void testAutoCreateQueueState() throws Exception { startScheduler(); createQueue("root.e.e1"); - csConf.setState(E, QueueState.STOPPED); - csConf.setState(E_E1, QueueState.STOPPED); - csConf.setState(A, QueueState.STOPPED); + csConf.setState("root.e", QueueState.STOPPED); + csConf.setState("root.e.e1", QueueState.STOPPED); + csConf.setState("root.a", QueueState.STOPPED); cs.reinitialize(csConf, mockRM.getRMContext()); // Make sure the static queue is stopped @@ -838,13 +827,13 @@ public void testAutoCreateQueueState() throws Exception { QueueState.STOPPED); // Make root.e state to RUNNING - csConf.setState(E, QueueState.RUNNING); + csConf.setState("root.e", QueueState.RUNNING); cs.reinitialize(csConf, mockRM.getRMContext()); Assert.assertEquals(cs.getQueue("root.e.e1").getState(), QueueState.STOPPED); // Make root.e.e1 state to RUNNING - csConf.setState(E_E1, QueueState.RUNNING); + csConf.setState("root.e.e1", QueueState.RUNNING); cs.reinitialize(csConf, mockRM.getRMContext()); Assert.assertEquals(cs.getQueue("root.e.e1").getState(), QueueState.RUNNING); @@ -970,7 +959,7 @@ public void testCapacitySchedulerAutoQueueDeletionDisabled() startScheduler(); // Test for disabled auto deletion csConf.setAutoExpiredDeletionEnabled( - A_A1_AUTO_A2_AUTO, false); + "root.a.a1-auto.a2-auto", false); csConf.setBoolean( YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true); csConf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES, @@ -1037,7 +1026,7 @@ public void testCapacitySchedulerAutoQueueDeletionDisabled() // Enabled now // The auto deletion will work. csConf.setAutoExpiredDeletionEnabled( - A_A1_AUTO_A2_AUTO, true); + "root.a.a1-auto.a2-auto", true); cs.reinitialize(csConf, mockRM.getRMContext()); // Wait the a2 deleted. @@ -1182,12 +1171,12 @@ public void testRemoveDanglingAutoCreatedQueuesOnReinit() throws Exception { getQueue("root.a.a-auto"); Assert.assertTrue(aAuto.isDynamicQueue()); - csConf.setState(A, QueueState.STOPPED); + csConf.setState("root.a", QueueState.STOPPED); cs.reinitialize(csConf, mockRM.getRMContext()); aAuto = (AbstractCSQueue) cs. getQueue("root.a.a-auto"); Assert.assertEquals("root.a.a-auto is not in STOPPED state", QueueState.STOPPED, aAuto.getState()); - csConf.setQueues(ROOT, new String[]{"b"}); + csConf.setQueues("root", new String[]{"b"}); cs.reinitialize(csConf, mockRM.getRMContext()); CSQueue aAutoNew = cs.getQueue("root.a.a-auto"); Assert.assertNull(aAutoNew); @@ -1197,24 +1186,24 @@ public void testRemoveDanglingAutoCreatedQueuesOnReinit() throws Exception { Assert.assertNotNull(aAutoNew); // Validate static grandparent deletion - csConf.setQueues(ROOT, new String[]{"a", "b"}); - csConf.setQueues(A, new String[]{"a1"}); - csConf.setAutoQueueCreationV2Enabled(A1, true); + csConf.setQueues("root", new String[]{"a", "b"}); + csConf.setQueues("root.a", new String[]{"a1"}); + csConf.setAutoQueueCreationV2Enabled("root.a.a1", true); cs.reinitialize(csConf, mockRM.getRMContext()); createQueue("root.a.a1.a1-auto"); CSQueue a1Auto = cs.getQueue("root.a.a1.a1-auto"); Assert.assertNotNull("a1-auto should exist", a1Auto); - csConf.setQueues(ROOT, new String[]{"b"}); + csConf.setQueues("root", new String[]{"b"}); cs.reinitialize(csConf, mockRM.getRMContext()); a1Auto = cs.getQueue("root.a.a1.a1-auto"); Assert.assertNull("a1-auto has no parent and should not exist", a1Auto); // Validate dynamic parent deletion - csConf.setState(B, QueueState.STOPPED); + csConf.setState("root.b", QueueState.STOPPED); cs.reinitialize(csConf, mockRM.getRMContext()); - csConf.setAutoQueueCreationV2Enabled(B, true); + csConf.setAutoQueueCreationV2Enabled("root.b", true); cs.reinitialize(csConf, mockRM.getRMContext()); createQueue("root.b.b-auto-parent.b-auto-leaf"); 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/TestCapacitySchedulerNewQueueAutoCreationWithCapacityVectors.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/TestCapacitySchedulerNewQueueAutoCreationWithCapacityVectors.java index d7083b3ba0c35..ccbea250f5f13 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/TestCapacitySchedulerNewQueueAutoCreationWithCapacityVectors.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/TestCapacitySchedulerNewQueueAutoCreationWithCapacityVectors.java @@ -90,14 +90,14 @@ private void createPercentageConfig() { // root // - a 25% // a and root has AQCv2 enabled - csConf.setQueues(ROOT, new String[]{"a"}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a"}); csConf.setCapacity(A, 25f); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); + csConf.setAutoQueueCreationV2Enabled(CapacitySchedulerConfiguration.ROOT, true); csConf.setAutoQueueCreationV2Enabled(A, true); // Set up dynamic queue templates - csConf.set(getTemplateKey(ROOT, "capacity"), "6.25"); + csConf.set(getTemplateKey(CapacitySchedulerConfiguration.ROOT, "capacity"), "6.25"); csConf.set(getLeafTemplateKey(A, "capacity"), "[memory=25%, vcores=50%]"); } @@ -105,14 +105,14 @@ private void createAbsoluteConfig() { // root // - a [memory=8000, vcores=8] // a and root has AQCv2 enabled - csConf.setQueues(ROOT, new String[]{"a"}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a"}); csConf.setCapacity(A, "[memory=8000, vcores=8]"); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); + csConf.setAutoQueueCreationV2Enabled(CapacitySchedulerConfiguration.ROOT, true); csConf.setAutoQueueCreationV2Enabled(A, true); // Set up dynamic queue templates - csConf.set(getTemplateKey(ROOT, "capacity"), + csConf.set(getTemplateKey(CapacitySchedulerConfiguration.ROOT, "capacity"), "[memory=2000mb, vcores=2]"); csConf.set(getLeafTemplateKey(A, "capacity"), "[memory=2000, vcores=4]"); @@ -131,23 +131,23 @@ private void createMixedConfig() { // - e1-auto [memory=2048, vcores=2] // Set up static queues - csConf.setQueues(ROOT, new String[]{"a", "b", "d"}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b", "d"}); csConf.setCapacityVector(A, NL, "[memory=10%, vcores=2]"); csConf.setCapacityVector(B, NL, "[memory=2000, vcores=10%]"); csConf.setCapacityVector(D, NL, "[memory=10%, vcores=10%]"); csConf.setQueues(A, new String[]{"a1"}); csConf.setCapacityVector(A1, NL, "[memory=100%, vcores=100%]"); - csConf.setAutoQueueCreationV2Enabled(ROOT, true); + csConf.setAutoQueueCreationV2Enabled(CapacitySchedulerConfiguration.ROOT, true); csConf.setAutoQueueCreationV2Enabled(A, true); csConf.setAutoQueueCreationV2Enabled(D, true); // Set up dynamic queue templates - csConf.set(getTemplateKey(ROOT, "capacity"), + csConf.set(getTemplateKey(CapacitySchedulerConfiguration.ROOT, "capacity"), "[memory=2w, vcores=5w]"); - csConf.set(getParentTemplateKey(ROOT, "capacity"), + csConf.set(getParentTemplateKey(CapacitySchedulerConfiguration.ROOT, "capacity"), "[memory=2w, vcores=10]"); - csConf.set(getLeafTemplateKey(new QueuePath(CapacitySchedulerConfiguration.ROOT + ".*"), - "capacity"), "[memory=2000, vcores=2]"); + csConf.set(getLeafTemplateKey(CapacitySchedulerConfiguration.ROOT + ".*", "capacity"), + "[memory=2000, vcores=2]"); csConf.set(getLeafTemplateKey(D, "capacity"), "[memory=1000, vcores=1]"); } @@ -185,7 +185,7 @@ public void testBasicAbsoluteConfiguration() throws Exception { } private void validateBasicConfiguration() throws Exception { - CSQueue a = cs.getQueue(A_PATH); + CSQueue a = cs.getQueue(A); Assert.assertEquals(8 / 32f, a.getAbsoluteCapacity(), EPSILON); Assert.assertEquals(-1f, a.getQueueCapacities().getWeight(), EPSILON); Assert.assertEquals(8000, @@ -316,18 +316,18 @@ protected AbstractLeafQueue createQueue(String queuePath) throws YarnException, return autoQueueHandler.createQueue(new QueuePath(queuePath)); } - private String getTemplateKey(QueuePath queuePath, String entryKey) { - return QueuePrefixes.getQueuePrefix(queuePath) + private String getTemplateKey(String queuePath, String entryKey) { + return CapacitySchedulerConfiguration.getQueuePrefix(queuePath) + AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX + entryKey; } - private String getParentTemplateKey(QueuePath queuePath, String entryKey) { - return QueuePrefixes.getQueuePrefix(queuePath) + private String getParentTemplateKey(String queuePath, String entryKey) { + return CapacitySchedulerConfiguration.getQueuePrefix(queuePath) + AutoCreatedQueueTemplate.AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + entryKey; } - private String getLeafTemplateKey(QueuePath queuePath, String entryKey) { - return QueuePrefixes.getQueuePrefix(queuePath) + private String getLeafTemplateKey(String queuePath, String entryKey) { + return CapacitySchedulerConfiguration.getQueuePrefix(queuePath) + AutoCreatedQueueTemplate.AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + entryKey; } } 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/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.java index 90a563245a183..5776fea619150 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/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.java @@ -62,7 +62,6 @@ public class TestCapacitySchedulerNodeLabelUpdate { private final int GB = 1024; private YarnConfiguration conf; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); RMNodeLabelsManager mgr; @@ -80,18 +79,17 @@ private Configuration getConfigurationWithQueueLabels(Configuration config) { new CapacitySchedulerConfiguration(config); // Define top-level queues - conf.setQueues(ROOT, new String[] {"a"}); - conf.setCapacityByLabel(ROOT, "x", 100); - conf.setCapacityByLabel(ROOT, "y", 100); - conf.setCapacityByLabel(ROOT, "z", 100); - - final String aPath = CapacitySchedulerConfiguration.ROOT + ".a"; - final QueuePath a = new QueuePath(aPath); - conf.setCapacity(a, 100); - conf.setAccessibleNodeLabels(a, ImmutableSet.of("x", "y", "z")); - conf.setCapacityByLabel(a, "x", 100); - conf.setCapacityByLabel(a, "y", 100); - conf.setCapacityByLabel(a, "z", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100); + + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + conf.setCapacity(A, 100); + conf.setAccessibleNodeLabels(A, ImmutableSet.of("x", "y", "z")); + conf.setCapacityByLabel(A, "x", 100); + conf.setCapacityByLabel(A, "y", 100); + conf.setCapacityByLabel(A, "z", 100); return conf; } @@ -102,25 +100,17 @@ private Configuration getConfigurationWithSubQueueLabels( new CapacitySchedulerConfiguration(config); // Define top-level queues - conf2.setQueues(ROOT, + conf2.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); - conf2.setCapacityByLabel(ROOT, "x", 100); - conf2.setCapacityByLabel(ROOT, "y", 100); - - final String aPath = CapacitySchedulerConfiguration.ROOT + ".a"; - final String bPath = CapacitySchedulerConfiguration.ROOT + ".b"; - final String aa1Path = aPath + ".a1"; - final String aa2Path = aPath + ".a2"; - final String aa3Path = aPath + ".a3"; - final String aa4Path = aPath + ".a4"; - - final QueuePath a = new QueuePath(aPath); - final QueuePath b = new QueuePath(bPath); - final QueuePath aa1 = new QueuePath(aa1Path); - final QueuePath aa2 = new QueuePath(aa2Path); - final QueuePath aa3= new QueuePath(aa3Path); - final QueuePath aa4 = new QueuePath(aa4Path); - + conf2.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf2.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + + final String a = CapacitySchedulerConfiguration.ROOT + ".a"; + final String b = CapacitySchedulerConfiguration.ROOT + ".b"; + final String aa1 = a + ".a1"; + final String aa2 = a + ".a2"; + final String aa3 = a + ".a3"; + final String aa4 = a + ".a4"; conf2.setQueues(a, new String[] {"a1", "a2", "a3", "a4"}); conf2.setCapacity(a, 50); conf2.setCapacity(b, 50); 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/TestCapacitySchedulerNodes.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/TestCapacitySchedulerNodes.java index be1645f27f449..e1cb44599ea22 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/TestCapacitySchedulerNodes.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/TestCapacitySchedulerNodes.java @@ -194,8 +194,8 @@ public void testDefaultNodeLabelExpressionQueueConfig() throws Exception { CapacityScheduler cs = new CapacityScheduler(); CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); setupQueueConfiguration(conf); - conf.setDefaultNodeLabelExpression(new QueuePath("root.a"), " x"); - conf.setDefaultNodeLabelExpression(new QueuePath("root.b"), " y "); + conf.setDefaultNodeLabelExpression("root.a", " x"); + conf.setDefaultNodeLabelExpression("root.b", " y "); cs.setConf(new YarnConfiguration()); cs.setRMContext(resourceManager.getRMContext()); cs.init(conf); 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/TestCapacitySchedulerOvercommit.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/TestCapacitySchedulerOvercommit.java index 4a2a10f6032eb..27eb3ac7bfd55 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/TestCapacitySchedulerOvercommit.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/TestCapacitySchedulerOvercommit.java @@ -29,9 +29,6 @@ */ public class TestCapacitySchedulerOvercommit extends TestSchedulerOvercommit { - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath DEFAULT = new QueuePath(CapacitySchedulerConfiguration.ROOT - + ".default"); @Override protected Configuration getConfiguration() { Configuration conf = super.getConfiguration(); @@ -41,10 +38,14 @@ protected Configuration getConfiguration() { // Remove limits on AMs to allow multiple applications running CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf); - csConf.setMaximumApplicationMasterResourcePerQueuePercent(ROOT, 100.0f); - csConf.setMaximumAMResourcePercentPerPartition(ROOT, "", 100.0f); - csConf.setMaximumApplicationMasterResourcePerQueuePercent(DEFAULT, 100.0f); - csConf.setMaximumAMResourcePercentPerPartition(DEFAULT, "", 100.0f); + csConf.setMaximumApplicationMasterResourcePerQueuePercent( + CapacitySchedulerConfiguration.ROOT, 100.0f); + csConf.setMaximumAMResourcePercentPerPartition( + CapacitySchedulerConfiguration.ROOT, "", 100.0f); + csConf.setMaximumApplicationMasterResourcePerQueuePercent( + CapacitySchedulerConfiguration.ROOT + ".default", 100.0f); + csConf.setMaximumAMResourcePercentPerPartition( + CapacitySchedulerConfiguration.ROOT + ".default", "", 100.0f); return csConf; } 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 24038ebeb8f0b..b8209a54952e7 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 @@ -69,9 +69,6 @@ import static org.mockito.Mockito.when; public class TestCapacitySchedulerPerf { - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath DEFAULT = new QueuePath(CapacitySchedulerConfiguration.ROOT - + ".default"); private final int GB = 1024; private String getResourceName(int idx) { @@ -451,16 +448,16 @@ CapacitySchedulerConfiguration createCSConfWithManyQueues(int numQueues) CapacitySchedulerConfiguration csconf = new CapacitySchedulerConfiguration(); csconf.setResourceComparator(DominantResourceCalculator.class); - csconf.setMaximumApplicationMasterResourcePerQueuePercent(ROOT, 100.0f); - csconf.setMaximumAMResourcePercentPerPartition(ROOT, "", 100.0f); - csconf.setCapacity(DEFAULT, 0.0f); + csconf.setMaximumApplicationMasterResourcePerQueuePercent("root", 100.0f); + csconf.setMaximumAMResourcePercentPerPartition("root", "", 100.0f); + csconf.setCapacity("root.default", 0.0f); csconf.setOffSwitchPerHeartbeatLimit(numQueues); float capacity = 100.0f / numQueues; String[] subQueues = new String[numQueues]; for (int i = 0; i < numQueues; i++) { String queueName = String.format("%03d", i); - QueuePath queuePath = new QueuePath("root." + queueName); + String queuePath = "root." + queueName; subQueues[i] = queueName; csconf.setMaximumApplicationMasterResourcePerQueuePercent( queuePath, 100.0f); @@ -470,7 +467,7 @@ CapacitySchedulerConfiguration createCSConfWithManyQueues(int numQueues) csconf.setMaximumCapacity(queuePath, 100.0f); } - csconf.setQueues(ROOT, subQueues); + csconf.setQueues("root", subQueues); return csconf; } 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/TestCapacitySchedulerQueueACLs.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/TestCapacitySchedulerQueueACLs.java index 841b8f9193fba..bb5b790fe2fc5 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/TestCapacitySchedulerQueueACLs.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/TestCapacitySchedulerQueueACLs.java @@ -39,11 +39,13 @@ public class TestCapacitySchedulerQueueACLs extends QueueACLsTestBase { protected Configuration createConfiguration() { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] { + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { QUEUEA, QUEUEB }); - setQueueCapacity(csConf, 50, A_QUEUE_PATH); - setQueueCapacity(csConf, 50, B_QUEUE_PATH); + setQueueCapacity(csConf, 50, + CapacitySchedulerConfiguration.ROOT + "." + QUEUEA); + setQueueCapacity(csConf, 50, + CapacitySchedulerConfiguration.ROOT + "." + QUEUEB); Map aclsOnQueueA = new HashMap(); @@ -52,7 +54,8 @@ protected Configuration createConfiguration() { AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN); aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA); aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA); - csConf.setAcls(A_QUEUE_PATH, aclsOnQueueA); + csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, + aclsOnQueueA); Map aclsOnQueueB = new HashMap(); @@ -61,7 +64,8 @@ protected Configuration createConfiguration() { AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN); aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB); aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB); - csConf.setAcls(B_QUEUE_PATH, aclsOnQueueB); + csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, + aclsOnQueueB); Map aclsOnRootQueue = new HashMap(); @@ -69,7 +73,7 @@ protected Configuration createConfiguration() { AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN); aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot); aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot); - csConf.setAcls(ROOT, aclsOnRootQueue); + csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue); csConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true); csConf.set(YarnConfiguration.RM_SCHEDULER, @@ -106,30 +110,30 @@ public void updateConfigWithDAndD1Queues(String rootAcl, String queueDAcl, CapacitySchedulerConfiguration csConf = (CapacitySchedulerConfiguration) getConf(); csConf.clear(); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {QUEUED, QUEUEA, QUEUEB}); String dPath = CapacitySchedulerConfiguration.ROOT + "." + QUEUED; String d1Path = dPath + "." + QUEUED1; - QueuePath dQueuePath = new QueuePath(dPath); - QueuePath d1QueuePath = new QueuePath(d1Path); - - csConf.setQueues(dQueuePath, new String[] {QUEUED1}); - setQueueCapacity(csConf, 100, d1QueuePath); - setQueueCapacity(csConf, 30, A_QUEUE_PATH); - setQueueCapacity(csConf, 50, B_QUEUE_PATH); - setQueueCapacity(csConf, 20, dQueuePath); + csConf.setQueues(dPath, new String[] {QUEUED1}); + setQueueCapacity(csConf, 100, d1Path); + setQueueCapacity(csConf, 30, CapacitySchedulerConfiguration.ROOT + + "." + QUEUEA); + setQueueCapacity(csConf, 50, CapacitySchedulerConfiguration.ROOT + + "." + QUEUEB); + setQueueCapacity(csConf, 20, dPath); if (rootAcl != null) { - setAdminAndSubmitACL(csConf, rootAcl, ROOT); + setAdminAndSubmitACL(csConf, rootAcl, + CapacitySchedulerConfiguration.ROOT); } if (queueDAcl != null) { - setAdminAndSubmitACL(csConf, queueDAcl, dQueuePath); + setAdminAndSubmitACL(csConf, queueDAcl, dPath); } if (queueD1Acl != null) { - setAdminAndSubmitACL(csConf, queueD1Acl, d1QueuePath); + setAdminAndSubmitACL(csConf, queueD1Acl, d1Path); } resourceManager.getResourceScheduler() .reinitialize(csConf, resourceManager.getRMContext()); @@ -137,12 +141,12 @@ public void updateConfigWithDAndD1Queues(String rootAcl, String queueDAcl, private void setQueueCapacity(CapacitySchedulerConfiguration csConf, - float capacity, QueuePath queuePath) { + float capacity, String queuePath) { csConf.setCapacity(queuePath, capacity); } private void setAdminAndSubmitACL(CapacitySchedulerConfiguration csConf, - String queueAcl, QueuePath queuePath) { + String queueAcl, String queuePath) { csConf.setAcl(queuePath, QueueACL.ADMINISTER_QUEUE, queueAcl); csConf.setAcl(queuePath, QueueACL.SUBMIT_APPLICATIONS, queueAcl); } 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/TestCapacitySchedulerQueues.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/TestCapacitySchedulerQueues.java index 2f445e636c8dc..77c37d78478cb 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/TestCapacitySchedulerQueues.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/TestCapacitySchedulerQueues.java @@ -50,7 +50,6 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A1; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A2; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.A1_B1; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B1; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B1_CAPACITY; @@ -58,7 +57,6 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B2_CAPACITY; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B3; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B3_CAPACITY; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.ROOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.checkQueueStructureCapacities; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.ExpectedCapacities; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.findQueue; @@ -131,9 +129,9 @@ public void testParseQueue() throws IOException { cs.init(conf); cs.start(); - conf.setQueues(A1, new String[]{"b1"}); - conf.setCapacity(A1_B1, 100.0f); - conf.setUserLimitFactor(A1_B1, 100.0f); + conf.setQueues(CapacitySchedulerConfiguration.ROOT + ".a.a1", new String[]{"b1"}); + conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a.a1.b1", 100.0f); + conf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".a.a1.b1", 100.0f); cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null, new RMContainerTokenSecretManager(conf), @@ -171,8 +169,7 @@ public void testRefreshQueuesWithNewQueue() throws Exception { checkQueueStructureCapacities(cs); // Add a new queue b4 - final String b4Path = B + ".b4"; - final QueuePath b4 = new QueuePath(b4Path); + final String b4 = B + ".b4"; final float b4Capacity = 10; final float modifiedB3Capacity = B3_CAPACITY - b4Capacity; @@ -190,15 +187,15 @@ public void testRefreshQueuesWithNewQueue() throws Exception { final float capB = 20f / 100.0f; Map expectedCapacities = getDefaultCapacities(capA, capB); - expectedCapacities.put(B3.getFullPath(), + expectedCapacities.put(B3, new ExpectedCapacities(modifiedB3Capacity / 100.0f, capB)); - expectedCapacities.put(b4Path, new ExpectedCapacities(b4Capacity / 100.0f, capB)); + expectedCapacities.put(b4, new ExpectedCapacities(b4Capacity / 100.0f, capB)); checkQueueStructureCapacities(cs, expectedCapacities); // Verify parent for B4 CSQueue rootQueue = cs.getRootQueue(); - CSQueue queueB = findQueue(rootQueue, B.getFullPath()); - CSQueue queueB4 = findQueue(queueB, b4Path); + CSQueue queueB = findQueue(rootQueue, B); + CSQueue queueB4 = findQueue(queueB, b4); assertEquals(queueB, queueB4.getParent()); } finally { @@ -229,8 +226,8 @@ public void testRefreshQueuesMaxAllocationRefresh() throws Exception { ResourceUtils.fetchMaximumAllocationFromConfig(conf).getMemorySize()); CSQueue rootQueue = cs.getRootQueue(); - CSQueue queueA = findQueue(rootQueue, A.getFullPath()); - CSQueue queueA1 = findQueue(queueA, A1.getFullPath()); + CSQueue queueA = findQueue(rootQueue, A); + CSQueue queueA1 = findQueue(queueA, A1); assertEquals("queue max allocation", ((LeafQueue) queueA1) .getMaximumAllocation().getMemorySize(), 8192); @@ -312,8 +309,8 @@ public void testRefreshQueuesMaxAllocationRefreshLarger() throws Exception { checkQueueStructureCapacities(cs); CSQueue rootQueue = cs.getRootQueue(); - CSQueue queueA = findQueue(rootQueue, A.getFullPath()); - CSQueue queueA1 = findQueue(queueA, A1.getFullPath()); + CSQueue queueA = findQueue(rootQueue, A); + CSQueue queueA1 = findQueue(queueA, A1); assertEquals("max capability MB in CS", YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, @@ -433,11 +430,11 @@ public void testRefreshQueuesMaxAllocationCSLarger() throws Exception { cs.getMaximumResourceCapability().getVirtualCores()); CSQueue rootQueue = cs.getRootQueue(); - CSQueue queueA = findQueue(rootQueue, A.getFullPath()); - CSQueue queueB = findQueue(rootQueue, B.getFullPath()); - CSQueue queueA1 = findQueue(queueA, A1.getFullPath()); - CSQueue queueA2 = findQueue(queueA, A2.getFullPath()); - CSQueue queueB2 = findQueue(queueB, B2.getFullPath()); + CSQueue queueA = findQueue(rootQueue, A); + CSQueue queueB = findQueue(rootQueue, B); + CSQueue queueA1 = findQueue(queueA, A1); + CSQueue queueA2 = findQueue(queueA, A2); + CSQueue queueB2 = findQueue(queueB, B2); assertEquals("queue A1 max allocation MB", 4096, queueA1.getMaximumAllocation().getMemorySize()); @@ -523,8 +520,8 @@ public void testRefreshQueuesWithQueueDelete() throws Exception { + " a queue WITHOUT running apps"); } CSQueue rootQueue = cs.getRootQueue(); - CSQueue queueB = findQueue(rootQueue, B.getFullPath()); - CSQueue queueB3 = findQueue(queueB, B1.getFullPath()); + CSQueue queueB = findQueue(rootQueue, B); + CSQueue queueB3 = findQueue(queueB, B1); assertNull("Refresh needs to support delete of leaf queue ", queueB3); // reset back to default configuration for testing parent queue delete @@ -574,7 +571,7 @@ public void testRefreshQueuesWithQueueDelete() throws Exception { + " a queue without running apps"); } rootQueue = cs.getRootQueue(); - queueB = findQueue(rootQueue, B.getFullPath()); + queueB = findQueue(rootQueue, B); String message = "Refresh needs to support delete of Parent queue and its children."; assertNull(message, queueB); @@ -643,7 +640,7 @@ public void testRefreshQueuesWithAllChildQueuesDeleted() throws Exception { + " all children of a parent queue(without running apps)."); } CSQueue rootQueue = cs.getRootQueue(); - CSQueue queueB = findQueue(rootQueue, B.getFullPath()); + CSQueue queueB = findQueue(rootQueue, B); assertNotNull("Parent Queue B should not be deleted", queueB); Assert.assertTrue("As Queue'B children are not deleted", queueB instanceof LeafQueue); @@ -727,7 +724,7 @@ public void testQueuesMaxAllocationInheritance() throws Exception { YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES); // Test the child queue overrides - setMaxAllocation(conf, ROOT, + setMaxAllocation(conf, CapacitySchedulerConfiguration.ROOT, "memory-mb=4096,vcores=2"); setMaxAllocation(conf, A1, "memory-mb=6144,vcores=2"); setMaxAllocation(conf, B, "memory-mb=5120, vcores=2"); @@ -739,12 +736,12 @@ public void testQueuesMaxAllocationInheritance() throws Exception { checkQueueStructureCapacities(cs); CSQueue rootQueue = cs.getRootQueue(); - CSQueue queueA = findQueue(rootQueue, A.getFullPath()); - CSQueue queueB = findQueue(rootQueue, B.getFullPath()); - CSQueue queueA1 = findQueue(queueA, A1.getFullPath()); - CSQueue queueA2 = findQueue(queueA, A2.getFullPath()); - CSQueue queueB1 = findQueue(queueB, B1.getFullPath()); - CSQueue queueB2 = findQueue(queueB, B2.getFullPath()); + CSQueue queueA = findQueue(rootQueue, A); + CSQueue queueB = findQueue(rootQueue, B); + CSQueue queueA1 = findQueue(queueA, A1); + CSQueue queueA2 = findQueue(queueA, A2); + CSQueue queueB1 = findQueue(queueB, B1); + CSQueue queueB2 = findQueue(queueB, B2); assertEquals("max capability MB in CS", YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, @@ -774,7 +771,7 @@ public void testQueuesMaxAllocationInheritance() throws Exception { unsetMaxAllocation(conf, A1); unsetMaxAllocation(conf, B); unsetMaxAllocation(conf, B1); - setMaxAllocation(conf, ROOT, + setMaxAllocation(conf, CapacitySchedulerConfiguration.ROOT, "memory-mb=6144,vcores=2"); setMaxAllocation(conf, A, "memory-mb=8192,vcores=2"); @@ -800,7 +797,7 @@ public void testQueuesMaxAllocationInheritance() throws Exception { queueB1.getMaximumAllocation().getVirtualCores()); // Test the default - unsetMaxAllocation(conf, ROOT); + unsetMaxAllocation(conf, CapacitySchedulerConfiguration.ROOT); unsetMaxAllocation(conf, A); unsetMaxAllocation(conf, A1); cs.reinitialize(conf, rm.getRMContext()); @@ -848,7 +845,7 @@ public void testVerifyQueuesMaxAllocationConf() throws Exception { cs.reinitialize(conf, rm.getRMContext()); checkQueueStructureCapacities(cs); - setMaxAllocation(conf, ROOT, + setMaxAllocation(conf, CapacitySchedulerConfiguration.ROOT, "memory-mb=" + largerMem + ",vcores=2"); try { cs.reinitialize(conf, rm.getRMContext()); @@ -858,7 +855,7 @@ public void testVerifyQueuesMaxAllocationConf() throws Exception { e.getCause().getMessage().contains("maximum allocation")); } - setMaxAllocation(conf, ROOT, + setMaxAllocation(conf, CapacitySchedulerConfiguration.ROOT, "memory-mb=4096,vcores=2"); setMaxAllocation(conf, A, "memory-mb=6144,vcores=2"); setMaxAllocation(conf, A1, "memory-mb=" + largerMem + ",vcores=2"); 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/TestCapacitySchedulerSurgicalPreemption.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/TestCapacitySchedulerSurgicalPreemption.java index 322b8f5b6cb36..ce79528c2599a 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/TestCapacitySchedulerSurgicalPreemption.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/TestCapacitySchedulerSurgicalPreemption.java @@ -62,17 +62,6 @@ public class TestCapacitySchedulerSurgicalPreemption extends CapacitySchedulerPreemptionTestBase { private static final int NUM_NM = 5; - - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".a"); - private static final QueuePath B = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".b"); - private static final QueuePath C = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".c"); - private static final QueuePath D = new QueuePath(CapacitySchedulerConfiguration.ROOT + - ".d"); - @Override @Before public void setUp() throws Exception { @@ -361,11 +350,11 @@ public void testPriorityPreemptionWhenAllQueuesAreBelowGuaranteedCapacities() */ conf.setPUOrderingPolicyUnderUtilizedPreemptionEnabled(true); conf.setPUOrderingPolicyUnderUtilizedPreemptionDelay(1000); - conf.setQueueOrderingPolicy(ROOT, + conf.setQueueOrderingPolicy(CapacitySchedulerConfiguration.ROOT, CapacitySchedulerConfiguration.QUEUE_PRIORITY_UTILIZATION_ORDERING_POLICY); // Queue c has higher priority than a/b - conf.setQueuePriority(C, 1); + conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".c", 1); MockRM rm1 = new MockRM(conf); rm1.getRMContext().setNodeLabelManager(mgr); @@ -481,12 +470,12 @@ public void testPriorityPreemptionRequiresMoveReservation() */ conf.setPUOrderingPolicyUnderUtilizedPreemptionEnabled(true); conf.setPUOrderingPolicyUnderUtilizedPreemptionDelay(1000); - conf.setQueueOrderingPolicy(ROOT, + conf.setQueueOrderingPolicy(CapacitySchedulerConfiguration.ROOT, CapacitySchedulerConfiguration.QUEUE_PRIORITY_UTILIZATION_ORDERING_POLICY); conf.setPUOrderingPolicyUnderUtilizedPreemptionMoveReservation(true); // Queue c has higher priority than a/b - conf.setQueuePriority(C, 1); + conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".c", 1); MockRM rm1 = new MockRM(conf); rm1.getRMContext().setNodeLabelManager(mgr); @@ -621,11 +610,11 @@ public void testPriorityPreemptionOnlyTriggeredWhenDemandingQueueUnsatisfied() */ conf.setPUOrderingPolicyUnderUtilizedPreemptionEnabled(true); conf.setPUOrderingPolicyUnderUtilizedPreemptionDelay(1000); - conf.setQueueOrderingPolicy(ROOT, + conf.setQueueOrderingPolicy(CapacitySchedulerConfiguration.ROOT, CapacitySchedulerConfiguration.QUEUE_PRIORITY_UTILIZATION_ORDERING_POLICY); // Queue c has higher priority than a/b - conf.setQueuePriority(C, 1); + conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".c", 1); MockRM rm1 = new MockRM(conf); rm1.getRMContext().setNodeLabelManager(mgr); @@ -761,11 +750,11 @@ public void testPriorityPreemptionFromHighestPriorityQueueAndOldestContainer() */ // A/B has higher priority - conf.setQueuePriority(A, 1); - conf.setQueuePriority(B, 2); - conf.setCapacity(A, 45f); - conf.setCapacity(B, 45f); - conf.setCapacity(C, 10f); + conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".a" , 1); + conf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".b", 2); + conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a", 45f); + conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".b", 45f); + conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".c", 10f); testPriorityPreemptionFromHighestPriorityQueueAndOldestContainer(new String[] {"a", "b", "c"}, new String[] {"user", "user", "user"}); @@ -781,7 +770,7 @@ public void testPriorityPreemptionFromHighestPriorityQueueAndOldestContainer() conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512); conf.setPUOrderingPolicyUnderUtilizedPreemptionEnabled(true); conf.setPUOrderingPolicyUnderUtilizedPreemptionDelay(1000); - conf.setQueueOrderingPolicy(ROOT, + conf.setQueueOrderingPolicy(CapacitySchedulerConfiguration.ROOT, CapacitySchedulerConfiguration.QUEUE_PRIORITY_UTILIZATION_ORDERING_POLICY); MockRM rm1 = new MockRM(conf) { @@ -998,14 +987,12 @@ protected RMNodeLabelsManager createNodeLabelManager() { private void initializeConfProperties(CapacitySchedulerConfiguration conf) throws IOException { - QueuePath aQueuePath = new QueuePath("root.A"); - QueuePath bQueuePath = new QueuePath("root.B"); - conf.setQueues(ROOT, new String[] {"A", "B"}); - conf.setCapacity(aQueuePath, 50); - conf.setCapacity(bQueuePath, 50); - conf.setQueuePriority(aQueuePath, 1); - conf.setQueuePriority(bQueuePath, 2); + conf.setQueues("root", new String[] {"A", "B"}); + conf.setCapacity("root.A", 50); + conf.setCapacity("root.B", 50); + conf.setQueuePriority("root.A", 1); + conf.setQueuePriority("root.B", 2); conf.set(PREFIX + "root.ordering-policy", "priority-utilization"); conf.set(PREFIX + "ordering-policy.priority-utilization.underutilized-preemption.enabled", "true"); @@ -1184,11 +1171,11 @@ public void testPreemptionForFragmentatedCluster() throws Exception { this.conf); conf.setLong(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 1024 * 21); - conf.setQueues(ROOT, new String[] {"a", "b"}); - conf.setCapacity(A, 50); - conf.setUserLimitFactor(A, 100); - conf.setCapacity(B, 50); - conf.setUserLimitFactor(B, 100); + conf.setQueues("root", new String[] { "a", "b" }); + conf.setCapacity("root.a", 50); + conf.setUserLimitFactor("root.a", 100); + conf.setCapacity("root.b", 50); + conf.setUserLimitFactor("root.b", 100); MockRM rm1 = new MockRM(conf); rm1.getRMContext().setNodeLabelManager(mgr); rm1.start(); 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/TestCapacitySchedulerWeightMode.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/TestCapacitySchedulerWeightMode.java index 216ec0f9b9261..115813a01717f 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/TestCapacitySchedulerWeightMode.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/TestCapacitySchedulerWeightMode.java @@ -52,19 +52,11 @@ import java.util.Set; public class TestCapacitySchedulerWeightMode { - private static final String DEFAULT_PATH = CapacitySchedulerConfiguration.ROOT + ".default"; - private static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - private static final String B_PATH = CapacitySchedulerConfiguration.ROOT + ".b"; - private static final String A1_PATH = A_PATH + ".a1"; - private static final String B1_PATH = B_PATH + ".b1"; - private static final String B2_PATH = B_PATH + ".b2"; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath DEFAULT = new QueuePath(DEFAULT_PATH); - private static final QueuePath A = new QueuePath(A_PATH); - private static final QueuePath B = new QueuePath(B_PATH); - private static final QueuePath A1 = new QueuePath(A1_PATH); - private static final QueuePath B1 = new QueuePath(B1_PATH); - private static final QueuePath B2 = new QueuePath(B2_PATH); + private static final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + private static final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + private static final String A1 = A + ".a1"; + private static final String B1 = B + ".b1"; + private static final String B2 = B + ".b2"; private YarnConfiguration conf; @@ -90,7 +82,7 @@ public static CapacitySchedulerConfiguration getConfigWithInheritedAccessibleNod config); // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a"}); conf.setCapacityByLabel(A, RMNodeLabelsManager.NO_LABEL, 100f); @@ -122,11 +114,11 @@ public static Configuration getCSConfWithQueueLabelsWeightOnly( config); // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" }); - conf.setLabeledQueueWeight(ROOT, "x", 100); - conf.setLabeledQueueWeight(ROOT, "y", 100); - conf.setLabeledQueueWeight(ROOT, "z", 100); + conf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "y", 100); + conf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "z", 100); conf.setLabeledQueueWeight(A, RMNodeLabelsManager.NO_LABEL, 1); conf.setMaximumCapacity(A, 10); @@ -181,11 +173,11 @@ public static Configuration getCSConfWithLabelsParentUseWeightChildUsePct( config); // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" }); - conf.setLabeledQueueWeight(ROOT, "x", 100); - conf.setLabeledQueueWeight(ROOT, "y", 100); - conf.setLabeledQueueWeight(ROOT, "z", 100); + conf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "y", 100); + conf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "z", 100); conf.setLabeledQueueWeight(A, RMNodeLabelsManager.NO_LABEL, 1); conf.setMaximumCapacity(A, 10); @@ -240,11 +232,11 @@ public static Configuration getCSConfWithLabelsParentUsePctChildUseWeight( config); // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" }); - conf.setCapacityByLabel(ROOT, "x", 100); - conf.setCapacityByLabel(ROOT, "y", 100); - conf.setCapacityByLabel(ROOT, "z", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100); conf.setCapacityByLabel(A, RMNodeLabelsManager.NO_LABEL, 10); conf.setMaximumCapacity(A, 10); @@ -330,15 +322,15 @@ public void testGetCapacityOrWeightStringUsingWeights() throws IOException { rm.start(); CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); - String capacityOrWeightString = ((ParentQueue) cs.getQueue(A.getFullPath())) + String capacityOrWeightString = ((ParentQueue) cs.getQueue(A)) .getCapacityOrWeightString(); validateCapacityOrWeightString(capacityOrWeightString, true); - capacityOrWeightString = ((LeafQueue) cs.getQueue(A1.getFullPath())) + capacityOrWeightString = ((LeafQueue) cs.getQueue(A1)) .getCapacityOrWeightString(); validateCapacityOrWeightString(capacityOrWeightString, true); - capacityOrWeightString = ((LeafQueue) cs.getQueue(A1.getFullPath())) + capacityOrWeightString = ((LeafQueue) cs.getQueue(A1)) .getExtendedCapacityOrWeightString(); validateCapacityOrWeightString(capacityOrWeightString, true); } @@ -356,15 +348,15 @@ public void testGetCapacityOrWeightStringParentPctLeafWeights() rm.start(); CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); - String capacityOrWeightString = ((ParentQueue) cs.getQueue(A.getFullPath())) + String capacityOrWeightString = ((ParentQueue) cs.getQueue(A)) .getCapacityOrWeightString(); validateCapacityOrWeightString(capacityOrWeightString, true); - capacityOrWeightString = ((LeafQueue) cs.getQueue(A1.getFullPath())) + capacityOrWeightString = ((LeafQueue) cs.getQueue(A1)) .getCapacityOrWeightString(); validateCapacityOrWeightString(capacityOrWeightString, false); - capacityOrWeightString = ((LeafQueue) cs.getQueue(A1.getFullPath())) + capacityOrWeightString = ((LeafQueue) cs.getQueue(A1)) .getExtendedCapacityOrWeightString(); validateCapacityOrWeightString(capacityOrWeightString, false); } @@ -387,7 +379,7 @@ public void testChildAccessibleNodeLabelsWeightMode() throws Exception { CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler(); - LeafQueue b1 = (LeafQueue) cs.getQueue(B1.getFullPath()); + LeafQueue b1 = (LeafQueue) cs.getQueue(B1); Assert.assertNotNull(b1); Assert.assertTrue(b1.getAccessibleNodeLabels().isEmpty()); @@ -430,11 +422,11 @@ public void testQueueInfoWeight() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration( conf); - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "default"}); - csConf.setNonLabeledQueueWeight(A, 1); - csConf.setNonLabeledQueueWeight(B, 2); - csConf.setNonLabeledQueueWeight(DEFAULT, 3); + csConf.setNonLabeledQueueWeight("root.a", 1); + csConf.setNonLabeledQueueWeight("root.b", 2); + csConf.setNonLabeledQueueWeight("root.default", 3); // Check queue info capacity CapacityScheduler cs = 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/TestCapacitySchedulerWithMultiResourceTypes.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/TestCapacitySchedulerWithMultiResourceTypes.java index deebe3d59be4c..98cffda51cc81 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/TestCapacitySchedulerWithMultiResourceTypes.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/TestCapacitySchedulerWithMultiResourceTypes.java @@ -57,7 +57,6 @@ import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; - import static org.junit.Assert.assertEquals; import org.junit.Test; @@ -76,13 +75,9 @@ */ public class TestCapacitySchedulerWithMultiResourceTypes { private static String RESOURCE_1 = "res1"; - private static final String DEFAULT_PATH = CapacitySchedulerConfiguration.ROOT + ".default"; - private static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - private static final String B_PATH = CapacitySchedulerConfiguration.ROOT + ".b"; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath DEFAULT = new QueuePath(DEFAULT_PATH); - private static final QueuePath A = new QueuePath(A_PATH); - private static final QueuePath B = new QueuePath(B_PATH); + + private static final String A_QUEUE = CapacitySchedulerConfiguration.ROOT + ".a"; + private static final String B_QUEUE = CapacitySchedulerConfiguration.ROOT + ".b"; private static float A_CAPACITY = 50.0f; private static float B_CAPACITY = 50.0f; @@ -121,11 +116,11 @@ public void testMaximumAllocationRefreshWithMultipleResourceTypes() throws Excep CapacitySchedulerConfiguration csconf = new CapacitySchedulerConfiguration(); - csconf.setMaximumApplicationMasterResourcePerQueuePercent(ROOT, 100.0f); - csconf.setMaximumAMResourcePercentPerPartition(ROOT, "", 100.0f); - csconf.setMaximumApplicationMasterResourcePerQueuePercent(DEFAULT, + csconf.setMaximumApplicationMasterResourcePerQueuePercent("root", 100.0f); + csconf.setMaximumAMResourcePercentPerPartition("root", "", 100.0f); + csconf.setMaximumApplicationMasterResourcePerQueuePercent("root.default", 100.0f); - csconf.setMaximumAMResourcePercentPerPartition(DEFAULT, "", 100.0f); + csconf.setMaximumAMResourcePercentPerPartition("root.default", "", 100.0f); csconf.setResourceComparator(DominantResourceCalculator.class); csconf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_1); csconf.setInt(YarnConfiguration.RESOURCE_TYPES + "." + RESOURCE_1 @@ -276,22 +271,23 @@ public void testMaxLimitsOfQueueWithMultipleResources() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setMaximumApplicationMasterResourcePerQueuePercent(ROOT, 100.0f); - csConf.setMaximumAMResourcePercentPerPartition(ROOT, "", 100.0f); - csConf.setMaximumApplicationMasterResourcePerQueuePercent(DEFAULT, + csConf.setMaximumApplicationMasterResourcePerQueuePercent("root", 100.0f); + csConf.setMaximumAMResourcePercentPerPartition("root", "", 100.0f); + csConf.setMaximumApplicationMasterResourcePerQueuePercent("root.default", 100.0f); - csConf.setMaximumAMResourcePercentPerPartition(DEFAULT, "", 100.0f); + csConf.setMaximumAMResourcePercentPerPartition("root.default", "", 100.0f); csConf.setResourceComparator(DominantResourceCalculator.class); csConf.set(YarnConfiguration.RESOURCE_TYPES, ResourceInformation.GPU_URI); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a", "b"}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a", "b"}); // Set each queue to consider 50% each. - csConf.setCapacity(A, A_CAPACITY); - csConf.setCapacity(B, B_CAPACITY); - csConf.setMaximumCapacity(A, 100.0f); - csConf.setUserLimitFactor(A, 2); + csConf.setCapacity(A_QUEUE, A_CAPACITY); + csConf.setCapacity(B_QUEUE, B_CAPACITY); + csConf.setMaximumCapacity(A_QUEUE, 100.0f); + csConf.setUserLimitFactor(A_QUEUE, 2); YarnConfiguration conf = new YarnConfiguration(csConf); // Don't reset resource types since we have already configured resource @@ -415,13 +411,14 @@ public void testConsumeAllExtendedResourcesWithSmallMinUserLimitPct() csconf.setResourceComparator(DominantResourceCalculator.class); // Define top-level queues - csconf.setQueues(ROOT, new String[] {"a", "b"}); + csconf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a", "b"}); // Set each queue to contain 50% each. - csconf.setCapacity(A, A_CAPACITY); - csconf.setCapacity(B, B_CAPACITY); - csconf.setMaximumCapacity(A, 100.0f); - csconf.setUserLimitFactor(A, 2); + csconf.setCapacity(A_QUEUE, A_CAPACITY); + csconf.setCapacity(B_QUEUE, B_CAPACITY); + csconf.setMaximumCapacity(A_QUEUE, 100.0f); + csconf.setUserLimitFactor(A_QUEUE, 2); YarnConfiguration yarnConf = new YarnConfiguration(csconf); // Don't reset resource types since we have already configured resource 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/TestCapacitySchedulerWorkflowPriorityMapping.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/TestCapacitySchedulerWorkflowPriorityMapping.java index 1108e76a3ec1b..07b2562a0a843 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/TestCapacitySchedulerWorkflowPriorityMapping.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/TestCapacitySchedulerWorkflowPriorityMapping.java @@ -32,7 +32,6 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B3; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B3_CAPACITY; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.B_CAPACITY; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.ROOT; import static org.junit.Assert.assertEquals; @@ -68,7 +67,8 @@ public void tearDown() { private static void setWorkFlowPriorityMappings( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b"}); + conf.setQueues( + CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); conf.setCapacity(A, A_CAPACITY); conf.setCapacity(B, B_CAPACITY); @@ -84,9 +84,9 @@ private static void setWorkFlowPriorityMappings( conf.setCapacity(B3, B3_CAPACITY); List mappings = Arrays.asList( - new WorkflowPriorityMapping("workflow1", B.getFullPath(), Priority.newInstance(2)), - new WorkflowPriorityMapping("workflow2", A1.getFullPath(), Priority.newInstance(3)), - new WorkflowPriorityMapping("Workflow3", A.getFullPath(), Priority.newInstance(4))); + new WorkflowPriorityMapping("workflow1", B, Priority.newInstance(2)), + new WorkflowPriorityMapping("workflow2", A1, Priority.newInstance(3)), + new WorkflowPriorityMapping("Workflow3", A, Priority.newInstance(4))); conf.setWorkflowPriorityMappings(mappings); } @@ -108,9 +108,9 @@ public void testWorkflowPriorityMappings() throws Exception { cs.start(); Map expected = ImmutableMap.of( - A.getFullPath(), ImmutableMap.of("workflow3", Priority.newInstance(4)), - B.getFullPath(), ImmutableMap.of("workflow1", Priority.newInstance(2)), - A1.getFullPath(), ImmutableMap.of("workflow2", Priority.newInstance(3))); + A, ImmutableMap.of("workflow3", Priority.newInstance(4)), + B, ImmutableMap.of("workflow1", Priority.newInstance(2)), + A1, ImmutableMap.of("workflow2", Priority.newInstance(3))); assertEquals(expected, cs.getWorkflowPriorityMappingsManager() .getWorkflowPriorityMappings()); 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 c03392da11018..52bafa792ef98 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 @@ -204,20 +204,19 @@ private void verifyQueueMetrics(CSQueue queue, private void setupSortedQueues(CapacitySchedulerConfiguration conf) { // Define queues - final QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - csConf.setQueues(root, new String[] {A, B, C, D}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {A, B, C, D}); final String Q_A = CapacitySchedulerConfiguration.ROOT + "." + A; - conf.setCapacity(new QueuePath(Q_A), 25); + conf.setCapacity(Q_A, 25); final String Q_B = CapacitySchedulerConfiguration.ROOT + "." + B; - conf.setCapacity(new QueuePath(Q_B), 25); + conf.setCapacity(Q_B, 25); final String Q_C = CapacitySchedulerConfiguration.ROOT + "." + C; - conf.setCapacity(new QueuePath(Q_C), 25); + conf.setCapacity(Q_C, 25); final String Q_D = CapacitySchedulerConfiguration.ROOT + "." + D; - conf.setCapacity(new QueuePath(Q_D), 25); + conf.setCapacity(Q_D, 25); } @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/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java index df1db68f7d73d..7e00c218aa1f2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java @@ -78,19 +78,7 @@ public class TestContainerAllocation { private static final Logger LOG = LoggerFactory .getLogger(TestContainerAllocation.class); - public static final String DEFAULT_PATH = CapacitySchedulerConfiguration.ROOT + ".default"; - public static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - public static final String B_PATH = CapacitySchedulerConfiguration.ROOT + ".b"; - public static final String C_PATH = CapacitySchedulerConfiguration.ROOT + ".c"; - public static final String C1_PATH = C_PATH + ".c1"; - public static final String C2_PATH = C_PATH + ".c2"; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath DEFAULT = new QueuePath(DEFAULT_PATH); - private static final QueuePath A = new QueuePath(A_PATH); - private static final QueuePath B = new QueuePath(B_PATH); - private static final QueuePath C = new QueuePath(C_PATH); - private static final QueuePath C1 = new QueuePath(C1_PATH); - private static final QueuePath C2 = new QueuePath(C2_PATH); + private final int GB = 1024; private YarnConfiguration conf; @@ -736,7 +724,7 @@ public void testContinuousReservationLookingWhenUsedEqualsMax() throws Exception (CapacitySchedulerConfiguration) TestUtils .getConfigurationWithMultipleQueues(conf); // Set maximum capacity of A to 10 - newConf.setMaximumCapacity(A, 10); + newConf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + ".a", 10); MockRM rm1 = new MockRM(newConf); rm1.getRMContext().setNodeLabelManager(mgr); @@ -801,8 +789,10 @@ public void testPendingResourcesConsideringUserLimit() throws Exception { // Set maximum capacity of A to 10 CapacitySchedulerConfiguration newConf = new CapacitySchedulerConfiguration( conf); - newConf.setUserLimitFactor(DEFAULT, 0.5f); - newConf.setMaximumAMResourcePercentPerPartition(DEFAULT, "", 1.0f); + newConf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".default", + 0.5f); + newConf.setMaximumAMResourcePercentPerPartition( + CapacitySchedulerConfiguration.ROOT + ".default", "", 1.0f); MockRM rm1 = new MockRM(newConf); rm1.getRMContext().setNodeLabelManager(mgr); @@ -902,14 +892,14 @@ public void testQueuePriorityOrdering() throws Exception { .getConfigurationWithMultipleQueues(conf); // Set ordering policy - newConf.setQueueOrderingPolicy(ROOT, + newConf.setQueueOrderingPolicy(CapacitySchedulerConfiguration.ROOT, CapacitySchedulerConfiguration.QUEUE_PRIORITY_UTILIZATION_ORDERING_POLICY); // Set maximum capacity of A to 20 - newConf.setMaximumCapacity(A, 20); - newConf.setQueuePriority(C, 1); - newConf.setQueuePriority(B, 2); - newConf.setQueuePriority(A, 3); + newConf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + ".a", 20); + newConf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".c", 1); + newConf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".b", 2); + newConf.setQueuePriority(CapacitySchedulerConfiguration.ROOT + ".a", 3); MockRM rm1 = new MockRM(newConf); @@ -1039,7 +1029,7 @@ public void testUserLimitAllocationMultipleContainers() throws Exception { // make sure an unlimited number of containers can be assigned, // overriding the default of 100 after YARN-8896 newConf.set(MAX_ASSIGN_PER_HEARTBEAT, "-1"); - newConf.setUserLimit(C, 50); + newConf.setUserLimit("root.c", 50); MockRM rm1 = new MockRM(newConf); rm1.getRMContext().setNodeLabelManager(mgr); @@ -1102,7 +1092,8 @@ public void testActiveUsersWithOnlyPendingApps() throws Exception { CapacitySchedulerConfiguration newConf = new CapacitySchedulerConfiguration(conf); - newConf.setMaximumAMResourcePercentPerPartition(DEFAULT, "", 0.2f); + newConf.setMaximumAMResourcePercentPerPartition( + CapacitySchedulerConfiguration.ROOT + ".default", "", 0.2f); MockRM rm1 = new MockRM(newConf); rm1.getRMContext().setNodeLabelManager(mgr); @@ -1196,7 +1187,7 @@ public void testUnreserveWhenClusterResourceHasEmptyResourceType() newConf .setBoolean(TestResourceProfiles.TEST_CONF_RESET_RESOURCE_TYPES, false); // Set maximum capacity of queue "a" to 50 - newConf.setMaximumCapacity(A, 50); + newConf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + ".a", 50); MockRM rm1 = new MockRM(newConf); RMNodeLabelsManager nodeLabelsManager = new NullRMNodeLabelsManager(); @@ -1285,12 +1276,12 @@ public void testAllocationCannotBeBlockedWhenFormerQueueReachedItsLimit() CapacitySchedulerConfiguration newConf = (CapacitySchedulerConfiguration) TestUtils .getConfigurationWithMultipleQueues(conf); - newConf.setQueues(C, + newConf.setQueues(CapacitySchedulerConfiguration.ROOT + ".c", new String[] { "c1", "c2" }); - newConf.setCapacity(C1, 10); + newConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".c.c1", 10); newConf - .setMaximumCapacity(C1, 10); - newConf.setCapacity(C2, 90); + .setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + ".c.c1", 10); + newConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".c.c2", 90); newConf.setClass(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, DominantResourceCalculator.class, ResourceCalculator.class); @@ -1354,7 +1345,7 @@ public void testContainerRejectionWhenAskBeyondDynamicMax() .getConfigurationWithMultipleQueues(conf); newConf.setClass(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, DominantResourceCalculator.class, ResourceCalculator.class); - newConf.set(QueuePrefixes.getQueuePrefix(A) + newConf.set(CapacitySchedulerConfiguration.getQueuePrefix("root.a") + MAXIMUM_ALLOCATION_MB, "4096"); MockRM rm1 = new MockRM(newConf); 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 870d61539b02b..c92e94c6cfb75 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 @@ -240,7 +240,7 @@ private void setUpInternal(ResourceCalculator rC, boolean withNodeLabels) root = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, - ROOT.getFullPath(), + ROOT, queues, queues, TestUtils.spyHook); queueManager.setRootQueue(root); @@ -268,69 +268,66 @@ private void setUpInternal(ResourceCalculator rC, boolean withNodeLabels) private static final String C1 = "c1"; private static final String D = "d"; private static final String E = "e"; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A_QUEUE_PATH = ROOT.createNewLeaf(A); private void setupQueueConfiguration( CapacitySchedulerConfiguration conf, - final String newRootName, boolean withNodeLabels) { + final String newRoot, boolean withNodeLabels) { // Define top-level queues - conf.setQueues(ROOT, new String[] {newRootName}); + conf.setQueues(ROOT, new String[] {newRoot}); conf.setMaximumCapacity(ROOT, 100); conf.setAcl(ROOT, QueueACL.SUBMIT_APPLICATIONS, " "); if (withNodeLabels) { - conf.setCapacityByLabel(ROOT, LABEL, 100); - conf.setMaximumCapacityByLabel(ROOT, LABEL, 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, LABEL, 100); + conf.setMaximumCapacityByLabel(CapacitySchedulerConfiguration.ROOT, + LABEL, 100); } - final String newRootPath = CapacitySchedulerConfiguration.ROOT + "." + newRootName; - final QueuePath newRoot = new QueuePath(newRootPath); - conf.setQueues(newRoot, new String[] {A, B, C, D, E}); - conf.setCapacity(newRoot, 100); - conf.setMaximumCapacity(newRoot, 100); - conf.setAcl(newRoot, QueueACL.SUBMIT_APPLICATIONS, " "); + final String Q_newRoot = ROOT + "." + newRoot; + conf.setQueues(Q_newRoot, new String[] {A, B, C, D, E}); + conf.setCapacity(Q_newRoot, 100); + conf.setMaximumCapacity(Q_newRoot, 100); + conf.setAcl(Q_newRoot, QueueACL.SUBMIT_APPLICATIONS, " "); if (withNodeLabels) { - conf.setAccessibleNodeLabels(newRoot, Collections.singleton(LABEL)); - conf.setCapacityByLabel(newRoot, LABEL, 100); - conf.setMaximumCapacityByLabel(newRoot, LABEL, 100); + conf.setAccessibleNodeLabels(Q_newRoot, Collections.singleton(LABEL)); + conf.setCapacityByLabel(Q_newRoot, LABEL, 100); + conf.setMaximumCapacityByLabel(Q_newRoot, LABEL, 100); } - final QueuePath a = new QueuePath(newRootPath, A); - conf.setCapacity(a, 8.5f); - conf.setMaximumCapacity(a, 20); - conf.setAcl(a, QueueACL.SUBMIT_APPLICATIONS, "*"); + final String Q_A = Q_newRoot + "." + A; + conf.setCapacity(Q_A, 8.5f); + conf.setMaximumCapacity(Q_A, 20); + conf.setAcl(Q_A, QueueACL.SUBMIT_APPLICATIONS, "*"); if (withNodeLabels) { - conf.setAccessibleNodeLabels(a, Collections.singleton(LABEL)); - conf.setCapacityByLabel(a, LABEL, 100); - conf.setMaximumCapacityByLabel(a, LABEL, 100); + conf.setAccessibleNodeLabels(Q_A, Collections.singleton(LABEL)); + conf.setCapacityByLabel(Q_A, LABEL, 100); + conf.setMaximumCapacityByLabel(Q_A, LABEL, 100); } - - final QueuePath b = new QueuePath(newRootPath, B); - conf.setCapacity(b, 80); - conf.setMaximumCapacity(b, 99); - conf.setAcl(b, QueueACL.SUBMIT_APPLICATIONS, "*"); - - final String cPath = newRootPath + "." + C; - final QueuePath c = new QueuePath(cPath); - conf.setCapacity(c, 1.5f); - conf.setMaximumCapacity(c, 10); - conf.setAcl(c, QueueACL.SUBMIT_APPLICATIONS, " "); - conf.setQueues(c, new String[] {C1}); - - final QueuePath c1 = new QueuePath(cPath, C1); - conf.setCapacity(c1, 100); + final String Q_B = Q_newRoot + "." + B; + conf.setCapacity(Q_B, 80); + conf.setMaximumCapacity(Q_B, 99); + conf.setAcl(Q_B, QueueACL.SUBMIT_APPLICATIONS, "*"); + + final String Q_C = Q_newRoot + "." + C; + conf.setCapacity(Q_C, 1.5f); + conf.setMaximumCapacity(Q_C, 10); + conf.setAcl(Q_C, QueueACL.SUBMIT_APPLICATIONS, " "); + + conf.setQueues(Q_C, new String[] {C1}); - final QueuePath d = new QueuePath(newRootPath, D); - conf.setCapacity(d, 9); - conf.setMaximumCapacity(d, 11); - conf.setAcl(d, QueueACL.SUBMIT_APPLICATIONS, "user_d"); + final String Q_C1 = Q_C + "." + C1; + conf.setCapacity(Q_C1, 100); - final QueuePath e = new QueuePath(newRootPath, E); - conf.setCapacity(e, 1); - conf.setMaximumCapacity(e, 1); - conf.setAcl(e, QueueACL.SUBMIT_APPLICATIONS, "user_e"); + final String Q_D = Q_newRoot + "." + D; + conf.setCapacity(Q_D, 9); + conf.setMaximumCapacity(Q_D, 11); + conf.setAcl(Q_D, QueueACL.SUBMIT_APPLICATIONS, "user_d"); + + final String Q_E = Q_newRoot + "." + E; + conf.setCapacity(Q_E, 1); + conf.setMaximumCapacity(Q_E, 1); + conf.setAcl(Q_E, QueueACL.SUBMIT_APPLICATIONS, "user_e"); } @@ -483,7 +480,7 @@ public void testPolicyConfiguration() throws Exception { "testPolicyRoot" + System.currentTimeMillis(); OrderingPolicy comPol = - testConf.getAppOrderingPolicy(new QueuePath(tproot)); + testConf.getAppOrderingPolicy(tproot); } @@ -630,9 +627,8 @@ public void testFairConfiguration() throws Exception { CapacitySchedulerConfiguration testConf = new CapacitySchedulerConfiguration(); - String tprootPath = ROOT + "." + + String tproot = ROOT + "." + "testPolicyRoot" + System.currentTimeMillis(); - QueuePath tproot = new QueuePath(tprootPath); OrderingPolicy schedOrder = testConf.getAppOrderingPolicy(tproot); @@ -1097,9 +1093,9 @@ public void testUserLimitCache() throws Exception { when(csContext.getClusterResource()).thenReturn(clusterResource); // working with just one queue - csConf.setQueues(ROOT, new String[]{A}); - csConf.setCapacity(A_QUEUE_PATH, 100); - csConf.setMaximumCapacity(A_QUEUE_PATH, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{A}); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 100); + csConf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 100); queueContext.reinitialize(); @@ -1316,9 +1312,9 @@ public void testUserLimitCacheActiveUsersChanged() throws Exception { when(csContext.getClusterResource()).thenReturn(clusterResource); // working with just one queue - csConf.setQueues(ROOT, new String[] {A}); - csConf.setCapacity(A_QUEUE_PATH, 100); - csConf.setMaximumCapacity(A_QUEUE_PATH, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {A}); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 100); + csConf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 100); queueContext.reinitialize(); @@ -1923,7 +1919,7 @@ public void testUserSpecificUserLimits() throws Exception { // Mock the queue LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A)); // Set minimum-user-limit-percent for queue "a" in the configs. - csConf.setUserLimit(a.getQueuePathObject(), 50); + csConf.setUserLimit(a.getQueuePath(), 50); // Set weight for "user_0" to be 1.5f for the a queue in the configs. csConf.setFloat("yarn.scheduler.capacity." + a.getQueuePath() + ".user-settings.user_0." + CapacitySchedulerConfiguration.USER_WEIGHT, @@ -3236,7 +3232,7 @@ public void testRackLocalityDelayScheduling() throws Exception { queueContext.reinitialize(); CSQueueStore newQueues = new CSQueueStore(); CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(queueContext, - csConf, null, ROOT.getFullPath(), newQueues, queues, + csConf, null, ROOT, newQueues, queues, TestUtils.spyHook); csContext.getCapacitySchedulerQueueManager().setRootQueue(newRoot); root.reinitialize(newRoot, cs.getClusterResource()); @@ -3675,7 +3671,7 @@ public void testActivateApplicationAfterQueueRefresh() throws Exception { CSQueueStore newQueues = new CSQueueStore(); CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, - ROOT.getFullPath(), + ROOT, newQueues, queues, TestUtils.spyHook); queues = newQueues; @@ -3707,7 +3703,7 @@ public void testLocalityDelaysAfterQueueRefresh() throws Exception { CSQueueStore newQueues = new CSQueueStore(); CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(queueContext, csConf, null, - ROOT.getFullPath(), + ROOT, newQueues, queues, TestUtils.spyHook); csContext.getCapacitySchedulerQueueManager().setRootQueue(newRoot); @@ -5138,19 +5134,19 @@ public void testSetupQueueConfigsWithSpecifiedConfiguration() final String leafQueueName = "testSetupQueueConfigsWithSpecifiedConfiguration"; - final QueuePath leafQueuePath = new QueuePath(ROOT.getFullPath(), leafQueueName); - final QueuePath bQueuePath = new QueuePath(ROOT + DOT + B); assertEquals(0, conf.size()); conf.setNodeLocalityDelay(60); csConf.setQueues(ROOT, new String[] {leafQueueName, B}); - csConf.setCapacity(leafQueuePath, 10); - csConf.setMaximumCapacity(leafQueuePath, 100); - csConf.setUserLimitFactor(leafQueuePath, 0.1f); + csConf.setCapacity(ROOT + DOT + leafQueueName, 10); + csConf.setMaximumCapacity(ROOT + DOT + leafQueueName, + 100); + csConf.setUserLimitFactor(ROOT + DOT + leafQueueName, 0.1f); - csConf.setCapacity(bQueuePath, 90); - csConf.setMaximumCapacity(bQueuePath, 100); + csConf.setCapacity(ROOT + DOT + B, 90); + csConf.setMaximumCapacity(ROOT + DOT + B, + 100); csConf.setNodeLocalityDelay(30); csConf.setGlobalMaximumApplicationsPerQueue(20); @@ -5176,9 +5172,9 @@ public void testSetupQueueConfigsWithSpecifiedConfiguration() assertEquals(2, leafQueue.getMaxApplicationsPerUser()); //check queue configs - conf.setMaximumAMResourcePercentPerPartition(leafQueue.getQueuePathObject(), + conf.setMaximumAMResourcePercentPerPartition(leafQueue.getQueuePath(), NO_LABEL, 10); - conf.setMaximumCapacity(leafQueue.getQueuePathObject(), 10); + conf.setMaximumCapacity(leafQueue.getQueuePath(), 10); assertEquals(0.1, leafQueue.getMaxAMResourcePerQueuePercent(), EPSILON); @@ -5248,13 +5244,13 @@ public void testMaxApplicationsWithNodeLabels() throws IOException { Resources.createResource(2 * 16 * GB, 2 * 32)); conf.setCapacityByLabel(ROOT, "test", 100); - conf.setCapacityByLabel(new QueuePath(rootChild), "test", 100); - conf.setCapacityByLabel(new QueuePath(rootChild, A), "test", 20); - conf.setCapacityByLabel(new QueuePath(rootChild, B), "test", 40); - conf.setCapacityByLabel(new QueuePath(rootChild, C), "test", 10); - conf.setCapacityByLabel(QueuePath.createFromQueues(rootChild, C, C1), "test", 100); - conf.setCapacityByLabel(new QueuePath(rootChild, D), "test", 30); - conf.setCapacityByLabel(new QueuePath(rootChild, E), "test", 0); + conf.setCapacityByLabel(rootChild, "test", 100); + conf.setCapacityByLabel(rootChild + "." + A, "test", 20); + conf.setCapacityByLabel(rootChild + "." + B, "test", 40); + conf.setCapacityByLabel(rootChild + "." + C, "test", 10); + conf.setCapacityByLabel(rootChild + "." + C + "." + C1, "test", 100); + conf.setCapacityByLabel(rootChild + "." + D, "test", 30); + conf.setCapacityByLabel(rootChild + "." + E, "test", 0); cs.getCapacitySchedulerQueueManager().reinitConfiguredNodeLabels(conf); cs.setMaxRunningAppsEnforcer(new CSMaxRunningAppsEnforcer(cs)); cs.reinitialize(conf, cs.getRMContext()); @@ -5265,11 +5261,11 @@ public void testMaxApplicationsWithNodeLabels() throws IOException { (int)(conf.getMaximumSystemApplications() * e.getAbsoluteCapacity()), e.getMaxApplications()); - conf.setCapacityByLabel(new QueuePath(rootChild, A), "test", 10); - conf.setCapacityByLabel(new QueuePath(rootChild, B), "test", 10); - conf.setCapacityByLabel(new QueuePath(rootChild, C), "test", 10); - conf.setCapacityByLabel(new QueuePath(rootChild, D), "test", 10); - conf.setCapacityByLabel(new QueuePath(rootChild, E), "test", 60); + conf.setCapacityByLabel(rootChild + "." + A, "test", 10); + conf.setCapacityByLabel(rootChild + "." + B, "test", 10); + conf.setCapacityByLabel(rootChild + "." + C, "test", 10); + conf.setCapacityByLabel(rootChild + "." + D, "test", 10); + conf.setCapacityByLabel(rootChild + "." + E, "test", 60); cs.reinitialize(conf, cs.getRMContext()); e = (LeafQueue) cs.getQueue("e"); @@ -5286,21 +5282,21 @@ public void testRootHasAllNodeLabelsOfItsDescendants() throws IOException { CapacitySchedulerConfiguration conf = csConf; String rootChild = root.getChildQueues().get(0).getQueuePath(); - conf.setCapacityByLabel(new QueuePath(rootChild), "test", 100); - conf.setCapacityByLabel(new QueuePath(rootChild, A), "test", 20); - conf.setCapacityByLabel(new QueuePath(rootChild, B), "test", 40); - conf.setCapacityByLabel(new QueuePath(rootChild, C), "test", 10); - conf.setCapacityByLabel(QueuePath.createFromQueues(rootChild, C, C1), "test", 100); - conf.setCapacityByLabel(new QueuePath(rootChild, D), "test", 30); - conf.setCapacityByLabel(new QueuePath(rootChild, E), "test", 0); - - conf.setCapacityByLabel(new QueuePath(rootChild), "test2", 100); - conf.setCapacityByLabel(new QueuePath(rootChild, A), "test2", 20); - conf.setCapacityByLabel(new QueuePath(rootChild, B), "test2", 40); - conf.setCapacityByLabel(new QueuePath(rootChild, C), "test2", 10); - conf.setCapacityByLabel(QueuePath.createFromQueues(rootChild, C, C1), "test2", 100); - conf.setCapacityByLabel(new QueuePath(rootChild, D), "test2", 30); - conf.setCapacityByLabel(new QueuePath(rootChild, E), "test2", 0); + conf.setCapacityByLabel(rootChild, "test", 100); + conf.setCapacityByLabel(rootChild + "." + A, "test", 20); + conf.setCapacityByLabel(rootChild + "." + B, "test", 40); + conf.setCapacityByLabel(rootChild + "." + C, "test", 10); + conf.setCapacityByLabel(rootChild + "." + C + "." + C1, "test", 100); + conf.setCapacityByLabel(rootChild + "." + D, "test", 30); + conf.setCapacityByLabel(rootChild + "." + E, "test", 0); + + conf.setCapacityByLabel(rootChild, "test2", 100); + conf.setCapacityByLabel(rootChild + "." + A, "test2", 20); + conf.setCapacityByLabel(rootChild + "." + B, "test2", 40); + conf.setCapacityByLabel(rootChild + "." + C, "test2", 10); + conf.setCapacityByLabel(rootChild + "." + C + "." + C1, "test2", 100); + conf.setCapacityByLabel(rootChild + "." + D, "test2", 30); + conf.setCapacityByLabel(rootChild + "." + E, "test2", 0); cs.getCapacitySchedulerQueueManager().reinitConfiguredNodeLabels(conf); cs.setMaxRunningAppsEnforcer(new CSMaxRunningAppsEnforcer(cs)); 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/TestMixedQueueResourceCalculation.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/TestMixedQueueResourceCalculation.java index d3084503e9305..fcfd1bbb0e822 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/TestMixedQueueResourceCalculation.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/TestMixedQueueResourceCalculation.java @@ -522,15 +522,15 @@ private void setupQueueHierarchyWithWarnings() throws IOException { } private void setQueues() { - csConf.setQueues(ROOT, new String[]{"a", "b", "c"}); + csConf.setQueues("root", new String[]{"a", "b", "c"}); csConf.setQueues(A, new String[]{"a1", "a2"}); csConf.setQueues(B, new String[]{"b1"}); } private Optional getSpecificWarning( Collection warnings, QueueUpdateWarningType warningTypeToSelect, - QueuePath queue) { + String queue) { return warnings.stream().filter((w) -> w.getWarningType().equals(warningTypeToSelect) - && w.getQueue().equals(queue.getFullPath())).findFirst(); + && w.getQueue().equals(queue)).findFirst(); } } 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/TestNodeLabelContainerAllocation.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/TestNodeLabelContainerAllocation.java index f960acd956116..db739377aaac8 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/TestNodeLabelContainerAllocation.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/TestNodeLabelContainerAllocation.java @@ -76,28 +76,6 @@ import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; public class TestNodeLabelContainerAllocation { - private static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - private static final String B_PATH = CapacitySchedulerConfiguration.ROOT + ".b"; - private static final String C_PATH = CapacitySchedulerConfiguration.ROOT + ".c"; - private static final String D_PATH = CapacitySchedulerConfiguration.ROOT + ".d"; - private static final String A1_PATH = A_PATH + ".a1"; - private static final String A2_PATH = A_PATH + ".a2"; - private static final String B1_PATH = B_PATH + ".b1"; - private static final String B2_PATH = B_PATH + ".b2"; - private static final String C1_PATH = C_PATH + ".c1"; - private static final String C2_PATH = C_PATH + ".c2"; - - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A = new QueuePath(A_PATH); - private static final QueuePath B = new QueuePath(B_PATH); - private static final QueuePath C = new QueuePath(C_PATH); - private static final QueuePath D = new QueuePath(D_PATH); - private static final QueuePath A1 = new QueuePath(A1_PATH); - private static final QueuePath A2 = new QueuePath(A2_PATH); - private static final QueuePath B1 = new QueuePath(B1_PATH); - private static final QueuePath B2 = new QueuePath(B2_PATH); - private static final QueuePath C1 = new QueuePath(C1_PATH); - private static final QueuePath C2 = new QueuePath(C2_PATH); private final int GB = 1024; private YarnConfiguration conf; @@ -118,37 +96,43 @@ private Configuration getConfigurationWithQueueLabels(Configuration config) { new CapacitySchedulerConfiguration(config); // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b", "c"}); - conf.setCapacityByLabel(ROOT, "x", 100); - conf.setCapacityByLabel(ROOT, "y", 100); - conf.setCapacityByLabel(ROOT, "z", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); conf.setAccessibleNodeLabels(A, toSet("x")); conf.setCapacityByLabel(A, "x", 100); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 20); conf.setAccessibleNodeLabels(B, toSet("y", "z")); conf.setCapacityByLabel(B, "y", 100); conf.setCapacityByLabel(B, "z", 100); + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; conf.setCapacity(C, 70); conf.setMaximumCapacity(C, 70); conf.setAccessibleNodeLabels(C, RMNodeLabelsManager.EMPTY_STRING_SET); - + // Define 2nd-level queues + final String A1 = A + ".a1"; conf.setQueues(A, new String[] {"a1"}); conf.setCapacity(A1, 100); conf.setMaximumCapacity(A1, 100); conf.setCapacityByLabel(A1, "x", 100); - + + final String B1 = B + ".b1"; conf.setQueues(B, new String[] {"b1"}); conf.setCapacity(B1, 100); conf.setMaximumCapacity(B1, 100); conf.setCapacityByLabel(B1, "y", 100); conf.setCapacityByLabel(B1, "z", 100); + final String C1 = C + ".c1"; conf.setQueues(C, new String[] {"c1"}); conf.setCapacity(C1, 100); conf.setMaximumCapacity(C1, 100); @@ -1389,43 +1373,52 @@ public void testPreferenceOfQueuesTowardsNodePartitions() new CapacitySchedulerConfiguration(this.conf); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a", "b", "c"}); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"}); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setCapacity(A, 33); csConf.setAccessibleNodeLabels(A, toSet("x")); csConf.setCapacityByLabel(A, "x", 33); csConf.setQueues(A, new String[] {"a1", "a2"}); - + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; csConf.setCapacity(B, 33); csConf.setAccessibleNodeLabels(B, toSet("x")); csConf.setCapacityByLabel(B, "x", 33); csConf.setQueues(B, new String[] {"b1", "b2"}); - + + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; csConf.setCapacity(C, 34); csConf.setAccessibleNodeLabels(C, toSet("x")); csConf.setCapacityByLabel(C, "x", 34); csConf.setQueues(C, new String[] {"c1", "c2"}); // Define 2nd-level queues + final String A1 = A + ".a1"; csConf.setCapacity(A1, 50); csConf.setCapacityByLabel(A1, "x", 100); csConf.setDefaultNodeLabelExpression(A1, "x"); - + + final String A2 = A + ".a2"; csConf.setCapacity(A2, 50); csConf.setCapacityByLabel(A2, "x", 0); - + + final String B1 = B + ".b1"; csConf.setCapacity(B1, 50); csConf.setCapacityByLabel(B1, "x", 100); csConf.setDefaultNodeLabelExpression(B1, "x"); - + + final String B2 = B + ".b2"; csConf.setCapacity(B2, 50); csConf.setCapacityByLabel(B2, "x", 0); - + + final String C1 = C + ".c1"; csConf.setCapacity(C1, 50); csConf.setCapacityByLabel(C1, "x", 100); csConf.setDefaultNodeLabelExpression(C1, "x"); - + + final String C2 = C + ".c2"; csConf.setCapacity(C2, 50); csConf.setCapacityByLabel(C2, "x", 0); @@ -1573,13 +1566,15 @@ public void testQueuesWithoutAccessUsingPartitionedNodes() throws Exception { new CapacitySchedulerConfiguration(this.conf); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a", "b"}); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setCapacity(A, 50); csConf.setAccessibleNodeLabels(A, toSet("x")); csConf.setCapacityByLabel(A, "x", 100); - + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; csConf.setCapacity(B, 50); csConf.setAccessibleNodeLabels(B, new HashSet()); csConf.setUserLimitFactor(B, 5); @@ -1735,15 +1730,18 @@ public RMNodeLabelsManager createNodeLabelManager() { new CapacitySchedulerConfiguration(this.conf); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a", "b"}); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", + "b" }); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setCapacity(A, 50); csConf.setAccessibleNodeLabels(A, toSet("x")); csConf.setCapacityByLabel(A, "x", 50); csConf.setMaximumCapacityByLabel(A, "x", 50); csConf.setUserLimit(A, 200); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; csConf.setCapacity(B, 50); csConf.setAccessibleNodeLabels(B, toSet("x")); csConf.setCapacityByLabel(B, "x", 50); @@ -1884,26 +1882,31 @@ public void testQueueUsedCapacitiesUpdate() new CapacitySchedulerConfiguration(this.conf); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a", "b"}); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", + "b" }); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); /** * Initially, we set A/B's resource 50:50 */ + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setCapacity(A, 50); csConf.setAccessibleNodeLabels(A, toSet("x")); csConf.setCapacityByLabel(A, "x", 50); csConf.setQueues(A, new String[] { "a1", "a2" }); - + + final String A1 = A + ".a1"; csConf.setCapacity(A1, 50); csConf.setAccessibleNodeLabels(A1, toSet("x")); csConf.setCapacityByLabel(A1, "x", 50); - + + final String A2 = A + ".a2"; csConf.setCapacity(A2, 50); csConf.setAccessibleNodeLabels(A2, toSet("x")); csConf.setCapacityByLabel(A2, "x", 50); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; csConf.setCapacity(B, 50); csConf.setAccessibleNodeLabels(B, toSet("x")); csConf.setCapacityByLabel(B, "x", 50); @@ -2140,20 +2143,25 @@ public void testOrderOfAllocationOnPartitions() new CapacitySchedulerConfiguration(this.conf); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a", "b", "c", "d"}); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", + "b", "c", "d" }); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setCapacity(A, 25); csConf.setAccessibleNodeLabels(A, toSet("x")); csConf.setCapacityByLabel(A, "x", 30); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; csConf.setCapacity(B, 25); csConf.setAccessibleNodeLabels(B, toSet("x")); csConf.setCapacityByLabel(B, "x", 70); - + + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; csConf.setAccessibleNodeLabels(C, Collections. emptySet()); csConf.setCapacity(C, 25); - + + final String D = CapacitySchedulerConfiguration.ROOT + ".d"; csConf.setAccessibleNodeLabels(D, Collections. emptySet()); csConf.setCapacity(D, 25); @@ -2338,14 +2346,16 @@ public void testOrderOfAllocationOnPartitionsWhenAccessibilityIsAll() new CapacitySchedulerConfiguration(this.conf); // Define top-level queues - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" }); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setCapacity(A, 25); csConf.setAccessibleNodeLabels(A, toSet("*")); csConf.setCapacityByLabel(A, "x", 60); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; csConf.setCapacity(B, 75); csConf.setAccessibleNodeLabels(B, toSet("x")); csConf.setCapacityByLabel(B, "x", 40); @@ -2424,14 +2434,17 @@ public void testParentQueueMaxCapsAreRespected() throws Exception { new CapacitySchedulerConfiguration(this.conf); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a", "b"}); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", + "b"}); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setCapacity(A, 10); csConf.setAccessibleNodeLabels(A, toSet("x")); csConf.setCapacityByLabel(A, "x", 50); csConf.setMaximumCapacityByLabel(A, "x", 50); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; csConf.setCapacity(B, 90); csConf.setAccessibleNodeLabels(B, toSet("x")); csConf.setCapacityByLabel(B, "x", 50); @@ -2441,12 +2454,14 @@ public void testParentQueueMaxCapsAreRespected() throws Exception { csConf.setQueues(A, new String[] { "a1", "a2"}); + final String A1 = A + ".a1"; csConf.setCapacity(A1, 50); csConf.setAccessibleNodeLabels(A1, toSet("x")); csConf.setCapacityByLabel(A1, "x", 50); csConf.setMaximumCapacityByLabel(A1, "x", 100); csConf.setUserLimitFactor(A1, 100.0f); + final String A2 = A + ".a2"; csConf.setCapacity(A2, 50); csConf.setAccessibleNodeLabels(A2, toSet("x")); csConf.setCapacityByLabel(A2, "x", 50); @@ -2535,19 +2550,20 @@ public void testQueueMetricsWithLabels() throws Exception { this.conf); // Define top-level queues - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" }); - csConf.setCapacityByLabel(ROOT, "x", 100); - - csConf.setCapacity(A, 25); - csConf.setAccessibleNodeLabels(A, toSet("x")); - csConf.setCapacityByLabel(A, "x", 50); - csConf.setMaximumCapacityByLabel(A, "x", 50); - - csConf.setCapacity(B, 75); - csConf.setAccessibleNodeLabels(B, toSet("x")); - csConf.setCapacityByLabel(B, "x", 50); - csConf.setMaximumCapacityByLabel(B, "x", 50); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + + final String queueA = CapacitySchedulerConfiguration.ROOT + ".a"; + csConf.setCapacity(queueA, 25); + csConf.setAccessibleNodeLabels(queueA, toSet("x")); + csConf.setCapacityByLabel(queueA, "x", 50); + csConf.setMaximumCapacityByLabel(queueA, "x", 50); + final String queueB = CapacitySchedulerConfiguration.ROOT + ".b"; + csConf.setCapacity(queueB, 75); + csConf.setAccessibleNodeLabels(queueB, toSet("x")); + csConf.setCapacityByLabel(queueB, "x", 50); + csConf.setMaximumCapacityByLabel(queueB, "x", 50); // set node -> label mgr.addToCluserNodeLabels( @@ -2659,19 +2675,20 @@ public void testQueueMetricsWithLabelsOnDefaultLabelNode() throws Exception { this.conf); // Define top-level queues - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" }); - csConf.setCapacityByLabel(ROOT, "x", 100); - - csConf.setCapacity(A, 25); - csConf.setAccessibleNodeLabels(A, toSet("x")); - csConf.setCapacityByLabel(A, "x", 50); - csConf.setMaximumCapacityByLabel(A, "x", 50); - - csConf.setCapacity(B, 75); - csConf.setAccessibleNodeLabels(B, toSet("x")); - csConf.setCapacityByLabel(B, "x", 50); - csConf.setMaximumCapacityByLabel(B, "x", 50); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + + final String queueA = CapacitySchedulerConfiguration.ROOT + ".a"; + csConf.setCapacity(queueA, 25); + csConf.setAccessibleNodeLabels(queueA, toSet("x")); + csConf.setCapacityByLabel(queueA, "x", 50); + csConf.setMaximumCapacityByLabel(queueA, "x", 50); + final String queueB = CapacitySchedulerConfiguration.ROOT + ".b"; + csConf.setCapacity(queueB, 75); + csConf.setAccessibleNodeLabels(queueB, toSet("x")); + csConf.setCapacityByLabel(queueB, "x", 50); + csConf.setMaximumCapacityByLabel(queueB, "x", 50); csConf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true); @@ -2939,14 +2956,15 @@ public void testQueueMetricsWithMixedLabels() throws Exception { this.conf); // Define top-level queues - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a"}); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); - csConf.setCapacity(A, 100); - csConf.setAccessibleNodeLabels(A, toSet("x")); - csConf.setCapacityByLabel(A, "x", 100); - csConf.setMaximumCapacityByLabel(A, "x", 100); + final String queueA = CapacitySchedulerConfiguration.ROOT + ".a"; + csConf.setCapacity(queueA, 100); + csConf.setAccessibleNodeLabels(queueA, toSet("x")); + csConf.setCapacityByLabel(queueA, "x", 100); + csConf.setMaximumCapacityByLabel(queueA, "x", 100); // set node -> label // label x exclusivity is set to true @@ -3074,21 +3092,23 @@ public void testTwoLevelQueueMetricsWithLabels() throws Exception { this.conf); // Define top-level queues - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a"}); - csConf.setCapacityByLabel(ROOT, "x", 100); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); - csConf.setCapacity(A, 100); - csConf.setAccessibleNodeLabels(A, toSet("x")); - csConf.setCapacityByLabel(A, "x", 100); - csConf.setMaximumCapacityByLabel(A, "x", 100); + final String queueA = CapacitySchedulerConfiguration.ROOT + ".a"; + csConf.setCapacity(queueA, 100); + csConf.setAccessibleNodeLabels(queueA, toSet("x")); + csConf.setCapacityByLabel(queueA, "x", 100); + csConf.setMaximumCapacityByLabel(queueA, "x", 100); - csConf.setQueues(A, new String[] {"a1"}); + csConf.setQueues(queueA, new String[] {"a1"}); + final String queueA1 = queueA + ".a1"; + csConf.setCapacity(queueA1, 100); - csConf.setCapacity(A1, 100); - csConf.setAccessibleNodeLabels(A1, toSet("x")); - csConf.setCapacityByLabel(A1, "x", 100); - csConf.setMaximumCapacityByLabel(A1, "x", 100); + csConf.setAccessibleNodeLabels(queueA1, toSet("x")); + csConf.setCapacityByLabel(queueA1, "x", 100); + csConf.setMaximumCapacityByLabel(queueA1, "x", 100); // set node -> label // label x exclusivity is set to true @@ -3292,31 +3312,35 @@ public void testQueueMetricsWithLabelsDisableElasticity() throws Exception { this.conf); // Define top-level queues - csConf.setQueues(ROOT, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b" }); - csConf.setCapacityByLabel(ROOT, "x", 100); - - csConf.setCapacity(A, 50); - csConf.setMaximumCapacity(A, 100); - csConf.setAccessibleNodeLabels(A, toSet("x")); - csConf.setCapacityByLabel(A, "x", 50); - csConf.setMaximumCapacityByLabel(A, "x", 100); - - csConf.setCapacity(B, 50); - csConf.setMaximumCapacity(B, 100); - csConf.setAccessibleNodeLabels(B, toSet("x")); - csConf.setCapacityByLabel(B, "x", 50); - csConf.setMaximumCapacityByLabel(B, "x", 100); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + + final String queueA = CapacitySchedulerConfiguration.ROOT + ".a"; + csConf.setCapacity(queueA, 50); + csConf.setMaximumCapacity(queueA, 100); + csConf.setAccessibleNodeLabels(queueA, toSet("x")); + csConf.setCapacityByLabel(queueA, "x", 50); + csConf.setMaximumCapacityByLabel(queueA, "x", 100); + final String queueB = CapacitySchedulerConfiguration.ROOT + ".b"; + csConf.setCapacity(queueB, 50); + csConf.setMaximumCapacity(queueB, 100); + csConf.setAccessibleNodeLabels(queueB, toSet("x")); + csConf.setCapacityByLabel(queueB, "x", 50); + csConf.setMaximumCapacityByLabel(queueB, "x", 100); // Define 2nd-level queues - csConf.setQueues(A, new String[] {"a1", "a2"}); + csConf.setQueues(queueA, new String[] { "a1", + "a2"}); + final String A1 = queueA + ".a1"; csConf.setCapacity(A1, 20); csConf.setMaximumCapacity(A1, 60); csConf.setAccessibleNodeLabels(A1, toSet("x")); csConf.setCapacityByLabel(A1, "x", 60); csConf.setMaximumCapacityByLabel(A1, "x", 30); + final String A2 = queueA + ".a2"; csConf.setCapacity(A2, 80); csConf.setMaximumCapacity(A2, 40); csConf.setAccessibleNodeLabels(A2, toSet("x")); 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 73db7c1c11bce..85e5515691db5 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 @@ -88,35 +88,7 @@ public class TestParentQueue { private final ResourceCalculator resourceComparator = new DefaultResourceCalculator(); - - private static final String A = "a"; - private static final String B = "b"; - private static final String C = "c"; - private static final String C1 = "c1"; - private static final String C11 = "c11"; - private static final String C111 = "c111"; - private static final String C1111 = "c1111"; - private static final String D = "d"; - private static final String A1 = "a1"; - private static final String A2 = "a2"; - private static final String B1 = "b1"; - private static final String B2 = "b2"; - private static final String B3 = "b3"; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath Q_A = ROOT.createNewLeaf(A); - private static final QueuePath Q_B = ROOT.createNewLeaf(B); - private static final QueuePath Q_C = ROOT.createNewLeaf(C); - private static final QueuePath Q_D = ROOT.createNewLeaf(D); - private static final QueuePath Q_A1 = Q_A.createNewLeaf(A1); - private static final QueuePath Q_A2 = Q_A.createNewLeaf(A2); - private static final QueuePath Q_B1 = Q_B.createNewLeaf(B1); - private static final QueuePath Q_B2 = Q_B.createNewLeaf(B2); - private static final QueuePath Q_B3 = Q_B.createNewLeaf(B3); - private static final QueuePath Q_C1 = Q_C.createNewLeaf(C1); - private static final QueuePath Q_C11 = Q_C1.createNewLeaf(C11); - private static final QueuePath Q_C111 = Q_C11.createNewLeaf(C111); - private static final QueuePath Q_C1111 = Q_C111.createNewLeaf(C1111); - + @Before public void setUp() throws Exception { rmContext = TestUtils.getMockRMContext(); @@ -141,11 +113,17 @@ public void setUp() throws Exception { queueContext = new CapacitySchedulerQueueContext(csContext); } - + + private static final String A = "a"; + private static final String B = "b"; + private static final String Q_A = + CapacitySchedulerConfiguration.ROOT + "." + A; + private static final String Q_B = + CapacitySchedulerConfiguration.ROOT + "." + B; private void setupSingleLevelQueues(CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {A, B}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {A, B}); conf.setCapacity(Q_A, 30); @@ -160,12 +138,12 @@ private void setupSingleLevelQueuesWithAbsoluteResource( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[]{A, B}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{A, B}); - conf.setMinimumResourceRequirement("", Q_A, + conf.setMinimumResourceRequirement("", new QueuePath(Q_A), QUEUE_A_RESOURCE); - conf.setMinimumResourceRequirement("", Q_B, + conf.setMinimumResourceRequirement("", new QueuePath(Q_B), QUEUE_B_RESOURCE); queueContext.reinitialize(); @@ -455,40 +433,60 @@ public void testSingleLevelQueuesPrecision() throws Exception { } } + private static final String C = "c"; + private static final String C1 = "c1"; + private static final String C11 = "c11"; + private static final String C111 = "c111"; + private static final String C1111 = "c1111"; + + private static final String D = "d"; + private static final String A1 = "a1"; + private static final String A2 = "a2"; + private static final String B1 = "b1"; + private static final String B2 = "b2"; + private static final String B3 = "b3"; + private static final String B4 = "b4"; + private void setupMultiLevelQueues(CapacitySchedulerConfiguration conf) { // Define top-level queues - csConf.setQueues(ROOT, new String[] {A, B, C, D}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {A, B, C, D}); conf.setCapacity(Q_A, 10); conf.setCapacity(Q_B, 50); - + + final String Q_C = CapacitySchedulerConfiguration.ROOT + "." + C; conf.setCapacity(Q_C, 19.5f); - + + final String Q_D = CapacitySchedulerConfiguration.ROOT + "." + D; conf.setCapacity(Q_D, 20.5f); // Define 2-nd level queues conf.setQueues(Q_A, new String[] {A1, A2}); - conf.setCapacity(Q_A1, 50); - conf.setCapacity(Q_A2, 50); + conf.setCapacity(Q_A + "." + A1, 50); + conf.setCapacity(Q_A + "." + A2, 50); conf.setQueues(Q_B, new String[] {B1, B2, B3}); - conf.setCapacity(Q_B1, 10); - conf.setCapacity(Q_B2, 20); - conf.setCapacity(Q_B3, 70); + conf.setCapacity(Q_B + "." + B1, 10); + conf.setCapacity(Q_B + "." + B2, 20); + conf.setCapacity(Q_B + "." + B3, 70); conf.setQueues(Q_C, new String[] {C1}); + final String Q_C1= Q_C + "." + C1; conf.setCapacity(Q_C1, 100); conf.setQueues(Q_C1, new String[] {C11}); + final String Q_C11= Q_C1 + "." + C11; conf.setCapacity(Q_C11, 100); conf.setQueues(Q_C11, new String[] {C111}); + final String Q_C111= Q_C11 + "." + C111; conf.setCapacity(Q_C111, 100); //Leaf Queue conf.setQueues(Q_C111, new String[] {C1111}); + final String Q_C1111= Q_C111 + "." + C1111; conf.setCapacity(Q_C1111, 100); queueContext.reinitialize(); } @@ -679,9 +677,9 @@ public void testQueueCapacitySettingChildZero() throws Exception { assumeThat(csConf.isLegacyQueueMode(), is(true)); // set child queues capacity to 0 when parents not 0 - csConf.setCapacity(Q_B1, 0); - csConf.setCapacity(Q_B2, 0); - csConf.setCapacity(Q_B3, 0); + csConf.setCapacity(Q_B + "." + B1, 0); + csConf.setCapacity(Q_B + "." + B2, 0); + csConf.setCapacity(Q_B + "." + B3, 0); queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); @@ -742,9 +740,9 @@ public void testQueueCapacitySettingParentZeroChildren50pctZeroSumAllowed() // and allow zero capacity sum csConf.setCapacity(Q_B, 0); csConf.setCapacity(Q_A, 100); - csConf.setCapacity(Q_B1, 10); - csConf.setCapacity(Q_B2, 20); - csConf.setCapacity(Q_B3, 20); + csConf.setCapacity(Q_B + "." + B1, 10); + csConf.setCapacity(Q_B + "." + B2, 20); + csConf.setCapacity(Q_B + "." + B3, 20); csConf.setAllowZeroCapacitySum(Q_B, true); queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); @@ -763,9 +761,9 @@ public void testQueueCapacitySettingParentNonZeroChildrenZeroSumAllowed() // and allow zero capacity sum csConf.setCapacity(Q_B, 10); csConf.setCapacity(Q_A, 50); - csConf.setCapacity(Q_B1, 0); - csConf.setCapacity(Q_B2, 0); - csConf.setCapacity(Q_B3, 0); + csConf.setCapacity(Q_B + "." + B1, 0); + csConf.setCapacity(Q_B + "." + B2, 0); + csConf.setCapacity(Q_B + "." + B3, 0); csConf.setAllowZeroCapacitySum(Q_B, true); queueContext.reinitialize(); CSQueueStore queues = new CSQueueStore(); @@ -781,9 +779,9 @@ public void testQueueCapacityZero() throws Exception { // set parent and child capacity to 0 csConf.setCapacity(Q_B, 0); - csConf.setCapacity(Q_B1, 0); - csConf.setCapacity(Q_B2, 0); - csConf.setCapacity(Q_B3, 0); + 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(); @@ -979,10 +977,12 @@ public boolean hasQueueACL(List aclInfos, QueueACL acl, String public void testQueueAcl() throws Exception { setupMultiLevelQueues(csConf); - csConf.setAcl(ROOT, QueueACL.SUBMIT_APPLICATIONS, " "); - csConf.setAcl(ROOT, QueueACL.ADMINISTER_QUEUE, " "); + csConf.setAcl(CapacitySchedulerConfiguration.ROOT, QueueACL.SUBMIT_APPLICATIONS, " "); + csConf.setAcl(CapacitySchedulerConfiguration.ROOT, QueueACL.ADMINISTER_QUEUE, " "); + final String Q_C = CapacitySchedulerConfiguration.ROOT + "." + C; csConf.setAcl(Q_C, QueueACL.ADMINISTER_QUEUE, "*"); + final String Q_C11= Q_C + "." + C1 + "." + C11; csConf.setAcl(Q_C11, QueueACL.SUBMIT_APPLICATIONS, "*"); queueContext.reinitialize(); 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/TestQueueMappings.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/TestQueueMappings.java index b30ea50bc9e57..e1e06d72fc853 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/TestQueueMappings.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/TestQueueMappings.java @@ -43,9 +43,6 @@ public class TestQueueMappings { CapacitySchedulerConfiguration.ROOT + "." + Q1; private final static String Q2_PATH = CapacitySchedulerConfiguration.ROOT + "." + Q2; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath Q1_QUEUE_PATH = new QueuePath(Q1_PATH); - private static final QueuePath Q2_QUEUE_PATH = new QueuePath(Q2_PATH); private CapacityScheduler cs; private YarnConfiguration conf; @@ -74,10 +71,10 @@ public void tearDown() { private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {Q1, Q2}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { Q1, Q2 }); - conf.setCapacity(Q1_QUEUE_PATH, 10); - conf.setCapacity(Q2_QUEUE_PATH, 90); + conf.setCapacity(Q1_PATH, 10); + conf.setCapacity(Q2_PATH, 90); LOG.info("Setup top-level queues q1 and q2"); } 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/TestQueueParsing.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/TestQueueParsing.java index 65bce5c9318a8..736293c5c4331 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/TestQueueParsing.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/TestQueueParsing.java @@ -54,46 +54,7 @@ public class TestQueueParsing { private static final Logger LOG = LoggerFactory.getLogger(TestQueueParsing.class); - private static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - private static final String B_PATH = CapacitySchedulerConfiguration.ROOT + ".b"; - private static final String C_PATH = CapacitySchedulerConfiguration.ROOT + ".c"; - private static final String A1_PATH = A_PATH + ".a1"; - private static final String A2_PATH = A_PATH + ".a2"; - private static final String B1_PATH = B_PATH + ".b1"; - private static final String B2_PATH = B_PATH + ".b2"; - private static final String B3_PATH = B_PATH + ".b3"; - private static final String B4_PATH = B_PATH + ".b4"; - private static final String C1_PATH = C_PATH + ".c1"; - private static final String C2_PATH = C_PATH + ".c2"; - private static final String C3_PATH = C_PATH + ".c3"; - private static final String C4_PATH = C_PATH + ".c4"; - private static final String C11_PATH = C1_PATH + ".c11"; - private static final String C12_PATH = C1_PATH + ".c12"; - private static final String C13_PATH = C1_PATH + ".c13"; - private static final String AX_PATH = "root.a.x"; - private static final String AY_PATH = "root.a.y"; - private static final String X_PATH = "root.x"; - - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A = new QueuePath(A_PATH); - private static final QueuePath B = new QueuePath(B_PATH); - private static final QueuePath C = new QueuePath(C_PATH); - private static final QueuePath A1 = new QueuePath(A1_PATH); - private static final QueuePath A2 = new QueuePath(A2_PATH); - private static final QueuePath B1 = new QueuePath(B1_PATH); - private static final QueuePath B2 = new QueuePath(B2_PATH); - private static final QueuePath B3 = new QueuePath(B3_PATH); - private static final QueuePath B4 = new QueuePath(B4_PATH); - private static final QueuePath C1 = new QueuePath(C1_PATH); - private static final QueuePath C2 = new QueuePath(C2_PATH); - private static final QueuePath C3 = new QueuePath(C3_PATH); - private static final QueuePath C4 = new QueuePath(C4_PATH); - private static final QueuePath C11 = new QueuePath(C11_PATH); - private static final QueuePath C12 = new QueuePath(C12_PATH); - private static final QueuePath C13 = new QueuePath(C13_PATH); - private static final QueuePath AX = new QueuePath(AX_PATH); - private static final QueuePath AY = new QueuePath(AY_PATH); - private static final QueuePath X = new QueuePath(X_PATH); + private static final double DELTA = 0.001; private static final int GB = 1024; @@ -114,23 +75,33 @@ public void setup() throws Exception{ private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b", "c"}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"}); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 20); + + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; conf.setCapacity(C, 70); conf.setMaximumCapacity(C, 70); LOG.info("Setup top-level queues"); // Define 2nd-level queues + final String A1 = A + ".a1"; + final String A2 = A + ".a2"; conf.setQueues(A, new String[] {"a1", "a2"}); conf.setCapacity(A1, 30); conf.setMaximumCapacity(A1, 45); conf.setCapacity(A2, 70); conf.setMaximumCapacity(A2, 85); + final String B1 = B + ".b1"; + final String B2 = B + ".b2"; + final String B3 = B + ".b3"; conf.setQueues(B, new String[] {"b1", "b2", "b3"}); conf.setCapacity(B1, 50); conf.setMaximumCapacity(B1, 85); @@ -139,6 +110,10 @@ private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) { conf.setCapacity(B3, 20); conf.setMaximumCapacity(B3, 35); + final String C1 = C + ".c1"; + final String C2 = C + ".c2"; + final String C3 = C + ".c3"; + final String C4 = C + ".c4"; conf.setQueues(C, new String[] {"c1", "c2", "c3", "c4"}); conf.setCapacity(C1, 50); conf.setMaximumCapacity(C1, 55); @@ -152,6 +127,9 @@ private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) { LOG.info("Setup 2nd-level queues"); // Define 3rd-level queues + final String C11 = C1 + ".c11"; + final String C12 = C1 + ".c12"; + final String C13 = C1 + ".c13"; conf.setQueues(C1, new String[] {"c11", "c12", "c13"}); conf.setCapacity(C11, 15); conf.setMaximumCapacity(C11, 30); @@ -167,12 +145,18 @@ private void setupQueueConfigurationWithSpacesShouldBeTrimmed( CapacitySchedulerConfiguration conf) { // Define top-level queues conf.set( - QueuePrefixes.getQueuePrefix(ROOT) + CapacitySchedulerConfiguration + .getQueuePrefix(CapacitySchedulerConfiguration.ROOT) + CapacitySchedulerConfiguration.QUEUES, " a ,b, c"); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 20); + + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; conf.setCapacity(C, 70); conf.setMaximumCapacity(C, 70); } @@ -181,40 +165,57 @@ private void setupNestedQueueConfigurationWithSpacesShouldBeTrimmed( CapacitySchedulerConfiguration conf) { // Define top-level queues conf.set( - QueuePrefixes.getQueuePrefix(ROOT) + CapacitySchedulerConfiguration + .getQueuePrefix(CapacitySchedulerConfiguration.ROOT) + CapacitySchedulerConfiguration.QUEUES, " a ,b, c"); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 20); + + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; conf.setCapacity(C, 70); conf.setMaximumCapacity(C, 70); // sub queues for A - conf.set(QueuePrefixes.getQueuePrefix(A) + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(A) + CapacitySchedulerConfiguration.QUEUES, "a1, a2 "); + final String A1 = CapacitySchedulerConfiguration.ROOT + ".a.a1"; conf.setCapacity(A1, 60); + + final String A2 = CapacitySchedulerConfiguration.ROOT + ".a.a2"; conf.setCapacity(A2, 40); } private void setupQueueConfigurationWithoutLabels(CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b"}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 90); LOG.info("Setup top-level queues"); // Define 2nd-level queues + final String A1 = A + ".a1"; + final String A2 = A + ".a2"; conf.setQueues(A, new String[] {"a1", "a2"}); conf.setCapacity(A1, 30); conf.setMaximumCapacity(A1, 45); conf.setCapacity(A2, 70); conf.setMaximumCapacity(A2, 85); + final String B1 = B + ".b1"; + final String B2 = B + ".b2"; + final String B3 = B + ".b3"; conf.setQueues(B, new String[] {"b1", "b2", "b3"}); conf.setCapacity(B1, 50); conf.setMaximumCapacity(B1, 85); @@ -226,17 +227,22 @@ private void setupQueueConfigurationWithoutLabels(CapacitySchedulerConfiguration private void setupQueueConfigurationWithLabels(CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b"}); - conf.setCapacityByLabel(ROOT, "red", 100); - conf.setCapacityByLabel(ROOT, "blue", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "red", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "blue", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 90); LOG.info("Setup top-level queues"); // Define 2nd-level queues + final String A1 = A + ".a1"; + final String A2 = A + ".a2"; conf.setQueues(A, new String[] {"a1", "a2"}); conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue")); conf.setCapacityByLabel(A, "red", 50); @@ -254,6 +260,9 @@ private void setupQueueConfigurationWithLabels(CapacitySchedulerConfiguration co conf.setCapacityByLabel(A2, "red", 50); conf.setMaximumCapacityByLabel(A2, "red", 60); + final String B1 = B + ".b1"; + final String B2 = B + ".b2"; + final String B3 = B + ".b3"; conf.setQueues(B, new String[] {"b1", "b2", "b3"}); conf.setAccessibleNodeLabels(B, ImmutableSet.of("red", "blue")); conf.setCapacityByLabel(B, "red", 50); @@ -278,20 +287,24 @@ private void setupQueueConfigurationWithLabels(CapacitySchedulerConfiguration co private void setupQueueConfigurationWithLabelsAndReleaseCheck( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b"}); - conf.setCapacityByLabel(ROOT, "red", 100); - conf.setCapacityByLabel(ROOT, "blue", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "red", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "blue", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; // The cap <= max-cap check is not needed conf.setCapacity(A, 50); conf.setMaximumCapacity(A, 100); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 50); conf.setMaximumCapacity(B, 100); LOG.info("Setup top-level queues"); // Define 2nd-level queues + final String A1 = A + ".a1"; + final String A2 = A + ".a2"; conf.setQueues(A, new String[] {"a1", "a2"}); conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue")); conf.setCapacityByLabel(A, "red", 50); @@ -312,6 +325,9 @@ private void setupQueueConfigurationWithLabelsAndReleaseCheck( conf.setCapacityByLabel(A2, "red", 40); conf.setMaximumCapacityByLabel(A2, "red", 60); + final String B1 = B + ".b1"; + final String B2 = B + ".b2"; + final String B3 = B + ".b3"; conf.setQueues(B, new String[] {"b1", "b2", "b3"}); conf.setAccessibleNodeLabels(B, ImmutableSet.of("red", "blue")); conf.setCapacityByLabel(B, "red", 50); @@ -340,11 +356,12 @@ private void setupQueueConfigurationWithLabelsAndReleaseCheck( private void setupQueueConfigurationWithLabelsInherit( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b"}); - conf.setCapacityByLabel(ROOT, "red", 100); - conf.setCapacityByLabel(ROOT, "blue", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "red", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "blue", 100); // Set A configuration + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); conf.setQueues(A, new String[] {"a1", "a2"}); @@ -353,10 +370,14 @@ private void setupQueueConfigurationWithLabelsInherit( conf.setCapacityByLabel(A, "blue", 100); // Set B configuraiton + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 90); conf.setAccessibleNodeLabels(B, CommonNodeLabelsManager.EMPTY_STRING_SET); // Define 2nd-level queues + final String A1 = A + ".a1"; + final String A2 = A + ".a2"; + conf.setCapacity(A1, 30); conf.setMaximumCapacity(A1, 45); conf.setCapacityByLabel(A1, "red", 50); @@ -371,9 +392,10 @@ private void setupQueueConfigurationWithLabelsInherit( private void setupQueueConfigurationWithSingleLevel( CapacitySchedulerConfiguration conf) { // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b"}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); // Set A configuration + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue")); @@ -381,6 +403,7 @@ private void setupQueueConfigurationWithSingleLevel( conf.setCapacityByLabel(A, "blue", 90); // Set B configuraiton + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 90); conf.setAccessibleNodeLabels(B, ImmutableSet.of("red", "blue")); conf.setCapacityByLabel(B, "red", 10); @@ -446,7 +469,7 @@ public void testQueueParsing() throws Exception { public void testRootQueueParsing() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); // non-100 percent value will throw IllegalArgumentException - csConf.setCapacity(ROOT, 90); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT, 90); } @Test @@ -689,7 +712,7 @@ public void testQueueParsingWithUnusedLabels() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); setupQueueConfiguration(csConf); - csConf.setAccessibleNodeLabels(ROOT, labels); + csConf.setAccessibleNodeLabels(CapacitySchedulerConfiguration.ROOT, labels); YarnConfiguration conf = new YarnConfiguration(csConf); MockRM rm = createMockRMWithLabels(conf, labels); @@ -784,7 +807,7 @@ public void testQueueParsingFailWhenSumOfChildrenNonLabeledCapacityNot100Percent // some of the resources of a parent queue unallocated assumeThat(csConf.isLegacyQueueMode(), is(true)); setupQueueConfiguration(csConf); - csConf.setCapacity(C2, 5); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".c.c2", 5); YarnConfiguration conf = new YarnConfiguration(csConf); MockRM rm = createMockRMWithLabels(conf, ImmutableSet.of("red", "blue")); @@ -806,7 +829,8 @@ public void testQueueParsingFailWhenSumOfChildrenLabeledCapacityNot100Percent() assumeThat(csConf.isLegacyQueueMode(), is(true)); setupQueueConfigurationWithLabels(csConf); - csConf.setCapacityByLabel(B3, "red", 24); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT + ".b.b3", + "red", 24); YarnConfiguration conf = new YarnConfiguration(csConf); MockRM rm = createMockRMWithLabels(conf, ImmutableSet.of("red", "blue")); @@ -828,9 +852,12 @@ public void testQueueParsingWithSumOfChildLabelCapacityNot100PercentWithWildCard assumeThat(csConf.isLegacyQueueMode(), is(true)); setupQueueConfigurationWithLabels(csConf); - csConf.setCapacityByLabel(B3, "red", 24); - csConf.setAccessibleNodeLabels(ROOT, ImmutableSet.of(RMNodeLabelsManager.ANY)); - csConf.setAccessibleNodeLabels(B, ImmutableSet.of(RMNodeLabelsManager.ANY)); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT + ".b.b3", + "red", 24); + csConf.setAccessibleNodeLabels(CapacitySchedulerConfiguration.ROOT, + ImmutableSet.of(RMNodeLabelsManager.ANY)); + csConf.setAccessibleNodeLabels(CapacitySchedulerConfiguration.ROOT + ".b", + ImmutableSet.of(RMNodeLabelsManager.ANY)); YarnConfiguration conf = new YarnConfiguration(csConf); MockRM rm = createMockRMWithLabels(conf, ImmutableSet.of("red", "blue")); @@ -841,21 +868,21 @@ public void testQueueParsingWithSumOfChildLabelCapacityNot100PercentWithWildCard public void testQueueParsingWithMoveQueue() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] {"a"}); - csConf.setQueues(A, new String[] {"x", "y"}); - csConf.setCapacity(A, 100); - csConf.setCapacity(AX, 50); - csConf.setCapacity(AY, 50); + csConf.setQueues("root", new String[] { "a" }); + csConf.setQueues("root.a", new String[] { "x", "y" }); + csConf.setCapacity("root.a", 100); + csConf.setCapacity("root.a.x", 50); + csConf.setCapacity("root.a.y", 50); YarnConfiguration conf = new YarnConfiguration(csConf); MockRM rm = createMockRMWithoutLabels(conf); CapacityScheduler capacityScheduler = (CapacityScheduler) rm.getResourceScheduler(); - csConf.setQueues(ROOT, new String[] {"a", "x"}); - csConf.setQueues(A, new String[] {"y"}); - csConf.setCapacity(X, 50); - csConf.setCapacity(A, 50); - csConf.setCapacity(AY, 100); + csConf.setQueues("root", new String[] { "a", "x" }); + csConf.setQueues("root.a", new String[] { "y" }); + csConf.setCapacity("root.x", 50); + csConf.setCapacity("root.a", 50); + csConf.setCapacity("root.a.y", 100); capacityScheduler.reinitialize(csConf, rm.getRMContext()); ServiceOperations.stopQuietly(rm); @@ -872,11 +899,12 @@ public void testRMStartWrongNodeCapacity() throws Exception { assumeThat(csConf.isLegacyQueueMode(), is(true)); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a"}); - csConf.setCapacityByLabel(ROOT, "x", 100); - csConf.setCapacityByLabel(ROOT, "y", 100); - csConf.setCapacityByLabel(ROOT, "z", 100); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a"}); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setCapacity(A, 100); csConf.setAccessibleNodeLabels(A, labels); csConf.setCapacityByLabel(A, "x", 100); @@ -900,9 +928,9 @@ public void testQueueOrderingPolicyUpdatedAfterReinitialize() throws Exception { CapacityScheduler capacityScheduler = (CapacityScheduler) rm.getResourceScheduler(); // Add a new b4 queue - csConf.setQueues(B, + csConf.setQueues(CapacitySchedulerConfiguration.ROOT + ".b", new String[] { "b1", "b2", "b3", "b4" }); - csConf.setCapacity(B4, 0f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".b.b4", 0f); ParentQueue bQ = (ParentQueue) capacityScheduler.getQueue("b"); checkEqualsToQueueSet(bQ.getChildQueues(), new String[] { "b1", "b2", "b3" }); @@ -927,11 +955,12 @@ public void testQueueCapacityWithWeight() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a"}); - csConf.setLabeledQueueWeight(ROOT, "x", 100); - csConf.setLabeledQueueWeight(ROOT, "y", 100); - csConf.setLabeledQueueWeight(ROOT, "z", 100); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a"}); + csConf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "x", 100); + csConf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "y", 100); + csConf.setLabeledQueueWeight(CapacitySchedulerConfiguration.ROOT, "z", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; csConf.setNonLabeledQueueWeight(A, 100); csConf.setAccessibleNodeLabels(A, labels); csConf.setLabeledQueueWeight(A, "x", 100); @@ -946,10 +975,10 @@ public void testQueueCapacityWithWeight() throws Exception { verifyQueueAbsCapacity(rm, CapacitySchedulerConfiguration.ROOT, "y", 1f); verifyQueueAbsCapacity(rm, CapacitySchedulerConfiguration.ROOT, "z", 1f); - verifyQueueAbsCapacity(rm, A_PATH, "", 1f); - verifyQueueAbsCapacity(rm, A_PATH, "x", 1f); - verifyQueueAbsCapacity(rm, A_PATH, "y", 1f); - verifyQueueAbsCapacity(rm, A_PATH, "z", 1f); + verifyQueueAbsCapacity(rm, A, "", 1f); + verifyQueueAbsCapacity(rm, A, "x", 1f); + verifyQueueAbsCapacity(rm, A, "y", 1f); + verifyQueueAbsCapacity(rm, A, "z", 1f); ServiceOperations.stopQuietly(rm); } @@ -962,17 +991,17 @@ public void testQueueParsingWithDefaultUserLimitValues() throws Exception { final String queueB = CapacitySchedulerConfiguration.ROOT + ".b"; // Define top-level queues - csConf.setQueues(ROOT, new String[] {"a", "b"}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); // Set default value csConf.setDefaultUserLimit(20); csConf.setDefaultUserLimitFactor(2.0f); // Set A configuration and let B use default values - csConf.setCapacity(A, 50); - csConf.setUserLimit(A, 15); - csConf.setUserLimitFactor(A, 1.5f); - csConf.setCapacity(B, 50); + csConf.setCapacity(queueA, 50); + csConf.setUserLimit(queueA, 15); + csConf.setUserLimitFactor(queueA, 1.5f); + csConf.setCapacity(queueB, 50); YarnConfiguration conf = new YarnConfiguration(csConf); MockRM rm = createMockRMWithoutLabels(conf); @@ -981,33 +1010,33 @@ public void testQueueParsingWithDefaultUserLimitValues() throws Exception { CapacityScheduler capacityScheduler = (CapacityScheduler) rm.getResourceScheduler(); Assert.assertEquals(15, - ((LeafQueue)capacityScheduler.getQueue(A_PATH)).getUserLimit(), DELTA); + ((LeafQueue)capacityScheduler.getQueue(queueA)).getUserLimit(), DELTA); Assert.assertEquals(1.5, - ((LeafQueue)capacityScheduler.getQueue(A_PATH)).getUserLimitFactor(), DELTA); + ((LeafQueue)capacityScheduler.getQueue(queueA)).getUserLimitFactor(), DELTA); Assert.assertEquals(20, - ((LeafQueue)capacityScheduler.getQueue(B_PATH)).getUserLimit(), DELTA); + ((LeafQueue)capacityScheduler.getQueue(queueB)).getUserLimit(), DELTA); Assert.assertEquals(2.0, - ((LeafQueue)capacityScheduler.getQueue(B_PATH)).getUserLimitFactor(), DELTA); + ((LeafQueue)capacityScheduler.getQueue(queueB)).getUserLimitFactor(), DELTA); // Use hadoop default value csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT, new String[] {"a", "b"}); - csConf.setCapacity(A, 50); - csConf.setUserLimit(A, 15); - csConf.setUserLimitFactor(A, 1.5f); - csConf.setCapacity(B, 50); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + csConf.setCapacity(queueA, 50); + csConf.setUserLimit(queueA, 15); + csConf.setUserLimitFactor(queueA, 1.5f); + csConf.setCapacity(queueB, 50); capacityScheduler.reinitialize(csConf, rm.getRMContext()); Assert.assertEquals(15, - ((LeafQueue)capacityScheduler.getQueue(A_PATH)).getUserLimit(), DELTA); + ((LeafQueue)capacityScheduler.getQueue(queueA)).getUserLimit(), DELTA); Assert.assertEquals(1.5, - ((LeafQueue)capacityScheduler.getQueue(A_PATH)).getUserLimitFactor(), DELTA); + ((LeafQueue)capacityScheduler.getQueue(queueA)).getUserLimitFactor(), DELTA); Assert.assertEquals(100, - ((LeafQueue)capacityScheduler.getQueue(B_PATH)).getUserLimit(), DELTA); + ((LeafQueue)capacityScheduler.getQueue(queueB)).getUserLimit(), DELTA); Assert.assertEquals(1, - ((LeafQueue)capacityScheduler.getQueue(B_PATH)).getUserLimitFactor(), DELTA); + ((LeafQueue)capacityScheduler.getQueue(queueB)).getUserLimitFactor(), DELTA); } private void verifyQueueAbsCapacity(MockRM rm, String queuePath, String label, 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/TestQueuePath.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/TestQueuePath.java index dc6437930676d..91171966c119c 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/TestQueuePath.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/TestQueuePath.java @@ -22,32 +22,23 @@ import org.junit.Assert; import org.junit.Test; -import java.util.ArrayList; import java.util.List; public class TestQueuePath { private static final String TEST_QUEUE = "root.level_1.level_2.level_3"; - private static final QueuePath TEST_QUEUE_PATH = new QueuePath(TEST_QUEUE); - private static final QueuePath QUEUE_PATH_WITH_EMPTY_PART = new QueuePath("root..level_2"); - private static final QueuePath QUEUE_PATH_WITH_EMPTY_LEAF = new QueuePath("root.level_1."); - private static final QueuePath ROOT_PATH = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath EMPTY_PATH = new QueuePath(""); - private static final QueuePath ONE_LEVEL_WILDCARDED_TEST_PATH = - new QueuePath("root.level_1.level_2.*"); - private static final QueuePath TWO_LEVEL_WILDCARDED_TEST_PATH = - new QueuePath("root.level_1.*.*"); - private static final QueuePath THREE_LEVEL_WILDCARDED_TEST_PATH = - new QueuePath("root.*.*.*"); @Test public void testCreation() { - Assert.assertEquals(TEST_QUEUE, TEST_QUEUE_PATH.getFullPath()); - Assert.assertEquals("root.level_1.level_2", TEST_QUEUE_PATH.getParent()); - Assert.assertEquals("level_3", TEST_QUEUE_PATH.getLeafName()); + QueuePath queuePath = new QueuePath(TEST_QUEUE); + + Assert.assertEquals(TEST_QUEUE, queuePath.getFullPath()); + Assert.assertEquals("root.level_1.level_2", queuePath.getParent()); + Assert.assertEquals("level_3", queuePath.getLeafName()); - Assert.assertNull(ROOT_PATH.getParent()); + QueuePath rootPath = new QueuePath(CapacitySchedulerConfiguration.ROOT); + Assert.assertNull(rootPath.getParent()); - QueuePath appendedPath = TEST_QUEUE_PATH.createNewLeaf("level_4"); + QueuePath appendedPath = queuePath.createNewLeaf("level_4"); Assert.assertEquals(TEST_QUEUE + CapacitySchedulerConfiguration.DOT + "level_4", appendedPath.getFullPath()); Assert.assertEquals("root.level_1.level_2.level_3", appendedPath.getParent()); @@ -56,9 +47,13 @@ public void testCreation() { @Test public void testEmptyPart() { - Assert.assertTrue(QUEUE_PATH_WITH_EMPTY_PART.hasEmptyPart()); - Assert.assertTrue(QUEUE_PATH_WITH_EMPTY_LEAF.hasEmptyPart()); - Assert.assertFalse(TEST_QUEUE_PATH.hasEmptyPart()); + QueuePath queuePathWithEmptyPart = new QueuePath("root..level_2"); + QueuePath queuePathWithEmptyLeaf = new QueuePath("root.level_1."); + QueuePath queuePathWithoutEmptyPart = new QueuePath(TEST_QUEUE); + + Assert.assertTrue(queuePathWithEmptyPart.hasEmptyPart()); + Assert.assertTrue(queuePathWithEmptyLeaf.hasEmptyPart()); + Assert.assertFalse(queuePathWithoutEmptyPart.hasEmptyPart()); } @Test @@ -73,10 +68,14 @@ public void testNullPath() { @Test public void testIterator() { - List queuePathCollection = ImmutableList.copyOf(TEST_QUEUE_PATH.iterator()); + QueuePath queuePath = new QueuePath(TEST_QUEUE); + QueuePath queuePathWithEmptyPart = new QueuePath("root..level_2"); + QueuePath rootPath = new QueuePath(CapacitySchedulerConfiguration.ROOT); + + List queuePathCollection = ImmutableList.copyOf(queuePath.iterator()); List queuePathWithEmptyPartCollection = ImmutableList.copyOf( - QUEUE_PATH_WITH_EMPTY_PART.iterator()); - List rootPathCollection = ImmutableList.copyOf(ROOT_PATH.iterator()); + queuePathWithEmptyPart.iterator()); + List rootPathCollection = ImmutableList.copyOf(rootPath.iterator()); Assert.assertEquals(4, queuePathCollection.size()); Assert.assertEquals(CapacitySchedulerConfiguration.ROOT, queuePathCollection.get(0)); @@ -93,10 +92,14 @@ public void testIterator() { @Test public void testReversePathIterator() { - List queuePathCollection = ImmutableList.copyOf(TEST_QUEUE_PATH.reverseIterator()); + QueuePath queuePath = new QueuePath(TEST_QUEUE); + QueuePath queuePathWithEmptyPart = new QueuePath("root..level_2"); + QueuePath rootPath = new QueuePath(CapacitySchedulerConfiguration.ROOT); + + List queuePathCollection = ImmutableList.copyOf(queuePath.reverseIterator()); List queuePathWithEmptyPartCollection = ImmutableList.copyOf( - QUEUE_PATH_WITH_EMPTY_PART.reverseIterator()); - List rootPathCollection = ImmutableList.copyOf(ROOT_PATH.reverseIterator()); + queuePathWithEmptyPart.reverseIterator()); + List rootPathCollection = ImmutableList.copyOf(rootPath.reverseIterator()); Assert.assertEquals(4, queuePathCollection.size()); Assert.assertEquals(CapacitySchedulerConfiguration.ROOT, @@ -125,101 +128,4 @@ public void testEquals() { Assert.assertEquals(empty, emptySame); Assert.assertNotEquals(null, queuePath); } - - @Test - public void testInvalidPath() { - Assert.assertFalse(TEST_QUEUE_PATH.isInvalid()); - Assert.assertFalse(ROOT_PATH.isInvalid()); - Assert.assertTrue(EMPTY_PATH.isInvalid()); - Assert.assertTrue(new QueuePath("invalidPath").isInvalid()); - } - - @Test - public void testGetParentObject() { - Assert.assertEquals(new QueuePath("root.level_1.level_2"), - TEST_QUEUE_PATH.getParentObject()); - Assert.assertEquals(ROOT_PATH, new QueuePath("root.level_1").getParentObject()); - Assert.assertNull(ROOT_PATH.getParentObject()); - } - - @Test - public void testGetPathComponents() { - Assert.assertArrayEquals(TEST_QUEUE_PATH.getPathComponents(), - new String[] {"root", "level_1", "level_2", "level_3"}); - Assert.assertArrayEquals(ROOT_PATH.getPathComponents(), new String[] {"root"}); - Assert.assertArrayEquals(EMPTY_PATH.getPathComponents(), new String[] {""}); - } - - @Test - public void testWildcardedQueuePathsWithOneLevelWildCard() { - int maxAutoCreatedQueueDepth = 1; - - List expectedPaths = new ArrayList<>(); - expectedPaths.add(TEST_QUEUE_PATH); - expectedPaths.add(ONE_LEVEL_WILDCARDED_TEST_PATH); - - List wildcardedPaths = TEST_QUEUE_PATH - .getWildcardedQueuePaths(maxAutoCreatedQueueDepth); - - Assert.assertEquals(expectedPaths, wildcardedPaths); - } - - @Test - public void testWildcardedQueuePathsWithTwoLevelWildCard() { - int maxAutoCreatedQueueDepth = 2; - - List expectedPaths = new ArrayList<>(); - expectedPaths.add(TEST_QUEUE_PATH); - expectedPaths.add(ONE_LEVEL_WILDCARDED_TEST_PATH); - expectedPaths.add(TWO_LEVEL_WILDCARDED_TEST_PATH); - - List wildcardedPaths = TEST_QUEUE_PATH - .getWildcardedQueuePaths(maxAutoCreatedQueueDepth); - - Assert.assertEquals(expectedPaths, wildcardedPaths); - } - - @Test - public void testWildcardedQueuePathsWithThreeLevelWildCard() { - int maxAutoCreatedQueueDepth = 3; - - List expectedPaths = new ArrayList<>(); - expectedPaths.add(TEST_QUEUE_PATH); - expectedPaths.add(ONE_LEVEL_WILDCARDED_TEST_PATH); - expectedPaths.add(TWO_LEVEL_WILDCARDED_TEST_PATH); - expectedPaths.add(THREE_LEVEL_WILDCARDED_TEST_PATH); - - List wildcardedPaths = TEST_QUEUE_PATH - .getWildcardedQueuePaths(maxAutoCreatedQueueDepth); - - Assert.assertEquals(expectedPaths, wildcardedPaths); - } - - @Test - public void testWildcardingWhenMaxACQDepthIsGreaterThanQueuePathDepth() { - int maxAutoCreatedQueueDepth = 4; - - List expectedPaths = new ArrayList<>(); - expectedPaths.add(TEST_QUEUE_PATH); - expectedPaths.add(ONE_LEVEL_WILDCARDED_TEST_PATH); - expectedPaths.add(TWO_LEVEL_WILDCARDED_TEST_PATH); - expectedPaths.add(THREE_LEVEL_WILDCARDED_TEST_PATH); - - List wildcardedPaths = TEST_QUEUE_PATH - .getWildcardedQueuePaths(maxAutoCreatedQueueDepth); - - Assert.assertEquals(expectedPaths, wildcardedPaths); - } - - @Test - public void testWildcardedQueuePathsWithRootPath() { - int maxAutoCreatedQueueDepth = 1; - - List expectedPaths = new ArrayList<>(); - expectedPaths.add(ROOT_PATH); - - List wildcardedPaths = ROOT_PATH.getWildcardedQueuePaths(maxAutoCreatedQueueDepth); - - Assert.assertEquals(expectedPaths, wildcardedPaths); - } } 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/TestQueueState.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/TestQueueState.java index 482f7f332b48c..aa3b5919fcc60 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/TestQueueState.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/TestQueueState.java @@ -53,15 +53,12 @@ public class TestQueueState { private static final String Q2 = "q2"; private static final String Q3 = "q3"; - - private final static QueuePath ROOT_PATH = - new QueuePath(CapacitySchedulerConfiguration.ROOT); - private final static QueuePath Q1_PATH = - new QueuePath(CapacitySchedulerConfiguration.ROOT + "." + Q1); - private final static QueuePath Q2_PATH = - new QueuePath(Q1_PATH + "." + Q2); - private final static QueuePath Q3_PATH = - new QueuePath(Q1_PATH + "." + Q3); + private final static String Q1_PATH = + CapacitySchedulerConfiguration.ROOT + "." + Q1; + private final static String Q2_PATH = + Q1_PATH + "." + Q2; + private final static String Q3_PATH = + Q1_PATH + "." + Q3; private CapacityScheduler cs; private YarnConfiguration conf; @@ -69,7 +66,7 @@ public class TestQueueState { public void testQueueState() throws IOException { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT_PATH, new String[] {Q1}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {Q1}); csConf.setQueues(Q1_PATH, new String[] {Q2}); csConf.setCapacity(Q1_PATH, 100); @@ -123,7 +120,7 @@ public void testQueueState() throws IOException { public void testQueueStateTransit() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT_PATH, new String[] {Q1}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {Q1}); csConf.setQueues(Q1_PATH, new String[] {Q2, Q3}); csConf.setCapacity(Q1_PATH, 100); @@ -235,7 +232,7 @@ public void testRecoverDrainingStateAfterRMRestart() throws Exception { false); newConf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName()); newConf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 1); - newConf.setQueues(ROOT_PATH, new String[]{Q1}); + newConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{Q1}); newConf.setQueues(Q1_PATH, new String[]{Q2}); newConf.setCapacity(Q1_PATH, 100); newConf.setCapacity(Q2_PATH, 100); 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/TestQueueStateManager.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/TestQueueStateManager.java index 27ae3c3ae0187..e893717a8dd0f 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/TestQueueStateManager.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/TestQueueStateManager.java @@ -53,11 +53,6 @@ public class TestQueueStateManager { Q1_PATH + "." + Q2; private final static String Q3_PATH = Q1_PATH + "." + Q3; - private final static QueuePath ROOT_QUEUE_PATH = - new QueuePath(CapacitySchedulerConfiguration.ROOT); - private final static QueuePath Q1_QUEUE_PATH = new QueuePath(Q1_PATH); - private final static QueuePath Q2_QUEUE_PATH = new QueuePath(Q2_PATH); - private final static QueuePath Q3_QUEUE_PATH = new QueuePath(Q3_PATH); private CapacityScheduler cs; private YarnConfiguration conf; @@ -66,12 +61,12 @@ public void testQueueStateManager() throws AccessControlException, YarnException { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(ROOT_QUEUE_PATH, new String[] {Q1}); - csConf.setQueues(Q1_QUEUE_PATH, new String[] {Q2, Q3}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {Q1}); + csConf.setQueues(Q1_PATH, new String[] {Q2, Q3}); - csConf.setCapacity(Q1_QUEUE_PATH, 100); - csConf.setCapacity(Q2_QUEUE_PATH, 50); - csConf.setCapacity(Q3_QUEUE_PATH, 50); + csConf.setCapacity(Q1_PATH, 100); + csConf.setCapacity(Q2_PATH, 50); + csConf.setCapacity(Q3_PATH, 50); conf = new YarnConfiguration(csConf); cs = new CapacityScheduler(); 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 ceab609f9e1fa..31bb9700762fb 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 @@ -176,26 +176,27 @@ private void setupQueueConfiguration(CapacitySchedulerConfiguration conf, final String newRoot, boolean addUserLimits) { // Define top-level queues - QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - QueuePath newRootPath = root.createNewLeaf(newRoot); - QueuePath aQueuePath = newRootPath.createNewLeaf(A); - - conf.setQueues(root, new String[] {newRoot}); - conf.setMaximumCapacity(root, 100); - conf.setAcl(root, QueueACL.SUBMIT_APPLICATIONS, " "); - - conf.setQueues(newRootPath, new String[] {A}); - conf.setCapacity(newRootPath, 100); - conf.setMaximumCapacity(newRootPath, 100); - conf.setAcl(newRootPath, QueueACL.SUBMIT_APPLICATIONS, " "); - - conf.setCapacity(aQueuePath, 100f); - conf.setMaximumCapacity(aQueuePath, 100); - conf.setAcl(aQueuePath, QueueACL.SUBMIT_APPLICATIONS, "*"); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] { newRoot }); + conf.setMaximumCapacity(CapacitySchedulerConfiguration.ROOT, 100); + conf.setAcl(CapacitySchedulerConfiguration.ROOT, + QueueACL.SUBMIT_APPLICATIONS, " "); + + final String Q_newRoot = CapacitySchedulerConfiguration.ROOT + "." + + newRoot; + conf.setQueues(Q_newRoot, new String[] { A }); + conf.setCapacity(Q_newRoot, 100); + conf.setMaximumCapacity(Q_newRoot, 100); + conf.setAcl(Q_newRoot, QueueACL.SUBMIT_APPLICATIONS, " "); + + final String Q_A = Q_newRoot + "." + A; + conf.setCapacity(Q_A, 100f); + conf.setMaximumCapacity(Q_A, 100); + conf.setAcl(Q_A, QueueACL.SUBMIT_APPLICATIONS, "*"); if (addUserLimits) { - conf.setUserLimit(aQueuePath, 25); - conf.setUserLimitFactor(aQueuePath, 0.25f); + conf.setUserLimit(Q_A, 25); + conf.setUserLimitFactor(Q_A, 0.25f); } } 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/TestUniformQueueResourceCalculation.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/TestUniformQueueResourceCalculation.java index 6125bd30d4941..863baaaaf95fb 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/TestUniformQueueResourceCalculation.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/TestUniformQueueResourceCalculation.java @@ -146,12 +146,12 @@ public void testPercentageResourceCalculation() throws IOException { @Test public void testAbsoluteResourceCalculation() throws IOException { - csConf.setMinimumResourceRequirement("", A, QUEUE_A_RES); - csConf.setMinimumResourceRequirement("", B, QUEUE_B_RES); - csConf.setMinimumResourceRequirement("", A1, QUEUE_A1_RES); - csConf.setMinimumResourceRequirement("", A2, QUEUE_A2_RES); - csConf.setMinimumResourceRequirement("", A11, QUEUE_A11_RES); - csConf.setMinimumResourceRequirement("", A12, QUEUE_A12_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A), QUEUE_A_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(B), QUEUE_B_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A1), QUEUE_A1_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A2), QUEUE_A2_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A11), QUEUE_A11_RES); + csConf.setMinimumResourceRequirement("", new QueuePath(A12), QUEUE_A12_RES); QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() .withQueue(A) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java index 9f928482a75b4..28ca66847de00 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java @@ -64,21 +64,6 @@ public class TestUtils { private static final Logger LOG = LoggerFactory.getLogger(TestUtils.class); - private static final String A_PATH = CapacitySchedulerConfiguration.ROOT + ".a"; - private static final String B_PATH = CapacitySchedulerConfiguration.ROOT + ".b"; - private static final String C_PATH = CapacitySchedulerConfiguration.ROOT + ".c"; - private static final String A1_PATH = A_PATH + ".a1"; - private static final String B1_PATH = B_PATH + ".b1"; - private static final String B2_PATH = B_PATH + ".b2"; - private static final String C1_PATH = C_PATH + ".c1"; - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A = new QueuePath(A_PATH); - private static final QueuePath B = new QueuePath(B_PATH); - private static final QueuePath C = new QueuePath(C_PATH); - private static final QueuePath A1 = new QueuePath(A1_PATH); - private static final QueuePath B1 = new QueuePath(B1_PATH); - private static final QueuePath B2 = new QueuePath(B2_PATH); - private static final QueuePath C1 = new QueuePath(C1_PATH); /** * Get a mock {@link RMContext} for use in test cases. @@ -294,35 +279,41 @@ public static Configuration getConfigurationWithQueueLabels(Configuration config new CapacitySchedulerConfiguration(config); // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b", "c"}); - conf.setCapacityByLabel(ROOT, "x", 100); - conf.setCapacityByLabel(ROOT, "y", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 15); conf.setAccessibleNodeLabels(A, toSet("x")); conf.setCapacityByLabel(A, "x", 100); - + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 20); conf.setAccessibleNodeLabels(B, toSet("y")); conf.setCapacityByLabel(B, "y", 100); - + + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; conf.setCapacity(C, 70); conf.setMaximumCapacity(C, 70); conf.setAccessibleNodeLabels(C, RMNodeLabelsManager.EMPTY_STRING_SET); // Define 2nd-level queues + final String A1 = A + ".a1"; conf.setQueues(A, new String[] {"a1"}); conf.setCapacity(A1, 100); conf.setMaximumCapacity(A1, 100); conf.setCapacityByLabel(A1, "x", 100); - + + final String B1 = B + ".b1"; conf.setQueues(B, new String[] {"b1"}); conf.setCapacity(B1, 100); conf.setMaximumCapacity(B1, 100); conf.setCapacityByLabel(B1, "y", 100); conf.setMaximumApplicationMasterResourcePerQueuePercent(B1, 1f); + final String C1 = C + ".c1"; conf.setQueues(C, new String[] {"c1"}); conf.setCapacity(C1, 100); conf.setMaximumCapacity(C1, 100); @@ -336,17 +327,19 @@ public static Configuration getComplexConfigurationWithQueueLabels( new CapacitySchedulerConfiguration(config); // Define top-level queues - conf.setQueues(ROOT, new String[] {"a", "b"}); - conf.setCapacityByLabel(ROOT, "x", 100); - conf.setCapacityByLabel(ROOT, "y", 100); - conf.setCapacityByLabel(ROOT, "z", 100); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 10); conf.setAccessibleNodeLabels(A, toSet("x", "y")); conf.setCapacityByLabel(A, "x", 100); conf.setCapacityByLabel(A, "y", 50); - + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 90); conf.setMaximumCapacity(B, 100); conf.setAccessibleNodeLabels(B, toSet("y", "z")); @@ -354,6 +347,7 @@ public static Configuration getComplexConfigurationWithQueueLabels( conf.setCapacityByLabel(B, "z", 100); // Define 2nd-level queues + final String A1 = A + ".a1"; conf.setQueues(A, new String[] {"a1"}); conf.setCapacity(A1, 100); conf.setMaximumCapacity(A1, 100); @@ -363,10 +357,12 @@ public static Configuration getComplexConfigurationWithQueueLabels( conf.setCapacityByLabel(A1, "y", 100); conf.setQueues(B, new String[] {"b1", "b2"}); + final String B1 = B + ".b1"; conf.setCapacity(B1, 50); conf.setMaximumCapacity(B1, 50); conf.setAccessibleNodeLabels(B1, RMNodeLabelsManager.EMPTY_STRING_SET); + final String B2 = B + ".b2"; conf.setCapacity(B2, 50); conf.setMaximumCapacity(B2, 50); conf.setAccessibleNodeLabels(B2, toSet("y", "z")); @@ -378,6 +374,9 @@ public static Configuration getComplexConfigurationWithQueueLabels( public static Configuration getConfigurationWithDefaultQueueLabels( Configuration config) { + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + CapacitySchedulerConfiguration conf = (CapacitySchedulerConfiguration) getConfigurationWithQueueLabels(config); new CapacitySchedulerConfiguration(config); @@ -407,17 +406,20 @@ public static FiCaSchedulerApp getFiCaSchedulerApp(MockRM rm, new CapacitySchedulerConfiguration(config); // Define top-level queues - conf.setQueues(ROOT, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a", "b", "c" }); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; conf.setCapacity(A, 10); conf.setMaximumCapacity(A, 100); conf.setUserLimitFactor(A, 100); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; conf.setCapacity(B, 20); conf.setMaximumCapacity(B, 100); conf.setUserLimitFactor(B, 100); + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; conf.setCapacity(C, 70); conf.setMaximumCapacity(C, 100); conf.setUserLimitFactor(C, 100); 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/conf/TestConfigurationUpdateAssembler.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/conf/TestConfigurationUpdateAssembler.java index fa767a5182652..890996ac23e57 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/conf/TestConfigurationUpdateAssembler.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/conf/TestConfigurationUpdateAssembler.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo; import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo; import org.junit.Before; @@ -164,7 +163,7 @@ public void testRemoveNonExistingQueue() { private CapacitySchedulerConfiguration crateInitialCSConfig() { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - csConf.setQueues(new QueuePath(CapacitySchedulerConfiguration.ROOT), new String[] {"a, b"}); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a, b"}); csConf.set(A_CONFIG_PATH, A_INIT_CONFIG_VALUE); csConf.set(B_CONFIG_PATH, B_INIT_CONFIG_VALUE); 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/conf/TestMutableCSConfigurationProvider.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/conf/TestMutableCSConfigurationProvider.java index 89b7dc4b2ff84..aec2c21956471 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/conf/TestMutableCSConfigurationProvider.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/conf/TestMutableCSConfigurationProvider.java @@ -29,7 +29,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStore.LogMutation; import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo; import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo; @@ -222,13 +221,12 @@ public void testHDFSBackedProvider() throws Exception { public void testAddRemoveQueueWithSpacesInConfig() throws Exception { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - QueuePath a = root.createNewLeaf("a"); - QueuePath b = root.createNewLeaf("b"); - QueuePath c = root.createNewLeaf("c"); - - csConf.setQueues(root, new String[] {" a , b, c" }); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {" a , b, c" }); + final String a = CapacitySchedulerConfiguration.ROOT + ".a"; + final String b = CapacitySchedulerConfiguration.ROOT + ".b"; + final String c = CapacitySchedulerConfiguration.ROOT + ".c"; csConf.setCapacity(a, 0); csConf.setCapacity(b, 50); csConf.setCapacity(c, 50); 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/conf/TestQueueCapacityConfigParser.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/conf/TestQueueCapacityConfigParser.java index 2859f3fdc139c..47701484f6a8f 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/conf/TestQueueCapacityConfigParser.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/conf/TestQueueCapacityConfigParser.java @@ -24,7 +24,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; import org.junit.Test; @@ -36,7 +35,6 @@ import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestQueueMetricsForCustomResources.GB; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getQueuePrefix; public class TestQueueCapacityConfigParser { @@ -52,8 +50,6 @@ public class TestQueueCapacityConfigParser { private static final float WEIGHT_VALUE = 6; private static final String QUEUE = "root.test"; - private static final QueuePath QUEUE_PATH = new QueuePath("root.test"); - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); private static final String ABSOLUTE_RESOURCE = String.format( ALL_RESOURCE_TEMPLATE, MEMORY_ABSOLUTE, VCORE_ABSOLUTE, GPU_ABSOLUTE); @@ -74,7 +70,7 @@ public class TestQueueCapacityConfigParser { @Test public void testPercentageCapacityConfig() { QueueCapacityVector percentageCapacityVector = - capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), QUEUE_PATH); + capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), QUEUE); QueueCapacityVectorEntry memory = percentageCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcore = percentageCapacityVector.getResource(VCORES_URI); @@ -85,7 +81,8 @@ public void testPercentageCapacityConfig() { Assert.assertEquals(PERCENTAGE_VALUE, vcore.getResourceValue(), EPSILON); QueueCapacityVector rootCapacityVector = - capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), ROOT); + capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), + CapacitySchedulerConfiguration.ROOT); QueueCapacityVectorEntry memoryRoot = rootCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcoreRoot = rootCapacityVector.getResource(VCORES_URI); @@ -100,7 +97,7 @@ public void testPercentageCapacityConfig() { @Test public void testWeightCapacityConfig() { QueueCapacityVector weightCapacityVector = capacityConfigParser.parse(WEIGHT_VALUE + "w", - QUEUE_PATH); + QUEUE); QueueCapacityVectorEntry memory = weightCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcore = weightCapacityVector.getResource(VCORES_URI); @@ -115,13 +112,13 @@ public void testWeightCapacityConfig() { @Test public void testAbsoluteCapacityVectorConfig() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.set(getQueuePrefix(QUEUE_PATH) + CapacitySchedulerConfiguration.CAPACITY, - ABSOLUTE_RESOURCE); + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY, ABSOLUTE_RESOURCE); conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); ResourceUtils.resetResourceTypes(conf); QueueCapacityVector absoluteCapacityVector = capacityConfigParser.parse(ABSOLUTE_RESOURCE, - QUEUE_PATH); + QUEUE); Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(MEMORY_URI).getVectorResourceType()); @@ -139,7 +136,7 @@ public void testAbsoluteCapacityVectorConfig() { .getResourceValue(), EPSILON); QueueCapacityVector withoutGpuVector = capacityConfigParser - .parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE_PATH); + .parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); Assert.assertEquals(3, withoutGpuVector.getResourceCount()); Assert.assertEquals(0f, withoutGpuVector.getResource(GPU_URI).getResourceValue(), EPSILON); @@ -152,7 +149,7 @@ public void testMixedCapacityConfig() { ResourceUtils.resetResourceTypes(conf); QueueCapacityVector mixedCapacityVector = - capacityConfigParser.parse(MIXED_RESOURCE, QUEUE_PATH); + capacityConfigParser.parse(MIXED_RESOURCE, QUEUE); Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, mixedCapacityVector.getResource(MEMORY_URI).getVectorResourceType()); @@ -171,7 +168,7 @@ public void testMixedCapacityConfig() { // Test undefined capacity type default value QueueCapacityVector mixedCapacityVectorWithGpuUndefined = - capacityConfigParser.parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE_PATH); + capacityConfigParser.parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, mixedCapacityVectorWithGpuUndefined.getResource(MEMORY_URI).getVectorResourceType()); Assert.assertEquals(0, mixedCapacityVectorWithGpuUndefined.getResource(GPU_URI) @@ -182,37 +179,37 @@ public void testMixedCapacityConfig() { @Test public void testInvalidCapacityConfigs() { QueueCapacityVector capacityVectorWithInvalidSuffix = - capacityConfigParser.parse(NONEXISTINGSUFFIX, QUEUE_PATH); + capacityConfigParser.parse(NONEXISTINGSUFFIX, QUEUE); List entriesWithInvalidSuffix = Lists.newArrayList(capacityVectorWithInvalidSuffix.iterator()); Assert.assertEquals(0, entriesWithInvalidSuffix.size()); QueueCapacityVector invalidDelimiterCapacityVector = - capacityConfigParser.parse(INVALID_CAPACITY_FORMAT, QUEUE_PATH); + capacityConfigParser.parse(INVALID_CAPACITY_FORMAT, QUEUE); List invalidDelimiterEntries = Lists.newArrayList(invalidDelimiterCapacityVector.iterator()); Assert.assertEquals(0, invalidDelimiterEntries.size()); QueueCapacityVector invalidCapacityVector = - capacityConfigParser.parse(INVALID_CAPACITY_BRACKET, QUEUE_PATH); + capacityConfigParser.parse(INVALID_CAPACITY_BRACKET, QUEUE); List resources = Lists.newArrayList(invalidCapacityVector.iterator()); Assert.assertEquals(0, resources.size()); QueueCapacityVector emptyBracketCapacityVector = - capacityConfigParser.parse(EMPTY_BRACKET, QUEUE_PATH); + capacityConfigParser.parse(EMPTY_BRACKET, QUEUE); List emptyEntries = Lists.newArrayList(emptyBracketCapacityVector.iterator()); Assert.assertEquals(0, emptyEntries.size()); QueueCapacityVector emptyCapacity = - capacityConfigParser.parse("", QUEUE_PATH); + capacityConfigParser.parse("", QUEUE); List emptyResources = Lists.newArrayList(emptyCapacity.iterator()); Assert.assertEquals(emptyResources.size(), 0); QueueCapacityVector nonSetCapacity = - capacityConfigParser.parse(null, QUEUE_PATH); + capacityConfigParser.parse(null, QUEUE); List nonSetResources = Lists.newArrayList(nonSetCapacity.iterator()); Assert.assertEquals(nonSetResources.size(), 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/constraint/TestPlacementProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementProcessor.java index 837d78df8da00..5d525c84639ca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementProcessor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementProcessor.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; @@ -417,14 +416,10 @@ public void testSchedulerRejection() throws Exception { stopRM(); CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); - - QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - QueuePath queueA = new QueuePath(CapacitySchedulerConfiguration.ROOT + ".a"); - QueuePath queueB = new QueuePath(CapacitySchedulerConfiguration.ROOT + ".b"); - - csConf.setQueues(root, new String[] {"a", "b"}); - csConf.setCapacity(queueA, 15.0f); - csConf.setCapacity(queueB, 85.0f); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a", "b"}); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a", 15.0f); + csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".b", 85.0f); YarnConfiguration conf = new YarnConfiguration(csConf); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); 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/fair/TestAllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java index 26ddb3aabe195..9fb76cf261ea3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java @@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileQueuePlacementPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileQueuePlacementRule; @@ -890,10 +889,8 @@ public void testReservableQueue() throws Exception { AllocationConfiguration allocConf = confHolder.allocConf; String reservableQueueName = "root.reservable"; String nonreservableQueueName = "root.other"; - QueuePath nonreservableQueuePath = new QueuePath(nonreservableQueueName); - QueuePath reservableQueuePath = new QueuePath(reservableQueueName); - assertFalse(allocConf.isReservable(nonreservableQueuePath)); - assertTrue(allocConf.isReservable(reservableQueuePath)); + assertFalse(allocConf.isReservable(nonreservableQueueName)); + assertTrue(allocConf.isReservable(reservableQueueName)); Map> configuredQueues = allocConf.getConfiguredQueues(); assertTrue("reservable queue is expected be to a parent queue", @@ -902,23 +899,23 @@ public void testReservableQueue() throws Exception { configuredQueues.get(FSQueueType.LEAF) .contains(reservableQueueName)); - assertTrue(allocConf.getMoveOnExpiry(reservableQueuePath)); + assertTrue(allocConf.getMoveOnExpiry(reservableQueueName)); assertEquals(ReservationSchedulerConfiguration.DEFAULT_RESERVATION_WINDOW, - allocConf.getReservationWindow(reservableQueuePath)); + allocConf.getReservationWindow(reservableQueueName)); assertEquals(100, - allocConf.getInstantaneousMaxCapacity(reservableQueuePath), 0.0001); + allocConf.getInstantaneousMaxCapacity(reservableQueueName), 0.0001); assertEquals("DummyAgentName", - allocConf.getReservationAgent(reservableQueuePath)); - assertEquals(100, allocConf.getAverageCapacity(reservableQueuePath), 0.001); - assertFalse(allocConf.getShowReservationAsQueues(reservableQueuePath)); + allocConf.getReservationAgent(reservableQueueName)); + assertEquals(100, allocConf.getAverageCapacity(reservableQueueName), 0.001); + assertFalse(allocConf.getShowReservationAsQueues(reservableQueueName)); assertEquals("AnyAdmissionPolicy", - allocConf.getReservationAdmissionPolicy(reservableQueuePath)); + allocConf.getReservationAdmissionPolicy(reservableQueueName)); assertEquals(ReservationSchedulerConfiguration .DEFAULT_RESERVATION_PLANNER_NAME, - allocConf.getReplanner(reservableQueuePath)); + allocConf.getReplanner(reservableQueueName)); assertEquals(ReservationSchedulerConfiguration .DEFAULT_RESERVATION_ENFORCEMENT_WINDOW, - allocConf.getEnforcementWindow(reservableQueuePath)); + allocConf.getEnforcementWindow(reservableQueueName)); } /** 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/fair/converter/TestFSConfigToCSConfigConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java index 8447261e6189e..55c43666cdb5b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java @@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.MappingRulesDescription; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; @@ -124,14 +123,6 @@ private static String prepareFileName(String f) { new File("src/test/resources/conversion-rules.properties") .getAbsolutePath(); - private static final QueuePath ROOT = new QueuePath("root"); - private static final QueuePath DEFAULT = new QueuePath("root.default"); - private static final QueuePath USERS = new QueuePath("root.users"); - private static final QueuePath USERS_JOE = new QueuePath("root.users.joe"); - private static final QueuePath USERS_JOHN = new QueuePath("root.users.john"); - private static final QueuePath ADMINS_ALICE = new QueuePath("root.admins.alice"); - private static final QueuePath ADMINS_BOB = new QueuePath("root.admins.bob"); - private ConversionOptions createDefaultConversionOptions() { return new ConversionOptions(new DryRunResultHolder(), false); } @@ -186,13 +177,13 @@ public void testDefaultMaxAMShare() throws Exception { assertEquals("Default max AM share", 0.16f, maxAmShare, 0.0f); assertEquals("root.admins.alice max-am-resource-percent", 0.15f, - conf.getMaximumApplicationMasterResourcePerQueuePercent(ADMINS_ALICE), + conf.getMaximumApplicationMasterResourcePerQueuePercent("root.admins.alice"), 0.0f); //root.users.joe don’t have maximum-am-resource-percent set // so falling back to the global value assertEquals("root.users.joe maximum-am-resource-percent", 0.16f, - conf.getMaximumApplicationMasterResourcePerQueuePercent(USERS_JOE), + conf.getMaximumApplicationMasterResourcePerQueuePercent("root.users.joe"), 0.0f); } @@ -203,20 +194,20 @@ public void testDefaultUserLimitFactor() throws Exception { CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig(); assertEquals("root.users user-limit-factor", 1.0f, - conf.getUserLimitFactor(USERS), 0.0f); + conf.getUserLimitFactor("root.users"), 0.0f); assertEquals("root.users auto-queue-creation-v2.enabled", true, - conf.isAutoQueueCreationV2Enabled(USERS)); + conf.isAutoQueueCreationV2Enabled("root.users")); assertEquals("root.default user-limit-factor", -1.0f, - conf.getUserLimitFactor(DEFAULT), 0.0f); + conf.getUserLimitFactor("root.default"), 0.0f); assertEquals("root.users.joe user-limit-factor", -1.0f, - conf.getUserLimitFactor(USERS_JOE), 0.0f); + conf.getUserLimitFactor("root.users.joe"), 0.0f); assertEquals("root.admins.bob user-limit-factor", -1.0f, - conf.getUserLimitFactor(ADMINS_BOB), 0.0f); + conf.getUserLimitFactor("root.admins.bob"), 0.0f); assertEquals("root.admin.bob auto-queue-creation-v2.enabled", false, - conf.isAutoQueueCreationV2Enabled(ADMINS_BOB)); + conf.isAutoQueueCreationV2Enabled("root.admin.bob")); } @Test @@ -236,15 +227,15 @@ public void testDefaultMaxAMShareDisabled() throws Exception { // root.admins.bob is unset,so falling back to the global value assertEquals("root.admins.bob maximum-am-resource-percent", 1.0f, - conf.getMaximumApplicationMasterResourcePerQueuePercent(ADMINS_BOB), 0.0f); + conf.getMaximumApplicationMasterResourcePerQueuePercent("root.admins.bob"), 0.0f); // root.admins.alice 0.15 != -1.0 assertEquals("root.admins.alice max-am-resource-percent", 0.15f, - conf.getMaximumApplicationMasterResourcePerQueuePercent(ADMINS_ALICE), 0.0f); + conf.getMaximumApplicationMasterResourcePerQueuePercent("root.admins.alice"), 0.0f); // root.users.joe is unset,so falling back to the global value assertEquals("root.users.joe maximum-am-resource-percent", 1.0f, - conf.getMaximumApplicationMasterResourcePerQueuePercent(USERS_JOE), 0.0f); + conf.getMaximumApplicationMasterResourcePerQueuePercent("root.users.joe"), 0.0f); } @Test @@ -255,33 +246,33 @@ public void testConvertACLs() throws Exception { // root assertEquals("root submit ACL", "alice,bob,joe,john hadoop_users", - conf.getAcl(ROOT, QueueACL.SUBMIT_APPLICATIONS).getAclString()); + conf.getAcl("root", QueueACL.SUBMIT_APPLICATIONS).getAclString()); assertEquals("root admin ACL", "alice,bob,joe,john hadoop_users", - conf.getAcl(ROOT, QueueACL.ADMINISTER_QUEUE).getAclString()); + conf.getAcl("root", QueueACL.ADMINISTER_QUEUE).getAclString()); // root.admins.bob assertEquals("root.admins.bob submit ACL", "bob ", - conf.getAcl(ADMINS_BOB, QueueACL.SUBMIT_APPLICATIONS).getAclString()); + conf.getAcl("root.admins.bob", QueueACL.SUBMIT_APPLICATIONS).getAclString()); assertEquals("root.admins.bob admin ACL", "bob ", - conf.getAcl(ADMINS_BOB, QueueACL.ADMINISTER_QUEUE).getAclString()); + conf.getAcl("root.admins.bob", QueueACL.ADMINISTER_QUEUE).getAclString()); // root.admins.alice assertEquals("root.admins.alice submit ACL", "alice ", - conf.getAcl(ADMINS_ALICE, QueueACL.SUBMIT_APPLICATIONS).getAclString()); + conf.getAcl("root.admins.alice", QueueACL.SUBMIT_APPLICATIONS).getAclString()); assertEquals("root.admins.alice admin ACL", "alice ", - conf.getAcl(ADMINS_ALICE, QueueACL.ADMINISTER_QUEUE).getAclString()); + conf.getAcl("root.admins.alice", QueueACL.ADMINISTER_QUEUE).getAclString()); // root.users.john assertEquals("root.users.john submit ACL", "*", - conf.getAcl(USERS_JOHN, QueueACL.SUBMIT_APPLICATIONS).getAclString()); + conf.getAcl("root.users.john", QueueACL.SUBMIT_APPLICATIONS).getAclString()); assertEquals("root.users.john admin ACL", "*", - conf.getAcl(USERS_JOHN, QueueACL.ADMINISTER_QUEUE).getAclString()); + conf.getAcl("root.users.john", QueueACL.ADMINISTER_QUEUE).getAclString()); // root.users.joe assertEquals("root.users.joe submit ACL", "joe ", - conf.getAcl(USERS_JOE, QueueACL.SUBMIT_APPLICATIONS).getAclString()); + conf.getAcl("root.users.joe", QueueACL.SUBMIT_APPLICATIONS).getAclString()); assertEquals("root.users.joe admin ACL", "joe ", - conf.getAcl(USERS_JOE, QueueACL.ADMINISTER_QUEUE).getAclString()); + conf.getAcl("root.users.joe", QueueACL.ADMINISTER_QUEUE).getAclString()); } @Test @@ -301,7 +292,7 @@ public void testSpecificQueueMaxParallelApps() throws Exception { CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig(); assertEquals("root.admins.alice max parallel apps", 2, - conf.getMaxParallelAppsForQueue(ADMINS_ALICE), 0); + conf.getMaxParallelAppsForQueue("root.admins.alice"), 0); } @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/fair/converter/TestFSQueueConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSQueueConverter.java index 47f8e1edfec3f..edab194a91817 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSQueueConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSQueueConverter.java @@ -18,6 +18,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DEFAULT_MAX_PARALLEL_APPLICATIONS; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -27,7 +28,6 @@ import java.io.File; import java.io.IOException; -import java.util.Arrays; import java.util.Set; import org.apache.hadoop.conf.Configuration; @@ -93,18 +93,6 @@ private static String prepareFileName(String f) { private FSQueueConverterBuilder builder; private String key; - private static final QueuePath ROOT = new QueuePath("root"); - private static final QueuePath DEFAULT = new QueuePath("root.default"); - private static final QueuePath USERS = new QueuePath("root.users"); - private static final QueuePath USERS_JOE = new QueuePath("root.users.joe"); - private static final QueuePath USERS_JOHN = new QueuePath("root.users.john"); - private static final QueuePath ADMINS = new QueuePath("root.admins"); - private static final QueuePath ADMINS_ALICE = new QueuePath("root.admins.alice"); - private static final QueuePath ADMINS_BOB = new QueuePath("root.admins.bob"); - private static final QueuePath MISC = new QueuePath("root.misc"); - private static final QueuePath MISC_A = new QueuePath("root.misc.a"); - private static final QueuePath MISC_B = new QueuePath("root.misc.b"); - @Mock private FSConfigToCSConfigRuleHandler ruleHandler; @@ -167,20 +155,20 @@ public void testConvertQueueHierarchy() { converter.convertQueueHierarchy(rootQueue); // root children - assertEquals("root children", Arrays.asList("admins", "users", "misc", "default"), - csConfig.getQueues(ROOT)); + assertArrayEquals("root children", new String[]{"admins", "users", "misc", "default"}, + csConfig.getQueues("root")); // root.admins children - assertEquals("root.admins children", Arrays.asList("bob", "alice"), - csConfig.getQueues(ADMINS)); + assertArrayEquals("root.admins children", new String[]{"bob", "alice"}, + csConfig.getQueues("root.admins")); // root.default children - none - assertTrue("root.default children", - csConfig.getQueues(DEFAULT).isEmpty()); + assertNull("root.default children", + csConfig.getQueues("root.default")); // root.users children - assertEquals("root.users children", Arrays.asList("john", "joe"), - csConfig.getQueues(USERS)); + assertArrayEquals("root.users children", new String[]{"john", "joe"}, + csConfig.getQueues("root.users")); Set leafs = Sets.difference(ALL_QUEUES, Sets.newHashSet("root", @@ -191,8 +179,8 @@ public void testConvertQueueHierarchy() { for (String queue : leafs) { key = PREFIX + queue + ".queues"; - assertTrue("Key " + key + " has value, but it should be empty", - csConfig.getQueues(new QueuePath(queue)).isEmpty()); + assertNull("Key " + key + " has value, but it should be null", + csConfig.getQueues(queue)); } } @@ -206,12 +194,12 @@ public void testQueueMaxAMShare() { // root.admins.bob assertEquals("root.admins.bob AM share", 1.0f, csConfig.getMaximumApplicationMasterResourcePerQueuePercent( - ADMINS_BOB), 0.0f); + "root.admins.bob"), 0.0f); // root.admins.alice assertEquals("root.admins.alice AM share", 0.15f, csConfig.getMaximumApplicationMasterResourcePerQueuePercent( - ADMINS_ALICE), 0.0f); + "root.admins.alice"), 0.0f); Set remaining = Sets.difference(ALL_QUEUES, Sets.newHashSet("root.admins.bob", "root.admins.alice")); @@ -220,7 +208,7 @@ public void testQueueMaxAMShare() { key = PREFIX + queue + ".maximum-am-resource-percent"; assertEquals("Key " + key + " has different value", 0.1f, csConfig - .getMaximumApplicationMasterResourcePerQueuePercent(new QueuePath(queue)), 0.0f); + .getMaximumApplicationMasterResourcePerQueuePercent(queue), 0.0f); } } @@ -231,7 +219,7 @@ public void testQueueMaxParallelApps() { converter.convertQueueHierarchy(rootQueue); assertEquals("root.admins.alice max apps", 2, - csConfig.getMaxParallelAppsForQueue(ADMINS_ALICE), 0); + csConfig.getMaxParallelAppsForQueue("root.admins.alice"), 0); Set remaining = Sets.difference(ALL_QUEUES, Sets.newHashSet("root.admins.alice")); @@ -240,7 +228,7 @@ public void testQueueMaxParallelApps() { key = PREFIX + queue + ".max-parallel-apps"; assertEquals("Key " + key + " has different value", DEFAULT_MAX_PARALLEL_APPLICATIONS, csConfig - .getMaxParallelAppsForQueue(new QueuePath(queue)), 0); + .getMaxParallelAppsForQueue(queue), 0); } } @@ -252,15 +240,15 @@ public void testQueueMaxAllocations() { // root.admins vcores + mb assertEquals("root.admins max vcores", 3, - csConfig.getQueueMaximumAllocationVcores(ADMINS)); + csConfig.getQueueMaximumAllocationVcores("root.admins")); assertEquals("root.admins max memory", 4096, - csConfig.getQueueMaximumAllocationMb(ADMINS)); + csConfig.getQueueMaximumAllocationMb("root.admins")); // root.users.john max vcores + mb assertEquals("root.users.john max vcores", 2, - csConfig.getQueueMaximumAllocationVcores(USERS_JOHN)); + csConfig.getQueueMaximumAllocationVcores("root.users.john")); assertEquals("root.users.john max memory", 8192, - csConfig.getQueueMaximumAllocationMb(USERS_JOHN)); + csConfig.getQueueMaximumAllocationMb("root.users.john")); Set remaining = Sets.difference(ALL_QUEUES, Sets.newHashSet("root.admins", "root.users.john")); @@ -269,12 +257,12 @@ public void testQueueMaxAllocations() { key = PREFIX + queue + ".maximum-allocation-vcores"; assertEquals("Key " + key + " has different value", -1.0, csConfig - .getQueueMaximumAllocationVcores(new QueuePath(queue)), 0.0f); + .getQueueMaximumAllocationVcores(queue), 0.0f); key = PREFIX + queue + ".maximum-allocation-mb"; assertEquals("Key " + key + " has different value", -1.0, csConfig - .getQueueMaximumAllocationMb(new QueuePath(queue)), 0.0f); + .getQueueMaximumAllocationMb(queue), 0.0f); } } @@ -286,10 +274,10 @@ public void testQueuePreemptionDisabled() { assertTrue("root.admins.alice preemption setting", csConfig.getPreemptionDisabled( - ADMINS_ALICE, false)); + "root.admins.alice", false)); assertTrue("root.users.joe preemption setting", csConfig.getPreemptionDisabled( - USERS_JOE, false)); + "root.users.joe", false)); Set remaining = Sets.difference(ALL_QUEUES, Sets.newHashSet("root.admins.alice", "root.users.joe")); @@ -297,7 +285,7 @@ public void testQueuePreemptionDisabled() { for (String queue : remaining) { key = PREFIX + queue + ".disable_preemption"; assertEquals("Key " + key + " has different value", - false, csConfig.getPreemptionDisabled(new QueuePath(queue), false)); + false, csConfig.getPreemptionDisabled(queue, false)); } } @@ -310,7 +298,7 @@ public void testQueuePreemptionDisabledWhenGlobalPreemptionDisabled() { for (String queue : ALL_QUEUES) { key = PREFIX + queue + ".disable_preemption"; assertEquals("Key " + key + " has different value", - false, csConfig.getPreemptionDisabled(new QueuePath(queue), false)); + false, csConfig.getPreemptionDisabled(queue, false)); } } @@ -322,31 +310,41 @@ public void testChildCapacityInCapacityMode() { // root assertEquals("root.default capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(DEFAULT), 0.0f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.default")), 0.0f); assertEquals("root.admins capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(ADMINS), 0.0f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.admins")), 0.0f); assertEquals("root.users capacity", 33.334f, - csConfig.getNonLabeledQueueCapacity(USERS), 0.0f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.users")), 0.0f); // root.users assertEquals("root.users.john capacity", 25.000f, - csConfig.getNonLabeledQueueCapacity(USERS_JOHN), 0.0f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.users.john")), 0.0f); assertEquals("root.users.joe capacity", 75.000f, - csConfig.getNonLabeledQueueCapacity(USERS_JOE), 0.0f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.users.joe")), 0.0f); // root.admins assertEquals("root.admins.alice capacity", 75.000f, - csConfig.getNonLabeledQueueCapacity(ADMINS_ALICE), 0.0f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.admins.alice")), 0.0f); assertEquals("root.admins.bob capacity", 25.000f, - csConfig.getNonLabeledQueueCapacity(ADMINS_BOB), 0.0f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.admins.bob")), 0.0f); // root.misc assertEquals("root.misc capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(MISC), 0.000f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.misc")), 0.000f); assertEquals("root.misc.a capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(MISC_A), 0.000f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.misc.a")), 0.000f); assertEquals("root.misc.b capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(MISC_B), 0.000f); + csConfig.getNonLabeledQueueCapacity( + new QueuePath("root.misc.b")), 0.000f); } @Test @@ -357,31 +355,31 @@ public void testChildCapacityInWeightMode() { // root assertEquals("root.default weight", 1.0f, - csConfig.getNonLabeledQueueWeight(DEFAULT), 0.01f); + csConfig.getNonLabeledQueueWeight("root.default"), 0.01f); assertEquals("root.admins weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ADMINS), 0.01f); + csConfig.getNonLabeledQueueWeight("root.admins"), 0.01f); assertEquals("root.users weight", 1.0f, - csConfig.getNonLabeledQueueWeight(USERS), 0.01f); + csConfig.getNonLabeledQueueWeight("root.users"), 0.01f); // root.users assertEquals("root.users.john weight", 1.0f, - csConfig.getNonLabeledQueueWeight(USERS_JOHN), 0.01f); + csConfig.getNonLabeledQueueWeight("root.users.john"), 0.01f); assertEquals("root.users.joe weight", 3.0f, - csConfig.getNonLabeledQueueWeight(USERS_JOE), 0.01f); + csConfig.getNonLabeledQueueWeight("root.users.joe"), 0.01f); // root.admins assertEquals("root.admins.alice weight", 3.0f, - csConfig.getNonLabeledQueueWeight(ADMINS_ALICE), 0.01f); + csConfig.getNonLabeledQueueWeight("root.admins.alice"), 0.01f); assertEquals("root.admins.bob weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ADMINS_BOB), 0.01f); + csConfig.getNonLabeledQueueWeight("root.admins.bob"), 0.01f); // root.misc assertEquals("root.misc weight", 0.0f, - csConfig.getNonLabeledQueueWeight(MISC), 0.00f); + csConfig.getNonLabeledQueueWeight("root.misc"), 0.00f); assertEquals("root.misc.a weight", 0.0f, - csConfig.getNonLabeledQueueWeight(MISC_A), 0.00f); + csConfig.getNonLabeledQueueWeight("root.misc.a"), 0.00f); assertEquals("root.misc.b weight", 0.0f, - csConfig.getNonLabeledQueueWeight(MISC_B), 0.00f); + csConfig.getNonLabeledQueueWeight("root.misc.b"), 0.00f); } @Test @@ -391,15 +389,15 @@ public void testAutoCreateV2FlagsInWeightMode() { converter.convertQueueHierarchy(rootQueue); assertTrue("root autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(ROOT)); + csConfig.isAutoQueueCreationV2Enabled("root")); assertTrue("root.admins autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(ADMINS)); + csConfig.isAutoQueueCreationV2Enabled("root.admins")); assertTrue("root.admins.alice autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(ADMINS_ALICE)); + csConfig.isAutoQueueCreationV2Enabled("root.admins.alice")); assertTrue("root.users autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(USERS)); + csConfig.isAutoQueueCreationV2Enabled("root.users")); assertTrue("root.misc autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(MISC)); + csConfig.isAutoQueueCreationV2Enabled("root.misc")); //leaf queue root.admins.alice is removed from the below list //adding reservation to a leaf, it's queueType changes to FSParentQueue @@ -414,7 +412,7 @@ public void testAutoCreateV2FlagsInWeightMode() { key = PREFIX + queue + ".auto-queue-creation-v2.enabled"; assertEquals("Key " + key + " has different value", false, csConfig - .isAutoQueueCreationV2Enabled(new QueuePath(queue))); + .isAutoQueueCreationV2Enabled(queue)); } } @@ -432,11 +430,11 @@ public void testZeroSumCapacityValidation() { key = PREFIX + queue + ".allow-zero-capacity-sum"; assertEquals("Key " + key + " has different value", false, csConfig - .getAllowZeroCapacitySum(new QueuePath(queue))); + .getAllowZeroCapacitySum(queue)); } assertTrue("root.misc allow zero capacities", - csConfig.getAllowZeroCapacitySum(MISC)); + csConfig.getAllowZeroCapacitySum("root.misc")); } @Test @@ -474,7 +472,7 @@ public void testQueueWithNoAutoCreateChildQueue() { for (String queue : ALL_QUEUES) { key = PREFIX + queue + ".auto-create-child-queue.enabled"; assertEquals("Key " + key + " has different value", - false, csConfig.isAutoCreateChildQueueEnabled(new QueuePath(queue))); + false, csConfig.isAutoCreateChildQueueEnabled(queue)); } } @@ -519,25 +517,25 @@ public void testQueueOrderingPolicy() throws Exception { converter.convertQueueHierarchy(rootQueue); // root assertEquals("root ordering policy", "fifo", - csConfig.getAppOrderingPolicy(ROOT).getConfigName()); + csConfig.getAppOrderingPolicy("root").getConfigName()); assertEquals("root.default ordering policy", "fair", - csConfig.getAppOrderingPolicy(DEFAULT).getConfigName()); + csConfig.getAppOrderingPolicy("root.default").getConfigName()); assertEquals("root.admins ordering policy", "fifo", - csConfig.getAppOrderingPolicy(ADMINS).getConfigName()); + csConfig.getAppOrderingPolicy("root.admins").getConfigName()); assertEquals("root.users ordering policy", "fifo", - csConfig.getAppOrderingPolicy(USERS).getConfigName()); + csConfig.getAppOrderingPolicy("root.users").getConfigName()); // root.users assertEquals("root.users.joe ordering policy", "fair", - csConfig.getAppOrderingPolicy(USERS_JOE).getConfigName()); + csConfig.getAppOrderingPolicy("root.users.joe").getConfigName()); assertEquals("root.users.john ordering policy", "fifo", - csConfig.getAppOrderingPolicy(USERS_JOHN).getConfigName()); + csConfig.getAppOrderingPolicy("root.users.john").getConfigName()); // root.admins assertEquals("root.admins.alice ordering policy", "fifo", - csConfig.getAppOrderingPolicy(ADMINS_ALICE).getConfigName()); + csConfig.getAppOrderingPolicy("root.admins.alice.").getConfigName()); assertEquals("root.admins.bob ordering policy", "fair", - csConfig.getAppOrderingPolicy(ADMINS_BOB).getConfigName()); + csConfig.getAppOrderingPolicy("root.admins.bob").getConfigName()); } @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/fair/converter/TestQueuePlacementConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestQueuePlacementConverter.java index d74d469f5d46e..6599080aab59c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestQueuePlacementConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestQueuePlacementConverter.java @@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.SpecifiedPlacementRule; import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.MappingRulesDescription; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.Rule; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.Rule.FallbackResult; @@ -467,7 +466,7 @@ public void testConvertNestedDefaultWithConflictingQueues() { when(fsRule.getParentRule()).thenReturn(parent); when(fsRule.getCreateFlag()).thenReturn(true); initPlacementManagerMock(fsRule); - csConf.setQueues(new QueuePath("root.users"), new String[] {"hadoop"}); + csConf.setQueues("root.users", new String[] {"hadoop"}); convert(); 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/fair/converter/weightconversion/TestWeightToPercentageConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToPercentageConverter.java index 33b18734a0a20..54fadf4571f43 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToPercentageConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToPercentageConverter.java @@ -38,10 +38,6 @@ public class TestWeightToPercentageConverter extends WeightConverterTestBase { private WeightToPercentConverter converter; private CapacitySchedulerConfiguration csConfig; - public static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - public static final QueuePath ROOT_A = new QueuePath("root", "a"); - public static final QueuePath ROOT_B = new QueuePath("root", "b"); - public static final QueuePath ROOT_C = new QueuePath("root", "c"); @Before public void setup() { @@ -56,7 +52,7 @@ public void testSingleWeightConversion() { converter.convertWeightsForChildQueues(root, csConfig); assertFalse("Capacity zerosum allowed", - csConfig.getAllowZeroCapacitySum(ROOT)); + csConfig.getAllowZeroCapacitySum("root")); assertEquals("root.a capacity", 100.000f, csConfig.getNonLabeledQueueCapacity(new QueuePath("root.a")), 0.0f); } @@ -80,11 +76,11 @@ public void testMultiWeightConversion() { csConfig.getPropsWithPrefix(PREFIX).size()); // this is no fixing - it's the result of BigDecimal rounding assertEquals("root.a capacity", 16.667f, - csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.a")), 0.0f); assertEquals("root.b capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.b")), 0.0f); assertEquals("root.c capacity", 50.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.c")), 0.0f); } @Test @@ -94,15 +90,15 @@ public void testMultiWeightConversionWhenOfThemIsZero() { converter.convertWeightsForChildQueues(root, csConfig); assertFalse("Capacity zerosum allowed", - csConfig.getAllowZeroCapacitySum(ROOT)); + csConfig.getAllowZeroCapacitySum("root")); assertEquals("Number of properties", 22, csConfig.getPropsWithPrefix(PREFIX).size()); assertEquals("root.a capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.a")), 0.0f); assertEquals("root.b capacity", 50.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.b")), 0.0f); assertEquals("root.c capacity", 50.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.c")), 0.0f); } @Test @@ -114,13 +110,13 @@ public void testMultiWeightConversionWhenAllOfThemAreZero() { assertEquals("Number of properties", 23, csConfig.getPropsWithPrefix(PREFIX).size()); assertTrue("Capacity zerosum allowed", - csConfig.getAllowZeroCapacitySum(ROOT)); + csConfig.getAllowZeroCapacitySum("root")); assertEquals("root.a capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.a")), 0.0f); assertEquals("root.b capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.b")), 0.0f); assertEquals("root.c capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.c")), 0.0f); } @Test @@ -132,11 +128,11 @@ public void testCapacityFixingWithThreeQueues() { assertEquals("Number of properties", 22, csConfig.getPropsWithPrefix(PREFIX).size()); assertEquals("root.a capacity", 33.334f, - csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.a")), 0.0f); assertEquals("root.b capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.b")), 0.0f); assertEquals("root.c capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f); + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.c")), 0.0f); } @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/fair/converter/weightconversion/TestWeightToWeightConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToWeightConverter.java index 38b69c66a2926..904dfe8b1f3d8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToWeightConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToWeightConverter.java @@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; import org.junit.Before; import org.junit.Test; @@ -35,11 +34,6 @@ public class TestWeightToWeightConverter extends WeightConverterTestBase { private WeightToWeightConverter converter; private CapacitySchedulerConfiguration csConfig; - public static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - public static final QueuePath ROOT_A = new QueuePath("root", "a"); - public static final QueuePath ROOT_B = new QueuePath("root", "b"); - public static final QueuePath ROOT_C = new QueuePath("root", "c"); - @Before public void setup() { converter = new WeightToWeightConverter(); @@ -53,7 +47,7 @@ public void testNoChildQueueConversion() { converter.convertWeightsForChildQueues(root, csConfig); assertEquals("root weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT), 0.0f); + csConfig.getNonLabeledQueueWeight("root"), 0.0f); assertEquals("Converted items", 21, csConfig.getPropsWithPrefix(PREFIX).size()); } @@ -64,9 +58,9 @@ public void testSingleWeightConversion() { converter.convertWeightsForChildQueues(root, csConfig); assertEquals("root weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT), 0.0f); + csConfig.getNonLabeledQueueWeight("root"), 0.0f); assertEquals("root.a weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT_A), 0.0f); + csConfig.getNonLabeledQueueWeight("root.a"), 0.0f); assertEquals("Number of properties", 22, csConfig.getPropsWithPrefix(PREFIX).size()); } @@ -80,13 +74,13 @@ public void testMultiWeightConversion() { assertEquals("Number of properties", 24, csConfig.getPropsWithPrefix(PREFIX).size()); assertEquals("root weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT), 0.0f); + csConfig.getNonLabeledQueueWeight("root"), 0.0f); assertEquals("root.a weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT_A), 0.0f); + csConfig.getNonLabeledQueueWeight("root.a"), 0.0f); assertEquals("root.b weight", 2.0f, - csConfig.getNonLabeledQueueWeight(ROOT_B), 0.0f); + csConfig.getNonLabeledQueueWeight("root.b"), 0.0f); assertEquals("root.c weight", 3.0f, - csConfig.getNonLabeledQueueWeight(ROOT_C), 0.0f); + csConfig.getNonLabeledQueueWeight("root.c"), 0.0f); } @Test @@ -95,7 +89,7 @@ public void testAutoCreateV2FlagOnParent() { converter.convertWeightsForChildQueues(root, csConfig); assertTrue("root autocreate v2 enabled", - csConfig.isAutoQueueCreationV2Enabled(ROOT)); + csConfig.isAutoQueueCreationV2Enabled("root")); } @Test @@ -106,6 +100,6 @@ public void testAutoCreateV2FlagOnParentWithoutChildren() { assertEquals("Number of properties", 21, csConfig.getPropsWithPrefix(PREFIX).size()); assertTrue("root autocreate v2 enabled", - csConfig.isAutoQueueCreationV2Enabled(ROOT)); + csConfig.isAutoQueueCreationV2Enabled("root")); } } 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/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java index 682a01d985fa4..b58dd0b6a653b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java @@ -37,7 +37,6 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.server.api.records.NodeStatus; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -861,9 +860,8 @@ public void testNonDefaultMinimumAllocation() throws Exception { @Test(timeout = 50000) public void testReconnectedNode() throws Exception { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - QueuePath defaultQueuePath = new QueuePath("default"); - conf.setQueues(defaultQueuePath, new String[] {"default"}); - conf.setCapacity(defaultQueuePath, 100); + conf.setQueues("default", new String[] { "default" }); + conf.setCapacity("default", 100); FifoScheduler fs = new FifoScheduler(); fs.init(conf); fs.start(); 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/policy/TestFairOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFairOrderingPolicy.java index 9cfbb7da197d2..ac5caedc9db0c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFairOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFairOrderingPolicy.java @@ -31,7 +31,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.junit.Assert; import org.junit.Test; @@ -164,8 +163,7 @@ public void testSizeBasedWeightNotAffectAppActivation() throws Exception { new CapacitySchedulerConfiguration(); // Define top-level queues - String defaultPath = CapacitySchedulerConfiguration.ROOT + ".default"; - QueuePath queuePath = new QueuePath(defaultPath); + String queuePath = CapacitySchedulerConfiguration.ROOT + ".default"; csConf.set(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class.getCanonicalName()); csConf.setOrderingPolicy(queuePath, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java index 93d4e695605fc..8120bb21b1eba 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java @@ -59,7 +59,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; @@ -330,51 +329,50 @@ static void setupQueueConfiguration(CapacitySchedulerConfiguration conf) { static void setupQueueConfiguration(CapacitySchedulerConfiguration conf, boolean useDRC) { // Define top-level queues - QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - conf.setQueues(root, new String[] {"a", "b", "c"}); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"}); - final QueuePath a = root.createNewLeaf("a"); - conf.setCapacity(a, 10); + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + conf.setCapacity(A, 10); - final QueuePath b = root.createNewLeaf("b"); - conf.setCapacity(b, 20); + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + conf.setCapacity(B, 20); - final QueuePath c = root.createNewLeaf("c"); - conf.setCapacity(c, 70); + final String C = CapacitySchedulerConfiguration.ROOT + ".c"; + conf.setCapacity(C, 70); // Define 2nd-level queues - final QueuePath a1 = a.createNewLeaf("a1"); - final QueuePath a2 = a.createNewLeaf("a2"); - conf.setQueues(a, new String[] {"a1", "a2"}); - conf.setCapacity(a1, 30); - conf.setCapacity(a2, 70); - - final QueuePath b1 = b.createNewLeaf("b1"); - final QueuePath b2 = b.createNewLeaf("b2"); - final QueuePath b3 = b.createNewLeaf("b3"); - conf.setQueues(b, new String[] {"b1", "b2", "b3"}); - conf.setCapacity(b1, 50); - conf.setCapacity(b2, 30); - conf.setCapacity(b3, 20); - - final QueuePath c1 = c.createNewLeaf("c1"); - final QueuePath c2 = c.createNewLeaf("c2"); - final QueuePath c3 = c.createNewLeaf("c3"); - final QueuePath c4 = c.createNewLeaf("c4"); - conf.setQueues(c, new String[] {"c1", "c2", "c3", "c4"}); - conf.setCapacity(c1, 50); - conf.setCapacity(c2, 10); - conf.setCapacity(c3, 35); - conf.setCapacity(c4, 5); + final String A1 = A + ".a1"; + final String A2 = A + ".a2"; + conf.setQueues(A, new String[] {"a1", "a2"}); + conf.setCapacity(A1, 30); + conf.setCapacity(A2, 70); + + final String B1 = B + ".b1"; + final String B2 = B + ".b2"; + final String B3 = B + ".b3"; + conf.setQueues(B, new String[] {"b1", "b2", "b3"}); + conf.setCapacity(B1, 50); + conf.setCapacity(B2, 30); + conf.setCapacity(B3, 20); + + final String C1 = C + ".c1"; + final String C2 = C + ".c2"; + final String C3 = C + ".c3"; + final String C4 = C + ".c4"; + conf.setQueues(C, new String[] {"c1", "c2", "c3", "c4"}); + conf.setCapacity(C1, 50); + conf.setCapacity(C2, 10); + conf.setCapacity(C3, 35); + conf.setCapacity(C4, 5); // Define 3rd-level queues - final QueuePath c11 = c1.createNewLeaf("c11"); - final QueuePath c12 = c1.createNewLeaf("c12"); - final QueuePath c13 = c1.createNewLeaf("c13"); - conf.setQueues(c1, new String[] {"c11", "c12", "c13"}); - conf.setCapacity(c11, 15); - conf.setCapacity(c12, 45); - conf.setCapacity(c13, 40); + final String C11 = C1 + ".c11"; + final String C12 = C1 + ".c12"; + final String C13 = C1 + ".c13"; + conf.setQueues(C1, new String[] {"c11", "c12", "c13"}); + conf.setCapacity(C11, 15); + conf.setCapacity(C12, 45); + conf.setCapacity(C13, 40); if (useDRC) { conf.set("yarn.scheduler.capacity.resource-calculator", "org.apache.hadoop.yarn.util.resource.DominantResourceCalculator"); @@ -407,8 +405,8 @@ public static FifoScheduler mockFifoScheduler(RMContext rmContext) static void setupFifoQueueConfiguration(CapacitySchedulerConfiguration conf) { // Define default queue - conf.setQueues(new QueuePath("default"), new String[] {"default"}); - conf.setCapacity(new QueuePath("default"), 100); + conf.setQueues("default", new String[] {"default"}); + conf.setCapacity("default", 100); } public static void main(String[] args) throws Exception { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServiceAppsNodelabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServiceAppsNodelabel.java index 422a9bb31a4cd..7397926732e95 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServiceAppsNodelabel.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServiceAppsNodelabel.java @@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; import org.apache.hadoop.yarn.webapp.GuiceServletConfig; import org.apache.hadoop.yarn.webapp.JerseyTestBase; @@ -117,19 +116,19 @@ private static void setupQueueConfiguration( CapacitySchedulerConfiguration config) { // Define top-level queues - QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - QueuePath queueA = root.createNewLeaf("a"); - QueuePath defaultQueue = root.createNewLeaf("default"); - - config.setQueues(root, + config.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "default"}); + final String queueA = CapacitySchedulerConfiguration.ROOT + ".a"; config.setCapacity(queueA, 50f); config.setMaximumCapacity(queueA, 50); + final String defaultQueue = + CapacitySchedulerConfiguration.ROOT + ".default"; config.setCapacity(defaultQueue, 50f); - config.setCapacityByLabel(root, "X", 100); - config.setMaximumCapacityByLabel(root, "X", 100); + config.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "X", 100); + config.setMaximumCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "X", + 100); // set for default queue config.setCapacityByLabel(defaultQueue, "X", 100); config.setMaximumCapacityByLabel(defaultQueue, "X", 100); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java index a51bd2afec675..650e83983b7d9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java @@ -81,7 +81,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; @@ -148,10 +147,7 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase { "test.build.data", "/tmp")).getAbsolutePath(); private static final String FS_ALLOC_FILE = new File(TEST_DIR, "test-fs-queues.xml").getAbsolutePath(); - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath DEFAULT = ROOT.createNewLeaf("default"); - private static final QueuePath TEST = ROOT.createNewLeaf("test"); - private static final QueuePath TEST_QUEUE = ROOT.createNewLeaf("testqueue"); + /* * Helper class to allow testing of RM web services which require * authorization Add this class as a filter in the Guice injector for the @@ -571,8 +567,8 @@ public void testSingleAppKillUnauthorized() throws Exception { // default root queue allows anyone to have admin acl CapacitySchedulerConfiguration csconf = new CapacitySchedulerConfiguration(); - csconf.setAcl(ROOT, QueueACL.ADMINISTER_QUEUE, "someuser"); - csconf.setAcl(DEFAULT, QueueACL.ADMINISTER_QUEUE, "someuser"); + csconf.setAcl("root", QueueACL.ADMINISTER_QUEUE, "someuser"); + csconf.setAcl("root.default", QueueACL.ADMINISTER_QUEUE, "someuser"); rm.getResourceScheduler().reinitialize(csconf, rm.getRMContext()); } @@ -794,9 +790,9 @@ public void testAppSubmit(String acceptMedia, String contentMedia) String[] queues = { "default", "testqueue" }; CapacitySchedulerConfiguration csconf = new CapacitySchedulerConfiguration(); - csconf.setQueues(ROOT, queues); - csconf.setCapacity(DEFAULT, 50.0f); - csconf.setCapacity(TEST_QUEUE, 50.0f); + csconf.setQueues("root", queues); + csconf.setCapacity("root.default", 50.0f); + csconf.setCapacity("root.testqueue", 50.0f); rm.getResourceScheduler().reinitialize(csconf, rm.getRMContext()); String appName = "test"; @@ -1102,12 +1098,12 @@ public void testUpdateAppPriority() throws Exception { CapacitySchedulerConfiguration csconf = new CapacitySchedulerConfiguration(); String[] queues = { "default", "test" }; - csconf.setQueues(ROOT, queues); - csconf.setCapacity(DEFAULT, 50.0f); - csconf.setCapacity(TEST, 50.0f); - csconf.setAcl(ROOT, QueueACL.ADMINISTER_QUEUE, "someuser"); - csconf.setAcl(DEFAULT, QueueACL.ADMINISTER_QUEUE, "someuser"); - csconf.setAcl(TEST, QueueACL.ADMINISTER_QUEUE, "someuser"); + csconf.setQueues("root", queues); + csconf.setCapacity("root.default", 50.0f); + csconf.setCapacity("root.test", 50.0f); + csconf.setAcl("root", QueueACL.ADMINISTER_QUEUE, "someuser"); + csconf.setAcl("root.default", QueueACL.ADMINISTER_QUEUE, "someuser"); + csconf.setAcl("root.test", QueueACL.ADMINISTER_QUEUE, "someuser"); rm.getResourceScheduler().reinitialize(csconf, rm.getRMContext()); rm.start(); @@ -1193,12 +1189,12 @@ public void testAppMove() throws Exception { CapacitySchedulerConfiguration csconf = new CapacitySchedulerConfiguration(); String[] queues = { "default", "test" }; - csconf.setQueues(ROOT, queues); - csconf.setCapacity(DEFAULT, 50.0f); - csconf.setCapacity(TEST, 50.0f); - csconf.setAcl(ROOT, QueueACL.ADMINISTER_QUEUE, "someuser"); - csconf.setAcl(DEFAULT, QueueACL.ADMINISTER_QUEUE, "someuser"); - csconf.setAcl(TEST, QueueACL.ADMINISTER_QUEUE, "someuser"); + csconf.setQueues("root", queues); + csconf.setCapacity("root.default", 50.0f); + csconf.setCapacity("root.test", 50.0f); + csconf.setAcl("root", QueueACL.ADMINISTER_QUEUE, "someuser"); + csconf.setAcl("root.default", QueueACL.ADMINISTER_QUEUE, "someuser"); + csconf.setAcl("root.test", QueueACL.ADMINISTER_QUEUE, "someuser"); rm.getResourceScheduler().reinitialize(csconf, rm.getRMContext()); rm.start(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java index 71465723a0109..859066c475ab0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java @@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmissionData; import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.webapp.JerseyTestBase; import org.junit.runner.RunWith; @@ -54,8 +53,6 @@ @RunWith(Parameterized.class) public class TestRMWebServicesCapacitySched extends JerseyTestBase { - private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT); - private static final QueuePath A = new QueuePath(CapacitySchedulerConfiguration.ROOT + ".a"); private final boolean legacyQueueMode; @Parameterized.Parameters(name = "{index}: legacy-queue-mode={0}") @@ -127,8 +124,8 @@ public void testPerUserResources() throws Exception { @Test public void testNodeLabelDefaultAPI() throws Exception { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(createConfig()); - conf.setDefaultNodeLabelExpression(ROOT, "ROOT-INHERITED"); - conf.setDefaultNodeLabelExpression(A, "root-a-default-label"); + conf.setDefaultNodeLabelExpression("root", "ROOT-INHERITED"); + conf.setDefaultNodeLabelExpression("root.a", "root-a-default-label"); try (MockRM rm = createRM(conf)) { rm.registerNode("h1:1234", 32 * GB, 32); ClientResponse response = resource().path("ws/v1/cluster/scheduler") diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java index 32dd1a0f43db0..40919a24162bc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java @@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; @@ -90,10 +89,6 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase { public static final QueuePath ROOT_A = new QueuePath("root", "a"); public static final QueuePath ROOT_A_A1 = QueuePath.createFromQueues("root", "a", "a1"); public static final QueuePath ROOT_A_A2 = QueuePath.createFromQueues("root", "a", "a2"); - public static final QueuePath ROOT_B = new QueuePath("root", "b"); - public static final QueuePath ROOT_C = new QueuePath("root", "c"); - public static final QueuePath ROOT_C_C1 = QueuePath.createFromQueues("root", "c", "c1"); - public static final QueuePath ROOT_D = new QueuePath("root", "d"); private static MockRM rm; private static String userName; private static CapacitySchedulerConfiguration csConf; @@ -155,23 +150,31 @@ public void setUp() throws Exception { private static void setupQueueConfiguration( CapacitySchedulerConfiguration config) { - config.setQueues(ROOT, new String[]{"a", "b", "c", "mappedqueue"}); + config.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[]{"a", "b", "c", "mappedqueue"}); - config.setCapacity(ROOT_A, 25f); - config.setMaximumCapacity(ROOT_A, 50f); + final String a = CapacitySchedulerConfiguration.ROOT + ".a"; + config.setCapacity(a, 25f); + config.setMaximumCapacity(a, 50f); - config.setQueues(ROOT_A, new String[]{"a1", "a2"}); - config.setCapacity(ROOT_A_A1, 100f); - config.setCapacity(ROOT_A_A2, 0f); + final String a1 = a + ".a1"; + final String a2 = a + ".a2"; + config.setQueues(a, new String[]{"a1", "a2"}); + config.setCapacity(a1, 100f); + config.setCapacity(a2, 0f); - config.setCapacity(ROOT_B, 75f); + final String b = CapacitySchedulerConfiguration.ROOT + ".b"; + config.setCapacity(b, 75f); - config.setCapacity(ROOT_C, 0f); + final String c = CapacitySchedulerConfiguration.ROOT + ".c"; + config.setCapacity(c, 0f); - config.setQueues(ROOT_C, new String[] {"c1"}); - config.setCapacity(ROOT_C_C1, 0f); + final String c1 = c + ".c1"; + config.setQueues(c, new String[] {"c1"}); + config.setCapacity(c1, 0f); - config.setCapacity(ROOT_D, 0f); + final String d = CapacitySchedulerConfiguration.ROOT + ".d"; + config.setCapacity(d, 0f); config.set(CapacitySchedulerConfiguration.QUEUE_MAPPING, "g:hadoop:mappedqueue"); } @@ -209,14 +212,14 @@ private CapacitySchedulerConfiguration getSchedulerConf() public void testGetSchedulerConf() throws Exception { CapacitySchedulerConfiguration orgConf = getSchedulerConf(); assertNotNull(orgConf); - assertEquals(4, orgConf.getQueues(ROOT).size()); + assertEquals(4, orgConf.getQueues("root").length); } @Test public void testFormatSchedulerConf() throws Exception { CapacitySchedulerConfiguration newConf = getSchedulerConf(); assertNotNull(newConf); - assertEquals(4, newConf.getQueues(ROOT).size()); + assertEquals(4, newConf.getQueues("root").length); SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo(); Map nearEmptyCapacity = new HashMap<>(); @@ -242,7 +245,7 @@ public void testFormatSchedulerConf() throws Exception { .put(ClientResponse.class); newConf = getSchedulerConf(); assertNotNull(newConf); - assertEquals(5, newConf.getQueues(ROOT).size()); + assertEquals(5, newConf.getQueues("root").length); // Format the scheduler config and validate root.formattest is not present response = r.path("ws").path("v1").path("cluster") @@ -251,7 +254,7 @@ public void testFormatSchedulerConf() throws Exception { .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); assertEquals(Status.OK.getStatusCode(), response.getStatus()); newConf = getSchedulerConf(); - assertEquals(4, newConf.getQueues(ROOT).size()); + assertEquals(4, newConf.getQueues("root").length); } private long getConfigVersion() throws Exception { @@ -277,7 +280,7 @@ public void testSchedulerConfigVersion() throws Exception { public void testAddNestedQueue() throws Exception { CapacitySchedulerConfiguration orgConf = getSchedulerConf(); assertNotNull(orgConf); - assertEquals(4, orgConf.getQueues(ROOT).size()); + assertEquals(4, orgConf.getQueues("root").length); WebResource r = resource(); @@ -312,8 +315,8 @@ public void testAddNestedQueue() throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - assertEquals(5, newCSConf.getQueues(ROOT).size()); - assertEquals(2, newCSConf.getQueues(ROOT_D).size()); + assertEquals(5, newCSConf.getQueues("root").length); + assertEquals(2, newCSConf.getQueues("root.d").length); assertEquals(25.0f, newCSConf.getNonLabeledQueueCapacity(new QueuePath("root.d.d1")), 0.01f); assertEquals(75.0f, newCSConf.getNonLabeledQueueCapacity(new QueuePath("root.d.d2")), @@ -321,7 +324,7 @@ public void testAddNestedQueue() throws Exception { CapacitySchedulerConfiguration newConf = getSchedulerConf(); assertNotNull(newConf); - assertEquals(5, newConf.getQueues(ROOT).size()); + assertEquals(5, newConf.getQueues("root").length); } @Test @@ -351,7 +354,7 @@ public void testAddWithUpdate() throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - assertEquals(5, newCSConf.getQueues(ROOT).size()); + assertEquals(5, newCSConf.getQueues("root").length); assertEquals(25.0f, newCSConf.getNonLabeledQueueCapacity(new QueuePath("root.d")), 0.01f); assertEquals(50.0f, newCSConf.getNonLabeledQueueCapacity(new QueuePath("root.b")), 0.01f); } @@ -381,7 +384,7 @@ public void testUnsetParentQueueOrderingPolicy() throws Exception { + "root.b" + CapacitySchedulerConfiguration.DOT + ORDERING_POLICY; assertEquals("fair", newCSConf.get(bOrderingPolicy)); - stopQueue(ROOT_B); + stopQueue("root.b"); // Add root.b.b1 which makes root.b a Parent Queue SchedConfUpdateInfo updateInfo2 = new SchedConfUpdateInfo(); @@ -432,7 +435,7 @@ public void testUnsetLeafQueueOrderingPolicy() throws Exception { + "root.c" + CapacitySchedulerConfiguration.DOT + ORDERING_POLICY; assertEquals("priority-utilization", newCSConf.get(cOrderingPolicy)); - stopQueue(ROOT_C_C1); + stopQueue("root.c.c1"); // Remove root.c.c1 which makes root.c a Leaf Queue SchedConfUpdateInfo updateInfo2 = new SchedConfUpdateInfo(); @@ -461,7 +464,7 @@ public void testRemoveQueue() throws Exception { ClientResponse response; - stopQueue(ROOT_A_A2); + stopQueue("root.a.a2"); // Remove root.a.a2 SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo(); updateInfo.getRemoveQueueInfo().add("root.a.a2"); @@ -477,9 +480,9 @@ public void testRemoveQueue() throws Exception { CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); assertEquals("Failed to remove the queue", - 1, newCSConf.getQueues(ROOT_A).size()); + 1, newCSConf.getQueues("root.a").length); assertEquals("Failed to remove the right queue", - "a1", newCSConf.getQueues(ROOT_A).get(0)); + "a1", newCSConf.getQueues("root.a")[0]); } @Test @@ -508,8 +511,8 @@ public void testStopWithRemoveQueue() throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - assertEquals(1, newCSConf.getQueues(ROOT_A).size()); - assertEquals("a1", newCSConf.getQueues(ROOT_A).get(0)); + assertEquals(1, newCSConf.getQueues("root.a").length); + assertEquals("a1", newCSConf.getQueues("root.a")[0]); } @Test @@ -547,7 +550,7 @@ public void testRemoveQueueWhichHasQueueMapping() throws Exception { // Validate queue 'mappedqueue' exists after above failure CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - assertEquals(4, newCSConf.getQueues(ROOT).size()); + assertEquals(4, newCSConf.getQueues("root").length); assertNotNull("CapacityScheduler Configuration is corrupt", cs.getQueue("mappedqueue")); } @@ -581,8 +584,8 @@ public void testStopWithConvertLeafToParentQueue() throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - assertEquals(1, newCSConf.getQueues(ROOT_B).size()); - assertEquals("b1", newCSConf.getQueues(ROOT_B).get(0)); + assertEquals(1, newCSConf.getQueues("root.b").length); + assertEquals("b1", newCSConf.getQueues("root.b")[0]); } @Test @@ -591,7 +594,7 @@ public void testRemoveParentQueue() throws Exception { ClientResponse response; - stopQueue(ROOT_C, ROOT_C_C1); + stopQueue("root.c", "root.c.c1"); // Remove root.c (parent queue) SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo(); updateInfo.getRemoveQueueInfo().add("root.c"); @@ -606,8 +609,8 @@ public void testRemoveParentQueue() throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - assertEquals(3, newCSConf.getQueues(ROOT).size()); - assertEquals(0, newCSConf.getQueues(ROOT_C).size()); + assertEquals(3, newCSConf.getQueues("root").length); + assertNull(newCSConf.getQueues("root.c")); } @Test @@ -616,7 +619,7 @@ public void testRemoveParentQueueWithCapacity() throws Exception { ClientResponse response; - stopQueue(ROOT_A, ROOT_A_A1, ROOT_A_A2); + stopQueue("root.a", "root.a.a1", "root.a.a2"); // Remove root.a (parent queue) with capacity 25 SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo(); updateInfo.getRemoveQueueInfo().add("root.a"); @@ -637,7 +640,7 @@ public void testRemoveParentQueueWithCapacity() throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - assertEquals(3, newCSConf.getQueues(ROOT).size()); + assertEquals(3, newCSConf.getQueues("root").length); assertEquals(100.0f, newCSConf.getNonLabeledQueueCapacity(new QueuePath("root.b")), 0.01f); } @@ -648,7 +651,7 @@ public void testRemoveMultipleQueues() throws Exception { ClientResponse response; - stopQueue(ROOT_B, ROOT_C, ROOT_C_C1); + stopQueue("root.b", "root.c", "root.c.c1"); // Remove root.b and root.c SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo(); updateInfo.getRemoveQueueInfo().add("root.b"); @@ -669,10 +672,10 @@ public void testRemoveMultipleQueues() throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - assertEquals(2, newCSConf.getQueues(ROOT).size()); + assertEquals(2, newCSConf.getQueues("root").length); } - private void stopQueue(QueuePath... queuePaths) throws Exception { + private void stopQueue(String... queuePaths) throws Exception { WebResource r = resource(); ClientResponse response; @@ -682,8 +685,8 @@ private void stopQueue(QueuePath... queuePaths) throws Exception { Map stoppedParam = new HashMap<>(); stoppedParam.put(CapacitySchedulerConfiguration.STATE, QueueState.STOPPED.toString()); - for (QueuePath queue : queuePaths) { - QueueConfigInfo stoppedInfo = new QueueConfigInfo(queue.getFullPath(), stoppedParam); + for (String queue : queuePaths) { + QueueConfigInfo stoppedInfo = new QueueConfigInfo(queue, stoppedParam); updateInfo.getUpdateQueueInfo().add(stoppedInfo); } response = @@ -696,7 +699,7 @@ private void stopQueue(QueuePath... queuePaths) throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration(); - for (QueuePath queue : queuePaths) { + for (String queue : queuePaths) { assertEquals(QueueState.STOPPED, newCSConf.getState(queue)); } } @@ -719,7 +722,7 @@ public void testUpdateQueue() throws Exception { assertEquals(CapacitySchedulerConfiguration .DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT, cs.getConfiguration() - .getMaximumApplicationMasterResourcePerQueuePercent(ROOT_A), + .getMaximumApplicationMasterResourcePerQueuePercent("root.a"), 0.001f); response = r.path("ws").path("v1").path("cluster") @@ -732,7 +735,7 @@ public void testUpdateQueue() throws Exception { assertEquals(Status.OK.getStatusCode(), response.getStatus()); CapacitySchedulerConfiguration newCSConf = cs.getConfiguration(); assertEquals(0.2f, newCSConf - .getMaximumApplicationMasterResourcePerQueuePercent(ROOT_A), 0.001f); + .getMaximumApplicationMasterResourcePerQueuePercent("root.a"), 0.001f); // Remove config. Config value should be reverted to default. updateParam.put(CapacitySchedulerConfiguration.MAXIMUM_AM_RESOURCE_SUFFIX, @@ -751,7 +754,7 @@ public void testUpdateQueue() throws Exception { newCSConf = cs.getConfiguration(); assertEquals(CapacitySchedulerConfiguration .DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT, newCSConf - .getMaximumApplicationMasterResourcePerQueuePercent(ROOT_A), + .getMaximumApplicationMasterResourcePerQueuePercent("root.a"), 0.001f); } @@ -887,9 +890,9 @@ public void testNodeLabelRemovalResidualConfigsAreCleared() throws Exception { CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); assertEquals(Sets.newHashSet("*"), - cs.getConfiguration().getAccessibleNodeLabels(ROOT)); + cs.getConfiguration().getAccessibleNodeLabels(ROOT.getFullPath())); assertEquals(Sets.newHashSet(LABEL_1), - cs.getConfiguration().getAccessibleNodeLabels(ROOT_A)); + cs.getConfiguration().getAccessibleNodeLabels(ROOT_A.getFullPath())); // 4. Set partition capacities to queues as below updateInfo = new SchedConfUpdateInfo(); @@ -986,8 +989,8 @@ public void testNodeLabelRemovalResidualConfigsAreCleared() throws Exception { .put(ClientResponse.class); assertEquals(Status.OK.getStatusCode(), response.getStatus()); assertEquals(Sets.newHashSet("*"), - cs.getConfiguration().getAccessibleNodeLabels(ROOT)); - assertNull(cs.getConfiguration().getAccessibleNodeLabels(ROOT_A)); + cs.getConfiguration().getAccessibleNodeLabels(ROOT.getFullPath())); + assertNull(cs.getConfiguration().getAccessibleNodeLabels(ROOT_A.getFullPath())); //6. Remove node label 'label1' MultivaluedMapImpl params = new MultivaluedMapImpl(); @@ -1022,8 +1025,8 @@ public void testNodeLabelRemovalResidualConfigsAreCleared() throws Exception { private String getConfValueForQueueAndLabelAndType(CapacityScheduler cs, QueuePath queuePath, String label, String type) { return cs.getConfiguration().get( - QueuePrefixes.getNodeLabelPrefix( - queuePath, label) + type); + CapacitySchedulerConfiguration.getNodeLabelPrefix( + queuePath.getFullPath(), label) + type); } private Object logAndReturnJson(WebResource ws, String json) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java index 7ecc4140e78a6..771f5c76681a3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java @@ -61,7 +61,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.webapp.JerseyTestBase; import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; @@ -98,8 +97,6 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase { private static final String QUEUE_A = "Qa"; private static final String LABEL_LY = "Ly"; private static final String LABEL_LX = "Lx"; - private static final QueuePath ROOT_QUEUE_PATH = - new QueuePath(CapacitySchedulerConfiguration.ROOT); private static final ImmutableSet CLUSTER_LABELS = ImmutableSet.of(LABEL_LX, LABEL_LY, DEFAULT_PARTITION); private static final String DOT = "."; @@ -124,63 +121,60 @@ private void setupQueueConfiguration( config.setLegacyQueueModeEnabled(legacyQueueMode); // Define top-level queues - config.setQueues(ROOT_QUEUE_PATH, + config.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { QUEUE_A, QUEUE_B, QUEUE_C }); String interMediateQueueC = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_C; - QueuePath interMediateQueueCPath = new QueuePath(interMediateQueueC); - config.setQueues(interMediateQueueCPath, + config.setQueues(interMediateQueueC, new String[] { LEAF_QUEUE_C1, LEAF_QUEUE_C2 }); - config.setCapacityByLabel(ROOT_QUEUE_PATH, LABEL_LX, 100); - config.setCapacityByLabel(ROOT_QUEUE_PATH, LABEL_LY, 100); + config.setCapacityByLabel( + CapacitySchedulerConfiguration.ROOT, LABEL_LX, 100); + config.setCapacityByLabel( + CapacitySchedulerConfiguration.ROOT, LABEL_LY, 100); String leafQueueA = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_A; - QueuePath leafQueueAPath = new QueuePath(leafQueueA); - config.setCapacity(leafQueueAPath, 30); - config.setMaximumCapacity(leafQueueAPath, 50); + config.setCapacity(leafQueueA, 30); + config.setMaximumCapacity(leafQueueA, 50); String leafQueueB = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_B; - QueuePath leafQueueBPath = new QueuePath(leafQueueB); - config.setCapacity(leafQueueBPath, 30); - config.setMaximumCapacity(leafQueueBPath, 50); + config.setCapacity(leafQueueB, 30); + config.setMaximumCapacity(leafQueueB, 50); - config.setCapacity(interMediateQueueCPath, 40); - config.setMaximumCapacity(interMediateQueueCPath, 50); + config.setCapacity(interMediateQueueC, 40); + config.setMaximumCapacity(interMediateQueueC, 50); String leafQueueC1 = interMediateQueueC + "." + LEAF_QUEUE_C1; - QueuePath leafQueueC1Path = new QueuePath(leafQueueC1); - config.setCapacity(leafQueueC1Path, 50); - config.setMaximumCapacity(leafQueueC1Path, 60); + config.setCapacity(leafQueueC1, 50); + config.setMaximumCapacity(leafQueueC1, 60); String leafQueueC2 = interMediateQueueC + "." + LEAF_QUEUE_C2; - QueuePath leafQueueC2Path = new QueuePath(leafQueueC2); - config.setCapacity(leafQueueC2Path, 50); - config.setMaximumCapacity(leafQueueC2Path, 70); + config.setCapacity(leafQueueC2, 50); + config.setMaximumCapacity(leafQueueC2, 70); // Define label specific configuration config.setAccessibleNodeLabels( - leafQueueAPath, ImmutableSet.of(DEFAULT_PARTITION)); - config.setAccessibleNodeLabels(leafQueueBPath, ImmutableSet.of(LABEL_LX)); - config.setAccessibleNodeLabels(interMediateQueueCPath, + leafQueueA, ImmutableSet.of(DEFAULT_PARTITION)); + config.setAccessibleNodeLabels(leafQueueB, ImmutableSet.of(LABEL_LX)); + config.setAccessibleNodeLabels(interMediateQueueC, ImmutableSet.of(LABEL_LX, LABEL_LY)); - config.setAccessibleNodeLabels(leafQueueC1Path, + config.setAccessibleNodeLabels(leafQueueC1, ImmutableSet.of(LABEL_LX, LABEL_LY)); - config.setAccessibleNodeLabels(leafQueueC2Path, + config.setAccessibleNodeLabels(leafQueueC2, ImmutableSet.of(LABEL_LX, LABEL_LY)); - config.setDefaultNodeLabelExpression(leafQueueBPath, LABEL_LX); - config.setDefaultNodeLabelExpression(leafQueueC1Path, LABEL_LX); - config.setDefaultNodeLabelExpression(leafQueueC2Path, LABEL_LY); - - config.setCapacityByLabel(leafQueueBPath, LABEL_LX, 30); - config.setCapacityByLabel(interMediateQueueCPath, LABEL_LX, 70); - config.setCapacityByLabel(leafQueueC1Path, LABEL_LX, 40); - config.setCapacityByLabel(leafQueueC2Path, LABEL_LX, 60); - - config.setCapacityByLabel(interMediateQueueCPath, LABEL_LY, 100); - config.setCapacityByLabel(leafQueueC1Path, LABEL_LY, 50); - config.setCapacityByLabel(leafQueueC2Path, LABEL_LY, 50); - config.setMaximumCapacityByLabel(leafQueueC1Path, LABEL_LY, 75); - config.setMaximumCapacityByLabel(leafQueueC2Path, LABEL_LY, 75); + config.setDefaultNodeLabelExpression(leafQueueB, LABEL_LX); + config.setDefaultNodeLabelExpression(leafQueueC1, LABEL_LX); + config.setDefaultNodeLabelExpression(leafQueueC2, LABEL_LY); + + config.setCapacityByLabel(leafQueueB, LABEL_LX, 30); + config.setCapacityByLabel(interMediateQueueC, LABEL_LX, 70); + config.setCapacityByLabel(leafQueueC1, LABEL_LX, 40); + config.setCapacityByLabel(leafQueueC2, LABEL_LX, 60); + + config.setCapacityByLabel(interMediateQueueC, LABEL_LY, 100); + config.setCapacityByLabel(leafQueueC1, LABEL_LY, 50); + config.setCapacityByLabel(leafQueueC2, LABEL_LY, 50); + config.setMaximumCapacityByLabel(leafQueueC1, LABEL_LY, 75); + config.setMaximumCapacityByLabel(leafQueueC2, LABEL_LY, 75); } @Before diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java index 13489dd9a0b8e..a0481283f0f05 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java @@ -49,7 +49,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; @@ -174,11 +173,10 @@ public void configureScheduler() { CapacitySchedulerConfiguration csconf = new CapacitySchedulerConfiguration(conf); String[] queues = { "default", "dedicated" }; - QueuePath dedicatedQueuePath = new QueuePath("root.dedicated"); - csconf.setQueues(new QueuePath("root"), queues); - csconf.setCapacity(new QueuePath("root.default"), 50.0f); - csconf.setCapacity(dedicatedQueuePath, 50.0f); - csconf.setReservable(dedicatedQueuePath, true); + csconf.setQueues("root", queues); + csconf.setCapacity("root.default", 50.0f); + csconf.setCapacity("root.dedicated", 50.0f); + csconf.setReservable("root.dedicated", true); conf = csconf; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivitiesWithMultiNodesEnabled.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivitiesWithMultiNodesEnabled.java index 017233cbf463f..e37a8d83c0560 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivitiesWithMultiNodesEnabled.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivitiesWithMultiNodesEnabled.java @@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.AllocationState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceUsageMultiNodeLookupPolicy; import org.apache.hadoop.yarn.util.resource.Resources; @@ -138,18 +137,16 @@ protected void configureServlets() { private static void setupQueueConfiguration( CapacitySchedulerConfiguration config) { // Define top-level queues - final QueuePath root = new QueuePath(CapacitySchedulerConfiguration.ROOT); - final QueuePath a = new QueuePath(CapacitySchedulerConfiguration.ROOT, "a"); - final QueuePath b = new QueuePath(CapacitySchedulerConfiguration.ROOT, "b"); - - config.setQueues(root, + config.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); - config.setCapacity(a, 10.5f); - config.setMaximumCapacity(a, 50); + final String queueA = CapacitySchedulerConfiguration.ROOT + ".a"; + config.setCapacity(queueA, 10.5f); + config.setMaximumCapacity(queueA, 50); - config.setCapacity(b, 89.5f); - config.setMaximumApplicationMasterResourcePerQueuePercent(b, 100); + final String queueB = CapacitySchedulerConfiguration.ROOT + ".b"; + config.setCapacity(queueB, 89.5f); + config.setMaximumApplicationMasterResourcePerQueuePercent(queueB, 100); } @Before diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java index 46508d91ed859..423e0e5a38cbe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java @@ -33,7 +33,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo; @@ -82,10 +81,6 @@ public abstract class BaseRouterWebServicesTest { public static final String QUEUE_DEDICATED = "dedicated"; public static final String QUEUE_DEDICATED_FULL = CapacitySchedulerConfiguration.ROOT + CapacitySchedulerConfiguration.DOT + QUEUE_DEDICATED; - public static final QueuePath ROOT_QUEUE_PATH = - new QueuePath(CapacitySchedulerConfiguration.ROOT); - public static final QueuePath DEFAULT_QUEUE_PATH = new QueuePath(QUEUE_DEFAULT_FULL); - public static final QueuePath DEDICATED_QUEUE_PATH = new QueuePath(QUEUE_DEDICATED_FULL); private RouterWebServices routerWebService; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java index ad9df4d4112ef..f162ab6be6593 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java @@ -166,10 +166,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.DEDICATED_QUEUE_PATH; -import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.DEFAULT_QUEUE_PATH; -import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.ROOT_QUEUE_PATH; import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEFAULT; +import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEFAULT_FULL; import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEDICATED; import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEDICATED_FULL; import static org.mockito.Mockito.mock; @@ -1107,12 +1105,12 @@ private MockRM setupResourceManager() throws Exception { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); // Define default queue - conf.setCapacity(DEFAULT_QUEUE_PATH, 20); + conf.setCapacity(QUEUE_DEFAULT_FULL, 20); // Define dedicated queues - conf.setQueues(ROOT_QUEUE_PATH, + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {QUEUE_DEFAULT, QUEUE_DEDICATED}); - conf.setCapacity(DEDICATED_QUEUE_PATH, 80); - conf.setReservable(DEDICATED_QUEUE_PATH, true); + conf.setCapacity(QUEUE_DEDICATED_FULL, 80); + conf.setReservable(QUEUE_DEDICATED_FULL, true); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); conf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestableFederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestableFederationInterceptorREST.java index 7a9dc1b85727d..0e37b7c9749ea 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestableFederationInterceptorREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestableFederationInterceptorREST.java @@ -33,9 +33,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.DEDICATED_QUEUE_PATH; -import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.DEFAULT_QUEUE_PATH; -import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.ROOT_QUEUE_PATH; +import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEDICATED_FULL; +import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEFAULT_FULL; import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEFAULT; import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEDICATED; @@ -83,12 +82,12 @@ protected void setupResourceManager() throws IOException { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); // Define default queue - conf.setCapacity(DEFAULT_QUEUE_PATH, 20); + conf.setCapacity(QUEUE_DEFAULT_FULL, 20); // Define dedicated queues String[] queues = new String[]{QUEUE_DEFAULT, QUEUE_DEDICATED}; - conf.setQueues(ROOT_QUEUE_PATH, queues); - conf.setCapacity(DEDICATED_QUEUE_PATH, 80); - conf.setReservable(DEDICATED_QUEUE_PATH, true); + conf.setQueues(CapacitySchedulerConfiguration.ROOT, queues); + conf.setCapacity(QUEUE_DEDICATED_FULL, 80); + conf.setReservable(QUEUE_DEDICATED_FULL, true); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class);