From 86b32bd587f3290e8fe5e2c5f5fd7deb1b185c61 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 31 Aug 2021 12:40:36 +0200 Subject: [PATCH 01/27] YARN-10930. Introduce universal capacity resource vector --- .../scheduler/capacity/AbstractCSQueue.java | 11 + .../scheduler/capacity/CSQueue.java | 8 + .../CapacitySchedulerConfiguration.java | 19 +- .../capacity/QueueCapacityVector.java | 141 ++++++++++++ .../scheduler/capacity/ResourceVector.java | 65 ++++++ .../conf/QueueCapacityConfigParser.java | 206 ++++++++++++++++++ .../conf/TestQueueCapacityConfigParser.java | 163 ++++++++++++++ 7 files changed, 611 insertions(+), 2 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacityVector.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ResourceVector.java create mode 100644 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 create mode 100644 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 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 7ddc765570747..ccdbec528bf7a 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 @@ -140,6 +140,8 @@ public enum CapacityConfigType { protected CapacityConfigType capacityConfigType = CapacityConfigType.NONE; + protected Map configuredCapacityVectors; + private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); protected CapacitySchedulerContext csContext; @@ -399,6 +401,9 @@ protected void setupQueueConfigs(Resource clusterResource, .getConfiguredNodeLabels(queuePath); } + configuredCapacityVectors = csContext.getConfiguration() + .parseConfiguredResourceVector(queuePath, configuredNodeLabels); + // After we setup labels, we can setup capacities setupConfigurableCapacities(configuration); updateAbsoluteCapacities(); @@ -704,6 +709,12 @@ public Resource getEffectiveMaxCapacityDown(String label, Resource factor) { minimumAllocation); } + @Override + public QueueCapacityVector getConfiguredCapacityVector( + String label) { + return configuredCapacityVectors.get(label); + } + private void initializeQueueState(QueueState previousState, QueueState configuredState, QueueState parentState) { // 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/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index 47bdeba45df57..d5704af0e206d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -414,6 +414,14 @@ public void validateSubmitApplication(ApplicationId applicationId, */ Resource getEffectiveCapacity(String label); + /** + * Get configured capacity resource vector parsed from the capacity config + * of the queue. + * @param label node label (partition) + * @return capacity resource vector + */ + QueueCapacityVector getConfiguredCapacityVector(String label); + /** * Get effective capacity of queue. If min/max resource is configured, * preference will be given to absolute configuration over normal capacity. 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 84302e57d76a2..d47f6ab75bb68 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 @@ -22,6 +22,7 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule.MappingRule; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.QueueCapacityConfigParser; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.MappingRuleCreator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,9 +74,9 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; -import java.util.Set; public class CapacitySchedulerConfiguration extends ReservationSchedulerConfiguration { @@ -406,6 +407,10 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur public static final String MAPPING_RULE_FORMAT_DEFAULT = MAPPING_RULE_FORMAT_LEGACY; + + private static final QueueCapacityConfigParser queueCapacityConfigParser + = new QueueCapacityConfigParser(); + private ConfigurationProperties configurationProperties; /** @@ -447,7 +452,7 @@ static String getUserPrefix(String user) { return PREFIX + "user." + user + DOT; } - private String getNodeLabelPrefix(String queue, String label) { + public static String getNodeLabelPrefix(String queue, String label) { if (label.equals(CommonNodeLabelsManager.NO_LABEL)) { return getQueuePrefix(queue); } @@ -2562,6 +2567,16 @@ public void setMaximumResourceRequirement(String label, String queue, updateMinMaxResourceToConf(label, queue, resource, MAXIMUM_CAPACITY); } + public Map parseConfiguredResourceVector( + String queuePath, Set labels) { + Map queueResourceVectors = new HashMap<>(); + for (String label : labels) { + queueResourceVectors.put(label, queueCapacityConfigParser.parse(this, queuePath, label)); + } + + return queueResourceVectors; + } + private void updateMinMaxResourceToConf(String label, String queue, Resource resource, String type) { if (queue.equals("root")) { 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/QueueCapacityVector.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/QueueCapacityVector.java new file mode 100644 index 0000000000000..bb9032f54ec2d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacityVector.java @@ -0,0 +1,141 @@ +/** + * 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.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.yarn.api.records.ResourceInformation; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +/** + * Contains capacity values with calculation types associated for each + * resource. + */ +public class QueueCapacityVector implements + Iterable { + private final ResourceVector resource; + private final Map resourceTypes + = new HashMap<>(); + private final Set + definedResourceTypes = new HashSet<>(); + + public QueueCapacityVector(ResourceVector resource) { + this.resource = resource; + } + + public static QueueCapacityVector empty() { + return new QueueCapacityVector(ResourceVector.empty()); + } + + public QueueResourceVectorEntry getResource(String resourceName) { + return new QueueResourceVectorEntry(resourceTypes.get(resourceName), + resourceName, resource.getValue(resourceName)); + } + + public void setResource(String resourceName, float value, + QueueVectorResourceType resourceType) { + resource.setValue(resourceName, value); + storeResourceType(resourceName, resourceType); + } + + public float getMemory() { + return resource.getValue(ResourceInformation.MEMORY_URI); + } + + public Set getDefinedResourceTypes() { + return definedResourceTypes; + } + + @Override + public Iterator iterator() { + return new Iterator() { + private final Iterator> resources = + resource.iterator(); + private int i = 0; + + @Override + public boolean hasNext() { + return resources.hasNext() && resourceTypes.size() > i; + } + + @Override + public QueueResourceVectorEntry next() { + Map.Entry resourceInformation = resources.next(); + i++; + return new QueueResourceVectorEntry( + resourceTypes.get(resourceInformation.getKey()), + resourceInformation.getKey(), resourceInformation.getValue()); + } + }; + } + + private void storeResourceType(String resourceName, QueueVectorResourceType resourceType) { + definedResourceTypes.add(resourceType); + resourceTypes.put(resourceName, resourceType); + } + + /** + * Represents a calculation type of a resource. + */ + public enum QueueVectorResourceType { + PERCENTAGE("%"), ABSOLUTE(""), WEIGHT("w"); + + private static final Set FLOAT_TYPES = + ImmutableSet.of(QueueVectorResourceType.PERCENTAGE, + QueueVectorResourceType.WEIGHT); + private final String postfix; + + QueueVectorResourceType(String postfix) { + this.postfix = postfix; + } + + public String getPostfix() { + return postfix; + } + } + + public static class QueueResourceVectorEntry { + private final QueueVectorResourceType vectorResourceType; + private final float resourceValue; + private final String resourceName; + + public QueueResourceVectorEntry(QueueVectorResourceType vectorResourceType, + String resourceName, float resourceValue) { + this.vectorResourceType = vectorResourceType; + this.resourceValue = resourceValue; + this.resourceName = resourceName; + } + + public QueueVectorResourceType getVectorResourceType() { + return vectorResourceType; + } + + public float getResourceValue() { + return resourceValue; + } + + public String getResourceName() { + return resourceName; + } + } +} 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/ResourceVector.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/ResourceVector.java new file mode 100644 index 0000000000000..c09f2c76a1301 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ResourceVector.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * Represents a simple resource floating point value storage + * grouped by resource names. + */ +public class ResourceVector implements Iterable> { + private final Map resourcesByName = new HashMap<>(); + + public static ResourceVector empty() { + ResourceVector emptyResourceVector = new ResourceVector(); + for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { + emptyResourceVector.setValue(resource.getName(), 0); + } + + return emptyResourceVector; + } + + public static ResourceVector uniform(float value) { + ResourceVector emptyResourceVector = new ResourceVector(); + for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { + emptyResourceVector.setValue(resource.getName(), value); + } + + return emptyResourceVector; + } + + public float getValue(String resourceName) { + return resourcesByName.getOrDefault(resourceName, 1f); + } + + public void setValue(String resourceName, float value) { + resourcesByName.put(resourceName, value); + } + + @Override + public Iterator> iterator() { + return resourcesByName.entrySet().iterator(); + } +} 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 new file mode 100644 index 0000000000000..489b3dcaaf5b6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/QueueCapacityConfigParser.java @@ -0,0 +1,206 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf; + +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AbsoluteResourceType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueVectorResourceType; +import org.apache.hadoop.yarn.util.UnitsConversionUtil; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * A class that parses {@code QueueResourcesVector} from the capacity + * configuration property set for a queue. + */ +public class QueueCapacityConfigParser { + private static final String UNIFORM_REGEX = "^([0-9.]+)(.*)"; + private static final String RESOURCE_REGEX = "^\\[[\\w\\.,\\-_=%\\ /]+\\]$"; + + private static final Pattern RESOURCE_PATTERN = Pattern.compile(RESOURCE_REGEX); + private static final Pattern UNIFORM_PATTERN = Pattern.compile(UNIFORM_REGEX); + + public static final Set ALLOWED_MEMORY_NAMES = ImmutableSet.of( + AbsoluteResourceType.MEMORY.name().toLowerCase(), + ResourceInformation.MEMORY_URI); + + private final List parsers = new ArrayList<>(); + + public QueueCapacityConfigParser() { + parsers.add(new Parser(RESOURCE_PATTERN, this::resourceParser)); + parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser)); + } + + /** + * Creates a {@code QueueResourceVector} parsed from the capacity configuration + * property set for a queue. + * @param conf configuration object + * @param queuePath queue for which the capacity property is parsed + * @param label node label + * @return a parsed resource vector + */ + public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, + String queuePath, String label) { + + if (queuePath.equals(CapacitySchedulerConfiguration.ROOT)) { + return createUniformResourceVector(QueueVectorResourceType.PERCENTAGE, 100f); + } + + String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( + queuePath, label) + CapacitySchedulerConfiguration.CAPACITY; + String capacityString = conf.get(propertyName); + + if (capacityString == null) { + return QueueCapacityVector.empty(); + } + + for (Parser parser : parsers) { + Matcher matcher = parser.regex.matcher(capacityString); + if (matcher.find()) { + return parser.parser.apply(matcher); + } + } + + return QueueCapacityVector.empty(); + } + + /** + * A parser method that is usable on uniform capacity values eg. percentage or + * weight. + * @param matcher a regex matcher that contains parsed value and its possible + * suffix + * @return a parsed resource vector + */ + private QueueCapacityVector uniformParser(Matcher matcher) { + QueueVectorResourceType vectorResourceType = QueueVectorResourceType.PERCENTAGE; + String value = matcher.group(1); + if (matcher.groupCount() == 2) { + String matchedSuffix = matcher.group(2); + if (!matchedSuffix.isEmpty()) { + for (QueueVectorResourceType suffix : QueueVectorResourceType.values()) { + // when capacity is given in percentage, we do not need % symbol + String uniformSuffix = suffix.getPostfix().replaceAll("%", ""); + if (uniformSuffix.equals(matchedSuffix)) { + vectorResourceType = suffix; + } + } + } + } + + return createUniformResourceVector(vectorResourceType, Float.parseFloat(value)); + } + + private QueueCapacityVector createUniformResourceVector(QueueVectorResourceType vectorResourceType, float parsedValue) { + Set resourceTypes = ResourceUtils.getResourceTypes().keySet(); + QueueCapacityVector resource = QueueCapacityVector.empty(); + + for (String resourceName : resourceTypes) { + resource.setResource(resourceName, parsedValue, vectorResourceType); + } + return resource; + } + + /** + * A parser method that is usable on resource capacity values eg. mixed or + * absolute resource. + * @param matcher a regex matcher that contains the matched resource string + * @return a parsed resource vector + */ + private QueueCapacityVector resourceParser(Matcher matcher) { + // Define resource here. + QueueCapacityVector resourceVector = QueueCapacityVector.empty(); + + /* + * Absolute resource configuration for a queue will be grouped by "[]". + * Syntax of absolute resource config could be like below + * "memory=4Gi vcores=2". Ideally this means "4GB of memory and 2 vcores". + */ + // Get the sub-group. + String bracketedGroup = matcher.group(0); + if (bracketedGroup.trim().isEmpty()) { + return resourceVector; + } + bracketedGroup = bracketedGroup.substring(1, bracketedGroup.length() - 1); + // Split by comma and equals delimiter eg. memory=1024, vcores=6 to + // [[memory, 1024], [vcores, 6]] + for (String kvPair : bracketedGroup.trim().split(",")) { + String[] splits = kvPair.split("="); + + // Ensure that each sub string is key value pair separated by '='. + if (splits.length > 1) { + setResourceVector(resourceVector, splits[0], splits[1]); + } + } + + // Memory has to be configured always. + if (resourceVector.getMemory() == 0L) { + return QueueCapacityVector.empty(); + } + + return resourceVector; + } + + private void setResourceVector(QueueCapacityVector resource, String resourceName, String resourceValue) { + QueueVectorResourceType resourceType = QueueVectorResourceType.ABSOLUTE; + + // Extract suffix from a value eg. for 6w extract w + String suffix = resourceValue.replaceAll("[0-9]", ""); + if (!resourceValue.endsWith(suffix)) { + return; + } + + String cleanResourceName = resourceName.replaceAll(" ", ""); + float parsedResourceValue = Float.parseFloat(resourceValue.substring( + 0, resourceValue.length() - suffix.length())); + float convertedValue = parsedResourceValue; + + if (!suffix.isEmpty() && UnitsConversionUtil.KNOWN_UNITS.contains(suffix)) { + // Convert all incoming units to MB if units is configured. + convertedValue = UnitsConversionUtil.convert(suffix, "Mi", (long) parsedResourceValue); + } else { + for (QueueVectorResourceType resourceTypeSuffix : QueueVectorResourceType.values()) { + if (resourceTypeSuffix.getPostfix().equals(suffix)) { + resourceType = resourceTypeSuffix; + } + } + } + + resource.setResource(cleanResourceName, convertedValue, resourceType); + } + + private static class Parser { + private final Pattern regex; + private final Function parser; + + Parser(Pattern regex, Function parser) { + this.regex = regex; + this.parser = parser; + } + } + +} 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 new file mode 100644 index 0000000000000..7a961e6ec6bee --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestQueueCapacityConfigParser.java @@ -0,0 +1,163 @@ +/** + * 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.conf; + +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +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.QueueResourceVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueVectorResourceType; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class TestQueueCapacityConfigParser { + + private static final String QUEUE = "root.test"; + private static final String ABSOLUTE_RESOURCE = "[memory-mb=12Gi, vcores=6, yarn.io/gpu=10]"; + private static final String MIXED_RESOURCE = "[memory-mb=1024, vcores=50%, yarn.io/gpu=6w]"; + private static final String RESOURCE_TYPES = "yarn.io/gpu"; + + private final QueueCapacityConfigParser capacityConfigParser + = new QueueCapacityConfigParser(); + + @Test + public void testPercentageCapacityConfig() { + CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); + conf.setCapacity(QUEUE, 50); + + QueueCapacityVector percentageResourceVector = capacityConfigParser.parse(conf, QUEUE, ""); + List resources = Lists.newArrayList(percentageResourceVector.iterator()); + + Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, resources.get(0).getVectorResourceType()); + Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6); + + Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, resources.get(1).getVectorResourceType()); + Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6); + + QueueCapacityVector rootResourceVector = capacityConfigParser.parse(conf, + CapacitySchedulerConfiguration.ROOT, ""); + List rootResources = + Lists.newArrayList(rootResourceVector.iterator()); + + Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, + rootResources.get(0).getVectorResourceType()); + Assert.assertEquals(100f, + rootResources.get(0).getResourceValue(), 1e-6); + + Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, + rootResources.get(1).getVectorResourceType()); + Assert.assertEquals(100f, + rootResources.get(1).getResourceValue(), 1e-6); + } + + @Test + public void testWeightCapacityConfig() { + CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); + conf.setNonLabeledQueueWeight(QUEUE, 6); + + QueueCapacityVector weightResourceVector = capacityConfigParser.parse(conf, QUEUE, ""); + List resources = Lists.newArrayList(weightResourceVector.iterator()); + + Assert.assertEquals(QueueVectorResourceType.WEIGHT, resources.get(0).getVectorResourceType()); + Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6); + + Assert.assertEquals(QueueVectorResourceType.WEIGHT, resources.get(1).getVectorResourceType()); + Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6); + } + + @Test + public void testAbsoluteResourceCapacityConfig() { + CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + CapacitySchedulerConfiguration.CAPACITY, ABSOLUTE_RESOURCE); + conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); + ResourceUtils.resetResourceTypes(conf); + + QueueCapacityVector absoluteResourceVector = capacityConfigParser.parse(conf, QUEUE, ""); + + Assert.assertEquals(QueueVectorResourceType.ABSOLUTE, absoluteResourceVector.getResource("memory-mb").getVectorResourceType()); + Assert.assertEquals(12 * 1024, absoluteResourceVector.getResource("memory-mb").getResourceValue(), 1e-6); + + Assert.assertEquals(QueueVectorResourceType.ABSOLUTE, absoluteResourceVector.getResource("vcores").getVectorResourceType()); + Assert.assertEquals(6f, absoluteResourceVector.getResource("vcores").getResourceValue(), 1e-6); + + Assert.assertEquals(QueueVectorResourceType.ABSOLUTE, absoluteResourceVector.getResource("yarn.io/gpu").getVectorResourceType()); + Assert.assertEquals(10f, absoluteResourceVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + } + + @Test + public void testMixedCapacityConfig() { + CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY, MIXED_RESOURCE); + conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); + ResourceUtils.resetResourceTypes(conf); + + QueueCapacityVector mixedResourceVector = + capacityConfigParser.parse(conf, QUEUE, ""); + + Assert.assertEquals(QueueVectorResourceType.ABSOLUTE, + mixedResourceVector.getResource("memory-mb").getVectorResourceType()); + Assert.assertEquals(1024, mixedResourceVector.getResource("memory-mb").getResourceValue(), 1e-6); + + Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, + mixedResourceVector.getResource("vcores").getVectorResourceType()); + Assert.assertEquals(50f, + mixedResourceVector.getResource("vcores").getResourceValue(), 1e-6); + + Assert.assertEquals(QueueVectorResourceType.WEIGHT, + mixedResourceVector.getResource("yarn.io/gpu").getVectorResourceType()); + Assert.assertEquals(6f, + mixedResourceVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + } + + @Test + public void testInvalidCapacityConfigs() { + CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY, "[invalid]"); + + QueueCapacityVector invalidResourceCapacity = + capacityConfigParser.parse(conf, QUEUE, ""); + List resources = + Lists.newArrayList(invalidResourceCapacity.iterator()); + Assert.assertEquals(resources.size(), 0); + + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY, ""); + + QueueCapacityVector emptyCapacity = + capacityConfigParser.parse(conf, QUEUE, ""); + List emptyResources = + Lists.newArrayList(emptyCapacity.iterator()); + Assert.assertEquals(emptyResources.size(), 0); + + conf.unset(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY); + + QueueCapacityVector nonSetCapacity = + capacityConfigParser.parse(conf, QUEUE, ""); + List nonSetResources = + Lists.newArrayList(nonSetCapacity.iterator()); + Assert.assertEquals(nonSetResources.size(), 0); + } +} \ No newline at end of file From b935b299b642b0fbe26edeb491af462a213ac9da Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 21 Sep 2021 14:29:33 +0200 Subject: [PATCH 02/27] YARN-10930. Cleanup for QueueCapacityVector --- .../capacity/QueueCapacityVector.java | 108 ++++++++++++------ .../scheduler/capacity/ResourceVector.java | 17 ++- .../conf/QueueCapacityConfigParser.java | 73 ++++++------ .../conf/TestQueueCapacityConfigParser.java | 76 ++++++------ 4 files changed, 167 insertions(+), 107 deletions(-) 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/QueueCapacityVector.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/QueueCapacityVector.java index bb9032f54ec2d..e5b117a2bbc3b 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/QueueCapacityVector.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/QueueCapacityVector.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.yarn.api.records.ResourceInformation; import java.util.HashMap; @@ -32,80 +31,121 @@ * resource. */ public class QueueCapacityVector implements - Iterable { + Iterable { private final ResourceVector resource; - private final Map resourceTypes + private final Map capacityTypes + = new HashMap<>(); + private final Map> capacityTypePerResource = new HashMap<>(); - private final Set - definedResourceTypes = new HashSet<>(); public QueueCapacityVector(ResourceVector resource) { this.resource = resource; } - public static QueueCapacityVector empty() { - return new QueueCapacityVector(ResourceVector.empty()); + /** + * Creates a new empty {@code QueueCapacityVector}. + * @return empty capacity vector + */ + public static QueueCapacityVector newInstance() { + return new QueueCapacityVector(new ResourceVector()); } - public QueueResourceVectorEntry getResource(String resourceName) { - return new QueueResourceVectorEntry(resourceTypes.get(resourceName), + public QueueCapacityVectorEntry getResource(String resourceName) { + return new QueueCapacityVectorEntry(capacityTypes.get(resourceName), resourceName, resource.getValue(resourceName)); } + /** + * Returns the number of resources defined for this vector. + * @return number of resources + */ + public int getResourceCount() { + return capacityTypes.size(); + } + + /** + * Set the value and capacity type of a resource. + * @param resourceName name of the resource + * @param value value of the resource + * @param capacityType type of the resource + */ public void setResource(String resourceName, float value, - QueueVectorResourceType resourceType) { - resource.setValue(resourceName, value); - storeResourceType(resourceName, resourceType); + QueueCapacityType capacityType) { + // Necessary due to backward compatibility (memory = memory-mb) + String convertedResourceName = resourceName; + if (resourceName.equals("memory")) { + convertedResourceName = ResourceInformation.MEMORY_URI; + } + resource.setValue(convertedResourceName, value); + storeResourceType(convertedResourceName, capacityType); } + /** + * A shorthand to retrieve the value stored for the memory resource. + * @return value of memory resource + */ public float getMemory() { return resource.getValue(ResourceInformation.MEMORY_URI); } - public Set getDefinedResourceTypes() { - return definedResourceTypes; + /** + * Returns the name of all resources of + * @param resourceType + * @return + */ + public Set getResourceNamesByCapacityType( + QueueCapacityType resourceType) { + return capacityTypePerResource.get(resourceType); } @Override - public Iterator iterator() { - return new Iterator() { + public Iterator iterator() { + return new Iterator() { private final Iterator> resources = resource.iterator(); private int i = 0; @Override public boolean hasNext() { - return resources.hasNext() && resourceTypes.size() > i; + return resources.hasNext() && capacityTypes.size() > i; } @Override - public QueueResourceVectorEntry next() { + public QueueCapacityVectorEntry next() { Map.Entry resourceInformation = resources.next(); i++; - return new QueueResourceVectorEntry( - resourceTypes.get(resourceInformation.getKey()), + return new QueueCapacityVectorEntry( + capacityTypes.get(resourceInformation.getKey()), resourceInformation.getKey(), resourceInformation.getValue()); } }; } - private void storeResourceType(String resourceName, QueueVectorResourceType resourceType) { - definedResourceTypes.add(resourceType); - resourceTypes.put(resourceName, resourceType); + /** + * Returns a set of all capacity type defined for this vector. + * @return capacity types + */ + public Set getDefinedCapacityTypes() { + return capacityTypePerResource.keySet(); + } + + private void storeResourceType( + String resourceName, QueueCapacityType resourceType) { + if (!capacityTypePerResource.containsKey(resourceType)) { + capacityTypePerResource.put(resourceType, new HashSet<>()); + } + capacityTypePerResource.get(resourceType).add(resourceName); + capacityTypes.put(resourceName, resourceType); } /** - * Represents a calculation type of a resource. + * Represents a capacity type associated with its syntax postfix. */ - public enum QueueVectorResourceType { + public enum QueueCapacityType { PERCENTAGE("%"), ABSOLUTE(""), WEIGHT("w"); - - private static final Set FLOAT_TYPES = - ImmutableSet.of(QueueVectorResourceType.PERCENTAGE, - QueueVectorResourceType.WEIGHT); private final String postfix; - QueueVectorResourceType(String postfix) { + QueueCapacityType(String postfix) { this.postfix = postfix; } @@ -114,19 +154,19 @@ public String getPostfix() { } } - public static class QueueResourceVectorEntry { - private final QueueVectorResourceType vectorResourceType; + public static class QueueCapacityVectorEntry { + private final QueueCapacityType vectorResourceType; private final float resourceValue; private final String resourceName; - public QueueResourceVectorEntry(QueueVectorResourceType vectorResourceType, + public QueueCapacityVectorEntry(QueueCapacityType vectorResourceType, String resourceName, float resourceValue) { this.vectorResourceType = vectorResourceType; this.resourceValue = resourceValue; this.resourceName = resourceName; } - public QueueVectorResourceType getVectorResourceType() { + public QueueCapacityType getVectorResourceType() { return vectorResourceType; } 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/ResourceVector.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/ResourceVector.java index c09f2c76a1301..5b9819c85d622 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/ResourceVector.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/ResourceVector.java @@ -32,7 +32,12 @@ public class ResourceVector implements Iterable> { private final Map resourcesByName = new HashMap<>(); - public static ResourceVector empty() { + /** + * Creates a new {@code ResourceVector} with all pre-defined resources set to + * zero. + * @return zero resource vector + */ + public static ResourceVector newInstance() { ResourceVector emptyResourceVector = new ResourceVector(); for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { emptyResourceVector.setValue(resource.getName(), 0); @@ -41,7 +46,13 @@ public static ResourceVector empty() { return emptyResourceVector; } - public static ResourceVector uniform(float value) { + /** + * Creates a new {@code ResourceVector} with all pre-defined resources set to + * the same value. + * @param value the value to set all resources to + * @return uniform resource vector + */ + public static ResourceVector of(float value) { ResourceVector emptyResourceVector = new ResourceVector(); for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { emptyResourceVector.setValue(resource.getName(), value); @@ -51,7 +62,7 @@ public static ResourceVector uniform(float value) { } public float getValue(String resourceName) { - return resourcesByName.getOrDefault(resourceName, 1f); + return resourcesByName.getOrDefault(resourceName, 0f); } public void setValue(String resourceName, float value) { 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 489b3dcaaf5b6..ab8bd5d7ace9d 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,12 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AbsoluteResourceType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueVectorResourceType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; import org.apache.hadoop.yarn.util.UnitsConversionUtil; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -35,8 +32,18 @@ import java.util.regex.Pattern; /** - * A class that parses {@code QueueResourcesVector} from the capacity + * A class that parses {@code QueueCapacityVector} from the capacity * configuration property set for a queue. + * + * A new syntax for capacity property could be implemented, by creating a parser + * with a regex to match the pattern and a method that creates a + * {@code QueueCapacityVector} from the matched pattern + * eg. root.capacity 20-50 + * + * A new capacity type for the existing parsers could be added by extending + * the {@code QueueCapacityVector.QueueCapacityType} with a new type and its + * associated postfix symbol. + * eg. root.capacity 20g */ public class QueueCapacityConfigParser { private static final String UNIFORM_REGEX = "^([0-9.]+)(.*)"; @@ -45,30 +52,26 @@ public class QueueCapacityConfigParser { private static final Pattern RESOURCE_PATTERN = Pattern.compile(RESOURCE_REGEX); private static final Pattern UNIFORM_PATTERN = Pattern.compile(UNIFORM_REGEX); - public static final Set ALLOWED_MEMORY_NAMES = ImmutableSet.of( - AbsoluteResourceType.MEMORY.name().toLowerCase(), - ResourceInformation.MEMORY_URI); - private final List parsers = new ArrayList<>(); public QueueCapacityConfigParser() { - parsers.add(new Parser(RESOURCE_PATTERN, this::resourceParser)); + parsers.add(new Parser(RESOURCE_PATTERN, this::heterogeneousParser)); parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser)); } /** - * Creates a {@code QueueResourceVector} parsed from the capacity configuration + * Creates a {@code QueueCapacityVector} parsed from the capacity configuration * property set for a queue. * @param conf configuration object * @param queuePath queue for which the capacity property is parsed * @param label node label - * @return a parsed resource vector + * @return a parsed capacity vector */ public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, String queuePath, String label) { if (queuePath.equals(CapacitySchedulerConfiguration.ROOT)) { - return createUniformResourceVector(QueueVectorResourceType.PERCENTAGE, 100f); + return createUniformCapacityVector(QueueCapacityType.PERCENTAGE, 100f); } String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( @@ -76,7 +79,7 @@ public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, String capacityString = conf.get(propertyName); if (capacityString == null) { - return QueueCapacityVector.empty(); + return QueueCapacityVector.newInstance(); } for (Parser parser : parsers) { @@ -86,7 +89,7 @@ public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, } } - return QueueCapacityVector.empty(); + return QueueCapacityVector.newInstance(); } /** @@ -97,27 +100,27 @@ public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, * @return a parsed resource vector */ private QueueCapacityVector uniformParser(Matcher matcher) { - QueueVectorResourceType vectorResourceType = QueueVectorResourceType.PERCENTAGE; + QueueCapacityType capacityType = QueueCapacityType.PERCENTAGE; String value = matcher.group(1); if (matcher.groupCount() == 2) { String matchedSuffix = matcher.group(2); if (!matchedSuffix.isEmpty()) { - for (QueueVectorResourceType suffix : QueueVectorResourceType.values()) { + for (QueueCapacityType suffix : QueueCapacityType.values()) { // when capacity is given in percentage, we do not need % symbol String uniformSuffix = suffix.getPostfix().replaceAll("%", ""); if (uniformSuffix.equals(matchedSuffix)) { - vectorResourceType = suffix; + capacityType = suffix; } } } } - return createUniformResourceVector(vectorResourceType, Float.parseFloat(value)); + return createUniformCapacityVector(capacityType, Float.parseFloat(value)); } - private QueueCapacityVector createUniformResourceVector(QueueVectorResourceType vectorResourceType, float parsedValue) { + private QueueCapacityVector createUniformCapacityVector(QueueCapacityType vectorResourceType, float parsedValue) { Set resourceTypes = ResourceUtils.getResourceTypes().keySet(); - QueueCapacityVector resource = QueueCapacityVector.empty(); + QueueCapacityVector resource = QueueCapacityVector.newInstance(); for (String resourceName : resourceTypes) { resource.setResource(resourceName, parsedValue, vectorResourceType); @@ -129,11 +132,11 @@ private QueueCapacityVector createUniformResourceVector(QueueVectorResourceType * A parser method that is usable on resource capacity values eg. mixed or * absolute resource. * @param matcher a regex matcher that contains the matched resource string - * @return a parsed resource vector + * @return a parsed capacity vector */ - private QueueCapacityVector resourceParser(Matcher matcher) { + private QueueCapacityVector heterogeneousParser(Matcher matcher) { // Define resource here. - QueueCapacityVector resourceVector = QueueCapacityVector.empty(); + QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); /* * Absolute resource configuration for a queue will be grouped by "[]". @@ -143,7 +146,7 @@ private QueueCapacityVector resourceParser(Matcher matcher) { // Get the sub-group. String bracketedGroup = matcher.group(0); if (bracketedGroup.trim().isEmpty()) { - return resourceVector; + return capacityVector; } bracketedGroup = bracketedGroup.substring(1, bracketedGroup.length() - 1); // Split by comma and equals delimiter eg. memory=1024, vcores=6 to @@ -153,20 +156,20 @@ private QueueCapacityVector resourceParser(Matcher matcher) { // Ensure that each sub string is key value pair separated by '='. if (splits.length > 1) { - setResourceVector(resourceVector, splits[0], splits[1]); + setCapacityVector(capacityVector, splits[0], splits[1]); } } // Memory has to be configured always. - if (resourceVector.getMemory() == 0L) { - return QueueCapacityVector.empty(); + if (capacityVector.getMemory() == 0L) { + return QueueCapacityVector.newInstance(); } - return resourceVector; + return capacityVector; } - private void setResourceVector(QueueCapacityVector resource, String resourceName, String resourceValue) { - QueueVectorResourceType resourceType = QueueVectorResourceType.ABSOLUTE; + private void setCapacityVector(QueueCapacityVector resource, String resourceName, String resourceValue) { + QueueCapacityType capacityType = QueueCapacityType.ABSOLUTE; // Extract suffix from a value eg. for 6w extract w String suffix = resourceValue.replaceAll("[0-9]", ""); @@ -183,14 +186,14 @@ private void setResourceVector(QueueCapacityVector resource, String resourceName // Convert all incoming units to MB if units is configured. convertedValue = UnitsConversionUtil.convert(suffix, "Mi", (long) parsedResourceValue); } else { - for (QueueVectorResourceType resourceTypeSuffix : QueueVectorResourceType.values()) { - if (resourceTypeSuffix.getPostfix().equals(suffix)) { - resourceType = resourceTypeSuffix; + for (QueueCapacityType capacityTypeSuffix : QueueCapacityType.values()) { + if (capacityTypeSuffix.getPostfix().equals(suffix)) { + capacityType = capacityTypeSuffix; } } } - resource.setResource(cleanResourceName, convertedValue, resourceType); + resource.setResource(cleanResourceName, convertedValue, capacityType); } private static class Parser { 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 7a961e6ec6bee..747eea2a83874 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 @@ -22,8 +22,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; 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.QueueResourceVectorEntry; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueVectorResourceType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; import org.junit.Test; @@ -34,6 +34,7 @@ public class TestQueueCapacityConfigParser { private static final String QUEUE = "root.test"; private static final String ABSOLUTE_RESOURCE = "[memory-mb=12Gi, vcores=6, yarn.io/gpu=10]"; + private static final String ABSOLUTE_RESOURCE_MEMORY_VCORE = "[memory-mb=12Gi, vcores=6]"; private static final String MIXED_RESOURCE = "[memory-mb=1024, vcores=50%, yarn.io/gpu=6w]"; private static final String RESOURCE_TYPES = "yarn.io/gpu"; @@ -45,26 +46,26 @@ public void testPercentageCapacityConfig() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); conf.setCapacity(QUEUE, 50); - QueueCapacityVector percentageResourceVector = capacityConfigParser.parse(conf, QUEUE, ""); - List resources = Lists.newArrayList(percentageResourceVector.iterator()); + QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, ""); + List resources = Lists.newArrayList(percentageCapacityVector.iterator()); - Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, resources.get(0).getVectorResourceType()); + Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType()); Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6); - Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, resources.get(1).getVectorResourceType()); + Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType()); Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6); - QueueCapacityVector rootResourceVector = capacityConfigParser.parse(conf, + QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf, CapacitySchedulerConfiguration.ROOT, ""); - List rootResources = - Lists.newArrayList(rootResourceVector.iterator()); + List rootResources = + Lists.newArrayList(rootCapacityVector.iterator()); - Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, + Assert.assertEquals(QueueCapacityType.PERCENTAGE, rootResources.get(0).getVectorResourceType()); Assert.assertEquals(100f, rootResources.get(0).getResourceValue(), 1e-6); - Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, + Assert.assertEquals(QueueCapacityType.PERCENTAGE, rootResources.get(1).getVectorResourceType()); Assert.assertEquals(100f, rootResources.get(1).getResourceValue(), 1e-6); @@ -75,13 +76,13 @@ public void testWeightCapacityConfig() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); conf.setNonLabeledQueueWeight(QUEUE, 6); - QueueCapacityVector weightResourceVector = capacityConfigParser.parse(conf, QUEUE, ""); - List resources = Lists.newArrayList(weightResourceVector.iterator()); + QueueCapacityVector weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, ""); + List resources = Lists.newArrayList(weightCapacityVector.iterator()); - Assert.assertEquals(QueueVectorResourceType.WEIGHT, resources.get(0).getVectorResourceType()); + Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType()); Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6); - Assert.assertEquals(QueueVectorResourceType.WEIGHT, resources.get(1).getVectorResourceType()); + Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType()); Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6); } @@ -92,16 +93,21 @@ public void testAbsoluteResourceCapacityConfig() { conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); ResourceUtils.resetResourceTypes(conf); - QueueCapacityVector absoluteResourceVector = capacityConfigParser.parse(conf, QUEUE, ""); + QueueCapacityVector absoluteCapacityVector = capacityConfigParser.parse(conf, QUEUE, ""); - Assert.assertEquals(QueueVectorResourceType.ABSOLUTE, absoluteResourceVector.getResource("memory-mb").getVectorResourceType()); - Assert.assertEquals(12 * 1024, absoluteResourceVector.getResource("memory-mb").getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource("memory-mb").getVectorResourceType()); + Assert.assertEquals(12 * 1024, absoluteCapacityVector.getResource("memory-mb").getResourceValue(), 1e-6); - Assert.assertEquals(QueueVectorResourceType.ABSOLUTE, absoluteResourceVector.getResource("vcores").getVectorResourceType()); - Assert.assertEquals(6f, absoluteResourceVector.getResource("vcores").getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource("vcores").getVectorResourceType()); + Assert.assertEquals(6f, absoluteCapacityVector.getResource("vcores").getResourceValue(), 1e-6); - Assert.assertEquals(QueueVectorResourceType.ABSOLUTE, absoluteResourceVector.getResource("yarn.io/gpu").getVectorResourceType()); - Assert.assertEquals(10f, absoluteResourceVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource("yarn.io/gpu").getVectorResourceType()); + Assert.assertEquals(10f, absoluteCapacityVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + CapacitySchedulerConfiguration.CAPACITY, ABSOLUTE_RESOURCE_MEMORY_VCORE); + QueueCapacityVector withoutGpuVector = capacityConfigParser.parse(conf, QUEUE, ""); + + Assert.assertEquals(2, withoutGpuVector.getResourceCount()); } @Test @@ -112,22 +118,22 @@ public void testMixedCapacityConfig() { conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); ResourceUtils.resetResourceTypes(conf); - QueueCapacityVector mixedResourceVector = + QueueCapacityVector mixedCapacityVector = capacityConfigParser.parse(conf, QUEUE, ""); - Assert.assertEquals(QueueVectorResourceType.ABSOLUTE, - mixedResourceVector.getResource("memory-mb").getVectorResourceType()); - Assert.assertEquals(1024, mixedResourceVector.getResource("memory-mb").getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.ABSOLUTE, + mixedCapacityVector.getResource("memory-mb").getVectorResourceType()); + Assert.assertEquals(1024, mixedCapacityVector.getResource("memory-mb").getResourceValue(), 1e-6); - Assert.assertEquals(QueueVectorResourceType.PERCENTAGE, - mixedResourceVector.getResource("vcores").getVectorResourceType()); + Assert.assertEquals(QueueCapacityType.PERCENTAGE, + mixedCapacityVector.getResource("vcores").getVectorResourceType()); Assert.assertEquals(50f, - mixedResourceVector.getResource("vcores").getResourceValue(), 1e-6); + mixedCapacityVector.getResource("vcores").getResourceValue(), 1e-6); - Assert.assertEquals(QueueVectorResourceType.WEIGHT, - mixedResourceVector.getResource("yarn.io/gpu").getVectorResourceType()); + Assert.assertEquals(QueueCapacityType.WEIGHT, + mixedCapacityVector.getResource("yarn.io/gpu").getVectorResourceType()); Assert.assertEquals(6f, - mixedResourceVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + mixedCapacityVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); } @Test @@ -138,7 +144,7 @@ public void testInvalidCapacityConfigs() { QueueCapacityVector invalidResourceCapacity = capacityConfigParser.parse(conf, QUEUE, ""); - List resources = + List resources = Lists.newArrayList(invalidResourceCapacity.iterator()); Assert.assertEquals(resources.size(), 0); @@ -147,7 +153,7 @@ public void testInvalidCapacityConfigs() { QueueCapacityVector emptyCapacity = capacityConfigParser.parse(conf, QUEUE, ""); - List emptyResources = + List emptyResources = Lists.newArrayList(emptyCapacity.iterator()); Assert.assertEquals(emptyResources.size(), 0); @@ -156,7 +162,7 @@ public void testInvalidCapacityConfigs() { QueueCapacityVector nonSetCapacity = capacityConfigParser.parse(conf, QUEUE, ""); - List nonSetResources = + List nonSetResources = Lists.newArrayList(nonSetCapacity.iterator()); Assert.assertEquals(nonSetResources.size(), 0); } From 5f3c0da4b80f986a4cd9bfbf9fec2a2d8bd2ca8b Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 23 Sep 2021 13:21:11 +0200 Subject: [PATCH 03/27] YARN-10965. Introduce enhanced queue calculation --- .../AbsoluteResourceCapacityCalculator.java | 127 +++++++++++++ .../scheduler/capacity/AbstractCSQueue.java | 7 +- .../AbstractQueueCapacityCalculator.java | 76 ++++++++ .../scheduler/capacity/CSQueue.java | 2 + ...CapacitySchedulerQueueCapacityHandler.java | 170 ++++++++++++++++++ .../PercentageQueueCapacityCalculator.java | 76 ++++++++ .../capacity/QueueBranchContext.java | 85 +++++++++ .../capacity/QueueHierarchyUpdateContext.java | 104 +++++++++++ .../scheduler/capacity/ResourceVector.java | 27 +++ .../capacity/RootQueueCapacityCalculator.java | 58 ++++++ .../WeightQueueCapacityCalculator.java | 64 +++++++ .../TestQueueResourceCalculation.java | 109 +++++++++++ 12 files changed, 904 insertions(+), 1 deletion(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbsoluteResourceCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractQueueCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueCapacityHandler.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PercentageQueueCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueBranchContext.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueHierarchyUpdateContext.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootQueueCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/WeightQueueCapacityCalculator.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueResourceCalculation.java 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java new file mode 100644 index 0000000000000..ec516696562e8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbsoluteResourceCapacityCalculator.java @@ -0,0 +1,127 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.util.UnitsConversionUtil; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; + +public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { + + @Override + public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { + + } + + @Override + protected QueueCapacityVector.QueueCapacityType getCapacityType() { + return null; + } + + @Override + public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String label) { + Resource minResource = Resource.newInstance(0, 0); + + for (String resourceName : queue.getConfiguredCapacityVector(label).getResourceNamesByCapacityType(QueueCapacityVector.QueueCapacityType.ABSOLUTE)) { + minResource.setResourceValue(resourceName, (long) queue.getConfiguredCapacityVector( + label).getResource(resourceName).getResourceValue()); + } + + queue.getQueueResourceQuotas().setEffectiveMinResource(label, minResource); + } + + public void calculateResources(QueueHierarchyUpdateContext updateContext, + CSQueue queue, + QueueCapacityVectorEntry configuredCapacityResource, + String label) { + if (!CollectionUtils.isEmpty(queue.getChildQueues()) && queue instanceof AbstractCSQueue) { + defineNormalizedResourceRatio(updateContext, queue, configuredCapacityResource, label); + } + + ResourceVector ratio = ResourceVector.of(1f); + + if (!queue.getQueuePath().equals(ROOT)) { + ratio = updateContext.getNormalizedMinResourceRatio( + queue.getParent().getQueuePath(), label); + } + + queue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(configuredCapacityResource.getResourceName(), + (long) (ratio.getValue(configuredCapacityResource.getResourceName()) * configuredCapacityResource.getResourceValue())); + + float relativeToParentPercentage = (float) queue.getQueueResourceQuotas() + .getEffectiveMinResource(label).getResourceValue( + configuredCapacityResource.getResourceName()) + / queue.getParent().getQueueResourceQuotas().getEffectiveMinResource( + label).getResourceValue(configuredCapacityResource.getResourceName()); + + updateContext.getRelativeResourceRatio(queue.getQueuePath(), label).setValue(configuredCapacityResource.getResourceName(), relativeToParentPercentage); + } + + @Override + public void setMetrics(QueueHierarchyUpdateContext updateContext, + CSQueue queue, String label) { + float sumCapacity = 0f; + float sumAbsoluteCapacity = 0f; + + for (String resourceName : queue.getConfiguredCapacityVector(label).getResourceNamesByCapacityType(QueueCapacityVector.QueueCapacityType.ABSOLUTE)) { + sumCapacity += updateContext.getRelativeResourceRatio(queue.getQueuePath(), label).getValue(resourceName); + sumAbsoluteCapacity += (float) queue.getQueueResourceQuotas().getEffectiveMinResource(label).getResourceValue(resourceName) + / updateContext.getUpdatedClusterResource().getResourceValue(resourceName); + } + + queue.getQueueCapacities().setCapacity(label, sumCapacity); + queue.getQueueCapacities().setAbsoluteCapacity(label, sumAbsoluteCapacity); + } + + private void defineNormalizedResourceRatio( + QueueHierarchyUpdateContext updateContext, CSQueue queue, + QueueCapacityVectorEntry configuredCapacityResource, + String label + ) { + AbstractCSQueue abstractQueue = (AbstractCSQueue) queue; + + Resource resourceByLabel = abstractQueue.labelManager.getResourceByLabel( + label, updateContext.getUpdatedClusterResource()); + + float childrenConfiguredResource = 0; + long effectiveMinResource = queue.getQueueResourceQuotas().getEffectiveMinResource(label).getResourceValue(configuredCapacityResource.getResourceName()); + long labeledResource = resourceByLabel.getResourceValue(configuredCapacityResource.getResourceName()); + String units = ""; + + // Total configured min resources of direct children of this given parent + // queue + + for (CSQueue childQueue : queue.getChildQueues()) { + units = childQueue.getQueueResourceQuotas().getConfiguredMinResource( + label).getResourceInformation( + configuredCapacityResource.getResourceName()).getUnits(); + childrenConfiguredResource += childQueue.getQueueResourceQuotas() + .getConfiguredMinResource(label).getResourceValue( + configuredCapacityResource.getResourceName()); + } + + // Factor to scale down effective resource: When cluster has sufficient + // resources, effective_min_resources will be same as configured + // min_resources. + float numeratorForMinRatio = labeledResource; + + if (!abstractQueue.getQueuePath().equals(ROOT)) { + if (effectiveMinResource > childrenConfiguredResource) { + numeratorForMinRatio = queue.getQueueResourceQuotas() + .getEffectiveMinResource(label).getResourceValue( + configuredCapacityResource.getResourceName()); + } + } + + long convertedValue = UnitsConversionUtil.convert(units, + resourceByLabel.getResourceInformation(configuredCapacityResource.getResourceName()).getUnits(), + (long) childrenConfiguredResource); + + if (convertedValue != 0) { + updateContext.getNormalizedMinResourceRatio(queue.getQueuePath(), label) + .setValue(configuredCapacityResource.getResourceName(), numeratorForMinRatio / convertedValue); + } + } +} 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 ccdbec528bf7a..9be20d72162c7 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 @@ -401,7 +401,7 @@ protected void setupQueueConfigs(Resource clusterResource, .getConfiguredNodeLabels(queuePath); } - configuredCapacityVectors = csContext.getConfiguration() + configuredCapacityVectors = configuration .parseConfiguredResourceVector(queuePath, configuredNodeLabels); // After we setup labels, we can setup capacities @@ -1199,6 +1199,11 @@ boolean canAssignToThisQueue(Resource clusterResource, } + @Override + public Set getConfiguredNodeLabels() { + return configuredNodeLabels; + } + @Override public void incReservedResource(String partition, Resource reservedRes) { if (partition == null) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java new file mode 100644 index 0000000000000..2db3035414fbb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractQueueCapacityCalculator.java @@ -0,0 +1,76 @@ +/** + * 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.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; + +import java.util.Set; + +/** + * A strategy class to encapsulate queue capacity setup and resource calculation + * logic. + */ +public abstract class AbstractQueueCapacityCalculator { + + /** + * Sets all field of the queue based on its configurations. + * @param queue queue to setup + * @param conf configuration the setup is based on + * @param label node label + */ + public abstract void setup( + CSQueue queue, CapacitySchedulerConfiguration conf, String label); + + /** + * Calculate the effective resource for a specific resource. + * @param updateContext context of the current update phase + * @param parentQueue the parent whose children will be updated + * @param label node label + */ + public abstract void calculateChildQueueResources( + QueueHierarchyUpdateContext updateContext, + CSQueue parentQueue, String label); + + /** + * Set the necessary metrics and statistics. + * @param updateContext context of the current update phase + * @param queue queue to update + * @param label node label + */ + public abstract void setMetrics( + QueueHierarchyUpdateContext updateContext, CSQueue queue, String label); + + /** + * Returns the capacity type the calculator could handle. + * @return capacity type + */ + protected abstract QueueCapacityType getCapacityType(); + + /** + * Returns all resource names that are defined for the capacity type that is + * handled by the calculator. + * @param queue queue for which the capacity vector is defined + * @param label node label + * @return resource names + */ + protected Set getResourceNames(CSQueue queue, String label) { + return queue.getConfiguredCapacityVector(label) + .getResourceNamesByCapacityType(getCapacityType()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index d5704af0e206d..4b20201e1e1b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -422,6 +422,8 @@ public void validateSubmitApplication(ApplicationId applicationId, */ QueueCapacityVector getConfiguredCapacityVector(String label); + Set getConfiguredNodeLabels(); + /** * Get effective capacity of queue. If min/max resource is configured, * preference will be given to absolute configuration over normal capacity. 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java new file mode 100644 index 0000000000000..09488b42121df --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueCapacityHandler.java @@ -0,0 +1,170 @@ +/** + * 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.commons.collections.CollectionUtils; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSortedSet; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; + +/** + * Controls how capacity and resource values are set and calculated for a queue. + * Resources are set for each label and resource separately. + */ +public class CapacitySchedulerQueueCapacityHandler { + + private static final Set CALCULATOR_PRECEDENCE = + ImmutableSortedSet.of( + QueueCapacityType.PERCENTAGE); + + private final Map + calculators; + private final AbstractQueueCapacityCalculator rootCalculator = + new RootQueueCapacityCalculator(); + private QueueHierarchyUpdateContext lastUpdateContext; + + public CapacitySchedulerQueueCapacityHandler() { + this.calculators = new HashMap<>(); + this.lastUpdateContext = new QueueHierarchyUpdateContext( + Resource.newInstance(0, 0)); + + this.calculators.put(QueueCapacityType.PERCENTAGE, + new PercentageQueueCapacityCalculator()); + } + + /** + * Set static capacity config values (mostly due to backward compatibility). + * These values are not calculated but defined at configuration time. + * + * @param queue queue to set capacity config values to + * @param conf configuration from which the values are derived + */ + public void setup(CSQueue queue, CapacitySchedulerConfiguration conf) { + for (String label : queue.getConfiguredNodeLabels()) { + for (QueueCapacityType capacityType : + queue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + AbstractQueueCapacityCalculator calculator = + calculators.get(capacityType); + calculator.setup(queue, conf, label); + } + } + } + + /** + * Updates the resource and metrics values for a queue and its descendants + * (and siblings if needed). These values are calculated at runtime. + * + * @param clusterResource resource of the cluster + * @param queue queue to update + */ + public void update(Resource clusterResource, CSQueue queue) { + QueueHierarchyUpdateContext newContext = + new QueueHierarchyUpdateContext(clusterResource, lastUpdateContext); + this.lastUpdateContext = newContext; + + if (queue.getQueuePath().equals(ROOT)) { + calculateResources(newContext, queue, ImmutableSet.of(rootCalculator)); + } + + update(queue, newContext); + } + + private void update( + CSQueue parent, QueueHierarchyUpdateContext queueHierarchyContext) { + if (parent == null || CollectionUtils.isEmpty(parent.getChildQueues())) { + return; + } + + collectCapacities(queueHierarchyContext, parent); + calculateResources(queueHierarchyContext, parent, + CALCULATOR_PRECEDENCE.stream().map((calculators::get)) + .collect(Collectors.toList())); + updateChildren(queueHierarchyContext, parent); + } + + private void calculateResources( + QueueHierarchyUpdateContext queueHierarchyContext, CSQueue parent, + Collection usableCalculators) { + for (String label : parent.getConfiguredNodeLabels()) { + queueHierarchyContext.getQueueBranchContext(parent.getQueuePath()) + .setRemainingResource(label, ResourceVector.of(parent.getEffectiveCapacity(label))); + for (AbstractQueueCapacityCalculator calculator : usableCalculators) { + calculator.calculateChildQueueResources(queueHierarchyContext, parent, label); + } + setMetrics(parent, label); + } + } + + private void updateChildren( + QueueHierarchyUpdateContext queueHierarchyContext, CSQueue parent) { + if (parent.getChildQueues() != null) { + for (CSQueue childQueue : parent.getChildQueues()) { + update(childQueue, queueHierarchyContext); + } + } + } + + /** + * Collects capacity values of all queue on the same level for each resource. + * + * @param queueHierarchyContext update context of the queue hierarchy + * @param parent parent of the branch + */ + private void collectCapacities( + QueueHierarchyUpdateContext queueHierarchyContext, CSQueue parent) { + List siblingsOfLevel = parent.getChildQueues(); + if (CollectionUtils.isEmpty(siblingsOfLevel)) { + return; + } + + for (CSQueue queue : siblingsOfLevel) { + for (String label : queue.getConfiguredNodeLabels()) { + for (QueueCapacityVectorEntry capacityResource : + queue.getConfiguredCapacityVector(label)) { + queueHierarchyContext.getQueueBranchContext( + parent.getQueuePath()).getSumByLabel(label) + .increment(capacityResource); + } + } + } + } + + private void setMetrics(CSQueue parent, String label) { + for (CSQueue childQueue : parent.getChildQueues()) { + for (QueueCapacityType capacityType : + childQueue.getConfiguredCapacityVector(label) + .getDefinedCapacityTypes()) { + AbstractQueueCapacityCalculator calculator = + calculators.get(capacityType); + calculator.setMetrics(lastUpdateContext, childQueue, label); + } + } + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java new file mode 100644 index 0000000000000..35db4f5be53b3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PercentageQueueCapacityCalculator.java @@ -0,0 +1,76 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; + +import java.util.Set; + +public class PercentageQueueCapacityCalculator extends AbstractQueueCapacityCalculator { + + @Override + public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String label) { + float sumCapacity = 0f; + QueueCapacityVector capacityVector = + queue.getConfiguredCapacityVector(label); + for (String resourceName : getResourceNames(queue, label)) { + sumCapacity += capacityVector.getResource(resourceName).getResourceValue(); + } + } + + @Override + public void calculateChildQueueResources( + QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { + ResourceVector aggregatedUsedResource = new ResourceVector(); + for (CSQueue childQueue : parentQueue.getChildQueues()) { + for (String resourceName : getResourceNames(childQueue, label)) { + QueueCapacityVectorEntry configuredCapacityResource = childQueue + .getConfiguredCapacityVector(label).getResource(resourceName); + + float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( + parentQueue.getQueuePath(), label).getValue(resourceName); + float queueAbsoluteCapacity = parentAbsoluteCapacity + * configuredCapacityResource.getResourceValue() / 100; + float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).getRemainingResource(label) + .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) + .getResourceValue(resourceName); + queueAbsoluteCapacity *= remainingPerEffectiveResourceRatio; + + long resource = Math.round(updateContext.getUpdatedClusterResource() + .getResourceValue(configuredCapacityResource.getResourceName()) + * queueAbsoluteCapacity); + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(configuredCapacityResource.getResourceName(), + resource); + + updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) + .setValue(configuredCapacityResource.getResourceName(), + queueAbsoluteCapacity); + aggregatedUsedResource.setValue( + configuredCapacityResource.getResourceName(), resource); + } + } + + updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .getRemainingResource(label).subtract(aggregatedUsedResource); + } + + @Override + public void setMetrics( + QueueHierarchyUpdateContext updateContext, CSQueue queue, String label) { + float sumAbsoluteCapacity = 0f; + Set resources = getResourceNames(queue, label); + for (String resourceName : resources) { + sumAbsoluteCapacity += updateContext.getRelativeResourceRatio( + queue.getQueuePath(), label).getValue(resourceName); + } + + queue.getQueueCapacities().setAbsoluteCapacity(sumAbsoluteCapacity + / resources.size()); + } + + @Override + protected QueueCapacityType getCapacityType() { + return QueueCapacityType.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/capacity/QueueBranchContext.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/QueueBranchContext.java new file mode 100644 index 0000000000000..ac3f34117b9b3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueBranchContext.java @@ -0,0 +1,85 @@ +/** + * 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.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * Contains all intermediate calculation values that are common for a queue + * branch (all siblings that have a common parent). + */ +public class QueueBranchContext { + private final Map sumCapacitiesByLabel = new HashMap<>(); + private final Map remainingResourceByLabel = new HashMap<>(); + + /** + * Returns summarized capacity values for a label. + * @param label node label + * @return summarized capacity values + */ + public CapacitySum getSumByLabel(String label) { + if (!sumCapacitiesByLabel.containsKey(label)) { + sumCapacitiesByLabel.put(label, new CapacitySum()); + } + + return sumCapacitiesByLabel.get(label); + } + + public void setRemainingResource(String label, ResourceVector resource) { + remainingResourceByLabel.put(label, resource); + } + + /** + * Returns the remaining resources of a parent that is still available for its + * children. + * @param label node label + * @return remaining resources + */ + public ResourceVector getRemainingResource(String label) { + return remainingResourceByLabel.get(label); + } + + public static class CapacitySum { + private Map> sum = new HashMap<>(); + + public void increment(QueueCapacityVectorEntry resource) { + Map sumByResourceName; + + if (sum.containsKey(resource.getResourceName())) { + sumByResourceName = sum.get(resource.getResourceName()); + } else { + sumByResourceName = new HashMap<>(); + sum.put(resource.getResourceName(), sumByResourceName); + } + + sumByResourceName.put(resource.getVectorResourceType(), sumByResourceName.getOrDefault( + resource.getVectorResourceType(), 0f) + resource.getResourceValue()); + } + + public float getSum(String resourceName, QueueCapacityType resourceType) { + return sum.getOrDefault(resourceName, + Collections.emptyMap()).getOrDefault(resourceType, 0f); + } + } +} 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/QueueHierarchyUpdateContext.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/QueueHierarchyUpdateContext.java new file mode 100644 index 0000000000000..e4718251391e2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueHierarchyUpdateContext.java @@ -0,0 +1,104 @@ +/** + * 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.commons.collections.map.LazyMap; +import org.apache.hadoop.yarn.api.records.Resource; + +import java.util.HashMap; +import java.util.Map; + +/** + * A storage that encapsulates intermediate calculation values throughout a + * full queue update phase. + */ +public class QueueHierarchyUpdateContext { + private final Resource updatedClusterResource; + + private final Map queueBranchContext + = LazyMap.decorate(new HashMap(), + QueueBranchContext::new); + private Map> normalizedResourceRatios = + createLazyResourceVector(); + private Map> relativeResourceRatio = + createLazyResourceVector(); + + public QueueHierarchyUpdateContext( + Resource updatedClusterResource, + QueueHierarchyUpdateContext queueHierarchyUpdateContext) { + this.updatedClusterResource = updatedClusterResource; + this.normalizedResourceRatios = queueHierarchyUpdateContext + .normalizedResourceRatios; + this.relativeResourceRatio = queueHierarchyUpdateContext + .relativeResourceRatio; + } + + public QueueHierarchyUpdateContext(Resource updatedClusterResource) { + this.updatedClusterResource = updatedClusterResource; + } + + private static Map> + createLazyResourceVector() { + return LazyMap.decorate( + new HashMap>(), + () -> LazyMap.decorate( + new HashMap(), + ResourceVector::newInstance)); + } + + /** + * Returns the overall cluster resource available for the update phase. + * @return cluster resource + */ + public Resource getUpdatedClusterResource() { + return updatedClusterResource; + } + + /** + * Returns the context for a queue branch, which is identified by the path of + * the parent. + * @param queuePath queue path of the parent + * @return queue branch context + */ + public QueueBranchContext getQueueBranchContext(String queuePath) { + return queueBranchContext.get(queuePath); + } + + /** + * Returns the normalized resource ratio calculated for a queue. + * @param queuePath queue path + * @param label node label + * @return normalized resource ratio + */ + public ResourceVector getNormalizedMinResourceRatio( + String queuePath, String label) { + return normalizedResourceRatios.get(queuePath).get(label); + } + + /** + * Returns the ratio of a child queue and its parent's resource. + * @param queuePath queue path + * @param label node label + * @return resource ratio + */ + public ResourceVector getRelativeResourceRatio( + String queuePath, String label) { + return relativeResourceRatio.get(queuePath).get(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/ResourceVector.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/ResourceVector.java index 5b9819c85d622..a57ad6795907f 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/ResourceVector.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/ResourceVector.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -61,6 +62,32 @@ public static ResourceVector of(float value) { return emptyResourceVector; } + /** + * Creates a new {@code ResourceVector} with the values set in a + * {@code Resource} object. + * @param resource resource object the resource vector will be based on + * @return uniform resource vector + */ + public static ResourceVector of(Resource resource) { + ResourceVector resourceVector = new ResourceVector(); + for (ResourceInformation resourceInformation : resource.getResources()) { + resourceVector.setValue(resourceInformation.getName(), + resourceInformation.getValue()); + } + + return resourceVector; + } + + /** + * Subtract values for each resource defined in the given resource vector. + * @param otherResourceVector rhs resource vector of the subtraction + */ + public void subtract(ResourceVector otherResourceVector) { + for (Map.Entry resource : otherResourceVector) { + setValue(resource.getKey(), getValue(resource.getKey()) - resource.getValue()); + } + } + public float getValue(String resourceName) { return resourcesByName.getOrDefault(resourceName, 0f); } 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java new file mode 100644 index 0000000000000..2e4e7e6bd0e4a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootQueueCapacityCalculator.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.PERCENTAGE; + +public class RootQueueCapacityCalculator extends + AbstractQueueCapacityCalculator { + @Override + public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String label) { + queue.getQueueCapacities().setCapacity(label, 100f); + } + + @Override + public void calculateChildQueueResources( + QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { + for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { + updateContext.getRelativeResourceRatio(ROOT, label).setValue( + resource.getName(), 1); + + parentQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(resource.getName(), + updateContext.getUpdatedClusterResource().getResourceValue( + resource.getName())); + } + } + + @Override + public void setMetrics( + QueueHierarchyUpdateContext updateContext, CSQueue queue, String label) { + queue.getQueueCapacities().setAbsoluteCapacity(label, 1); + } + + @Override + protected QueueCapacityVector.QueueCapacityType getCapacityType() { + return 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/capacity/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java new file mode 100644 index 0000000000000..07038c81a346c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/WeightQueueCapacityCalculator.java @@ -0,0 +1,64 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.PERCENTAGE; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.WEIGHT; + +public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculator { + + @Override + public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { + + } + + @Override + protected QueueCapacityVector.QueueCapacityType getCapacityType() { + return null; + } + + @Override + public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String label) { + float sumWeight = 0; + QueueCapacityVector capacityVector = queue.getConfiguredCapacityVector(label); + for (String resourceName : capacityVector.getResourceNamesByCapacityType(WEIGHT)) { + sumWeight += capacityVector.getResource(resourceName).getResourceValue(); + } + + queue.getQueueCapacities().setWeight(label, sumWeight); + } + + public void calculateResources(QueueHierarchyUpdateContext updateContext, CSQueue queue, QueueCapacityVectorEntry configuredCapacityResource, String label) { + QueueBranchContext.CapacitySum capacitySum = updateContext.getQueueBranchContext(queue.getParent().getQueuePath()).getSumByLabel(label); + + float capacityMultiplier = 1 - capacitySum.getSum(configuredCapacityResource.getResourceName(), PERCENTAGE); + float normalizedWeight = configuredCapacityResource.getResourceValue() / + capacitySum.getSum(configuredCapacityResource.getResourceName(), WEIGHT) + * capacityMultiplier; + + float parentAbsoluteCapacity = queue.getParent() != null ? + updateContext.getRelativeResourceRatio( + queue.getParent().getQueuePath(), label).getValue( + configuredCapacityResource.getResourceName()) : 1; + float queueAbsoluteCapacity = parentAbsoluteCapacity * normalizedWeight; + long resource = Math.round(queueAbsoluteCapacity * + updateContext.getUpdatedClusterResource().getResourceValue( + configuredCapacityResource.getResourceName())); + + queue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(configuredCapacityResource.getResourceName(), resource); + + updateContext.getRelativeResourceRatio(queue.getQueuePath(), label).setValue( + configuredCapacityResource.getResourceName(), queueAbsoluteCapacity); + } + + @Override + public void setMetrics(QueueHierarchyUpdateContext updateContext, CSQueue queue, String label) { + float sumNormalizedWeight = 0; + for (String resourceName : queue.getConfiguredCapacityVector(label).getResourceNamesByCapacityType(WEIGHT)) { + sumNormalizedWeight += updateContext.getRelativeResourceRatio(queue.getQueuePath(), label).getValue(resourceName); + } + + queue.getQueueCapacities().setNormalizedWeight(label, sumNormalizedWeight); + } + +} 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/TestQueueResourceCalculation.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/TestQueueResourceCalculation.java new file mode 100644 index 0000000000000..1bbbffaff15fb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueResourceCalculation.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class TestQueueResourceCalculation + extends TestCapacitySchedulerAutoCreatedQueueBase { + + private CapacitySchedulerConfiguration csConf; + private static final Resource QUEUE_A_RES = Resource.newInstance(80 * GB, + 10); + private static final Resource QUEUE_B_RES = Resource.newInstance( 120 * GB, + 15); + private static final Resource QUEUE_A1_RES = Resource.newInstance(50 * GB, + 15); + private static final Resource UPDATE_RES = Resource.newInstance(250 * GB, 40); + + public void setUp() throws Exception { + csConf = new CapacitySchedulerConfiguration(); + csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + 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", 6f); + csConf.setNonLabeledQueueWeight("root.b", 4f); + csConf.setQueues("root.a", new String[]{"a1", "a2"}); + csConf.setNonLabeledQueueWeight("root.a.a1", 1f); + + RMNodeLabelsManager mgr = new NullRMNodeLabelsManager(); + mgr.init(csConf); + mockRM = new MockRM(csConf) { + protected RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + cs = (CapacityScheduler) mockRM.getResourceScheduler(); + cs.updatePlacementRules(); + // Policy for new auto created queue's auto deletion when expired + mockRM.start(); + cs.start(); + mockRM.registerNode("h1:1234", 10 * GB); // label = x + } + + @Test + public void testPercentageResourceCalculation() throws IOException { + csConf.setCapacity("root.a", 30); + csConf.setCapacity("root.b", 70); + csConf.setCapacity("root.a.a1", 17); + csConf.setCapacity("root.a.a2", 83); + cs.reinitialize(csConf, mockRM.getRMContext()); + + CapacitySchedulerQueueCapacityHandler queueController = + new CapacitySchedulerQueueCapacityHandler(); + CSQueue a = cs.getQueue("root.a"); + CSQueue b = cs.getQueue("root.b"); + CSQueue a1 = cs.getQueue("root.a.a1"); + CSQueue a2 = cs.getQueue("root.a.a2"); + + queueController.setup(a, csConf); + queueController.setup(b, csConf); + queueController.setup(a1, csConf); + queueController.setup(a2, csConf); + queueController.update(Resource.newInstance(10 * GB, 20), cs.getQueue("root")); + + Assert.assertEquals(0.3 * 10 * GB, a.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize(), 1e-6); + Assert.assertEquals(0.3, a.getQueueCapacities().getAbsoluteCapacity(), 1e-6); + Assert.assertEquals(0.3, a.getQueueCapacities().getCapacity(), 1e-6); + + Assert.assertEquals(0.7 * 10 * GB, b.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize(), 1e-6); + Assert.assertEquals(0.7, b.getQueueCapacities().getAbsoluteCapacity(), 1e-6); + Assert.assertEquals(0.7, b.getQueueCapacities().getCapacity(), 1e-6); + + Assert.assertEquals(Math.round(0.3 * 0.17 * 10 * GB), a1.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize()); + Assert.assertEquals(0.3 * 0.17, a1.getQueueCapacities().getAbsoluteCapacity(), 1e-6); + Assert.assertEquals(0.17, a1.getQueueCapacities().getCapacity(), 1e-6); + + Assert.assertEquals(Math.round(0.3 * 0.83 * 10 * GB), a2.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize()); + Assert.assertEquals(0.3 * 0.83, a2.getQueueCapacities().getAbsoluteCapacity(), 1e-6); + Assert.assertEquals(0.83, a2.getQueueCapacities().getCapacity(), 1e-6); + } +} \ No newline at end of file From 6ee8dc2b6e3caf00ed238f51186fbc62d37c43d1 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 14 Oct 2021 16:27:03 +0200 Subject: [PATCH 04/27] YARN-10930. Cover additional test cases --- .../capacity/QueueCapacityVector.java | 99 ++++++++++++++--- .../scheduler/capacity/ResourceVector.java | 31 +++++- .../conf/QueueCapacityConfigParser.java | 37 +++---- .../capacity/TestQueueCapacityVector.java | 102 ++++++++++++++++++ .../capacity/TestResourceVector.java | 97 +++++++++++++++++ .../conf/TestQueueCapacityConfigParser.java | 21 +++- 6 files changed, 350 insertions(+), 37 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacityVector.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestResourceVector.java 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/QueueCapacityVector.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/QueueCapacityVector.java index e5b117a2bbc3b..968e8cf779fb9 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/QueueCapacityVector.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/QueueCapacityVector.java @@ -20,6 +20,7 @@ import org.apache.hadoop.yarn.api.records.ResourceInformation; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -32,22 +33,57 @@ */ public class QueueCapacityVector implements Iterable { + private static final String START_PARENTHESES = "["; + private static final String END_PARENTHESES = "]"; + private static final String RESOURCE_DELIMITER = ","; + private static final String VALUE_DELIMITER = "="; + private final ResourceVector resource; private final Map capacityTypes = new HashMap<>(); private final Map> capacityTypePerResource = new HashMap<>(); - public QueueCapacityVector(ResourceVector resource) { + public QueueCapacityVector() { + this.resource = new ResourceVector(); + } + + private QueueCapacityVector(ResourceVector resource) { this.resource = resource; } /** - * Creates a new empty {@code QueueCapacityVector}. - * @return empty capacity vector + * Creates a zero {@code QueueCapacityVector}. The resources are defined + * in absolute capacity type by default. + * + * @return zero capacity vector */ public static QueueCapacityVector newInstance() { - return new QueueCapacityVector(new ResourceVector()); + QueueCapacityVector newCapacityVector = + new QueueCapacityVector(ResourceVector.newInstance()); + for (Map.Entry resourceEntry : newCapacityVector.resource) { + newCapacityVector.storeResourceType(resourceEntry.getKey(), + QueueCapacityType.ABSOLUTE); + } + + return newCapacityVector; + } + + /** + * Creates a uniform and homogeneous {@code QueueCapacityVector}. + * The resources are defined in absolute capacity type by default. + * + * @return uniform capacity vector + */ + public static QueueCapacityVector of( + float value, QueueCapacityType capacityType) { + QueueCapacityVector newCapacityVector = + new QueueCapacityVector(ResourceVector.of(value)); + for (Map.Entry resourceEntry : newCapacityVector.resource) { + newCapacityVector.storeResourceType(resourceEntry.getKey(), capacityType); + } + + return newCapacityVector; } public QueueCapacityVectorEntry getResource(String resourceName) { @@ -57,6 +93,7 @@ public QueueCapacityVectorEntry getResource(String resourceName) { /** * Returns the number of resources defined for this vector. + * * @return number of resources */ public int getResourceCount() { @@ -65,8 +102,9 @@ public int getResourceCount() { /** * Set the value and capacity type of a resource. + * * @param resourceName name of the resource - * @param value value of the resource + * @param value value of the resource * @param capacityType type of the resource */ public void setResource(String resourceName, float value, @@ -82,6 +120,7 @@ public void setResource(String resourceName, float value, /** * A shorthand to retrieve the value stored for the memory resource. + * * @return value of memory resource */ public float getMemory() { @@ -89,13 +128,22 @@ public float getMemory() { } /** - * Returns the name of all resources of - * @param resourceType - * @return + * Returns the name of all resources that are defined in the given capacity + * type. + * + * @param capacityType the capacity type of the resources + * @return all resource names for the given capacity type */ public Set getResourceNamesByCapacityType( - QueueCapacityType resourceType) { - return capacityTypePerResource.get(resourceType); + QueueCapacityType capacityType) { + return capacityTypePerResource.getOrDefault(capacityType, + Collections.emptySet()); + } + + public boolean isResourceOfType( + String resourceName, QueueCapacityType capacityType) { + return capacityTypes.containsKey(resourceName) && + capacityTypes.get(resourceName).equals(capacityType); } @Override @@ -123,6 +171,7 @@ public QueueCapacityVectorEntry next() { /** * Returns a set of all capacity type defined for this vector. + * * @return capacity types */ public Set getDefinedCapacityTypes() { @@ -131,13 +180,39 @@ public Set getDefinedCapacityTypes() { private void storeResourceType( String resourceName, QueueCapacityType resourceType) { - if (!capacityTypePerResource.containsKey(resourceType)) { - capacityTypePerResource.put(resourceType, new HashSet<>()); + if (capacityTypes.get(resourceName) != null + && !capacityTypes.get(resourceName).equals(resourceType)) { + capacityTypePerResource.get(capacityTypes.get(resourceName)) + .remove(resourceName); } + + capacityTypePerResource.putIfAbsent(resourceType, new HashSet<>()); capacityTypePerResource.get(resourceType).add(resourceName); capacityTypes.put(resourceName, resourceType); } + @Override + public String toString() { + StringBuilder stringVector = new StringBuilder(); + stringVector.append(START_PARENTHESES); + + int resourceCount = 0; + for (Map.Entry resourceEntry : resource) { + resourceCount++; + stringVector.append(resourceEntry.getKey()) + .append(VALUE_DELIMITER) + .append(resourceEntry.getValue()) + .append(capacityTypes.get(resourceEntry.getKey()).postfix); + if (resourceCount < capacityTypes.size()) { + stringVector.append(RESOURCE_DELIMITER); + } + } + + stringVector.append(END_PARENTHESES); + + return stringVector.toString(); + } + /** * Represents a capacity type associated with its syntax postfix. */ 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/ResourceVector.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/ResourceVector.java index a57ad6795907f..404be82fc0ab1 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/ResourceVector.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/ResourceVector.java @@ -39,12 +39,12 @@ public class ResourceVector implements Iterable> { * @return zero resource vector */ public static ResourceVector newInstance() { - ResourceVector emptyResourceVector = new ResourceVector(); + ResourceVector zeroResourceVector = new ResourceVector(); for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { - emptyResourceVector.setValue(resource.getName(), 0); + zeroResourceVector.setValue(resource.getName(), 0); } - return emptyResourceVector; + return zeroResourceVector; } /** @@ -88,8 +88,12 @@ public void subtract(ResourceVector otherResourceVector) { } } - public float getValue(String resourceName) { - return resourcesByName.getOrDefault(resourceName, 0f); + public void increment(String resourceName, float value) { + setValue(resourceName, getValue(resourceName) + value); + } + + public Float getValue(String resourceName) { + return resourcesByName.get(resourceName); } public void setValue(String resourceName, float value) { @@ -100,4 +104,21 @@ public void setValue(String resourceName, float value) { public Iterator> iterator() { return resourcesByName.entrySet().iterator(); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + return this.resourcesByName.equals(((ResourceVector) o).resourcesByName); + } + + @Override + public int hashCode() { + return resourcesByName.hashCode(); + } } 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 ab8bd5d7ace9d..c46446b64d714 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 @@ -22,11 +22,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; import org.apache.hadoop.yarn.util.UnitsConversionUtil; -import org.apache.hadoop.yarn.util.resource.ResourceUtils; import java.util.ArrayList; import java.util.List; -import java.util.Set; import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -47,7 +45,7 @@ */ public class QueueCapacityConfigParser { private static final String UNIFORM_REGEX = "^([0-9.]+)(.*)"; - private static final String RESOURCE_REGEX = "^\\[[\\w\\.,\\-_=%\\ /]+\\]$"; + private static final String RESOURCE_REGEX = "^\\[([\\w\\.,\\-_%\\ /]+=[\\w\\.,\\-_%\\ /]+)+\\]$"; private static final Pattern RESOURCE_PATTERN = Pattern.compile(RESOURCE_REGEX); private static final Pattern UNIFORM_PATTERN = Pattern.compile(UNIFORM_REGEX); @@ -71,7 +69,7 @@ public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, String queuePath, String label) { if (queuePath.equals(CapacitySchedulerConfiguration.ROOT)) { - return createUniformCapacityVector(QueueCapacityType.PERCENTAGE, 100f); + return QueueCapacityVector.of(100f, QueueCapacityType.PERCENTAGE); } String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( @@ -79,7 +77,7 @@ public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, String capacityString = conf.get(propertyName); if (capacityString == null) { - return QueueCapacityVector.newInstance(); + return new QueueCapacityVector(); } for (Parser parser : parsers) { @@ -89,7 +87,7 @@ public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, } } - return QueueCapacityVector.newInstance(); + return new QueueCapacityVector(); } /** @@ -115,17 +113,7 @@ private QueueCapacityVector uniformParser(Matcher matcher) { } } - return createUniformCapacityVector(capacityType, Float.parseFloat(value)); - } - - private QueueCapacityVector createUniformCapacityVector(QueueCapacityType vectorResourceType, float parsedValue) { - Set resourceTypes = ResourceUtils.getResourceTypes().keySet(); - QueueCapacityVector resource = QueueCapacityVector.newInstance(); - - for (String resourceName : resourceTypes) { - resource.setResource(resourceName, parsedValue, vectorResourceType); - } - return resource; + return QueueCapacityVector.of(Float.parseFloat(value), capacityType); } /** @@ -168,7 +156,8 @@ private QueueCapacityVector heterogeneousParser(Matcher matcher) { return capacityVector; } - private void setCapacityVector(QueueCapacityVector resource, String resourceName, String resourceValue) { + private void setCapacityVector( + QueueCapacityVector resource, String resourceName, String resourceValue) { QueueCapacityType capacityType = QueueCapacityType.ABSOLUTE; // Extract suffix from a value eg. for 6w extract w @@ -196,6 +185,18 @@ private void setCapacityVector(QueueCapacityVector resource, String resourceName resource.setResource(cleanResourceName, convertedValue, capacityType); } + /** + * Checks whether the given capacity string is in a capacity vector compatible + * format. + * @param configuredCapacity capacity string + * @return true, if capacity string is in capacity vector format, + * false otherwise + */ + public boolean isCapacityVectorFormat(String configuredCapacity) { + return configuredCapacity != null + && RESOURCE_PATTERN.matcher(configuredCapacity).find(); + } + private static class Parser { private final Pattern regex; private final Function parser; 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/TestQueueCapacityVector.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/TestQueueCapacityVector.java new file mode 100644 index 0000000000000..345bea511620f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacityVector.java @@ -0,0 +1,102 @@ +/** + * 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.util.Lists; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; + +public class TestQueueCapacityVector { + + private final YarnConfiguration conf = new YarnConfiguration(); + + @Before + public void setUp() { + conf.set(YarnConfiguration.RESOURCE_TYPES, "custom"); + ResourceUtils.resetResourceTypes(conf); + } + + @Test + public void getResourceNamesByCapacityType() { + QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); + + capacityVector.setResource("memory-mb", 10, QueueCapacityType.PERCENTAGE); + capacityVector.setResource("vcores", 6, QueueCapacityType.PERCENTAGE); + + // custom is not set, defaults to 0 + Assert.assertEquals(1, capacityVector.getResourceNamesByCapacityType( + QueueCapacityType.ABSOLUTE).size()); + Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( + QueueCapacityType.ABSOLUTE).contains("custom")); + + Assert.assertEquals(2, capacityVector.getResourceNamesByCapacityType( + QueueCapacityType.PERCENTAGE).size()); + Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( + QueueCapacityType.PERCENTAGE).contains("vcores")); + Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( + QueueCapacityType.PERCENTAGE).contains("memory-mb")); + } + + @Test + public void isResourceOfType() { + QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); + + capacityVector.setResource("memory-mb", 10, QueueCapacityType.WEIGHT); + capacityVector.setResource("vcores", 6, QueueCapacityType.PERCENTAGE); + capacityVector.setResource("custom", 3, QueueCapacityType.ABSOLUTE); + + Assert.assertTrue(capacityVector.isResourceOfType("memory-mb", QueueCapacityType.WEIGHT)); + Assert.assertTrue(capacityVector.isResourceOfType("vcores", QueueCapacityType.PERCENTAGE)); + Assert.assertTrue(capacityVector.isResourceOfType("custom", QueueCapacityType.ABSOLUTE)); + } + + @Test + public void iterator() { + QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); + List entries = Lists.newArrayList(capacityVector); + + Assert.assertEquals(3, entries.size()); + + QueueCapacityVector emptyCapacityVector = new QueueCapacityVector(); + List emptyEntries = Lists.newArrayList(emptyCapacityVector); + + Assert.assertEquals(0, emptyEntries.size()); + } + + @Test + public void testToString() { + QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); + + capacityVector.setResource("memory-mb", 10, QueueCapacityType.WEIGHT); + capacityVector.setResource("vcores", 6, QueueCapacityType.PERCENTAGE); + capacityVector.setResource("custom", 3, QueueCapacityType.ABSOLUTE); + + Assert.assertEquals("[custom=3.0,memory-mb=10.0w,vcores=6.0%]", capacityVector.toString()); + + QueueCapacityVector emptyCapacityVector = new QueueCapacityVector(); + Assert.assertEquals("[]", emptyCapacityVector.toString()); + } +} \ 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/TestResourceVector.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/TestResourceVector.java new file mode 100644 index 0000000000000..a9f7a9e296217 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestResourceVector.java @@ -0,0 +1,97 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class TestResourceVector { + private final YarnConfiguration conf = new YarnConfiguration(); + + @Before + public void setUp() { + conf.set(YarnConfiguration.RESOURCE_TYPES, "custom"); + ResourceUtils.resetResourceTypes(conf); + } + + @Test + public void testCreation() { + ResourceVector zeroResourceVector = ResourceVector.newInstance(); + Assert.assertEquals(0, zeroResourceVector.getValue("memory-mb"), 1e-6); + Assert.assertEquals(0, zeroResourceVector.getValue("vcores"), 1e-6); + Assert.assertEquals(0, zeroResourceVector.getValue("custom"), 1e-6); + + ResourceVector uniformResourceVector = ResourceVector.of(10); + Assert.assertEquals(10, uniformResourceVector.getValue("memory-mb"), 1e-6); + Assert.assertEquals(10, uniformResourceVector.getValue("vcores"), 1e-6); + Assert.assertEquals(10, uniformResourceVector.getValue("custom"), 1e-6); + + Map customResources = new HashMap<>(); + customResources.put("custom", 2L); + Resource resource = Resource.newInstance(10, 5, customResources); + ResourceVector resourceVectorFromResource = ResourceVector.of(resource); + Assert.assertEquals(10, resourceVectorFromResource.getValue("memory-mb"), 1e-6); + Assert.assertEquals(5, resourceVectorFromResource.getValue("vcores"), 1e-6); + Assert.assertEquals(2, resourceVectorFromResource.getValue("custom"), 1e-6); + } + + @Test + public void subtract() { + ResourceVector lhsResourceVector = ResourceVector.of(13); + ResourceVector rhsResourceVector = ResourceVector.of(5); + lhsResourceVector.subtract(rhsResourceVector); + + Assert.assertEquals(8, lhsResourceVector.getValue("memory-mb"), 1e-6); + Assert.assertEquals(8, lhsResourceVector.getValue("vcores"), 1e-6); + Assert.assertEquals(8, lhsResourceVector.getValue("custom"), 1e-6); + } + + @Test + public void increment() { + ResourceVector resourceVector = ResourceVector.of(13); + resourceVector.increment("memory-mb", 5); + + Assert.assertEquals(18, resourceVector.getValue("memory-mb"), 1e-6); + Assert.assertEquals(13, resourceVector.getValue("vcores"), 1e-6); + Assert.assertEquals(13, resourceVector.getValue("custom"), 1e-6); + } + + @Test + public void testEquals() { + ResourceVector resourceVector = ResourceVector.of(13); + ResourceVector resourceVectorOther = ResourceVector.of(14); + Resource resource = Resource.newInstance(13, 13); + + Assert.assertFalse(resourceVector.equals(null)); + Assert.assertFalse(resourceVector.equals(resourceVectorOther)); + Assert.assertFalse(resourceVector.equals(resource)); + + ResourceVector resourceVectorOne = ResourceVector.of(1); + resourceVectorOther.subtract(resourceVectorOne); + + Assert.assertTrue(resourceVector.equals(resourceVectorOther)); + } +} \ 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/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 747eea2a83874..d3518a1ff527f 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 @@ -107,7 +107,8 @@ public void testAbsoluteResourceCapacityConfig() { conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + CapacitySchedulerConfiguration.CAPACITY, ABSOLUTE_RESOURCE_MEMORY_VCORE); QueueCapacityVector withoutGpuVector = capacityConfigParser.parse(conf, QUEUE, ""); - Assert.assertEquals(2, withoutGpuVector.getResourceCount()); + Assert.assertEquals(3, withoutGpuVector.getResourceCount()); + Assert.assertEquals(0f, withoutGpuVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); } @Test @@ -134,6 +135,17 @@ public void testMixedCapacityConfig() { mixedCapacityVector.getResource("yarn.io/gpu").getVectorResourceType()); Assert.assertEquals(6f, mixedCapacityVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + + // Test undefined capacity type default value + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY, "[memory-mb=12Gi,vcores=6]"); + + QueueCapacityVector mixedCapacityVectorWithGpuUndefined = + capacityConfigParser.parse(conf, QUEUE, ""); + Assert.assertEquals(QueueCapacityType.ABSOLUTE, + mixedCapacityVectorWithGpuUndefined.getResource("memory-mb").getVectorResourceType()); + Assert.assertEquals(0, mixedCapacityVectorWithGpuUndefined.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + } @Test @@ -146,7 +158,7 @@ public void testInvalidCapacityConfigs() { capacityConfigParser.parse(conf, QUEUE, ""); List resources = Lists.newArrayList(invalidResourceCapacity.iterator()); - Assert.assertEquals(resources.size(), 0); + Assert.assertEquals(0, resources.size()); conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + CapacitySchedulerConfiguration.CAPACITY, ""); @@ -166,4 +178,9 @@ public void testInvalidCapacityConfigs() { Lists.newArrayList(nonSetCapacity.iterator()); Assert.assertEquals(nonSetResources.size(), 0); } + + @Test + public void testResourceVector() { + + } } \ No newline at end of file From ca8016c6f9536fa6e6297278f498d9a4062e7836 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 14 Oct 2021 16:27:33 +0200 Subject: [PATCH 05/27] YARN-10965. Extend --- .../yarn/util/resource/ResourceUtils.java | 11 + .../AbsoluteResourceCapacityCalculator.java | 139 ++++------- .../AbstractQueueCapacityCalculator.java | 138 ++++++++++- .../CapacitySchedulerConfiguration.java | 14 +- ...CapacitySchedulerQueueCapacityHandler.java | 70 +++--- .../PercentageQueueCapacityCalculator.java | 59 ++--- .../capacity/QueueBranchContext.java | 40 +-- .../capacity/QueueHierarchyUpdateContext.java | 22 +- .../capacity/QueueUpdateWarning.java | 26 ++ .../capacity/RootQueueCapacityCalculator.java | 24 +- .../WeightQueueCapacityCalculator.java | 101 +++++--- .../nodelabels/NullRMNodeLabelsManager.java | 22 ++ ...citySchedulerQueueCalculationTestBase.java | 228 ++++++++++++++++++ .../TestMixedQueueResourceCalculation.java | 91 +++++++ .../TestQueueResourceCalculation.java | 109 --------- .../TestUniformQueueResourceCalculation.java | 146 +++++++++++ 16 files changed, 872 insertions(+), 368 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueUpdateWarning.java create mode 100644 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 create mode 100644 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 delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueResourceCalculation.java create mode 100644 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 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java index 3654965c05df1..312427ef87b4d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java @@ -818,6 +818,17 @@ public static Resource createResourceWithSameValue(long value) { return res; } + public static Resource multiply(Resource lhs, float rhs) { + Resource newResource = Resource.newInstance(0, 0); + + for (ResourceInformation resourceInformation : lhs.getResources()) { + newResource.setResourceValue(resourceInformation.getName(), + (long) Math.floor(resourceInformation.getValue() * rhs)); + } + + return newResource; + } + @InterfaceAudience.Private @InterfaceStability.Unstable public static Resource createResourceFromString( 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index ec516696562e8..d16aa0e3e3b31 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -1,62 +1,59 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import org.apache.hadoop.yarn.util.UnitsConversionUtil; - -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { + public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { + super.calculateChildQueueResources(updateContext, parentQueue); + + iterateThroughChildrenResources(parentQueue, updateContext, ((childQueue, label, capacityVectorEntry) -> { + String resourceName = capacityVectorEntry.getResourceName(); + ResourceVector ratio = updateContext.getNormalizedMinResourceRatio( + parentQueue.getQueuePath(), label); + + long resource = (long) Math.floor(ratio.getValue(resourceName) + * capacityVectorEntry.getResourceValue()); + long parentResource = parentQueue.getEffectiveCapacity(label) + .getResourceValue(resourceName); + if (resource > parentResource) { + updateContext.addUpdateWarning( + QueueUpdateWarning.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath())); + resource = parentResource; + } + + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(resourceName, resource); + float absolutePercentage = (float) childQueue.getQueueResourceQuotas() + .getEffectiveMinResource(label).getResourceValue(resourceName) + / updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName); + + updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), + label).setValue(resourceName, absolutePercentage); + return resource; + })); } @Override - protected QueueCapacityVector.QueueCapacityType getCapacityType() { - return null; + protected QueueCapacityType getCapacityType() { + return QueueCapacityType.ABSOLUTE; } @Override - public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String label) { + public void setup(CSQueue queue, String label) { Resource minResource = Resource.newInstance(0, 0); - for (String resourceName : queue.getConfiguredCapacityVector(label).getResourceNamesByCapacityType(QueueCapacityVector.QueueCapacityType.ABSOLUTE)) { - minResource.setResourceValue(resourceName, (long) queue.getConfiguredCapacityVector( - label).getResource(resourceName).getResourceValue()); - } - - queue.getQueueResourceQuotas().setEffectiveMinResource(label, minResource); - } - - public void calculateResources(QueueHierarchyUpdateContext updateContext, - CSQueue queue, - QueueCapacityVectorEntry configuredCapacityResource, - String label) { - if (!CollectionUtils.isEmpty(queue.getChildQueues()) && queue instanceof AbstractCSQueue) { - defineNormalizedResourceRatio(updateContext, queue, configuredCapacityResource, label); - } - - ResourceVector ratio = ResourceVector.of(1f); - - if (!queue.getQueuePath().equals(ROOT)) { - ratio = updateContext.getNormalizedMinResourceRatio( - queue.getParent().getQueuePath(), label); + for (String resourceName : getResourceNames(queue, label)) { + long resource = (long) queue.getConfiguredCapacityVector( + label).getResource(resourceName).getResourceValue(); + minResource.setResourceValue(resourceName, minResource.getResourceValue( + resourceName) + resource); } - queue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(configuredCapacityResource.getResourceName(), - (long) (ratio.getValue(configuredCapacityResource.getResourceName()) * configuredCapacityResource.getResourceValue())); - - float relativeToParentPercentage = (float) queue.getQueueResourceQuotas() - .getEffectiveMinResource(label).getResourceValue( - configuredCapacityResource.getResourceName()) - / queue.getParent().getQueueResourceQuotas().getEffectiveMinResource( - label).getResourceValue(configuredCapacityResource.getResourceName()); - - updateContext.getRelativeResourceRatio(queue.getQueuePath(), label).setValue(configuredCapacityResource.getResourceName(), relativeToParentPercentage); + queue.getQueueResourceQuotas().setConfiguredMinResource(label, minResource); } @Override @@ -65,63 +62,17 @@ public void setMetrics(QueueHierarchyUpdateContext updateContext, float sumCapacity = 0f; float sumAbsoluteCapacity = 0f; - for (String resourceName : queue.getConfiguredCapacityVector(label).getResourceNamesByCapacityType(QueueCapacityVector.QueueCapacityType.ABSOLUTE)) { - sumCapacity += updateContext.getRelativeResourceRatio(queue.getQueuePath(), label).getValue(resourceName); - sumAbsoluteCapacity += (float) queue.getQueueResourceQuotas().getEffectiveMinResource(label).getResourceValue(resourceName) - / updateContext.getUpdatedClusterResource().getResourceValue(resourceName); + for (String resourceName : getResourceNames(queue, label)) { + sumCapacity += queue.getConfiguredCapacityVector(label).getResource( + resourceName).getResourceValue() / queue.getParent() + .getQueueResourceQuotas().getEffectiveMinResource(label) + .getResourceValue(resourceName); + sumAbsoluteCapacity += updateContext.getRelativeResourceRatio( + queue.getQueuePath(), label).getValue(resourceName); } queue.getQueueCapacities().setCapacity(label, sumCapacity); queue.getQueueCapacities().setAbsoluteCapacity(label, sumAbsoluteCapacity); } - private void defineNormalizedResourceRatio( - QueueHierarchyUpdateContext updateContext, CSQueue queue, - QueueCapacityVectorEntry configuredCapacityResource, - String label - ) { - AbstractCSQueue abstractQueue = (AbstractCSQueue) queue; - - Resource resourceByLabel = abstractQueue.labelManager.getResourceByLabel( - label, updateContext.getUpdatedClusterResource()); - - float childrenConfiguredResource = 0; - long effectiveMinResource = queue.getQueueResourceQuotas().getEffectiveMinResource(label).getResourceValue(configuredCapacityResource.getResourceName()); - long labeledResource = resourceByLabel.getResourceValue(configuredCapacityResource.getResourceName()); - String units = ""; - - // Total configured min resources of direct children of this given parent - // queue - - for (CSQueue childQueue : queue.getChildQueues()) { - units = childQueue.getQueueResourceQuotas().getConfiguredMinResource( - label).getResourceInformation( - configuredCapacityResource.getResourceName()).getUnits(); - childrenConfiguredResource += childQueue.getQueueResourceQuotas() - .getConfiguredMinResource(label).getResourceValue( - configuredCapacityResource.getResourceName()); - } - - // Factor to scale down effective resource: When cluster has sufficient - // resources, effective_min_resources will be same as configured - // min_resources. - float numeratorForMinRatio = labeledResource; - - if (!abstractQueue.getQueuePath().equals(ROOT)) { - if (effectiveMinResource > childrenConfiguredResource) { - numeratorForMinRatio = queue.getQueueResourceQuotas() - .getEffectiveMinResource(label).getResourceValue( - configuredCapacityResource.getResourceName()); - } - } - - long convertedValue = UnitsConversionUtil.convert(units, - resourceByLabel.getResourceInformation(configuredCapacityResource.getResourceName()).getUnits(), - (long) childrenConfiguredResource); - - if (convertedValue != 0) { - updateContext.getNormalizedMinResourceRatio(queue.getQueuePath(), label) - .setValue(configuredCapacityResource.getResourceName(), numeratorForMinRatio / convertedValue); - } - } } 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 2db3035414fbb..a0db8dca4a374 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -19,7 +19,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.util.UnitsConversionUtil; +import java.util.HashMap; +import java.util.Map; import java.util.Set; /** @@ -31,24 +35,33 @@ public abstract class AbstractQueueCapacityCalculator { /** * Sets all field of the queue based on its configurations. * @param queue queue to setup - * @param conf configuration the setup is based on * @param label node label */ public abstract void setup( - CSQueue queue, CapacitySchedulerConfiguration conf, String label); + CSQueue queue, String label); /** * Calculate the effective resource for a specific resource. * @param updateContext context of the current update phase * @param parentQueue the parent whose children will be updated - * @param label node label */ - public abstract void calculateChildQueueResources( + public void calculateChildQueueResources( QueueHierarchyUpdateContext updateContext, - CSQueue parentQueue, String label); + CSQueue parentQueue) { + for (String label : parentQueue.getConfiguredNodeLabels()) { + // We need to set normalized resource ratio only once, not for each + // resource calculator + if (!updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).isParentAlreadyUpdated()) { + setNormalizedResourceRatio(updateContext, parentQueue, label); + updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .setUpdateFlag(); + } + } + } /** - * Set the necessary metrics and statistics. + * Sets the metrics and statistics after effective resource calculation. * @param updateContext context of the current update phase * @param queue queue to update * @param label node label @@ -70,7 +83,118 @@ public abstract void setMetrics( * @return resource names */ protected Set getResourceNames(CSQueue queue, String label) { + return getResourceNames(queue, label, getCapacityType()); + } + + /** + * Returns all resource names that are defined for a capacity type. + * @param queue queue for which the capacity vector is defined + * @param label node label + * @param capacityType capacity type for which the resource names are defined + * @return resource names + */ + protected Set getResourceNames(CSQueue queue, String label, + QueueCapacityType capacityType) { return queue.getConfiguredCapacityVector(label) - .getResourceNamesByCapacityType(getCapacityType()); + .getResourceNamesByCapacityType(capacityType); + } + + protected void setNormalizedResourceRatio( + QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, + String label + ) { + for (QueueCapacityVectorEntry capacityVectorEntry : + parentQueue.getConfiguredCapacityVector(label)) { + long childrenConfiguredResource = 0; + long effectiveMinResource = parentQueue.getQueueResourceQuotas() + .getEffectiveMinResource(label).getResourceValue( + capacityVectorEntry.getResourceName()); + + // Total configured min resources of direct children of this given parent + // queue + for (CSQueue childQueue : parentQueue.getChildQueues()) { + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + if (capacityVector.isResourceOfType( + capacityVectorEntry.getResourceName(), QueueCapacityType.ABSOLUTE)) { + childrenConfiguredResource += capacityVector.getResource( + capacityVectorEntry.getResourceName()).getResourceValue(); + } + } + + // If no children is using ABSOLUTE capacity type, normalization is + // not needed + if (childrenConfiguredResource == 0) { + continue; + } + + updateContext.addUpdateWarning(QueueUpdateWarning.BRANCH_DOWNSCALED.ofQueue( + parentQueue.getQueuePath())); + + // Factor to scale down effective resource: When cluster has sufficient + // resources, effective_min_resources will be same as configured + // min_resources. + float numeratorForMinRatio = childrenConfiguredResource; + if (effectiveMinResource < childrenConfiguredResource) { + numeratorForMinRatio = parentQueue.getQueueResourceQuotas() + .getEffectiveMinResource(label).getResourceValue( + capacityVectorEntry.getResourceName()); + } + + String unit = capacityVectorEntry.getResourceName().equals("memory-mb") + ? "Mi" : ""; + long convertedValue = UnitsConversionUtil.convert(unit, + updateContext.getUpdatedClusterResource(label).getResourceInformation( + capacityVectorEntry.getResourceName()).getUnits(), + childrenConfiguredResource); + + if (convertedValue != 0) { + updateContext.getNormalizedMinResourceRatio(parentQueue.getQueuePath(), label) + .setValue(capacityVectorEntry.getResourceName(), + numeratorForMinRatio / convertedValue); + } + } + } + + protected float sumCapacityValues(CSQueue queue, String label) { + float sumValue = 0f; + QueueCapacityVector capacityVector = + queue.getConfiguredCapacityVector(label); + for (String resourceName : getResourceNames(queue, label)) { + sumValue += capacityVector.getResource(resourceName).getResourceValue(); + } + + return sumValue; + } + + protected void iterateThroughChildrenResources( + CSQueue parentQueue, QueueHierarchyUpdateContext updateContext, + ChildrenResourceCalculator callable) { + Map aggregatedResources = new HashMap<>(); + for (CSQueue childQueue : parentQueue.getChildQueues()) { + for (String label : childQueue.getConfiguredNodeLabels()) { + ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault( + label, ResourceVector.newInstance()); + for (String resourceName : getResourceNames(childQueue, label)) { + float resource = callable.call(childQueue, label, childQueue + .getConfiguredCapacityVector(label).getResource(resourceName)); + if (resource == 0) { + updateContext.addUpdateWarning(QueueUpdateWarning. + QUEUE_ZERO_RESOURCE.ofQueue(childQueue.getQueuePath())); + } + aggregatedUsedResource.increment(resourceName, resource); + } + aggregatedResources.put(label, aggregatedUsedResource); + } + } + + for (Map.Entry entry : aggregatedResources.entrySet()){ + updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .getRemainingResource(entry.getKey()).subtract(entry.getValue()); + } + } + + protected interface ChildrenResourceCalculator { + float call(CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry); } } 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 d47f6ab75bb68..b2a32505dd36a 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 @@ -542,8 +542,10 @@ public float getNonLabeledQueueCapacity(String queue) { String configuredCapacity = get(getQueuePrefix(queue) + CAPACITY); boolean absoluteResourceConfigured = (configuredCapacity != null) && RESOURCE_PATTERN.matcher(configuredCapacity).find(); + boolean isCapacityVectorFormat = queueCapacityConfigParser + .isCapacityVectorFormat(configuredCapacity); if (absoluteResourceConfigured || configuredWeightAsCapacity( - configuredCapacity)) { + configuredCapacity) || isCapacityVectorFormat) { // Return capacity in percentage as 0 for non-root queues and 100 for // root.From AbstractCSQueue, absolute resource will be parsed and // updated. Once nodes are added/removed in cluster, capacity in @@ -777,6 +779,11 @@ public Set getAccessibleNodeLabels(String queue) { return Collections.unmodifiableSet(set); } + public void setCapacityVector(String queuePath, String label, String capacityVector) { + String capacityPropertyName = getNodeLabelPrefix(queuePath, label) + CAPACITY; + set(capacityPropertyName, capacityVector); + } + private boolean configuredWeightAsCapacity(String configureValue) { if (configureValue == null) { return false; @@ -2676,6 +2683,11 @@ private void updateResourceValuesFromConfig(Set resourceTypes, } String units = getUnits(splits[1]); + + if (!UnitsConversionUtil.KNOWN_UNITS.contains(units)) { + return; + } + Long resourceValue = Long .valueOf(splits[1].substring(0, splits[1].length() - units.length())); 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index 09488b42121df..8a9344fa675fa 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -20,14 +20,14 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSortedSet; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Collection; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -40,9 +40,14 @@ */ public class CapacitySchedulerQueueCapacityHandler { + private static final Logger LOG = + LoggerFactory.getLogger(CapacitySchedulerQueueCapacityHandler.class); + private static final Set CALCULATOR_PRECEDENCE = - ImmutableSortedSet.of( - QueueCapacityType.PERCENTAGE); + ImmutableSet.of( + QueueCapacityType.ABSOLUTE, + QueueCapacityType.PERCENTAGE, + QueueCapacityType.WEIGHT); private final Map calculators; @@ -50,13 +55,17 @@ public class CapacitySchedulerQueueCapacityHandler { new RootQueueCapacityCalculator(); private QueueHierarchyUpdateContext lastUpdateContext; - public CapacitySchedulerQueueCapacityHandler() { + public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) { this.calculators = new HashMap<>(); this.lastUpdateContext = new QueueHierarchyUpdateContext( - Resource.newInstance(0, 0)); + Resource.newInstance(0, 0), labelsManager); + this.calculators.put(QueueCapacityType.ABSOLUTE, + new AbsoluteResourceCapacityCalculator()); this.calculators.put(QueueCapacityType.PERCENTAGE, new PercentageQueueCapacityCalculator()); + this.calculators.put(QueueCapacityType.WEIGHT, + new WeightQueueCapacityCalculator()); } /** @@ -64,15 +73,14 @@ public CapacitySchedulerQueueCapacityHandler() { * These values are not calculated but defined at configuration time. * * @param queue queue to set capacity config values to - * @param conf configuration from which the values are derived */ - public void setup(CSQueue queue, CapacitySchedulerConfiguration conf) { + public void setup(CSQueue queue) { for (String label : queue.getConfiguredNodeLabels()) { for (QueueCapacityType capacityType : queue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); - calculator.setup(queue, conf, label); + calculator.setup(queue, label); } } } @@ -84,7 +92,7 @@ public void setup(CSQueue queue, CapacitySchedulerConfiguration conf) { * @param clusterResource resource of the cluster * @param queue queue to update */ - public void update(Resource clusterResource, CSQueue queue) { + public QueueHierarchyUpdateContext update(Resource clusterResource, CSQueue queue) { QueueHierarchyUpdateContext newContext = new QueueHierarchyUpdateContext(clusterResource, lastUpdateContext); this.lastUpdateContext = newContext; @@ -94,6 +102,8 @@ public void update(Resource clusterResource, CSQueue queue) { } update(queue, newContext); + + return newContext; } private void update( @@ -102,10 +112,19 @@ private void update( return; } - collectCapacities(queueHierarchyContext, parent); calculateResources(queueHierarchyContext, parent, CALCULATOR_PRECEDENCE.stream().map((calculators::get)) .collect(Collectors.toList())); + + for (String label : parent.getConfiguredNodeLabels()) { + if (!queueHierarchyContext.getQueueBranchContext(parent.getQueuePath()) + .getRemainingResource(label).equals(ResourceVector.newInstance())) { + queueHierarchyContext.addUpdateWarning( + QueueUpdateWarning.BRANCH_UNDERUTILIZED.ofQueue( + parent.getQueuePath())); + } + } + updateChildren(queueHierarchyContext, parent); } @@ -116,7 +135,7 @@ private void calculateResources( queueHierarchyContext.getQueueBranchContext(parent.getQueuePath()) .setRemainingResource(label, ResourceVector.of(parent.getEffectiveCapacity(label))); for (AbstractQueueCapacityCalculator calculator : usableCalculators) { - calculator.calculateChildQueueResources(queueHierarchyContext, parent, label); + calculator.calculateChildQueueResources(queueHierarchyContext, parent); } setMetrics(parent, label); } @@ -131,31 +150,6 @@ private void updateChildren( } } - /** - * Collects capacity values of all queue on the same level for each resource. - * - * @param queueHierarchyContext update context of the queue hierarchy - * @param parent parent of the branch - */ - private void collectCapacities( - QueueHierarchyUpdateContext queueHierarchyContext, CSQueue parent) { - List siblingsOfLevel = parent.getChildQueues(); - if (CollectionUtils.isEmpty(siblingsOfLevel)) { - return; - } - - for (CSQueue queue : siblingsOfLevel) { - for (String label : queue.getConfiguredNodeLabels()) { - for (QueueCapacityVectorEntry capacityResource : - queue.getConfiguredCapacityVector(label)) { - queueHierarchyContext.getQueueBranchContext( - parent.getQueuePath()).getSumByLabel(label) - .increment(capacityResource); - } - } - } - } - private void setMetrics(CSQueue parent, String label) { for (CSQueue childQueue : parent.getChildQueues()) { for (QueueCapacityType capacityType : 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index 35db4f5be53b3..e14fcfde5415c 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -1,14 +1,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import java.util.Set; public class PercentageQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String label) { + public void setup(CSQueue queue, String label) { float sumCapacity = 0f; QueueCapacityVector capacityVector = queue.getConfiguredCapacityVector(label); @@ -19,40 +18,34 @@ public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String lab @Override public void calculateChildQueueResources( - QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { - ResourceVector aggregatedUsedResource = new ResourceVector(); - for (CSQueue childQueue : parentQueue.getChildQueues()) { - for (String resourceName : getResourceNames(childQueue, label)) { - QueueCapacityVectorEntry configuredCapacityResource = childQueue - .getConfiguredCapacityVector(label).getResource(resourceName); + QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { + super.calculateChildQueueResources(updateContext, parentQueue); - float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( - parentQueue.getQueuePath(), label).getValue(resourceName); - float queueAbsoluteCapacity = parentAbsoluteCapacity - * configuredCapacityResource.getResourceValue() / 100; - float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).getRemainingResource(label) - .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) - .getResourceValue(resourceName); - queueAbsoluteCapacity *= remainingPerEffectiveResourceRatio; + iterateThroughChildrenResources( + parentQueue, updateContext, ((childQueue, label, capacityVectorEntry) -> { + String resourceName = capacityVectorEntry.getResourceName(); + float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( + parentQueue.getQueuePath(), label).getValue(resourceName); + float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).getRemainingResource(label) + .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) + .getResourceValue(resourceName); + float queueAbsoluteCapacity = parentAbsoluteCapacity * + remainingPerEffectiveResourceRatio + * capacityVectorEntry.getResourceValue() / 100; - long resource = Math.round(updateContext.getUpdatedClusterResource() - .getResourceValue(configuredCapacityResource.getResourceName()) - * queueAbsoluteCapacity); - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(configuredCapacityResource.getResourceName(), - resource); + long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) + .getResourceValue(capacityVectorEntry.getResourceName()) + * queueAbsoluteCapacity); + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(capacityVectorEntry.getResourceName(), + resource); - updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) - .setValue(configuredCapacityResource.getResourceName(), - queueAbsoluteCapacity); - aggregatedUsedResource.setValue( - configuredCapacityResource.getResourceName(), resource); - } - } - - updateContext.getQueueBranchContext(parentQueue.getQueuePath()) - .getRemainingResource(label).subtract(aggregatedUsedResource); + updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) + .setValue(capacityVectorEntry.getResourceName(), + queueAbsoluteCapacity); + return resource; + })); } @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/QueueBranchContext.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/QueueBranchContext.java index ac3f34117b9b3..bc2e824b6a205 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/QueueBranchContext.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/QueueBranchContext.java @@ -30,21 +30,8 @@ * branch (all siblings that have a common parent). */ public class QueueBranchContext { - private final Map sumCapacitiesByLabel = new HashMap<>(); private final Map remainingResourceByLabel = new HashMap<>(); - - /** - * Returns summarized capacity values for a label. - * @param label node label - * @return summarized capacity values - */ - public CapacitySum getSumByLabel(String label) { - if (!sumCapacitiesByLabel.containsKey(label)) { - sumCapacitiesByLabel.put(label, new CapacitySum()); - } - - return sumCapacitiesByLabel.get(label); - } + private boolean isUpdated = false; public void setRemainingResource(String label, ResourceVector resource) { remainingResourceByLabel.put(label, resource); @@ -60,26 +47,11 @@ public ResourceVector getRemainingResource(String label) { return remainingResourceByLabel.get(label); } - public static class CapacitySum { - private Map> sum = new HashMap<>(); - - public void increment(QueueCapacityVectorEntry resource) { - Map sumByResourceName; - - if (sum.containsKey(resource.getResourceName())) { - sumByResourceName = sum.get(resource.getResourceName()); - } else { - sumByResourceName = new HashMap<>(); - sum.put(resource.getResourceName(), sumByResourceName); - } - - sumByResourceName.put(resource.getVectorResourceType(), sumByResourceName.getOrDefault( - resource.getVectorResourceType(), 0f) + resource.getResourceValue()); - } + public void setUpdateFlag() { + isUpdated = true; + } - public float getSum(String resourceName, QueueCapacityType resourceType) { - return sum.getOrDefault(resourceName, - Collections.emptyMap()).getOrDefault(resourceType, 0f); - } + public boolean isParentAlreadyUpdated() { + return isUpdated; } } 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/QueueHierarchyUpdateContext.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/QueueHierarchyUpdateContext.java index e4718251391e2..51e42701a691d 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/QueueHierarchyUpdateContext.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/QueueHierarchyUpdateContext.java @@ -20,8 +20,11 @@ import org.apache.commons.collections.map.LazyMap; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; /** @@ -34,10 +37,12 @@ public class QueueHierarchyUpdateContext { private final Map queueBranchContext = LazyMap.decorate(new HashMap(), QueueBranchContext::new); + private final RMNodeLabelsManager labelsManager; private Map> normalizedResourceRatios = createLazyResourceVector(); private Map> relativeResourceRatio = createLazyResourceVector(); + private List warnings = new ArrayList<>(); public QueueHierarchyUpdateContext( Resource updatedClusterResource, @@ -47,10 +52,13 @@ public QueueHierarchyUpdateContext( .normalizedResourceRatios; this.relativeResourceRatio = queueHierarchyUpdateContext .relativeResourceRatio; + this.labelsManager = queueHierarchyUpdateContext.labelsManager; } - public QueueHierarchyUpdateContext(Resource updatedClusterResource) { + public QueueHierarchyUpdateContext(Resource updatedClusterResource, + RMNodeLabelsManager labelsManager) { this.updatedClusterResource = updatedClusterResource; + this.labelsManager = labelsManager; } private static Map> @@ -66,8 +74,8 @@ public QueueHierarchyUpdateContext(Resource updatedClusterResource) { * Returns the overall cluster resource available for the update phase. * @return cluster resource */ - public Resource getUpdatedClusterResource() { - return updatedClusterResource; + public Resource getUpdatedClusterResource(String label) { + return labelsManager.getResourceByLabel(label, updatedClusterResource); } /** @@ -101,4 +109,12 @@ public ResourceVector getRelativeResourceRatio( String queuePath, String label) { return relativeResourceRatio.get(queuePath).get(label); } + + public void addUpdateWarning(QueueUpdateWarning warning) { + warnings.add(warning); + } + + public List getUpdateWarnings() { + return warnings; + } } 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/QueueUpdateWarning.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/QueueUpdateWarning.java new file mode 100644 index 0000000000000..119e9f270e678 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueUpdateWarning.java @@ -0,0 +1,26 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +public enum QueueUpdateWarning { + BRANCH_UNDERUTILIZED("Remaining resource found in branch under parent queue '%s'."), + QUEUE_OVERUTILIZED("Queue '%' is configured to use more resources than what is available under its parent."), + QUEUE_ZERO_RESOURCE("Queue '%s' is assigned zero resource."), + BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are downscaled due to insufficient cluster resource."); + + private final String message; + private String queue; + + QueueUpdateWarning(String message) { + this.message = message; + } + + public QueueUpdateWarning ofQueue(String queuePath) { + this.queue = queuePath; + + return this; + } + + @Override + public String toString() { + return String.format(message, queue); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index 2e4e7e6bd0e4a..f0bb58c0bfb9e 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -27,21 +27,25 @@ public class RootQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String label) { + public void setup(CSQueue queue, String label) { queue.getQueueCapacities().setCapacity(label, 100f); } @Override public void calculateChildQueueResources( - QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { - for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { - updateContext.getRelativeResourceRatio(ROOT, label).setValue( - resource.getName(), 1); - - parentQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(resource.getName(), - updateContext.getUpdatedClusterResource().getResourceValue( - resource.getName())); + QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { + for (String label : parentQueue.getConfiguredNodeLabels()) { + for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { + updateContext.getRelativeResourceRatio(ROOT, label).setValue( + resource.getName(), 1); + + parentQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(resource.getName(), + updateContext.getUpdatedClusterResource(label).getResourceValue( + resource.getName())); + } + + setNormalizedResourceRatio(updateContext, parentQueue, 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 07038c81a346c..16c3c58dd2401 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -1,64 +1,87 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.PERCENTAGE; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; + +import java.util.HashMap; +import java.util.Map; + import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.WEIGHT; public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { + public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { + Map> sumWeightsPerLabel = summarizeWeights(parentQueue); - } - - @Override - protected QueueCapacityVector.QueueCapacityType getCapacityType() { - return null; - } + iterateThroughChildrenResources(parentQueue, updateContext, + ((childQueue, label, capacityVectorEntry) -> { + String resourceName = capacityVectorEntry.getResourceName(); + float normalizedWeight = capacityVectorEntry.getResourceValue() + / sumWeightsPerLabel.get(label).get(resourceName); - @Override - public void setup(CSQueue queue, CapacitySchedulerConfiguration conf, String label) { - float sumWeight = 0; - QueueCapacityVector capacityVector = queue.getConfiguredCapacityVector(label); - for (String resourceName : capacityVector.getResourceNamesByCapacityType(WEIGHT)) { - sumWeight += capacityVector.getResource(resourceName).getResourceValue(); - } + float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).getRemainingResource(label) + .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) + .getResourceValue(resourceName); - queue.getQueueCapacities().setWeight(label, sumWeight); - } + float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( + parentQueue.getQueuePath(), label).getValue(resourceName); + float queueAbsoluteCapacity = parentAbsoluteCapacity * + remainingPerEffectiveResourceRatio * normalizedWeight; + long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) + .getResourceValue(resourceName) + * queueAbsoluteCapacity); - public void calculateResources(QueueHierarchyUpdateContext updateContext, CSQueue queue, QueueCapacityVectorEntry configuredCapacityResource, String label) { - QueueBranchContext.CapacitySum capacitySum = updateContext.getQueueBranchContext(queue.getParent().getQueuePath()).getSumByLabel(label); + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(capacityVectorEntry.getResourceName(), + resource); - float capacityMultiplier = 1 - capacitySum.getSum(configuredCapacityResource.getResourceName(), PERCENTAGE); - float normalizedWeight = configuredCapacityResource.getResourceValue() / - capacitySum.getSum(configuredCapacityResource.getResourceName(), WEIGHT) - * capacityMultiplier; + updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) + .setValue(capacityVectorEntry.getResourceName(), + queueAbsoluteCapacity); - float parentAbsoluteCapacity = queue.getParent() != null ? - updateContext.getRelativeResourceRatio( - queue.getParent().getQueuePath(), label).getValue( - configuredCapacityResource.getResourceName()) : 1; - float queueAbsoluteCapacity = parentAbsoluteCapacity * normalizedWeight; - long resource = Math.round(queueAbsoluteCapacity * - updateContext.getUpdatedClusterResource().getResourceValue( - configuredCapacityResource.getResourceName())); + return resource; + })); + } - queue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(configuredCapacityResource.getResourceName(), resource); + @Override + protected QueueCapacityType getCapacityType() { + return WEIGHT; + } - updateContext.getRelativeResourceRatio(queue.getQueuePath(), label).setValue( - configuredCapacityResource.getResourceName(), queueAbsoluteCapacity); + @Override + public void setup(CSQueue queue, String label) { + queue.getQueueCapacities().setWeight(label, sumCapacityValues(queue, label)); } @Override - public void setMetrics(QueueHierarchyUpdateContext updateContext, CSQueue queue, String label) { + public void setMetrics(QueueHierarchyUpdateContext updateContext, + CSQueue queue, String label) { float sumNormalizedWeight = 0; - for (String resourceName : queue.getConfiguredCapacityVector(label).getResourceNamesByCapacityType(WEIGHT)) { - sumNormalizedWeight += updateContext.getRelativeResourceRatio(queue.getQueuePath(), label).getValue(resourceName); + for (String resourceName : getResourceNames(queue, label)) { + sumNormalizedWeight += updateContext.getRelativeResourceRatio( + queue.getQueuePath(), label).getValue(resourceName); } queue.getQueueCapacities().setNormalizedWeight(label, sumNormalizedWeight); } + private Map> summarizeWeights(CSQueue parentQueue) { + Map> sumWeightsPerResource = new HashMap<>(); + + for (String label : parentQueue.getConfiguredNodeLabels()) { + Map sumWeight = new HashMap<>(); + + for (CSQueue childQueue : parentQueue.getChildQueues()) { + for (String resourceName : childQueue.getConfiguredCapacityVector(label).getResourceNamesByCapacityType(getCapacityType())) { + sumWeight.put(resourceName, sumWeight.getOrDefault(resourceName, 0f) + + childQueue.getConfiguredCapacityVector(label).getResource(resourceName).getResourceValue()); + } + } + sumWeightsPerResource.put(label, sumWeight); + } + + return sumWeightsPerResource; + } + } 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/NullRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java index b8f3fae7da66e..180c2d6442c29 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java @@ -27,9 +27,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.nodelabels.NodeLabelsStore; +import org.apache.hadoop.yarn.nodelabels.RMNodeLabel; public class NullRMNodeLabelsManager extends RMNodeLabelsManager { Map> lastNodeToLabels = null; @@ -98,4 +100,24 @@ protected void serviceInit(Configuration conf) throws Exception { conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); super.serviceInit(conf); } + + public void setResourceForLabel(String label, Resource resource) { + if (label.equals(NO_LABEL)) { + noNodeLabel = new FakeLabel(resource); + return; + } + + labelCollections.put(label, new FakeLabel(label, resource)); + } + + private static class FakeLabel extends RMNodeLabel { + + FakeLabel(String label, Resource resource) { + super(label, resource, 1, false); + } + + FakeLabel(Resource resource) { + super(NO_LABEL, resource, 1, false); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/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 new file mode 100644 index 0000000000000..0d54a4a5f2ed8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueCalculationTestBase.java @@ -0,0 +1,228 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.junit.Assert; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; + +public class CapacitySchedulerQueueCalculationTestBase { + + protected static class QueueAssertionBuilder { + public static final String EFFECTIVE_MIN_RES_INFO = "Effective Minimum Resource"; + public static final Function EFFECTIVE_MIN_RES = + QueueResourceQuotas::getEffectiveMinResource; + + public static final String CAPACITY_INFO = "Capacity"; + public static final Function CAPACITY = + QueueCapacities::getCapacity; + + public static final String ABS_CAPACITY_INFO = "Absolute Capacity"; + public static final Function ABS_CAPACITY = + QueueCapacities::getAbsoluteCapacity; + + private static final String ASSERTION_ERROR_MESSAGE = + "'%s' of queue '%s' does not match %f"; + private static final String RESOURCE_ASSERTION_ERROR_MESSAGE = + "'%s' of queue '%s' does not match %s"; + private final CapacityScheduler cs; + + QueueAssertionBuilder(CapacityScheduler cs) { + this.cs = cs; + } + + public class QueueAssertion { + + public class ValueAssertion { + private float expectedValue = 0; + private Resource expectedResource = null; + private String assertionType; + private Supplier valueSupplier; + private Supplier resourceSupplier; + + ValueAssertion(float expectedValue) { + this.expectedValue = expectedValue; + } + + ValueAssertion(Resource expectedResource) { + this.expectedResource = expectedResource; + } + + public QueueAssertion assertEffectiveMinResource() { + return withResourceSupplier(EFFECTIVE_MIN_RES, EFFECTIVE_MIN_RES_INFO); + } + + public QueueAssertion assertCapacity() { + return withCapacitySupplier(CAPACITY, CAPACITY_INFO); + } + + public QueueAssertion assertAbsoluteCapacity() { + return withCapacitySupplier(ABS_CAPACITY, ABS_CAPACITY_INFO); + } + + public QueueAssertion withResourceSupplier( + Function assertion, String messageInfo) { + CSQueue queue = cs.getQueue(queuePath); + if (queue == null) { + Assert.fail("Queue " + queuePath + " is not found"); + } + + assertionType = messageInfo; + resourceSupplier = () -> assertion.apply(queue.getQueueResourceQuotas()); + QueueAssertion.this.assertions.add(this); + + return QueueAssertion.this; + } + + public QueueAssertion withCapacitySupplier( + Function assertion, String messageInfo) { + CSQueue queue = cs.getQueue(queuePath); + if (queue == null) { + Assert.fail("Queue " + queuePath + " is not found"); + } + assertionType = messageInfo; + valueSupplier = () -> assertion.apply(queue.getQueueCapacities()); + QueueAssertion.this.assertions.add(this); + + return QueueAssertion.this; + } + } + + private final String queuePath; + private final List assertions = new ArrayList<>(); + + QueueAssertion(String queuePath) { + this.queuePath = queuePath; + } + + public ValueAssertion toExpect(float expected) { + return new ValueAssertion(expected); + } + + public ValueAssertion toExpect(Resource expected) { + return new ValueAssertion(expected); + } + + + public QueueAssertion withQueue(String queuePath) { + return QueueAssertionBuilder.this.withQueue(queuePath); + } + + public QueueAssertionBuilder build() { + return QueueAssertionBuilder.this.build(); + } + } + + private final Map assertions = new LinkedHashMap<>(); + + public QueueAssertionBuilder build() { + return this; + } + + public QueueAssertion withQueue(String queuePath) { + assertions.putIfAbsent(queuePath, new QueueAssertion(queuePath)); + return assertions.get(queuePath); + } + + public void finishAssertion() { + 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, + assertion.assertionType, assertionEntry.getKey(), + assertion.expectedResource.toString()); + Assert.assertEquals(errorMessage, + assertion.expectedResource, + assertion.resourceSupplier.get()); + } else { + String errorMessage = String.format(ASSERTION_ERROR_MESSAGE, + assertion.assertionType, assertionEntry.getKey(), + assertion.expectedValue); + Assert.assertEquals(errorMessage, + assertion.expectedValue, + assertion.valueSupplier.get(), + 1e-6); + } + } + } + } + + public Set getQueues() { + return assertions.keySet(); + } + } + + protected MockRM mockRM; + protected CapacityScheduler cs; + protected CapacitySchedulerConfiguration csConf; + protected NullRMNodeLabelsManager mgr; + + @Before + public void setUp() throws Exception { + csConf = new CapacitySchedulerConfiguration(); + csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + + csConf.setQueues("root", new String[]{"a", "b"}); + csConf.setNonLabeledQueueWeight("root", 1f); + csConf.setNonLabeledQueueWeight("root.a", 6f); + csConf.setNonLabeledQueueWeight("root.b", 4f); + csConf.setQueues("root.a", new String[]{"a1", "a2"}); + csConf.setNonLabeledQueueWeight("root.a.a1", 1f); + + mgr = new NullRMNodeLabelsManager(); + mgr.init(csConf); + mockRM = new MockRM(csConf) { + protected RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + cs = (CapacityScheduler) mockRM.getResourceScheduler(); + cs.updatePlacementRules(); + // Policy for new auto created queue's auto deletion when expired + mockRM.start(); + cs.start(); + mockRM.registerNode("h1:1234", 10 * GB); // label = x + } + + protected QueueHierarchyUpdateContext update( + QueueAssertionBuilder assertions, Resource resource) throws IOException { + cs.reinitialize(csConf, mockRM.getRMContext()); + + CapacitySchedulerQueueCapacityHandler queueController = + new CapacitySchedulerQueueCapacityHandler(mgr); + mgr.setResourceForLabel(CommonNodeLabelsManager.NO_LABEL, resource); + for (String queueToAssert : assertions.getQueues()) { + CSQueue queue = cs.getQueue(queueToAssert); + queueController.setup(queue); + } + + QueueHierarchyUpdateContext updateContext = + queueController.update(resource, cs.getQueue("root")); + + assertions.finishAssertion(); + + return updateContext; + } + + protected QueueAssertionBuilder createAssertionBuilder() { + return new QueueAssertionBuilder(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 new file mode 100644 index 0000000000000..80d013c3728c6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestMixedQueueResourceCalculation.java @@ -0,0 +1,91 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.QueueState; +import org.apache.hadoop.yarn.api.records.Resource; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { + + private static final Resource UPDATE_RESOURCE = Resource.newInstance(16384, 16); + + @Test + public void testMixedCapacitiesWithoutRemainingResource() throws IOException { + setupQueueHierarchyWithoutRemainingResource(); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue("root.a") + .toExpect(Resource.newInstance(2486, 9)) + .assertEffectiveMinResource() + .withQueue("root.a.a1") + .toExpect(Resource.newInstance(621, 4)) + .assertEffectiveMinResource() + .withQueue("root.a.a2") + .toExpect(Resource.newInstance(1865, 5)) + .assertEffectiveMinResource() + .withQueue("root.b") + .toExpect(Resource.newInstance(8095, 3)) + .assertEffectiveMinResource() + .withQueue("root.b.b1") + .toExpect(Resource.newInstance(8095, 3)) + .assertEffectiveMinResource() + .withQueue("root.c") + .toExpect(Resource.newInstance(5803, 4)) + .assertEffectiveMinResource() + .build(); + + QueueHierarchyUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + + Assert.assertEquals("No warning should occur", 0, + updateContext.getUpdateWarnings().size()); + } + + @Test + public void testMixedCapacitiesWithWarnings() throws IOException { + setupQueueHierarchyWithWarnings(); + QueueAssertionBuilder assertionBuilder = createAssertionBuilder(); + + QueueHierarchyUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + + } + + private void setupQueueHierarchyWithoutRemainingResource() throws IOException { + csConf.setState("root.b", QueueState.STOPPED); + cs.reinitialize(csConf, mockRM.getRMContext()); + + csConf.setQueues("root", new String[]{"a", "b", "c"}); + csConf.setQueues("root.a", new String[]{"a1", "a2"}); + csConf.setQueues("root.b", new String[]{"b1"}); + + csConf.setState("root.b", QueueState.RUNNING); + csConf.setCapacityVector("root.a", "", "[memory=30%,vcores=6w]"); + csConf.setCapacityVector("root.a.a1", "", "[memory=1w,vcores=4]"); + csConf.setCapacityVector("root.a.a2", "", "[memory=3w,vcores=100%]"); + csConf.setCapacityVector("root.b", "", "[memory=8095,vcores=30%]"); + csConf.setCapacityVector("root.b.b1", "", "[memory=5w,vcores=3]"); + csConf.setCapacityVector("root.c", "", "[memory=3w,vcores=4]"); + + cs.reinitialize(csConf, mockRM.getRMContext()); + } + + private void setupQueueHierarchyWithWarnings() throws IOException { + csConf.setState("root.b", QueueState.STOPPED); + cs.reinitialize(csConf, mockRM.getRMContext()); + + csConf.setQueues("root", new String[]{"a", "b", "c"}); + csConf.setQueues("root.a", new String[]{"a1", "a2"}); + csConf.setQueues("root.b", new String[]{"b1"}); + + csConf.setState("root.b", QueueState.RUNNING); + csConf.setCapacityVector("root.a", "", "[memory=100%,vcores=6w]"); + csConf.setCapacityVector("root.a.a1", "", "[memory=2048,vcores=4]"); + csConf.setCapacityVector("root.a.a2", "", "[memory=2048,vcores=100%]"); + csConf.setCapacityVector("root.b", "", "[memory=8096,vcores=30%]"); + csConf.setCapacityVector("root.b.b1", "", "[memory=10256,vcores=3]"); + csConf.setCapacityVector("root.c", "", "[memory=3w,vcores=4]"); + + cs.reinitialize(csConf, mockRM.getRMContext()); + } +} 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/TestQueueResourceCalculation.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/TestQueueResourceCalculation.java deleted file mode 100644 index 1bbbffaff15fb..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueResourceCalculation.java +++ /dev/null @@ -1,109 +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.api.records.Resource; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; -import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; - -public class TestQueueResourceCalculation - extends TestCapacitySchedulerAutoCreatedQueueBase { - - private CapacitySchedulerConfiguration csConf; - private static final Resource QUEUE_A_RES = Resource.newInstance(80 * GB, - 10); - private static final Resource QUEUE_B_RES = Resource.newInstance( 120 * GB, - 15); - private static final Resource QUEUE_A1_RES = Resource.newInstance(50 * GB, - 15); - private static final Resource UPDATE_RES = Resource.newInstance(250 * GB, 40); - - public void setUp() throws Exception { - csConf = new CapacitySchedulerConfiguration(); - csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, - 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", 6f); - csConf.setNonLabeledQueueWeight("root.b", 4f); - csConf.setQueues("root.a", new String[]{"a1", "a2"}); - csConf.setNonLabeledQueueWeight("root.a.a1", 1f); - - RMNodeLabelsManager mgr = new NullRMNodeLabelsManager(); - mgr.init(csConf); - mockRM = new MockRM(csConf) { - protected RMNodeLabelsManager createNodeLabelManager() { - return mgr; - } - }; - cs = (CapacityScheduler) mockRM.getResourceScheduler(); - cs.updatePlacementRules(); - // Policy for new auto created queue's auto deletion when expired - mockRM.start(); - cs.start(); - mockRM.registerNode("h1:1234", 10 * GB); // label = x - } - - @Test - public void testPercentageResourceCalculation() throws IOException { - csConf.setCapacity("root.a", 30); - csConf.setCapacity("root.b", 70); - csConf.setCapacity("root.a.a1", 17); - csConf.setCapacity("root.a.a2", 83); - cs.reinitialize(csConf, mockRM.getRMContext()); - - CapacitySchedulerQueueCapacityHandler queueController = - new CapacitySchedulerQueueCapacityHandler(); - CSQueue a = cs.getQueue("root.a"); - CSQueue b = cs.getQueue("root.b"); - CSQueue a1 = cs.getQueue("root.a.a1"); - CSQueue a2 = cs.getQueue("root.a.a2"); - - queueController.setup(a, csConf); - queueController.setup(b, csConf); - queueController.setup(a1, csConf); - queueController.setup(a2, csConf); - queueController.update(Resource.newInstance(10 * GB, 20), cs.getQueue("root")); - - Assert.assertEquals(0.3 * 10 * GB, a.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize(), 1e-6); - Assert.assertEquals(0.3, a.getQueueCapacities().getAbsoluteCapacity(), 1e-6); - Assert.assertEquals(0.3, a.getQueueCapacities().getCapacity(), 1e-6); - - Assert.assertEquals(0.7 * 10 * GB, b.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize(), 1e-6); - Assert.assertEquals(0.7, b.getQueueCapacities().getAbsoluteCapacity(), 1e-6); - Assert.assertEquals(0.7, b.getQueueCapacities().getCapacity(), 1e-6); - - Assert.assertEquals(Math.round(0.3 * 0.17 * 10 * GB), a1.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize()); - Assert.assertEquals(0.3 * 0.17, a1.getQueueCapacities().getAbsoluteCapacity(), 1e-6); - Assert.assertEquals(0.17, a1.getQueueCapacities().getCapacity(), 1e-6); - - Assert.assertEquals(Math.round(0.3 * 0.83 * 10 * GB), a2.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize()); - Assert.assertEquals(0.3 * 0.83, a2.getQueueCapacities().getAbsoluteCapacity(), 1e-6); - Assert.assertEquals(0.83, a2.getQueueCapacities().getCapacity(), 1e-6); - } -} \ 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/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 new file mode 100644 index 0000000000000..27f83322ba95c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUniformQueueResourceCalculation.java @@ -0,0 +1,146 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; + +public class TestUniformQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { + + private static final Resource QUEUE_A_RES = Resource.newInstance(80 * GB, + 15); + private static final Resource QUEUE_B_RES = Resource.newInstance( 170 * GB, + 25); + private static final Resource QUEUE_A1_RES = Resource.newInstance(50 * GB, + 5); + private static final Resource QUEUE_A2_RES = Resource.newInstance(30 * GB, + 5); + private static final Resource UPDATE_RES = Resource.newInstance(250 * GB, 40); + private static final Resource PERCENTAGE_ALL_RES = Resource.newInstance(10 * GB, 20); + private static final Resource WEIGHT_ALL_RES = Resource.newInstance(10 * GB, 20); + + @Test + public void testWeightResourceCalculation() throws IOException { +// CapacitySchedulerQueueCapacityHandler queueController = +// new CapacitySchedulerQueueCapacityHandler(mgr); +// update(WEIGHT_ALL_RES); +// queueController.update(WEIGHT_ALL_RES, cs.getQueue("root.a")); +// CSQueue a = cs.getQueue("root.a"); +// +// Assert.assertEquals(6 * GB, a.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize(), 1e-6); + } + + @Test + public void testPercentageResourceCalculation() throws IOException { + csConf.setCapacity("root.a", 30); + csConf.setCapacity("root.b", 70); + csConf.setCapacity("root.a.a1", 17); + csConf.setCapacity("root.a.a2", 83); + + QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() + .withQueue("root.a") + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.3f)) + .assertEffectiveMinResource() + .toExpect(0.3f) + .assertCapacity() + .toExpect(0.3f) + .assertAbsoluteCapacity() + + .withQueue("root.b") + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.7f)) + .assertEffectiveMinResource() + .toExpect(0.7f) + .assertCapacity() + .toExpect(0.7f) + .assertAbsoluteCapacity() + + .withQueue("root.a.a1") + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.3f * 0.17f)) + .assertEffectiveMinResource() + .toExpect(0.17f) + .assertCapacity() + .toExpect(0.3f * 0.17f) + .assertAbsoluteCapacity() + + .withQueue("root.a.a2") + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.3f * 0.83f)) + .assertEffectiveMinResource() + .toExpect(0.83f) + .assertCapacity() + .toExpect(0.3f * 0.83f) + .assertAbsoluteCapacity() + .build(); + + update(queueAssertionBuilder, PERCENTAGE_ALL_RES); + } + + @Test + public void testAbsoluteResourceCalculation() throws IOException { + csConf.setMinimumResourceRequirement("", "root.a", QUEUE_A_RES); + csConf.setMinimumResourceRequirement("", "root.b", QUEUE_B_RES); + csConf.setMinimumResourceRequirement("", "root.a.a1", QUEUE_A1_RES); + csConf.setMinimumResourceRequirement("", "root.a.a2", QUEUE_A2_RES); + + QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() + .withQueue("root.a") + .toExpect(QUEUE_A_RES) + .assertEffectiveMinResource() + + .withQueue("root.b") + .toExpect(QUEUE_B_RES) + .assertEffectiveMinResource() + + .withQueue("root.a.a1") + .toExpect(QUEUE_A1_RES) + .assertEffectiveMinResource() + + .withQueue("root.a.a2") + .toExpect(QUEUE_A2_RES) + .assertEffectiveMinResource() + .build(); + + update(queueAssertionBuilder, UPDATE_RES); + + QueueAssertionBuilder queueAssertionHalfClusterResource = createAssertionBuilder() + .withQueue("root.a") + .toExpect(ResourceUtils.multiply(QUEUE_A_RES, 0.5f)) + .assertEffectiveMinResource() + + .withQueue("root.b") + .toExpect(ResourceUtils.multiply(QUEUE_B_RES, 0.5f)) + .assertEffectiveMinResource() + + .withQueue("root.a.a1") + .toExpect(ResourceUtils.multiply(QUEUE_A1_RES, 0.5f)) + .assertEffectiveMinResource() + + .withQueue("root.a.a2") + .toExpect(ResourceUtils.multiply(QUEUE_A2_RES, 0.5f)) + .assertEffectiveMinResource() + .build(); + + update(queueAssertionHalfClusterResource, ResourceUtils.multiply(UPDATE_RES, 0.5f)); + } + +} \ No newline at end of file From 4fb353cf3eb515cb253608ee43387da7e15499cf Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 19 Oct 2021 16:48:41 +0200 Subject: [PATCH 06/27] YARN-10965. Extend 2 --- .../AbsoluteResourceCapacityCalculator.java | 8 ++----- .../AbstractQueueCapacityCalculator.java | 21 ++++++++++++++----- .../PercentageQueueCapacityCalculator.java | 5 +---- .../WeightQueueCapacityCalculator.java | 12 +++++------ .../TestMixedQueueResourceCalculation.java | 1 + 5 files changed, 25 insertions(+), 22 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index d16aa0e3e3b31..b25af74082eb0 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -9,7 +9,7 @@ public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCal public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { super.calculateChildQueueResources(updateContext, parentQueue); - iterateThroughChildrenResources(parentQueue, updateContext, ((childQueue, label, capacityVectorEntry) -> { + setChildrenResources(parentQueue, updateContext, ((childQueue, label, capacityVectorEntry) -> { String resourceName = capacityVectorEntry.getResourceName(); ResourceVector ratio = updateContext.getNormalizedMinResourceRatio( parentQueue.getQueuePath(), label); @@ -24,11 +24,7 @@ public void calculateChildQueueResources(QueueHierarchyUpdateContext updateConte resource = parentResource; } - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(resourceName, resource); - - float absolutePercentage = (float) childQueue.getQueueResourceQuotas() - .getEffectiveMinResource(label).getResourceValue(resourceName) + float absolutePercentage = (float) resource / updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName); updateContext.getRelativeResourceRatio(childQueue.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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index a0db8dca4a374..cae1f93dddf37 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -166,21 +166,32 @@ protected float sumCapacityValues(CSQueue queue, String label) { return sumValue; } - protected void iterateThroughChildrenResources( + protected void setChildrenResources( CSQueue parentQueue, QueueHierarchyUpdateContext updateContext, - ChildrenResourceCalculator callable) { + ChildResourceCalculator callable) { Map aggregatedResources = new HashMap<>(); for (CSQueue childQueue : parentQueue.getChildQueues()) { for (String label : childQueue.getConfiguredNodeLabels()) { ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault( label, ResourceVector.newInstance()); for (String resourceName : getResourceNames(childQueue, label)) { - float resource = callable.call(childQueue, label, childQueue + long parentResource = parentQueue.getEffectiveCapacity(label) + .getResourceValue(resourceName); + long resource = callable.call(childQueue, label, childQueue .getConfiguredCapacityVector(label).getResource(resourceName)); + + if (resource > parentResource) { + updateContext.addUpdateWarning( + QueueUpdateWarning.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath())); + resource = parentResource; + } if (resource == 0) { updateContext.addUpdateWarning(QueueUpdateWarning. QUEUE_ZERO_RESOURCE.ofQueue(childQueue.getQueuePath())); } + + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(resourceName, resource); aggregatedUsedResource.increment(resourceName, resource); } aggregatedResources.put(label, aggregatedUsedResource); @@ -193,8 +204,8 @@ protected void iterateThroughChildrenResources( } } - protected interface ChildrenResourceCalculator { - float call(CSQueue childQueue, String label, + protected interface ChildResourceCalculator { + long call(CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry); } } 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index e14fcfde5415c..afd5bb300aa0a 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -21,7 +21,7 @@ public void calculateChildQueueResources( QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { super.calculateChildQueueResources(updateContext, parentQueue); - iterateThroughChildrenResources( + setChildrenResources( parentQueue, updateContext, ((childQueue, label, capacityVectorEntry) -> { String resourceName = capacityVectorEntry.getResourceName(); float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( @@ -37,9 +37,6 @@ public void calculateChildQueueResources( long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) .getResourceValue(capacityVectorEntry.getResourceName()) * queueAbsoluteCapacity); - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), - resource); updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) .setValue(capacityVectorEntry.getResourceName(), 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 16c3c58dd2401..5e604488761ba 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -13,7 +13,7 @@ public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculat public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { Map> sumWeightsPerLabel = summarizeWeights(parentQueue); - iterateThroughChildrenResources(parentQueue, updateContext, + setChildrenResources(parentQueue, updateContext, ((childQueue, label, capacityVectorEntry) -> { String resourceName = capacityVectorEntry.getResourceName(); float normalizedWeight = capacityVectorEntry.getResourceValue() @@ -32,10 +32,6 @@ public void calculateChildQueueResources(QueueHierarchyUpdateContext updateConte .getResourceValue(resourceName) * queueAbsoluteCapacity); - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), - resource); - updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) .setValue(capacityVectorEntry.getResourceName(), queueAbsoluteCapacity); @@ -73,9 +69,11 @@ private Map> summarizeWeights(CSQueue parentQueue) { Map sumWeight = new HashMap<>(); for (CSQueue childQueue : parentQueue.getChildQueues()) { - for (String resourceName : childQueue.getConfiguredCapacityVector(label).getResourceNamesByCapacityType(getCapacityType())) { + for (String resourceName : childQueue.getConfiguredCapacityVector(label) + .getResourceNamesByCapacityType(getCapacityType())) { sumWeight.put(resourceName, sumWeight.getOrDefault(resourceName, 0f) - + childQueue.getConfiguredCapacityVector(label).getResource(resourceName).getResourceValue()); + + childQueue.getConfiguredCapacityVector(label) + .getResource(resourceName).getResourceValue()); } } sumWeightsPerResource.put(label, sumWeight); 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 80d013c3728c6..95f70b1e35451 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 @@ -48,6 +48,7 @@ public void testMixedCapacitiesWithWarnings() throws IOException { QueueAssertionBuilder assertionBuilder = createAssertionBuilder(); QueueHierarchyUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + System.out.println(""); } From 8df390157c8bfff47e6544b69f2e880a38132731 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Wed, 20 Oct 2021 10:51:32 +0200 Subject: [PATCH 07/27] YARN-10930. Fix review feedbacks (cherry picked from commit efc535e90e7d9fe83622410944c0dcea4b5928f0) --- .../capacity/QueueCapacityVector.java | 2 + .../scheduler/capacity/ResourceVector.java | 5 ++ .../conf/QueueCapacityConfigParser.java | 49 ++++++++++--------- .../capacity/TestResourceVector.java | 5 ++ .../conf/TestQueueCapacityConfigParser.java | 34 ++++++++++--- 5 files changed, 66 insertions(+), 29 deletions(-) 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/QueueCapacityVector.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/QueueCapacityVector.java index 968e8cf779fb9..9f6e0e264a320 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/QueueCapacityVector.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/QueueCapacityVector.java @@ -73,6 +73,8 @@ public static QueueCapacityVector newInstance() { * Creates a uniform and homogeneous {@code QueueCapacityVector}. * The resources are defined in absolute capacity type by default. * + * @param value value to be set for each resource + * @param capacityType capacity type to be set for each resource * @return uniform capacity vector */ public static QueueCapacityVector of( 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/ResourceVector.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/ResourceVector.java index 404be82fc0ab1..88c09af6b0911 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/ResourceVector.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/ResourceVector.java @@ -88,6 +88,11 @@ public void subtract(ResourceVector otherResourceVector) { } } + /** + * Increments the given resource by the specified value. + * @param resourceName name of the resource + * @param value value to be added to the resource's current value + */ public void increment(String resourceName, float value) { setValue(resourceName, getValue(resourceName) + value); } 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 c46446b64d714..51f6b41ef5a16 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 @@ -35,13 +35,13 @@ * * A new syntax for capacity property could be implemented, by creating a parser * with a regex to match the pattern and a method that creates a - * {@code QueueCapacityVector} from the matched pattern - * eg. root.capacity 20-50 + * {@code QueueCapacityVector} from the matched pattern. + * Extending the parsers field with a {@code Parser} object in the constructor + * is needed in this case. * * A new capacity type for the existing parsers could be added by extending * the {@code QueueCapacityVector.QueueCapacityType} with a new type and its * associated postfix symbol. - * eg. root.capacity 20g */ public class QueueCapacityConfigParser { private static final String UNIFORM_REGEX = "^([0-9.]+)(.*)"; @@ -91,39 +91,44 @@ public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, } /** - * A parser method that is usable on uniform capacity values eg. percentage or + * A parser method that is usable on uniform capacity values e.g. percentage or * weight. * @param matcher a regex matcher that contains parsed value and its possible * suffix - * @return a parsed resource vector + * @return a parsed capacity vector */ private QueueCapacityVector uniformParser(Matcher matcher) { - QueueCapacityType capacityType = QueueCapacityType.PERCENTAGE; + QueueCapacityType capacityType = null; String value = matcher.group(1); if (matcher.groupCount() == 2) { String matchedSuffix = matcher.group(2); - if (!matchedSuffix.isEmpty()) { - for (QueueCapacityType suffix : QueueCapacityType.values()) { - // when capacity is given in percentage, we do not need % symbol - String uniformSuffix = suffix.getPostfix().replaceAll("%", ""); - if (uniformSuffix.equals(matchedSuffix)) { - capacityType = suffix; - } + for (QueueCapacityType suffix : QueueCapacityType.values()) { + // Absolute uniform syntax is not supported + if (suffix.equals(QueueCapacityType.ABSOLUTE)) { + continue; + } + // when capacity is given in percentage, we do not need % symbol + String uniformSuffix = suffix.getPostfix().replaceAll("%", ""); + if (uniformSuffix.equals(matchedSuffix)) { + capacityType = suffix; } } } + if (capacityType == null) { + return new QueueCapacityVector(); + } + return QueueCapacityVector.of(Float.parseFloat(value), capacityType); } /** - * A parser method that is usable on resource capacity values eg. mixed or + * A parser method that is usable on resource capacity values e.g. mixed or * absolute resource. * @param matcher a regex matcher that contains the matched resource string * @return a parsed capacity vector */ private QueueCapacityVector heterogeneousParser(Matcher matcher) { - // Define resource here. QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); /* @@ -133,12 +138,10 @@ private QueueCapacityVector heterogeneousParser(Matcher matcher) { */ // Get the sub-group. String bracketedGroup = matcher.group(0); - if (bracketedGroup.trim().isEmpty()) { - return capacityVector; - } + // Get the string inside starting and closing [] bracketedGroup = bracketedGroup.substring(1, bracketedGroup.length() - 1); - // Split by comma and equals delimiter eg. memory=1024, vcores=6 to - // [[memory, 1024], [vcores, 6]] + // Split by comma and equals delimiter eg. the string memory=1024,vcores=6 + // is converted to an array of array as {{memory,1024}, {vcores, 6}} for (String kvPair : bracketedGroup.trim().split(",")) { String[] splits = kvPair.split("="); @@ -148,9 +151,9 @@ private QueueCapacityVector heterogeneousParser(Matcher matcher) { } } - // Memory has to be configured always. + // Memory always have to be defined if (capacityVector.getMemory() == 0L) { - return QueueCapacityVector.newInstance(); + return new QueueCapacityVector(); } return capacityVector; @@ -160,7 +163,7 @@ private void setCapacityVector( QueueCapacityVector resource, String resourceName, String resourceValue) { QueueCapacityType capacityType = QueueCapacityType.ABSOLUTE; - // Extract suffix from a value eg. for 6w extract w + // Extract suffix from a value e.g. for 6w extract w String suffix = resourceValue.replaceAll("[0-9]", ""); if (!resourceValue.endsWith(suffix)) { return; 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/TestResourceVector.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/TestResourceVector.java index a9f7a9e296217..6b6b9753189e8 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/TestResourceVector.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/TestResourceVector.java @@ -77,6 +77,11 @@ public void increment() { Assert.assertEquals(18, resourceVector.getValue("memory-mb"), 1e-6); Assert.assertEquals(13, resourceVector.getValue("vcores"), 1e-6); Assert.assertEquals(13, resourceVector.getValue("custom"), 1e-6); + + // Check whether overflow causes any issues + ResourceVector maxFloatResourceVector = ResourceVector.of(Float.MAX_VALUE); + maxFloatResourceVector.increment("memory-mb", 100); + Assert.assertEquals(Float.MAX_VALUE, maxFloatResourceVector.getValue("memory-mb"), 1e-6); } @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/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 d3518a1ff527f..c3055cf050e1c 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 @@ -87,7 +87,7 @@ public void testWeightCapacityConfig() { } @Test - public void testAbsoluteResourceCapacityConfig() { + public void testAbsoluteCapacityVectorConfig() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + CapacitySchedulerConfiguration.CAPACITY, ABSOLUTE_RESOURCE); conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); @@ -151,18 +151,41 @@ public void testMixedCapacityConfig() { @Test public void testInvalidCapacityConfigs() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, "[invalid]"); + + CapacitySchedulerConfiguration.CAPACITY, "50nonexistingsuffix"); + QueueCapacityVector capacityVectorWithInvalidSuffix = + capacityConfigParser.parse(conf, QUEUE, ""); + List entriesWithInvalidSuffix = + Lists.newArrayList(capacityVectorWithInvalidSuffix.iterator()); + Assert.assertEquals(0, entriesWithInvalidSuffix.size()); - QueueCapacityVector invalidResourceCapacity = + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY, "[memory-100,vcores-60]"); + QueueCapacityVector invalidDelimiterCapacityVector = + capacityConfigParser.parse(conf, QUEUE, ""); + List invalidDelimiterEntries = + Lists.newArrayList(invalidDelimiterCapacityVector.iterator()); + Assert.assertEquals(0, invalidDelimiterEntries.size()); + + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY, "[invalid]"); + QueueCapacityVector invalidCapacityVector = capacityConfigParser.parse(conf, QUEUE, ""); List resources = - Lists.newArrayList(invalidResourceCapacity.iterator()); + Lists.newArrayList(invalidCapacityVector.iterator()); Assert.assertEquals(0, resources.size()); conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, ""); + + CapacitySchedulerConfiguration.CAPACITY, "[]"); + QueueCapacityVector emptyBracketCapacityVector = + capacityConfigParser.parse(conf, QUEUE, ""); + List emptyEntries = + Lists.newArrayList(emptyBracketCapacityVector.iterator()); + Assert.assertEquals(0, resources.size()); + conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + + CapacitySchedulerConfiguration.CAPACITY, ""); QueueCapacityVector emptyCapacity = capacityConfigParser.parse(conf, QUEUE, ""); List emptyResources = @@ -171,7 +194,6 @@ public void testInvalidCapacityConfigs() { conf.unset(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + CapacitySchedulerConfiguration.CAPACITY); - QueueCapacityVector nonSetCapacity = capacityConfigParser.parse(conf, QUEUE, ""); List nonSetResources = From 8c6559fe3cab28045a6917011058e68ca37597f5 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 21 Oct 2021 12:22:39 +0200 Subject: [PATCH 08/27] YARN-10965. Simplify calculation --- .../AbsoluteResourceCapacityCalculator.java | 39 ++++---- .../AbstractQueueCapacityCalculator.java | 81 ++++++++++++--- .../PercentageQueueCapacityCalculator.java | 49 ++++----- .../capacity/QueueBranchContext.java | 11 +++ .../capacity/QueueUpdateWarning.java | 17 +++- .../capacity/RootQueueCapacityCalculator.java | 24 +++-- .../WeightQueueCapacityCalculator.java | 99 +++++++++---------- 7 files changed, 202 insertions(+), 118 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index b25af74082eb0..f57921c3b9d12 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -2,35 +2,32 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { - super.calculateChildQueueResources(updateContext, parentQueue); + protected long calculateMinimumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry) { + String resourceName = capacityVectorEntry.getResourceName(); + ResourceVector ratio = updateContext.getNormalizedMinResourceRatio( + childQueue.getParent().getQueuePath(), label); - setChildrenResources(parentQueue, updateContext, ((childQueue, label, capacityVectorEntry) -> { - String resourceName = capacityVectorEntry.getResourceName(); - ResourceVector ratio = updateContext.getNormalizedMinResourceRatio( - parentQueue.getQueuePath(), label); + long resource = (long) Math.floor(ratio.getValue(resourceName) + * capacityVectorEntry.getResourceValue()); - long resource = (long) Math.floor(ratio.getValue(resourceName) - * capacityVectorEntry.getResourceValue()); - long parentResource = parentQueue.getEffectiveCapacity(label) - .getResourceValue(resourceName); - if (resource > parentResource) { - updateContext.addUpdateWarning( - QueueUpdateWarning.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath())); - resource = parentResource; - } + float absolutePercentage = (float) resource + / updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName); - float absolutePercentage = (float) resource - / updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName); + updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), + label).setValue(resourceName, absolutePercentage); + return resource; + } - updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), - label).setValue(resourceName, absolutePercentage); - return resource; - })); + @Override + protected long calculateMaximumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + return 0; } @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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index cae1f93dddf37..2ce5b8fdda987 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -48,15 +48,50 @@ public abstract void setup( public void calculateChildQueueResources( QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { - for (String label : parentQueue.getConfiguredNodeLabels()) { - // We need to set normalized resource ratio only once, not for each - // resource calculator - if (!updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).isParentAlreadyUpdated()) { - setNormalizedResourceRatio(updateContext, parentQueue, label); - updateContext.getQueueBranchContext(parentQueue.getQueuePath()) - .setUpdateFlag(); + calculateResourcePrerequisites(updateContext, parentQueue); + + Map aggregatedResources = new HashMap<>(); + for (CSQueue childQueue : parentQueue.getChildQueues()) { + for (String label : childQueue.getConfiguredNodeLabels()) { + ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault( + label, ResourceVector.newInstance()); + calculateMinResForAllResource(updateContext, childQueue, label, aggregatedUsedResource); + aggregatedResources.put(label, aggregatedUsedResource); + } + } + + for (Map.Entry entry : aggregatedResources.entrySet()){ + updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .getRemainingResource(entry.getKey()).subtract(entry.getValue()); + } + } + + private void calculateMinResForAllResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + ResourceVector aggregatedUsedResource) { + CSQueue parentQueue = childQueue.getParent(); + for (String resourceName : getResourceNames(childQueue, label)) { + long parentResource = parentQueue.getEffectiveCapacity(label) + .getResourceValue(resourceName); + long minimumResource = calculateMinimumResource(updateContext, childQueue, + label, childQueue.getConfiguredCapacityVector(label) + .getResource(resourceName)); + + if (minimumResource > parentResource) { + updateContext.addUpdateWarning( + QueueUpdateWarning.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath()) + .withInfo("Resource name: " + resourceName + " resource value: " + minimumResource)); + minimumResource = parentResource; + } + if (minimumResource == 0) { + updateContext.addUpdateWarning(QueueUpdateWarning. + QUEUE_ZERO_RESOURCE.ofQueue(childQueue.getQueuePath())); } + + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(resourceName, minimumResource); + + aggregatedUsedResource.increment(resourceName, minimumResource); } } @@ -75,6 +110,28 @@ public abstract void setMetrics( */ protected abstract QueueCapacityType getCapacityType(); + protected abstract long calculateMinimumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry); + + protected abstract long calculateMaximumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry); + + protected void calculateResourcePrerequisites( + QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { + for (String label : parentQueue.getConfiguredNodeLabels()) { + // We need to set normalized resource ratio only once, not for each + // resource calculator + if (!updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).isParentAlreadyUpdated()) { + setNormalizedResourceRatio(updateContext, parentQueue, label); + updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .setUpdateFlag(); + } + } + } + /** * Returns all resource names that are defined for the capacity type that is * handled by the calculator. @@ -127,9 +184,6 @@ protected void setNormalizedResourceRatio( continue; } - updateContext.addUpdateWarning(QueueUpdateWarning.BRANCH_DOWNSCALED.ofQueue( - parentQueue.getQueuePath())); - // Factor to scale down effective resource: When cluster has sufficient // resources, effective_min_resources will be same as configured // min_resources. @@ -138,6 +192,8 @@ protected void setNormalizedResourceRatio( numeratorForMinRatio = parentQueue.getQueueResourceQuotas() .getEffectiveMinResource(label).getResourceValue( capacityVectorEntry.getResourceName()); + updateContext.addUpdateWarning(QueueUpdateWarning.BRANCH_DOWNSCALED.ofQueue( + parentQueue.getQueuePath())); } String unit = capacityVectorEntry.getResourceName().equals("memory-mb") @@ -174,6 +230,7 @@ protected void setChildrenResources( for (String label : childQueue.getConfiguredNodeLabels()) { ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault( label, ResourceVector.newInstance()); + for (String resourceName : getResourceNames(childQueue, label)) { long parentResource = parentQueue.getEffectiveCapacity(label) .getResourceValue(resourceName); @@ -192,8 +249,10 @@ protected void setChildrenResources( childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) .setResourceValue(resourceName, resource); + aggregatedUsedResource.increment(resourceName, resource); } + aggregatedResources.put(label, aggregatedUsedResource); } } 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index afd5bb300aa0a..4bef6a9aba63c 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -1,6 +1,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import java.util.Set; @@ -17,32 +18,34 @@ public void setup(CSQueue queue, String label) { } @Override - public void calculateChildQueueResources( - QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { - super.calculateChildQueueResources(updateContext, parentQueue); + protected long calculateMinimumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + CSQueue parentQueue = childQueue.getParent(); + String resourceName = capacityVectorEntry.getResourceName(); - setChildrenResources( - parentQueue, updateContext, ((childQueue, label, capacityVectorEntry) -> { - String resourceName = capacityVectorEntry.getResourceName(); - float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( - parentQueue.getQueuePath(), label).getValue(resourceName); - float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).getRemainingResource(label) - .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) - .getResourceValue(resourceName); - float queueAbsoluteCapacity = parentAbsoluteCapacity * - remainingPerEffectiveResourceRatio - * capacityVectorEntry.getResourceValue() / 100; + float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( + parentQueue.getQueuePath(), label).getValue(resourceName); + float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).getRemainingResource(label) + .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) + .getResourceValue(resourceName); + float queueAbsoluteCapacity = parentAbsoluteCapacity * + remainingPerEffectiveResourceRatio + * capacityVectorEntry.getResourceValue() / 100; - long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) - .getResourceValue(capacityVectorEntry.getResourceName()) - * queueAbsoluteCapacity); + long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) + .getResourceValue(capacityVectorEntry.getResourceName()) + * queueAbsoluteCapacity); - updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) - .setValue(capacityVectorEntry.getResourceName(), - queueAbsoluteCapacity); - return resource; - })); + updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) + .setValue(capacityVectorEntry.getResourceName(), + queueAbsoluteCapacity); + + return resource; +} + + @Override + protected long calculateMaximumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + return 0; } @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/QueueBranchContext.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/QueueBranchContext.java index bc2e824b6a205..48bd1772f1161 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/QueueBranchContext.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/QueueBranchContext.java @@ -31,8 +31,19 @@ */ public class QueueBranchContext { private final Map remainingResourceByLabel = new HashMap<>(); + private final Map> sumWeightsPerLabel = new HashMap<>(); private boolean isUpdated = false; + public void incrementWeight(String label, String resourceName, float value) { + sumWeightsPerLabel.putIfAbsent(label, new HashMap<>()); + sumWeightsPerLabel.get(label).put(resourceName, + sumWeightsPerLabel.get(label).getOrDefault(resourceName, 0f) + value); + } + + public float getSumWeightsByResource(String label, String resourceName) { + return sumWeightsPerLabel.get(label).get(resourceName); + } + public void setRemainingResource(String label, ResourceVector resource) { remainingResourceByLabel.put(label, resource); } 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/QueueUpdateWarning.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/QueueUpdateWarning.java index 119e9f270e678..503a0f7e5d195 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/QueueUpdateWarning.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/QueueUpdateWarning.java @@ -1,13 +1,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; public enum QueueUpdateWarning { - BRANCH_UNDERUTILIZED("Remaining resource found in branch under parent queue '%s'."), - QUEUE_OVERUTILIZED("Queue '%' is configured to use more resources than what is available under its parent."), - QUEUE_ZERO_RESOURCE("Queue '%s' is assigned zero resource."), - BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are downscaled due to insufficient cluster resource."); + BRANCH_UNDERUTILIZED("Remaining resource found in branch under parent queue '%s'. %s"), + QUEUE_OVERUTILIZED("Queue '%' is configured to use more resources than what is available under its parent. %s"), + QUEUE_ZERO_RESOURCE("Queue '%s' is assigned zero resource. %s"), + BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are downscaled due to insufficient cluster resource. %s"); private final String message; private String queue; + private String additionalInfo = ""; QueueUpdateWarning(String message) { this.message = message; @@ -19,8 +20,14 @@ public QueueUpdateWarning ofQueue(String queuePath) { return this; } + public QueueUpdateWarning withInfo(String info) { + additionalInfo = info; + + return this; + } + @Override public String toString() { - return String.format(message, queue); + return String.format(message, queue, additionalInfo); } } 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index f0bb58c0bfb9e..722d56a3cbb6e 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; @@ -35,18 +36,27 @@ public void setup(CSQueue queue, String label) { public void calculateChildQueueResources( QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { for (String label : parentQueue.getConfiguredNodeLabels()) { - for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { + for (QueueCapacityVectorEntry capacityVectorEntry : parentQueue.getConfiguredCapacityVector(label)) { updateContext.getRelativeResourceRatio(ROOT, label).setValue( - resource.getName(), 1); + capacityVectorEntry.getResourceName(), 1); + long minimumResource = calculateMinimumResource(updateContext, parentQueue, label, capacityVectorEntry); parentQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(resource.getName(), - updateContext.getUpdatedClusterResource(label).getResourceValue( - resource.getName())); + .setResourceValue(capacityVectorEntry.getResourceName(), minimumResource); } - - setNormalizedResourceRatio(updateContext, parentQueue, label); } + + calculateResourcePrerequisites(updateContext, parentQueue); + } + + @Override + protected long calculateMinimumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + return updateContext.getUpdatedClusterResource(label).getResourceValue(capacityVectorEntry.getResourceName()); + } + + @Override + protected long calculateMaximumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + return 0; } @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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 5e604488761ba..19e50e02d1250 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -1,43 +1,60 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; - -import java.util.HashMap; -import java.util.Map; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.WEIGHT; public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { - Map> sumWeightsPerLabel = summarizeWeights(parentQueue); - - setChildrenResources(parentQueue, updateContext, - ((childQueue, label, capacityVectorEntry) -> { - String resourceName = capacityVectorEntry.getResourceName(); - float normalizedWeight = capacityVectorEntry.getResourceValue() - / sumWeightsPerLabel.get(label).get(resourceName); - - float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).getRemainingResource(label) - .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) - .getResourceValue(resourceName); - - float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( - parentQueue.getQueuePath(), label).getValue(resourceName); - float queueAbsoluteCapacity = parentAbsoluteCapacity * - remainingPerEffectiveResourceRatio * normalizedWeight; - long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) - .getResourceValue(resourceName) - * queueAbsoluteCapacity); - - updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) - .setValue(capacityVectorEntry.getResourceName(), - queueAbsoluteCapacity); - - return resource; - })); + protected void calculateResourcePrerequisites(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { + super.calculateResourcePrerequisites(updateContext, parentQueue); + + for (CSQueue childQueue : parentQueue.getChildQueues()) { + for (String label : childQueue.getConfiguredNodeLabels()) { + for (String resourceName : childQueue.getConfiguredCapacityVector(label) + .getResourceNamesByCapacityType(getCapacityType())) { + updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .incrementWeight(label, resourceName, childQueue.getConfiguredCapacityVector(label).getResource(resourceName).getResourceValue()); + } + } + } + } + + @Override + protected long calculateMinimumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + CSQueue parentQueue = childQueue.getParent(); + String resourceName = capacityVectorEntry.getResourceName(); + float normalizedWeight = capacityVectorEntry.getResourceValue() + / updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .getSumWeightsByResource(label, resourceName); + + float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).getRemainingResource(label) + .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) + .getResourceValue(resourceName); + + float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( + parentQueue.getQueuePath(), label).getValue(resourceName); + float queueAbsoluteCapacity = parentAbsoluteCapacity * + remainingPerEffectiveResourceRatio * normalizedWeight; + long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) + .getResourceValue(resourceName) + * queueAbsoluteCapacity); + + updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) + .setValue(capacityVectorEntry.getResourceName(), + queueAbsoluteCapacity); + + return resource; + } + + @Override + protected long calculateMaximumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + return 0; } @Override @@ -62,24 +79,4 @@ public void setMetrics(QueueHierarchyUpdateContext updateContext, queue.getQueueCapacities().setNormalizedWeight(label, sumNormalizedWeight); } - private Map> summarizeWeights(CSQueue parentQueue) { - Map> sumWeightsPerResource = new HashMap<>(); - - for (String label : parentQueue.getConfiguredNodeLabels()) { - Map sumWeight = new HashMap<>(); - - for (CSQueue childQueue : parentQueue.getChildQueues()) { - for (String resourceName : childQueue.getConfiguredCapacityVector(label) - .getResourceNamesByCapacityType(getCapacityType())) { - sumWeight.put(resourceName, sumWeight.getOrDefault(resourceName, 0f) - + childQueue.getConfiguredCapacityVector(label) - .getResource(resourceName).getResourceValue()); - } - } - sumWeightsPerResource.put(label, sumWeight); - } - - return sumWeightsPerResource; - } - } From 0b7adc89f28e7d928548c18454259f35b7cfaa13 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Fri, 29 Oct 2021 17:54:25 +0200 Subject: [PATCH 09/27] YARN-10965. Simplify calculators --- .../AbsoluteResourceCapacityCalculator.java | 20 +- .../scheduler/capacity/AbstractCSQueue.java | 15 + .../AbstractQueueCapacityCalculator.java | 262 ++++++++---------- .../scheduler/capacity/CSQueue.java | 17 +- .../CapacitySchedulerConfiguration.java | 28 +- ...CapacitySchedulerQueueCapacityHandler.java | 3 +- .../scheduler/capacity/ParentQueue.java | 3 +- .../PercentageQueueCapacityCalculator.java | 36 ++- .../capacity/QueueBranchContext.java | 11 + .../capacity/QueueHierarchyUpdateContext.java | 17 +- .../capacity/QueueUpdateWarning.java | 55 ++-- .../capacity/RootQueueCapacityCalculator.java | 20 +- .../WeightQueueCapacityCalculator.java | 58 ++-- .../conf/QueueCapacityConfigParser.java | 19 +- ...citySchedulerQueueCalculationTestBase.java | 3 + .../TestMixedQueueResourceCalculation.java | 39 ++- .../capacity/TestQueueCapacityVector.java | 43 +-- .../capacity/TestResourceVector.java | 68 +++-- .../TestUniformQueueResourceCalculation.java | 50 +++- .../conf/TestQueueCapacityConfigParser.java | 176 ++++++------ 20 files changed, 572 insertions(+), 371 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index f57921c3b9d12..620da28d0dbf7 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -7,27 +7,21 @@ public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - protected long calculateMinimumResource( + protected float calculateMinimumResource( QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { String resourceName = capacityVectorEntry.getResourceName(); ResourceVector ratio = updateContext.getNormalizedMinResourceRatio( childQueue.getParent().getQueuePath(), label); - long resource = (long) Math.floor(ratio.getValue(resourceName) - * capacityVectorEntry.getResourceValue()); - - float absolutePercentage = (float) resource - / updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName); - - updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), - label).setValue(resourceName, absolutePercentage); - return resource; + return ratio.getValue(resourceName) * capacityVectorEntry.getResourceValue(); } @Override - protected long calculateMaximumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { - return 0; + protected float calculateMaximumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry) { + return capacityVectorEntry.getResourceValue(); } @Override @@ -60,7 +54,7 @@ public void setMetrics(QueueHierarchyUpdateContext updateContext, resourceName).getResourceValue() / queue.getParent() .getQueueResourceQuotas().getEffectiveMinResource(label) .getResourceValue(resourceName); - sumAbsoluteCapacity += updateContext.getRelativeResourceRatio( + sumAbsoluteCapacity += updateContext.getAbsoluteMinCapacity( queue.getQueuePath(), label).getValue(resourceName); } 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 9be20d72162c7..c5d6e24241d25 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 @@ -25,6 +25,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.util.Sets; import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -141,6 +142,7 @@ public enum CapacityConfigType { CapacityConfigType.NONE; protected Map configuredCapacityVectors; + protected Map configuredMaximumCapacityVectors; private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); @@ -403,6 +405,8 @@ protected void setupQueueConfigs(Resource clusterResource, configuredCapacityVectors = configuration .parseConfiguredResourceVector(queuePath, configuredNodeLabels); + configuredMaximumCapacityVectors = configuration + .parseConfiguredMaximumCapacityVector(queuePath, configuredNodeLabels); // After we setup labels, we can setup capacities setupConfigurableCapacities(configuration); @@ -715,6 +719,12 @@ public QueueCapacityVector getConfiguredCapacityVector( return configuredCapacityVectors.get(label); } + @Override + public QueueCapacityVector getConfiguredMaximumCapacityVector( + String label) { + return configuredCapacityVectors.get(label); + } + private void initializeQueueState(QueueState previousState, QueueState configuredState, QueueState parentState) { // verify that we can not any value for State other than RUNNING/STOPPED @@ -937,6 +947,11 @@ public ReentrantReadWriteLock.ReadLock getReadLock() { return readLock; } + @Override + public ReentrantReadWriteLock.WriteLock getWriteLock() { + return writeLock; + } + /** * The specified queue is cross-queue preemptable if system-wide cross-queue * preemption is turned on unless any queue in the qPath hierarchy 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 2ce5b8fdda987..2979a096b13da 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -20,12 +20,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; import org.apache.hadoop.yarn.util.UnitsConversionUtil; import java.util.HashMap; import java.util.Map; import java.util.Set; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; + /** * A strategy class to encapsulate queue capacity setup and resource calculation * logic. @@ -34,97 +37,74 @@ public abstract class AbstractQueueCapacityCalculator { /** * Sets all field of the queue based on its configurations. + * * @param queue queue to setup * @param label node label */ - public abstract void setup( - CSQueue queue, String label); + public abstract void setup(CSQueue queue, String label); + + /** + * Sets the metrics and statistics after effective resource calculation. + * + * @param updateContext context of the current update phase + * @param queue queue to update + * @param label node label + */ + public abstract void setMetrics(QueueHierarchyUpdateContext updateContext, CSQueue queue, + String label); /** * Calculate the effective resource for a specific resource. + * * @param updateContext context of the current update phase - * @param parentQueue the parent whose children will be updated + * @param parentQueue the parent whose children will be updated */ - public void calculateChildQueueResources( - QueueHierarchyUpdateContext updateContext, - CSQueue parentQueue) { + public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, + CSQueue parentQueue) { calculateResourcePrerequisites(updateContext, parentQueue); Map aggregatedResources = new HashMap<>(); for (CSQueue childQueue : parentQueue.getChildQueues()) { - for (String label : childQueue.getConfiguredNodeLabels()) { - ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault( - label, ResourceVector.newInstance()); - calculateMinResForAllResource(updateContext, childQueue, label, aggregatedUsedResource); - aggregatedResources.put(label, aggregatedUsedResource); + childQueue.getWriteLock().lock(); + try { + for (String label : childQueue.getConfiguredNodeLabels()) { + ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault(label, + ResourceVector.newInstance()); + calculateResources(updateContext, childQueue, label, aggregatedUsedResource); + aggregatedResources.put(label, aggregatedUsedResource); + } + } finally { + childQueue.getWriteLock().unlock(); } } - for (Map.Entry entry : aggregatedResources.entrySet()){ - updateContext.getQueueBranchContext(parentQueue.getQueuePath()) - .getRemainingResource(entry.getKey()).subtract(entry.getValue()); - } - } - - private void calculateMinResForAllResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, - ResourceVector aggregatedUsedResource) { - CSQueue parentQueue = childQueue.getParent(); - for (String resourceName : getResourceNames(childQueue, label)) { - long parentResource = parentQueue.getEffectiveCapacity(label) - .getResourceValue(resourceName); - long minimumResource = calculateMinimumResource(updateContext, childQueue, - label, childQueue.getConfiguredCapacityVector(label) - .getResource(resourceName)); - - if (minimumResource > parentResource) { - updateContext.addUpdateWarning( - QueueUpdateWarning.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath()) - .withInfo("Resource name: " + resourceName + " resource value: " + minimumResource)); - minimumResource = parentResource; - } - if (minimumResource == 0) { - updateContext.addUpdateWarning(QueueUpdateWarning. - QUEUE_ZERO_RESOURCE.ofQueue(childQueue.getQueuePath())); - } - - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(resourceName, minimumResource); - - aggregatedUsedResource.increment(resourceName, minimumResource); + for (Map.Entry entry : aggregatedResources.entrySet()) { + updateContext.getQueueBranchContext(parentQueue.getQueuePath()).getRemainingResource( + entry.getKey()).subtract(entry.getValue()); } } - /** - * Sets the metrics and statistics after effective resource calculation. - * @param updateContext context of the current update phase - * @param queue queue to update - * @param label node label - */ - public abstract void setMetrics( - QueueHierarchyUpdateContext updateContext, CSQueue queue, String label); - /** * Returns the capacity type the calculator could handle. + * * @return capacity type */ protected abstract QueueCapacityType getCapacityType(); - protected abstract long calculateMinimumResource( + protected abstract float calculateMinimumResource( QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry); - protected abstract long calculateMaximumResource( + protected abstract float calculateMaximumResource( QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry); - protected void calculateResourcePrerequisites( - QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { + protected void calculateResourcePrerequisites(QueueHierarchyUpdateContext updateContext, + CSQueue parentQueue) { for (String label : parentQueue.getConfiguredNodeLabels()) { - // We need to set normalized resource ratio only once, not for each - // resource calculator - if (!updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).isParentAlreadyUpdated()) { + // We need to set normalized resource ratio only once per parent + if (!updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .isParentAlreadyUpdated()) { setNormalizedResourceRatio(updateContext, parentQueue, label); updateContext.getQueueBranchContext(parentQueue.getQueuePath()) .setUpdateFlag(); @@ -135,6 +115,7 @@ protected void calculateResourcePrerequisites( /** * Returns all resource names that are defined for the capacity type that is * handled by the calculator. + * * @param queue queue for which the capacity vector is defined * @param label node label * @return resource names @@ -145,8 +126,9 @@ protected Set getResourceNames(CSQueue queue, String label) { /** * Returns all resource names that are defined for a capacity type. - * @param queue queue for which the capacity vector is defined - * @param label node label + * + * @param queue queue for which the capacity vector is defined + * @param label node label * @param capacityType capacity type for which the resource names are defined * @return resource names */ @@ -156,115 +138,117 @@ protected Set getResourceNames(CSQueue queue, String label, .getResourceNamesByCapacityType(capacityType); } - protected void setNormalizedResourceRatio( - QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, - String label - ) { - for (QueueCapacityVectorEntry capacityVectorEntry : - parentQueue.getConfiguredCapacityVector(label)) { + protected float sumCapacityValues(CSQueue queue, String label) { + float sumValue = 0f; + QueueCapacityVector capacityVector = queue.getConfiguredCapacityVector(label); + for (String resourceName : getResourceNames(queue, label)) { + sumValue += capacityVector.getResource(resourceName).getResourceValue(); + } + return sumValue; + } + + private void setNormalizedResourceRatio( + QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { + for (QueueCapacityVectorEntry capacityVectorEntry : parentQueue.getConfiguredCapacityVector( + label)) { + String resourceName = capacityVectorEntry.getResourceName(); long childrenConfiguredResource = 0; - long effectiveMinResource = parentQueue.getQueueResourceQuotas() - .getEffectiveMinResource(label).getResourceValue( - capacityVectorEntry.getResourceName()); + long effectiveMinResource = parentQueue.getQueueResourceQuotas().getEffectiveMinResource( + label).getResourceValue(resourceName); // Total configured min resources of direct children of this given parent // queue for (CSQueue childQueue : parentQueue.getChildQueues()) { QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); - if (capacityVector.isResourceOfType( - capacityVectorEntry.getResourceName(), QueueCapacityType.ABSOLUTE)) { - childrenConfiguredResource += capacityVector.getResource( - capacityVectorEntry.getResourceName()).getResourceValue(); + if (capacityVector.isResourceOfType(resourceName, QueueCapacityType.ABSOLUTE)) { + childrenConfiguredResource += capacityVector.getResource(resourceName).getResourceValue(); } } - // If no children is using ABSOLUTE capacity type, normalization is // not needed if (childrenConfiguredResource == 0) { continue; } - // Factor to scale down effective resource: When cluster has sufficient // resources, effective_min_resources will be same as configured // min_resources. float numeratorForMinRatio = childrenConfiguredResource; if (effectiveMinResource < childrenConfiguredResource) { - numeratorForMinRatio = parentQueue.getQueueResourceQuotas() - .getEffectiveMinResource(label).getResourceValue( - capacityVectorEntry.getResourceName()); - updateContext.addUpdateWarning(QueueUpdateWarning.BRANCH_DOWNSCALED.ofQueue( + numeratorForMinRatio = parentQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .getResourceValue(resourceName); + updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_DOWNSCALED.ofQueue( parentQueue.getQueuePath())); } - String unit = capacityVectorEntry.getResourceName().equals("memory-mb") - ? "Mi" : ""; + String unit = resourceName.equals(MEMORY_URI) ? "Mi" : ""; long convertedValue = UnitsConversionUtil.convert(unit, - updateContext.getUpdatedClusterResource(label).getResourceInformation( - capacityVectorEntry.getResourceName()).getUnits(), - childrenConfiguredResource); + updateContext.getUpdatedClusterResource(label).getResourceInformation(resourceName) + .getUnits(), childrenConfiguredResource); if (convertedValue != 0) { - updateContext.getNormalizedMinResourceRatio(parentQueue.getQueuePath(), label) - .setValue(capacityVectorEntry.getResourceName(), - numeratorForMinRatio / convertedValue); + updateContext.getNormalizedMinResourceRatio(parentQueue.getQueuePath(), label).setValue( + resourceName, numeratorForMinRatio / convertedValue); } } } - protected float sumCapacityValues(CSQueue queue, String label) { - float sumValue = 0f; - QueueCapacityVector capacityVector = - queue.getConfiguredCapacityVector(label); - for (String resourceName : getResourceNames(queue, label)) { - sumValue += capacityVector.getResource(resourceName).getResourceValue(); - } + private void calculateResources( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + ResourceVector aggregatedUsedResource) { + CSQueue parentQueue = childQueue.getParent(); + for (String resourceName : getResourceNames(childQueue, label)) { + long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( + resourceName); + float minimumResource = calculateMinimumResource(updateContext, childQueue, label, + childQueue.getConfiguredCapacityVector(label).getResource(resourceName)); + long parentMinimumResource = parentQueue.getEffectiveCapacity(label).getResourceValue( + resourceName); + + long parentMaximumResource = parentQueue.getEffectiveMaxCapacity(label).getResourceValue( + resourceName); + float maximumResource = calculateMaximumResource(updateContext, childQueue, label, + childQueue.getConfiguredMaximumCapacityVector(label).getResource(resourceName)); + + if (maximumResource != 0 && maximumResource > parentMaximumResource) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( + childQueue.getQueuePath())); + } + maximumResource = maximumResource == 0 ? parentMaximumResource + : Math.min(maximumResource, parentMaximumResource); - return sumValue; - } + if (maximumResource < minimumResource) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( + childQueue.getQueuePath())); + minimumResource = maximumResource; + } - protected void setChildrenResources( - CSQueue parentQueue, QueueHierarchyUpdateContext updateContext, - ChildResourceCalculator callable) { - Map aggregatedResources = new HashMap<>(); - for (CSQueue childQueue : parentQueue.getChildQueues()) { - for (String label : childQueue.getConfiguredNodeLabels()) { - ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault( - label, ResourceVector.newInstance()); - - for (String resourceName : getResourceNames(childQueue, label)) { - long parentResource = parentQueue.getEffectiveCapacity(label) - .getResourceValue(resourceName); - long resource = callable.call(childQueue, label, childQueue - .getConfiguredCapacityVector(label).getResource(resourceName)); - - if (resource > parentResource) { - updateContext.addUpdateWarning( - QueueUpdateWarning.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath())); - resource = parentResource; - } - if (resource == 0) { - updateContext.addUpdateWarning(QueueUpdateWarning. - QUEUE_ZERO_RESOURCE.ofQueue(childQueue.getQueuePath())); - } - - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(resourceName, resource); - - aggregatedUsedResource.increment(resourceName, resource); - } - - aggregatedResources.put(label, aggregatedUsedResource); + if (minimumResource > parentMinimumResource) { + updateContext.addUpdateWarning( + QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath()).withInfo( + "Resource name: " + resourceName + " resource value: " + minimumResource)); + minimumResource = parentMinimumResource; } - } - for (Map.Entry entry : aggregatedResources.entrySet()){ - updateContext.getQueueBranchContext(parentQueue.getQueuePath()) - .getRemainingResource(entry.getKey()).subtract(entry.getValue()); - } - } + if (minimumResource == 0) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( + childQueue.getQueuePath()).withInfo("Resource name: " + resourceName)); + } - protected interface ChildResourceCalculator { - long call(CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry); + float absoluteMinCapacity = minimumResource / clusterResource; + float absoluteMaxCapacity = maximumResource / clusterResource; + updateContext.getAbsoluteMinCapacity(childQueue.getQueuePath(), label).setValue(resourceName, + absoluteMinCapacity); + updateContext.getAbsoluteMaxCapacity(childQueue.getQueuePath(), label).setValue(resourceName, + absoluteMaxCapacity); + + long roundedMinResource = (long) Math.floor(minimumResource); + long roundedMaxResource = (long) Math.floor(maximumResource); + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + resourceName, roundedMinResource); + childQueue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + resourceName, roundedMaxResource); + + aggregatedUsedResource.increment(resourceName, roundedMinResource); + } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index 4b20201e1e1b7..43d4617603651 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -370,6 +370,12 @@ void apply(Resource cluster, */ public ReentrantReadWriteLock.ReadLock getReadLock(); + /** + * Get writeLock associated with the Queue. + * @return writeLock of corresponding queue. + */ + ReentrantReadWriteLock.WriteLock getWriteLock(); + /** * Validate submitApplication api so that moveApplication do a pre-check. * @param applicationId Application ID @@ -415,13 +421,22 @@ public void validateSubmitApplication(ApplicationId applicationId, Resource getEffectiveCapacity(String label); /** - * Get configured capacity resource vector parsed from the capacity config + * Get configured capacity vector parsed from the capacity config * of the queue. * @param label node label (partition) * @return capacity resource vector */ QueueCapacityVector getConfiguredCapacityVector(String label); + /** + * Get configured maximum capacity vector parsed from the capacity config + * of the queue. + * @param label node label (partition) + * @return capacity resource vector + */ + QueueCapacityVector getConfiguredMaximumCapacityVector(String label); + + Set getConfiguredNodeLabels(); /** 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 b2a32505dd36a..e19dfe907c279 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 @@ -410,6 +410,8 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur private static final QueueCapacityConfigParser queueCapacityConfigParser = new QueueCapacityConfigParser(); + private static final String LEGACY_QUEUE_MODE_ENABLED = PREFIX + "legacy-queue-mode.enabled"; + public static final boolean DEFAULT_LEGACY_QUEUE_MODE = true; private ConfigurationProperties configurationProperties; @@ -2578,7 +2580,23 @@ public Map parseConfiguredResourceVector( String queuePath, Set labels) { Map queueResourceVectors = new HashMap<>(); for (String label : labels) { - queueResourceVectors.put(label, queueCapacityConfigParser.parse(this, queuePath, label)); + String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( + queuePath, label) + CapacitySchedulerConfiguration.CAPACITY; + String capacityString = get(propertyName); + queueResourceVectors.put(label, queueCapacityConfigParser.parse(capacityString, queuePath)); + } + + return queueResourceVectors; + } + + public Map parseConfiguredMaximumCapacityVector( + String queuePath, Set labels) { + Map queueResourceVectors = new HashMap<>(); + for (String label : labels) { + String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( + queuePath, label) + CapacitySchedulerConfiguration.MAXIMUM_CAPACITY; + String capacityString = get(propertyName); + queueResourceVectors.put(label, queueCapacityConfigParser.parse(capacityString, queuePath)); } return queueResourceVectors; @@ -2770,6 +2788,14 @@ public String getMultiNodesSortingAlgorithmPolicy( return normalizePolicyName(policyClassName.trim()); } + public boolean isLegacyQueueMode() { + return getBoolean(LEGACY_QUEUE_MODE_ENABLED, DEFAULT_LEGACY_QUEUE_MODE); + } + + public void setLegacyQueueModeEnabled(boolean value) { + setBoolean(LEGACY_QUEUE_MODE_ENABLED, false); + } + public boolean getMultiNodePlacementEnabled() { return getBoolean(MULTI_NODE_PLACEMENT_ENABLED, DEFAULT_MULTI_NODE_PLACEMENT_ENABLED); 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index 8a9344fa675fa..95dfbbeafb191 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -120,7 +121,7 @@ private void update( if (!queueHierarchyContext.getQueueBranchContext(parent.getQueuePath()) .getRemainingResource(label).equals(ResourceVector.newInstance())) { queueHierarchyContext.addUpdateWarning( - QueueUpdateWarning.BRANCH_UNDERUTILIZED.ofQueue( + QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( parent.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/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 e1307d7bd3f26..123817b13d647 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 @@ -311,7 +311,8 @@ void setChildQueues(Collection childQueues) throws IOException { // We don't allow any mixed absolute + {weight, percentage} between // children and parent if (childrenCapacityType != parentCapacityType && !this.getQueuePath() - .equals(CapacitySchedulerConfiguration.ROOT)) { + .equals(CapacitySchedulerConfiguration.ROOT) + && csContext.getConfiguration().isLegacyQueueMode()) { throw new IOException("Parent=" + this.getQueuePath() + ": When absolute minResource is used, we must make sure both " + "parent and child all use absolute minResource"); 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index 4bef6a9aba63c..33addfe19d946 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -18,34 +18,40 @@ public void setup(CSQueue queue, String label) { } @Override - protected long calculateMinimumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + protected float calculateMinimumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry) { CSQueue parentQueue = childQueue.getParent(); String resourceName = capacityVectorEntry.getResourceName(); - float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( + float parentAbsoluteCapacity = updateContext.getAbsoluteMinCapacity( parentQueue.getQueuePath(), label).getValue(resourceName); float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( parentQueue.getQueuePath()).getRemainingResource(label) .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) .getResourceValue(resourceName); - float queueAbsoluteCapacity = parentAbsoluteCapacity * + float absoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio * capacityVectorEntry.getResourceValue() / 100; - long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) - .getResourceValue(capacityVectorEntry.getResourceName()) - * queueAbsoluteCapacity); - - updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) - .setValue(capacityVectorEntry.getResourceName(), - queueAbsoluteCapacity); - - return resource; + return updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName) + * absoluteCapacity; } @Override - protected long calculateMaximumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { - return 0; + protected float calculateMaximumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry) { + CSQueue parentQueue = childQueue.getParent(); + String resourceName = capacityVectorEntry.getResourceName(); + + float parentAbsoluteMaxCapacity = updateContext.getAbsoluteMaxCapacity( + parentQueue.getQueuePath(), label).getValue(resourceName); + float absoluteMaxCapacity = parentAbsoluteMaxCapacity + * capacityVectorEntry.getResourceValue() / 100; + + return updateContext.getUpdatedClusterResource(label).getResourceValue( + capacityVectorEntry.getResourceName()) * absoluteMaxCapacity; } @Override @@ -54,7 +60,7 @@ public void setMetrics( float sumAbsoluteCapacity = 0f; Set resources = getResourceNames(queue, label); for (String resourceName : resources) { - sumAbsoluteCapacity += updateContext.getRelativeResourceRatio( + sumAbsoluteCapacity += updateContext.getAbsoluteMinCapacity( queue.getQueuePath(), label).getValue(resourceName); } 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/QueueBranchContext.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/QueueBranchContext.java index 48bd1772f1161..207cbf6a19e1d 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/QueueBranchContext.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/QueueBranchContext.java @@ -32,6 +32,7 @@ public class QueueBranchContext { private final Map remainingResourceByLabel = new HashMap<>(); private final Map> sumWeightsPerLabel = new HashMap<>(); + private final Map> sumMaxWeightsPerLabel = new HashMap<>(); private boolean isUpdated = false; public void incrementWeight(String label, String resourceName, float value) { @@ -44,6 +45,16 @@ public float getSumWeightsByResource(String label, String resourceName) { return sumWeightsPerLabel.get(label).get(resourceName); } + public void incrementMaxWeight(String label, String resourceName, float value) { + sumMaxWeightsPerLabel.putIfAbsent(label, new HashMap<>()); + sumMaxWeightsPerLabel.get(label).put(resourceName, + sumMaxWeightsPerLabel.get(label).getOrDefault(resourceName, 0f) + value); + } + + public float getSumMaxWeightsByResource(String label, String resourceName) { + return sumMaxWeightsPerLabel.get(label).get(resourceName); + } + public void setRemainingResource(String label, ResourceVector resource) { remainingResourceByLabel.put(label, resource); } 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/QueueHierarchyUpdateContext.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/QueueHierarchyUpdateContext.java index 51e42701a691d..f62279eed1bc4 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/QueueHierarchyUpdateContext.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/QueueHierarchyUpdateContext.java @@ -42,7 +42,9 @@ public class QueueHierarchyUpdateContext { createLazyResourceVector(); private Map> relativeResourceRatio = createLazyResourceVector(); - private List warnings = new ArrayList<>(); + private Map> relativeMaxResourceRatio = + createLazyResourceVector(); + private List warnings = new ArrayList(); public QueueHierarchyUpdateContext( Resource updatedClusterResource, @@ -105,11 +107,20 @@ public ResourceVector getNormalizedMinResourceRatio( * @param label node label * @return resource ratio */ - public ResourceVector getRelativeResourceRatio( - String queuePath, String label) { + public ResourceVector getAbsoluteMinCapacity(String queuePath, String label) { return relativeResourceRatio.get(queuePath).get(label); } + /** + * Returns the ratio of a child queue and its parent's maximum resource. + * @param queuePath queue path + * @param label node label + * @return resource ratio + */ + public ResourceVector getAbsoluteMaxCapacity(String queuePath, String label) { + return relativeMaxResourceRatio.get(queuePath).get(label); + } + public void addUpdateWarning(QueueUpdateWarning warning) { warnings.add(warning); } 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/QueueUpdateWarning.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/QueueUpdateWarning.java index 503a0f7e5d195..f26d4649b9628 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/QueueUpdateWarning.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/QueueUpdateWarning.java @@ -1,33 +1,54 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -public enum QueueUpdateWarning { - BRANCH_UNDERUTILIZED("Remaining resource found in branch under parent queue '%s'. %s"), - QUEUE_OVERUTILIZED("Queue '%' is configured to use more resources than what is available under its parent. %s"), - QUEUE_ZERO_RESOURCE("Queue '%s' is assigned zero resource. %s"), - BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are downscaled due to insufficient cluster resource. %s"); - - private final String message; - private String queue; - private String additionalInfo = ""; - - QueueUpdateWarning(String message) { - this.message = message; +public class QueueUpdateWarning { + private final String queue; + private final QueueUpdateWarningType warningType; + private String info = ""; + + public QueueUpdateWarning(QueueUpdateWarningType queueUpdateWarningType, String queue) { + this.warningType = queueUpdateWarningType; + this.queue = queue; } - public QueueUpdateWarning ofQueue(String queuePath) { - this.queue = queuePath; + public enum QueueUpdateWarningType { + BRANCH_UNDERUTILIZED("Remaining resource found in branch under parent queue '%s'. %s"), + QUEUE_OVERUTILIZED("Queue '%' is configured to use more resources than what is available under its parent. %s"), + QUEUE_ZERO_RESOURCE("Queue '%s' is assigned zero resource. %s"), + BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are downscaled due to insufficient cluster resource. %s"), + QUEUE_EXCEEDS_MAX_RESOURCE("Queue '%s' exceeds its maximum available resources. %s"), + QUEUE_MAX_RESOURCE_EXCEEDS_PARENT("Maximum resources of queue '%s' are greater than its parent's. %s"); - return this; + private final String template; + + QueueUpdateWarningType(String template) { + this.template = template; + } + + public QueueUpdateWarning ofQueue(String queue) { + return new QueueUpdateWarning(this, queue); + } + + public String getTemplate() { + return template; + } } public QueueUpdateWarning withInfo(String info) { - additionalInfo = info; + this.info = info; return this; } + public String getQueue() { + return queue; + } + + public QueueUpdateWarningType getWarningType() { + return warningType; + } + @Override public String toString() { - return String.format(message, queue, additionalInfo); + return String.format(warningType.getTemplate(), queue, info); } } 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index 722d56a3cbb6e..051ff7acfffe3 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -18,9 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import org.apache.hadoop.yarn.util.resource.ResourceUtils; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.PERCENTAGE; @@ -30,6 +28,7 @@ public class RootQueueCapacityCalculator extends @Override public void setup(CSQueue queue, String label) { queue.getQueueCapacities().setCapacity(label, 100f); + queue.getQueueCapacities().setMaximumCapacity(label, 100f); } @Override @@ -37,12 +36,17 @@ public void calculateChildQueueResources( QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { for (String label : parentQueue.getConfiguredNodeLabels()) { for (QueueCapacityVectorEntry capacityVectorEntry : parentQueue.getConfiguredCapacityVector(label)) { - updateContext.getRelativeResourceRatio(ROOT, label).setValue( + updateContext.getAbsoluteMinCapacity(ROOT, label).setValue( capacityVectorEntry.getResourceName(), 1); - long minimumResource = calculateMinimumResource(updateContext, parentQueue, label, capacityVectorEntry); + float minimumResource = calculateMinimumResource(updateContext, parentQueue, label, capacityVectorEntry); + float maximumResource = calculateMinimumResource(updateContext, parentQueue, label, capacityVectorEntry); + long roundedMinResource = (long) Math.floor(minimumResource); + long roundedMaxResource = (long) Math.floor(maximumResource); parentQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), minimumResource); + .setResourceValue(capacityVectorEntry.getResourceName(), roundedMinResource); + parentQueue.getQueueResourceQuotas().getEffectiveMaxResource(label) + .setResourceValue(capacityVectorEntry.getResourceName(), roundedMaxResource); } } @@ -50,13 +54,13 @@ public void calculateChildQueueResources( } @Override - protected long calculateMinimumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + protected float calculateMinimumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { return updateContext.getUpdatedClusterResource(label).getResourceValue(capacityVectorEntry.getResourceName()); } @Override - protected long calculateMaximumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { - return 0; + protected float calculateMaximumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + return updateContext.getUpdatedClusterResource(label).getResourceValue(capacityVectorEntry.getResourceName()); } @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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 19e50e02d1250..c7d95999c01f0 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -11,50 +11,68 @@ public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculat protected void calculateResourcePrerequisites(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { super.calculateResourcePrerequisites(updateContext, parentQueue); - for (CSQueue childQueue : parentQueue.getChildQueues()) { - for (String label : childQueue.getConfiguredNodeLabels()) { + for (CSQueue childQueue : parentQueue.getChildQueues()) { + for (String label : childQueue.getConfiguredNodeLabels()) { for (String resourceName : childQueue.getConfiguredCapacityVector(label) .getResourceNamesByCapacityType(getCapacityType())) { updateContext.getQueueBranchContext(parentQueue.getQueuePath()) - .incrementWeight(label, resourceName, childQueue.getConfiguredCapacityVector(label).getResource(resourceName).getResourceValue()); + .incrementWeight(label, resourceName, childQueue.getConfiguredCapacityVector(label) + .getResource(resourceName).getResourceValue()); + updateContext.getQueueBranchContext(parentQueue.getQueuePath()).incrementMaxWeight( + label, resourceName, childQueue.getConfiguredMaximumCapacityVector(label) + .getResource(resourceName).getResourceValue()); } } } } @Override - protected long calculateMinimumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + protected float calculateMinimumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry) { CSQueue parentQueue = childQueue.getParent(); String resourceName = capacityVectorEntry.getResourceName(); float normalizedWeight = capacityVectorEntry.getResourceValue() / updateContext.getQueueBranchContext(parentQueue.getQueuePath()) .getSumWeightsByResource(label, resourceName); - float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( + float remainingResource = updateContext.getQueueBranchContext( parentQueue.getQueuePath()).getRemainingResource(label) - .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) - .getResourceValue(resourceName); + .getValue(resourceName); + float remainingPerEffectiveResourceRatio = remainingResource / parentQueue.getEffectiveCapacity( + label).getResourceValue(resourceName); - float parentAbsoluteCapacity = updateContext.getRelativeResourceRatio( + float parentAbsoluteCapacity = updateContext.getAbsoluteMinCapacity( parentQueue.getQueuePath(), label).getValue(resourceName); float queueAbsoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio * normalizedWeight; - long resource = (long) Math.floor(updateContext.getUpdatedClusterResource(label) - .getResourceValue(resourceName) - * queueAbsoluteCapacity); - updateContext.getRelativeResourceRatio(childQueue.getQueuePath(), label) - .setValue(capacityVectorEntry.getResourceName(), - queueAbsoluteCapacity); + // Due to rounding loss it is better to use all remaining resources + // if no other resource uses weight + if (normalizedWeight == 1) { + return remainingResource; + } - return resource; + return updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName) + * queueAbsoluteCapacity; } @Override - protected long calculateMaximumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { - return 0; + protected float calculateMaximumResource( + QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + QueueCapacityVectorEntry capacityVectorEntry) { + CSQueue parentQueue = childQueue.getParent(); + String resourceName = capacityVectorEntry.getResourceName(); + float normalizedMaxWeight = capacityVectorEntry.getResourceValue() + / updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .getSumMaxWeightsByResource(label, resourceName); + + float parentAbsoluteMaxCapacity = updateContext.getAbsoluteMaxCapacity( + parentQueue.getQueuePath(), label).getValue(resourceName); + float absoluteMaxCapacity = parentAbsoluteMaxCapacity * normalizedMaxWeight; + + return updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName) + * absoluteMaxCapacity; } @Override @@ -72,7 +90,7 @@ public void setMetrics(QueueHierarchyUpdateContext updateContext, CSQueue queue, String label) { float sumNormalizedWeight = 0; for (String resourceName : getResourceNames(queue, label)) { - sumNormalizedWeight += updateContext.getRelativeResourceRatio( + sumNormalizedWeight += updateContext.getAbsoluteMinCapacity( queue.getQueuePath(), label).getValue(resourceName); } 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 51f6b41ef5a16..e6703ea4c5c51 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,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf; +import org.apache.hadoop.util.StringUtils; 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.QueueCapacityType; @@ -49,6 +50,7 @@ public class QueueCapacityConfigParser { private static final Pattern RESOURCE_PATTERN = Pattern.compile(RESOURCE_REGEX); private static final Pattern UNIFORM_PATTERN = Pattern.compile(UNIFORM_REGEX); + public static final String FLOAT_DIGIT_REGEX = "[0-9.]"; private final List parsers = new ArrayList<>(); @@ -60,25 +62,21 @@ public QueueCapacityConfigParser() { /** * Creates a {@code QueueCapacityVector} parsed from the capacity configuration * property set for a queue. - * @param conf configuration object + * @param capacityString capacity string to parse * @param queuePath queue for which the capacity property is parsed - * @param label node label * @return a parsed capacity vector */ - public QueueCapacityVector parse(CapacitySchedulerConfiguration conf, - String queuePath, String label) { + public QueueCapacityVector parse(String capacityString, String queuePath) { if (queuePath.equals(CapacitySchedulerConfiguration.ROOT)) { return QueueCapacityVector.of(100f, QueueCapacityType.PERCENTAGE); } - String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( - queuePath, label) + CapacitySchedulerConfiguration.CAPACITY; - String capacityString = conf.get(propertyName); - if (capacityString == null) { return new QueueCapacityVector(); } + // Trim all spaces from capacity string + capacityString = capacityString.replaceAll(" ", ""); for (Parser parser : parsers) { Matcher matcher = parser.regex.matcher(capacityString); @@ -164,12 +162,11 @@ private void setCapacityVector( QueueCapacityType capacityType = QueueCapacityType.ABSOLUTE; // Extract suffix from a value e.g. for 6w extract w - String suffix = resourceValue.replaceAll("[0-9]", ""); + String suffix = resourceValue.replaceAll(FLOAT_DIGIT_REGEX, ""); if (!resourceValue.endsWith(suffix)) { return; } - String cleanResourceName = resourceName.replaceAll(" ", ""); float parsedResourceValue = Float.parseFloat(resourceValue.substring( 0, resourceValue.length() - suffix.length())); float convertedValue = parsedResourceValue; @@ -185,7 +182,7 @@ private void setCapacityVector( } } - resource.setResource(cleanResourceName, convertedValue, capacityType); + resource.setResource(resourceName, convertedValue, capacityType); } /** 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 0d54a4a5f2ed8..a4436127e7f11 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 @@ -186,6 +186,9 @@ public void setUp() throws Exception { csConf.setNonLabeledQueueWeight("root.b", 4f); csConf.setQueues("root.a", new String[]{"a1", "a2"}); csConf.setNonLabeledQueueWeight("root.a.a1", 1f); + csConf.setQueues("root.a.a1", new String[]{"a11", "a12"}); + csConf.setNonLabeledQueueWeight("root.a.a1.a11", 1f); + csConf.setNonLabeledQueueWeight("root.a.a1.a12", 1f); 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/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 95f70b1e35451..02b721d4255a5 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 @@ -2,10 +2,13 @@ import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; import org.junit.Assert; import org.junit.Test; import java.io.IOException; +import java.util.Collection; +import java.util.Optional; public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { @@ -22,6 +25,12 @@ public void testMixedCapacitiesWithoutRemainingResource() throws IOException { .withQueue("root.a.a1") .toExpect(Resource.newInstance(621, 4)) .assertEffectiveMinResource() + .withQueue("root.a.a1.a11") + .toExpect(Resource.newInstance(217, 1)) + .assertEffectiveMinResource() + .withQueue("root.a.a1.a12") + .toExpect(Resource.newInstance(404, 3)) + .assertEffectiveMinResource() .withQueue("root.a.a2") .toExpect(Resource.newInstance(1865, 5)) .assertEffectiveMinResource() @@ -44,12 +53,27 @@ public void testMixedCapacitiesWithoutRemainingResource() throws IOException { @Test public void testMixedCapacitiesWithWarnings() throws IOException { + csConf.setLegacyQueueModeEnabled(false); setupQueueHierarchyWithWarnings(); QueueAssertionBuilder assertionBuilder = createAssertionBuilder(); QueueHierarchyUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); - System.out.println(""); - + Optional queueCZeroResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, "root.c"); + Optional queueARemainingResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_UNDERUTILIZED, "root.a"); + Optional queueBDownscalingWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_DOWNSCALED, "root.b"); + Optional queueA11ZeroResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, "root.a.a1.a11"); + Optional queueA12ZeroResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, "root.a.a1.a12"); + + Assert.assertTrue(queueCZeroResourceWarning.isPresent()); + Assert.assertTrue(queueARemainingResourceWarning.isPresent()); + Assert.assertTrue(queueBDownscalingWarning.isPresent()); + Assert.assertTrue(queueA11ZeroResourceWarning.isPresent()); + Assert.assertTrue(queueA12ZeroResourceWarning.isPresent()); } private void setupQueueHierarchyWithoutRemainingResource() throws IOException { @@ -63,6 +87,8 @@ private void setupQueueHierarchyWithoutRemainingResource() throws IOException { csConf.setState("root.b", QueueState.RUNNING); csConf.setCapacityVector("root.a", "", "[memory=30%,vcores=6w]"); csConf.setCapacityVector("root.a.a1", "", "[memory=1w,vcores=4]"); + csConf.setCapacityVector("root.a.a1.a11", "", "[memory=35%,vcores=25%]"); + csConf.setCapacityVector("root.a.a1.a12", "", "[memory=65%,vcores=75%]"); csConf.setCapacityVector("root.a.a2", "", "[memory=3w,vcores=100%]"); csConf.setCapacityVector("root.b", "", "[memory=8095,vcores=30%]"); csConf.setCapacityVector("root.b.b1", "", "[memory=5w,vcores=3]"); @@ -82,6 +108,8 @@ private void setupQueueHierarchyWithWarnings() throws IOException { csConf.setState("root.b", QueueState.RUNNING); csConf.setCapacityVector("root.a", "", "[memory=100%,vcores=6w]"); csConf.setCapacityVector("root.a.a1", "", "[memory=2048,vcores=4]"); + csConf.setCapacityVector("root.a.a1.a11", "", "[memory=1w,vcores=4]"); + csConf.setCapacityVector("root.a.a1.a12", "", "[memory=100%,vcores=100%]"); csConf.setCapacityVector("root.a.a2", "", "[memory=2048,vcores=100%]"); csConf.setCapacityVector("root.b", "", "[memory=8096,vcores=30%]"); csConf.setCapacityVector("root.b.b1", "", "[memory=10256,vcores=3]"); @@ -89,4 +117,11 @@ private void setupQueueHierarchyWithWarnings() throws IOException { cs.reinitialize(csConf, mockRM.getRMContext()); } + + private Optional getSpecificWarning( + Collection warnings, QueueUpdateWarningType warningTypeToSelect, + String queue) { + return warnings.stream().filter((w) -> w.getWarningType().equals(warningTypeToSelect) && 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/TestQueueCapacityVector.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/TestQueueCapacityVector.java index 345bea511620f..058e14bfaf272 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/TestQueueCapacityVector.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/TestQueueCapacityVector.java @@ -29,13 +29,20 @@ import java.util.List; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; + public class TestQueueCapacityVector { + private static final String CUSTOM_RESOURCE = "custom"; + public static final String MIXED_CAPACITY_VECTOR_STRING = + "[custom=3.0,memory-mb=10.0w,vcores=6.0%]"; private final YarnConfiguration conf = new YarnConfiguration(); @Before public void setUp() { - conf.set(YarnConfiguration.RESOURCE_TYPES, "custom"); + conf.set(YarnConfiguration.RESOURCE_TYPES, CUSTOM_RESOURCE); ResourceUtils.resetResourceTypes(conf); } @@ -43,38 +50,40 @@ public void setUp() { public void getResourceNamesByCapacityType() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource("memory-mb", 10, QueueCapacityType.PERCENTAGE); - capacityVector.setResource("vcores", 6, QueueCapacityType.PERCENTAGE); + capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.PERCENTAGE); + capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); // custom is not set, defaults to 0 Assert.assertEquals(1, capacityVector.getResourceNamesByCapacityType( QueueCapacityType.ABSOLUTE).size()); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.ABSOLUTE).contains("custom")); + QueueCapacityType.ABSOLUTE).contains(CUSTOM_RESOURCE)); Assert.assertEquals(2, capacityVector.getResourceNamesByCapacityType( QueueCapacityType.PERCENTAGE).size()); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.PERCENTAGE).contains("vcores")); + QueueCapacityType.PERCENTAGE).contains(VCORES_URI)); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.PERCENTAGE).contains("memory-mb")); + QueueCapacityType.PERCENTAGE).contains(MEMORY_URI)); + Assert.assertEquals(10, capacityVector.getResource(MEMORY_URI).getResourceValue(), EPSILON); + Assert.assertEquals(6, capacityVector.getResource(VCORES_URI).getResourceValue(), EPSILON); } @Test public void isResourceOfType() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource("memory-mb", 10, QueueCapacityType.WEIGHT); - capacityVector.setResource("vcores", 6, QueueCapacityType.PERCENTAGE); - capacityVector.setResource("custom", 3, QueueCapacityType.ABSOLUTE); + capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.WEIGHT); + capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); + capacityVector.setResource(CUSTOM_RESOURCE, 3, QueueCapacityType.ABSOLUTE); - Assert.assertTrue(capacityVector.isResourceOfType("memory-mb", QueueCapacityType.WEIGHT)); - Assert.assertTrue(capacityVector.isResourceOfType("vcores", QueueCapacityType.PERCENTAGE)); - Assert.assertTrue(capacityVector.isResourceOfType("custom", QueueCapacityType.ABSOLUTE)); + Assert.assertTrue(capacityVector.isResourceOfType(MEMORY_URI, QueueCapacityType.WEIGHT)); + Assert.assertTrue(capacityVector.isResourceOfType(VCORES_URI, QueueCapacityType.PERCENTAGE)); + Assert.assertTrue(capacityVector.isResourceOfType(CUSTOM_RESOURCE, QueueCapacityType.ABSOLUTE)); } @Test - public void iterator() { + public void testIterator() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); List entries = Lists.newArrayList(capacityVector); @@ -90,11 +99,11 @@ public void iterator() { public void testToString() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource("memory-mb", 10, QueueCapacityType.WEIGHT); - capacityVector.setResource("vcores", 6, QueueCapacityType.PERCENTAGE); - capacityVector.setResource("custom", 3, QueueCapacityType.ABSOLUTE); + capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.WEIGHT); + capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); + capacityVector.setResource(CUSTOM_RESOURCE, 3, QueueCapacityType.ABSOLUTE); - Assert.assertEquals("[custom=3.0,memory-mb=10.0w,vcores=6.0%]", capacityVector.toString()); + Assert.assertEquals(MIXED_CAPACITY_VECTOR_STRING, capacityVector.toString()); QueueCapacityVector emptyCapacityVector = new QueueCapacityVector(); Assert.assertEquals("[]", emptyCapacityVector.toString()); 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/TestResourceVector.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/TestResourceVector.java index 6b6b9753189e8..fd6edb1fa5d49 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/TestResourceVector.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/TestResourceVector.java @@ -28,60 +28,76 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; + public class TestResourceVector { + private final static String CUSTOM_RESOURCE = "custom"; + private final YarnConfiguration conf = new YarnConfiguration(); @Before public void setUp() { - conf.set(YarnConfiguration.RESOURCE_TYPES, "custom"); + conf.set(YarnConfiguration.RESOURCE_TYPES, CUSTOM_RESOURCE); ResourceUtils.resetResourceTypes(conf); } @Test public void testCreation() { ResourceVector zeroResourceVector = ResourceVector.newInstance(); - Assert.assertEquals(0, zeroResourceVector.getValue("memory-mb"), 1e-6); - Assert.assertEquals(0, zeroResourceVector.getValue("vcores"), 1e-6); - Assert.assertEquals(0, zeroResourceVector.getValue("custom"), 1e-6); + Assert.assertEquals(0, zeroResourceVector.getValue(MEMORY_URI), EPSILON); + Assert.assertEquals(0, zeroResourceVector.getValue(VCORES_URI), EPSILON); + Assert.assertEquals(0, zeroResourceVector.getValue(CUSTOM_RESOURCE), EPSILON); ResourceVector uniformResourceVector = ResourceVector.of(10); - Assert.assertEquals(10, uniformResourceVector.getValue("memory-mb"), 1e-6); - Assert.assertEquals(10, uniformResourceVector.getValue("vcores"), 1e-6); - Assert.assertEquals(10, uniformResourceVector.getValue("custom"), 1e-6); + Assert.assertEquals(10, uniformResourceVector.getValue(MEMORY_URI), EPSILON); + Assert.assertEquals(10, uniformResourceVector.getValue(VCORES_URI), EPSILON); + Assert.assertEquals(10, uniformResourceVector.getValue(CUSTOM_RESOURCE), EPSILON); Map customResources = new HashMap<>(); - customResources.put("custom", 2L); + customResources.put(CUSTOM_RESOURCE, 2L); Resource resource = Resource.newInstance(10, 5, customResources); ResourceVector resourceVectorFromResource = ResourceVector.of(resource); - Assert.assertEquals(10, resourceVectorFromResource.getValue("memory-mb"), 1e-6); - Assert.assertEquals(5, resourceVectorFromResource.getValue("vcores"), 1e-6); - Assert.assertEquals(2, resourceVectorFromResource.getValue("custom"), 1e-6); + Assert.assertEquals(10, resourceVectorFromResource.getValue(MEMORY_URI), EPSILON); + Assert.assertEquals(5, resourceVectorFromResource.getValue(VCORES_URI), EPSILON); + Assert.assertEquals(2, resourceVectorFromResource.getValue(CUSTOM_RESOURCE), EPSILON); } @Test - public void subtract() { + public void testSubtract() { ResourceVector lhsResourceVector = ResourceVector.of(13); ResourceVector rhsResourceVector = ResourceVector.of(5); lhsResourceVector.subtract(rhsResourceVector); - Assert.assertEquals(8, lhsResourceVector.getValue("memory-mb"), 1e-6); - Assert.assertEquals(8, lhsResourceVector.getValue("vcores"), 1e-6); - Assert.assertEquals(8, lhsResourceVector.getValue("custom"), 1e-6); + Assert.assertEquals(8, lhsResourceVector.getValue(MEMORY_URI), EPSILON); + Assert.assertEquals(8, lhsResourceVector.getValue(VCORES_URI), EPSILON); + Assert.assertEquals(8, lhsResourceVector.getValue(CUSTOM_RESOURCE), EPSILON); + + ResourceVector negativeResourceVector = ResourceVector.of(-100); + + // Check whether overflow causes any issues + negativeResourceVector.subtract(ResourceVector.of(Float.MAX_VALUE)); + Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(MEMORY_URI), EPSILON); + Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(VCORES_URI), EPSILON); + Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(CUSTOM_RESOURCE), + EPSILON); + } @Test - public void increment() { + public void testIncrement() { ResourceVector resourceVector = ResourceVector.of(13); - resourceVector.increment("memory-mb", 5); + resourceVector.increment(MEMORY_URI, 5); - Assert.assertEquals(18, resourceVector.getValue("memory-mb"), 1e-6); - Assert.assertEquals(13, resourceVector.getValue("vcores"), 1e-6); - Assert.assertEquals(13, resourceVector.getValue("custom"), 1e-6); + Assert.assertEquals(18, resourceVector.getValue(MEMORY_URI), EPSILON); + Assert.assertEquals(13, resourceVector.getValue(VCORES_URI), EPSILON); + Assert.assertEquals(13, resourceVector.getValue(CUSTOM_RESOURCE), EPSILON); // Check whether overflow causes any issues ResourceVector maxFloatResourceVector = ResourceVector.of(Float.MAX_VALUE); - maxFloatResourceVector.increment("memory-mb", 100); - Assert.assertEquals(Float.MAX_VALUE, maxFloatResourceVector.getValue("memory-mb"), 1e-6); + maxFloatResourceVector.increment(MEMORY_URI, 100); + Assert.assertEquals(Float.MAX_VALUE, maxFloatResourceVector.getValue(MEMORY_URI), EPSILON); } @Test @@ -90,13 +106,13 @@ public void testEquals() { ResourceVector resourceVectorOther = ResourceVector.of(14); Resource resource = Resource.newInstance(13, 13); - Assert.assertFalse(resourceVector.equals(null)); - Assert.assertFalse(resourceVector.equals(resourceVectorOther)); - Assert.assertFalse(resourceVector.equals(resource)); + Assert.assertNotEquals(null, resourceVector); + Assert.assertNotEquals(resourceVectorOther, resourceVector); + Assert.assertNotEquals(resource, resourceVector); ResourceVector resourceVectorOne = ResourceVector.of(1); resourceVectorOther.subtract(resourceVectorOne); - Assert.assertTrue(resourceVector.equals(resourceVectorOther)); + Assert.assertEquals(resourceVectorOther, resourceVector); } } \ 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/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 27f83322ba95c..8ea9f56690707 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 @@ -29,13 +29,17 @@ public class TestUniformQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { private static final Resource QUEUE_A_RES = Resource.newInstance(80 * GB, - 15); + 10); private static final Resource QUEUE_B_RES = Resource.newInstance( 170 * GB, - 25); + 30); private static final Resource QUEUE_A1_RES = Resource.newInstance(50 * GB, - 5); + 4); private static final Resource QUEUE_A2_RES = Resource.newInstance(30 * GB, - 5); + 6); + private static final Resource QUEUE_A11_RES = Resource.newInstance(40 * GB, + 2); + private static final Resource QUEUE_A12_RES = Resource.newInstance(10 * GB, + 2); private static final Resource UPDATE_RES = Resource.newInstance(250 * GB, 40); private static final Resource PERCENTAGE_ALL_RES = Resource.newInstance(10 * GB, 20); private static final Resource WEIGHT_ALL_RES = Resource.newInstance(10 * GB, 20); @@ -56,6 +60,8 @@ public void testPercentageResourceCalculation() throws IOException { csConf.setCapacity("root.a", 30); csConf.setCapacity("root.b", 70); csConf.setCapacity("root.a.a1", 17); + csConf.setCapacity("root.a.a1.a11", 25); + csConf.setCapacity("root.a.a1.a12", 75); csConf.setCapacity("root.a.a2", 83); QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() @@ -90,6 +96,22 @@ public void testPercentageResourceCalculation() throws IOException { .assertCapacity() .toExpect(0.3f * 0.83f) .assertAbsoluteCapacity() + + .withQueue("root.a.a1.a11") + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.25f * 0.3f * 0.17f)) + .assertEffectiveMinResource() + .toExpect(0.25f) + .assertCapacity() + .toExpect(0.25f * 0.3f * 0.17f) + .assertAbsoluteCapacity() + + .withQueue("root.a.a1.a12") + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.75f * 0.3f * 0.17f)) + .assertEffectiveMinResource() + .toExpect(0.75f) + .assertCapacity() + .toExpect(0.75f * 0.3f * 0.17f) + .assertAbsoluteCapacity() .build(); update(queueAssertionBuilder, PERCENTAGE_ALL_RES); @@ -101,23 +123,28 @@ public void testAbsoluteResourceCalculation() throws IOException { csConf.setMinimumResourceRequirement("", "root.b", QUEUE_B_RES); csConf.setMinimumResourceRequirement("", "root.a.a1", QUEUE_A1_RES); csConf.setMinimumResourceRequirement("", "root.a.a2", QUEUE_A2_RES); + csConf.setMinimumResourceRequirement("", "root.a.a1.a11", QUEUE_A11_RES); + csConf.setMinimumResourceRequirement("", "root.a.a1.a12", QUEUE_A12_RES); QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() .withQueue("root.a") .toExpect(QUEUE_A_RES) .assertEffectiveMinResource() - .withQueue("root.b") .toExpect(QUEUE_B_RES) .assertEffectiveMinResource() - .withQueue("root.a.a1") .toExpect(QUEUE_A1_RES) .assertEffectiveMinResource() - .withQueue("root.a.a2") .toExpect(QUEUE_A2_RES) .assertEffectiveMinResource() + .withQueue("root.a.a1.a11") + .toExpect(QUEUE_A11_RES) + .assertEffectiveMinResource() + .withQueue("root.a.a1.a12") + .toExpect(QUEUE_A12_RES) + .assertEffectiveMinResource() .build(); update(queueAssertionBuilder, UPDATE_RES); @@ -126,18 +153,21 @@ public void testAbsoluteResourceCalculation() throws IOException { .withQueue("root.a") .toExpect(ResourceUtils.multiply(QUEUE_A_RES, 0.5f)) .assertEffectiveMinResource() - .withQueue("root.b") .toExpect(ResourceUtils.multiply(QUEUE_B_RES, 0.5f)) .assertEffectiveMinResource() - .withQueue("root.a.a1") .toExpect(ResourceUtils.multiply(QUEUE_A1_RES, 0.5f)) .assertEffectiveMinResource() - .withQueue("root.a.a2") .toExpect(ResourceUtils.multiply(QUEUE_A2_RES, 0.5f)) .assertEffectiveMinResource() + .withQueue("root.a.a1.a11") + .toExpect(ResourceUtils.multiply(QUEUE_A11_RES, 0.5f)) + .assertEffectiveMinResource() + .withQueue("root.a.a1.a12") + .toExpect(ResourceUtils.multiply(QUEUE_A12_RES, 0.5f)) + .assertEffectiveMinResource() .build(); update(queueAssertionHalfClusterResource, ResourceUtils.multiply(UPDATE_RES, 0.5f)); 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 c3055cf050e1c..7a3d4e9048d0c 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 @@ -30,179 +30,183 @@ import java.util.List; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.GPU_URI; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +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; + public class TestQueueCapacityConfigParser { + private static final String ALL_RESOURCE_TEMPLATE = "[memory-mb=%s, vcores=%s, yarn.io/gpu=%s]"; + private static final String MEMORY_VCORE_TEMPLATE = "[memory-mb=%s, vcores=%s]"; + + private static final String MEMORY_ABSOLUTE = "12Gi"; + private static final float VCORE_ABSOLUTE = 6; + private static final float GPU_ABSOLUTE = 10; + + private static final float PERCENTAGE_VALUE = 50f; + private static final float MEMORY_MIXED = 1024; + private static final float WEIGHT_VALUE = 6; + private static final String QUEUE = "root.test"; - private static final String ABSOLUTE_RESOURCE = "[memory-mb=12Gi, vcores=6, yarn.io/gpu=10]"; - private static final String ABSOLUTE_RESOURCE_MEMORY_VCORE = "[memory-mb=12Gi, vcores=6]"; - private static final String MIXED_RESOURCE = "[memory-mb=1024, vcores=50%, yarn.io/gpu=6w]"; - private static final String RESOURCE_TYPES = "yarn.io/gpu"; + + private static final String ABSOLUTE_RESOURCE = String.format( + ALL_RESOURCE_TEMPLATE, MEMORY_ABSOLUTE, VCORE_ABSOLUTE, GPU_ABSOLUTE); + private static final String ABSOLUTE_RESOURCE_MEMORY_VCORE = String.format( + MEMORY_VCORE_TEMPLATE, MEMORY_ABSOLUTE, VCORE_ABSOLUTE); + private static final String MIXED_RESOURCE = String.format( + ALL_RESOURCE_TEMPLATE, MEMORY_MIXED, PERCENTAGE_VALUE + "%", WEIGHT_VALUE + "w"); + private static final String RESOURCE_TYPES = GPU_URI; + + public static final String NONEXISTINGSUFFIX = "50nonexistingsuffix"; + public static final String EMPTY_BRACKET = "[]"; + public static final String INVALID_CAPACITY_BRACKET = "[invalid]"; + public static final String INVALID_CAPACITY_FORMAT = "[memory-100,vcores-60]"; private final QueueCapacityConfigParser capacityConfigParser = new QueueCapacityConfigParser(); @Test public void testPercentageCapacityConfig() { - CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.setCapacity(QUEUE, 50); + QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), QUEUE); + QueueCapacityVectorEntry memory = percentageCapacityVector.getResource(MEMORY_URI); + QueueCapacityVectorEntry vcore = percentageCapacityVector.getResource(VCORES_URI); - QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, ""); - List resources = Lists.newArrayList(percentageCapacityVector.iterator()); + Assert.assertEquals(QueueCapacityType.PERCENTAGE, memory.getVectorResourceType()); + Assert.assertEquals(PERCENTAGE_VALUE, memory.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType()); - Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.PERCENTAGE, vcore.getVectorResourceType()); + Assert.assertEquals(PERCENTAGE_VALUE, vcore.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType()); - Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6); + QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), + CapacitySchedulerConfiguration.ROOT); - QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf, - CapacitySchedulerConfiguration.ROOT, ""); - List rootResources = - Lists.newArrayList(rootCapacityVector.iterator()); + QueueCapacityVectorEntry memoryRoot = rootCapacityVector.getResource(MEMORY_URI); + QueueCapacityVectorEntry vcoreRoot = rootCapacityVector.getResource(VCORES_URI); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, - rootResources.get(0).getVectorResourceType()); - Assert.assertEquals(100f, - rootResources.get(0).getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.PERCENTAGE, memoryRoot.getVectorResourceType()); + Assert.assertEquals(100f, memoryRoot.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, - rootResources.get(1).getVectorResourceType()); - Assert.assertEquals(100f, - rootResources.get(1).getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.PERCENTAGE, vcoreRoot.getVectorResourceType()); + Assert.assertEquals(100f, vcoreRoot.getResourceValue(), EPSILON); } @Test public void testWeightCapacityConfig() { - CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.setNonLabeledQueueWeight(QUEUE, 6); + QueueCapacityVector weightCapacityVector = capacityConfigParser.parse(WEIGHT_VALUE + "w", QUEUE); - QueueCapacityVector weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, ""); - List resources = Lists.newArrayList(weightCapacityVector.iterator()); + QueueCapacityVectorEntry memory = weightCapacityVector.getResource(MEMORY_URI); + QueueCapacityVectorEntry vcore = weightCapacityVector.getResource(VCORES_URI); - Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType()); - Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.WEIGHT, memory.getVectorResourceType()); + Assert.assertEquals(WEIGHT_VALUE, memory.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType()); - Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.WEIGHT, vcore.getVectorResourceType()); + Assert.assertEquals(WEIGHT_VALUE, vcore.getResourceValue(), EPSILON); } @Test public void testAbsoluteCapacityVectorConfig() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + 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(conf, QUEUE, ""); + QueueCapacityVector absoluteCapacityVector = capacityConfigParser.parse(ABSOLUTE_RESOURCE, QUEUE); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource("memory-mb").getVectorResourceType()); - Assert.assertEquals(12 * 1024, absoluteCapacityVector.getResource("memory-mb").getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(MEMORY_URI) + .getVectorResourceType()); + Assert.assertEquals(12 * GB, absoluteCapacityVector.getResource(MEMORY_URI) + .getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource("vcores").getVectorResourceType()); - Assert.assertEquals(6f, absoluteCapacityVector.getResource("vcores").getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) + .getVectorResourceType()); + Assert.assertEquals(VCORE_ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) + .getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource("yarn.io/gpu").getVectorResourceType()); - Assert.assertEquals(10f, absoluteCapacityVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) + .getVectorResourceType()); + Assert.assertEquals(GPU_ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) + .getResourceValue(), EPSILON); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) + CapacitySchedulerConfiguration.CAPACITY, ABSOLUTE_RESOURCE_MEMORY_VCORE); - QueueCapacityVector withoutGpuVector = capacityConfigParser.parse(conf, QUEUE, ""); + QueueCapacityVector withoutGpuVector = capacityConfigParser.parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); Assert.assertEquals(3, withoutGpuVector.getResourceCount()); - Assert.assertEquals(0f, withoutGpuVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + Assert.assertEquals(0f, withoutGpuVector.getResource(GPU_URI).getResourceValue(), EPSILON); } @Test public void testMixedCapacityConfig() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, MIXED_RESOURCE); conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); ResourceUtils.resetResourceTypes(conf); QueueCapacityVector mixedCapacityVector = - capacityConfigParser.parse(conf, QUEUE, ""); + capacityConfigParser.parse(MIXED_RESOURCE, QUEUE); Assert.assertEquals(QueueCapacityType.ABSOLUTE, - mixedCapacityVector.getResource("memory-mb").getVectorResourceType()); - Assert.assertEquals(1024, mixedCapacityVector.getResource("memory-mb").getResourceValue(), 1e-6); + mixedCapacityVector.getResource(MEMORY_URI).getVectorResourceType()); + Assert.assertEquals(MEMORY_MIXED, mixedCapacityVector.getResource(MEMORY_URI) + .getResourceValue(), EPSILON); Assert.assertEquals(QueueCapacityType.PERCENTAGE, - mixedCapacityVector.getResource("vcores").getVectorResourceType()); - Assert.assertEquals(50f, - mixedCapacityVector.getResource("vcores").getResourceValue(), 1e-6); + mixedCapacityVector.getResource(VCORES_URI).getVectorResourceType()); + Assert.assertEquals(PERCENTAGE_VALUE, + mixedCapacityVector.getResource(VCORES_URI).getResourceValue(), EPSILON); Assert.assertEquals(QueueCapacityType.WEIGHT, - mixedCapacityVector.getResource("yarn.io/gpu").getVectorResourceType()); - Assert.assertEquals(6f, - mixedCapacityVector.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + mixedCapacityVector.getResource(GPU_URI).getVectorResourceType()); + Assert.assertEquals(WEIGHT_VALUE, + mixedCapacityVector.getResource(GPU_URI).getResourceValue(), EPSILON); // Test undefined capacity type default value - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, "[memory-mb=12Gi,vcores=6]"); - QueueCapacityVector mixedCapacityVectorWithGpuUndefined = - capacityConfigParser.parse(conf, QUEUE, ""); + capacityConfigParser.parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); Assert.assertEquals(QueueCapacityType.ABSOLUTE, - mixedCapacityVectorWithGpuUndefined.getResource("memory-mb").getVectorResourceType()); - Assert.assertEquals(0, mixedCapacityVectorWithGpuUndefined.getResource("yarn.io/gpu").getResourceValue(), 1e-6); + mixedCapacityVectorWithGpuUndefined.getResource(MEMORY_URI).getVectorResourceType()); + Assert.assertEquals(0, mixedCapacityVectorWithGpuUndefined.getResource(GPU_URI) + .getResourceValue(), EPSILON); } @Test public void testInvalidCapacityConfigs() { - CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); - - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, "50nonexistingsuffix"); QueueCapacityVector capacityVectorWithInvalidSuffix = - capacityConfigParser.parse(conf, QUEUE, ""); + capacityConfigParser.parse(NONEXISTINGSUFFIX, QUEUE); List entriesWithInvalidSuffix = Lists.newArrayList(capacityVectorWithInvalidSuffix.iterator()); Assert.assertEquals(0, entriesWithInvalidSuffix.size()); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, "[memory-100,vcores-60]"); QueueCapacityVector invalidDelimiterCapacityVector = - capacityConfigParser.parse(conf, QUEUE, ""); + capacityConfigParser.parse(INVALID_CAPACITY_FORMAT, QUEUE); List invalidDelimiterEntries = Lists.newArrayList(invalidDelimiterCapacityVector.iterator()); Assert.assertEquals(0, invalidDelimiterEntries.size()); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, "[invalid]"); QueueCapacityVector invalidCapacityVector = - capacityConfigParser.parse(conf, QUEUE, ""); + capacityConfigParser.parse(INVALID_CAPACITY_BRACKET, QUEUE); List resources = Lists.newArrayList(invalidCapacityVector.iterator()); Assert.assertEquals(0, resources.size()); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, "[]"); QueueCapacityVector emptyBracketCapacityVector = - capacityConfigParser.parse(conf, QUEUE, ""); + capacityConfigParser.parse(EMPTY_BRACKET, QUEUE); List emptyEntries = Lists.newArrayList(emptyBracketCapacityVector.iterator()); - Assert.assertEquals(0, resources.size()); + Assert.assertEquals(0, emptyEntries.size()); - conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY, ""); QueueCapacityVector emptyCapacity = - capacityConfigParser.parse(conf, QUEUE, ""); + capacityConfigParser.parse("", QUEUE); List emptyResources = Lists.newArrayList(emptyCapacity.iterator()); Assert.assertEquals(emptyResources.size(), 0); - conf.unset(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE) - + CapacitySchedulerConfiguration.CAPACITY); QueueCapacityVector nonSetCapacity = - capacityConfigParser.parse(conf, QUEUE, ""); + capacityConfigParser.parse(null, QUEUE); List nonSetResources = Lists.newArrayList(nonSetCapacity.iterator()); Assert.assertEquals(nonSetResources.size(), 0); } - - @Test - public void testResourceVector() { - - } } \ No newline at end of file From 83aabd01eca910987b57285fe54167019ac9145a Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Mon, 15 Nov 2021 14:23:54 +0100 Subject: [PATCH 10/27] YARN-10965. Simplify logic in QueueHandler --- .../yarn/util/resource/ResourceUtils.java | 12 + .../AbsoluteResourceCapacityCalculator.java | 70 ++-- .../scheduler/capacity/AbstractCSQueue.java | 30 +- .../AbstractQueueCapacityCalculator.java | 236 +++++++------- .../scheduler/capacity/CSQueue.java | 22 +- .../capacity/CSQueueUsageTracker.java | 14 + .../CapacitySchedulerConfiguration.java | 16 +- ...CapacitySchedulerQueueCapacityHandler.java | 289 ++++++++++++----- .../CapacitySchedulerQueueManager.java | 6 + .../DefaultQueueResourceRoundingStrategy.java | 38 +++ .../scheduler/capacity/LeafQueue.java | 47 +++ .../scheduler/capacity/ParentQueue.java | 178 +++++----- .../PercentageQueueCapacityCalculator.java | 67 ++-- .../capacity/QueueBranchContext.java | 55 ++-- ...t.java => QueueCapacityUpdateContext.java} | 78 ++--- .../capacity/QueueCapacityVector.java | 58 +++- .../QueueResourceRoundingStrategy.java | 36 +++ .../capacity/QueueUpdateWarning.java | 23 +- .../scheduler/capacity/ResourceVector.java | 25 +- .../capacity/RootQueueCapacityCalculator.java | 43 +-- .../WeightQueueCapacityCalculator.java | 107 ++++--- .../conf/QueueCapacityConfigParser.java | 22 +- ...citySchedulerQueueCalculationTestBase.java | 62 +++- .../TestMixedQueueResourceCalculation.java | 303 ++++++++++++++---- .../capacity/TestQueueCapacityVector.java | 34 +- .../TestUniformQueueResourceCalculation.java | 171 ++++++---- .../conf/TestQueueCapacityConfigParser.java | 28 +- 27 files changed, 1348 insertions(+), 722 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/DefaultQueueResourceRoundingStrategy.java rename hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/{QueueHierarchyUpdateContext.java => QueueCapacityUpdateContext.java} (52%) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueResourceRoundingStrategy.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java index fbc0a73ec3a90..454bdf6438768 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java @@ -51,6 +51,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -829,6 +830,17 @@ public static Resource multiply(Resource lhs, float rhs) { return newResource; } + public static Resource multiplyRound(Resource lhs, float rhs) { + Resource newResource = Resource.newInstance(0, 0); + + for (ResourceInformation resourceInformation : lhs.getResources()) { + newResource.setResourceValue(resourceInformation.getName(), + Math.round(resourceInformation.getValue() * rhs)); + } + + return newResource; + } + @InterfaceAudience.Private @InterfaceStability.Unstable public static Resource createResourceFromString( 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index 620da28d0dbf7..25357e06095a2 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -1,65 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - protected float calculateMinimumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + public float calculateMinimumResource( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { String resourceName = capacityVectorEntry.getResourceName(); - ResourceVector ratio = updateContext.getNormalizedMinResourceRatio( - childQueue.getParent().getQueuePath(), label); + ResourceVector ratio = updateContext.getQueueBranchContext(childQueue.getParent().getQueuePath()) + .getNormalizedResourceRatios().getOrDefault(label, ResourceVector.of(1)); return ratio.getValue(resourceName) * capacityVectorEntry.getResourceValue(); } @Override - protected float calculateMaximumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + public float calculateMaximumResource( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { return capacityVectorEntry.getResourceValue(); } @Override - protected QueueCapacityType getCapacityType() { - return QueueCapacityType.ABSOLUTE; - } - - @Override - public void setup(CSQueue queue, String label) { - Resource minResource = Resource.newInstance(0, 0); - - for (String resourceName : getResourceNames(queue, label)) { - long resource = (long) queue.getConfiguredCapacityVector( - label).getResource(resourceName).getResourceValue(); - minResource.setResourceValue(resourceName, minResource.getResourceValue( - resourceName) + resource); - } - - queue.getQueueResourceQuotas().setConfiguredMinResource(label, minResource); + public void updateCapacitiesAfterCalculation( + QueueCapacityUpdateContext updateContext, CSQueue queue, String label) { + setQueueCapacities(updateContext.getUpdatedClusterResource(label), queue, label); } @Override - public void setMetrics(QueueHierarchyUpdateContext updateContext, - CSQueue queue, String label) { - float sumCapacity = 0f; - float sumAbsoluteCapacity = 0f; - - for (String resourceName : getResourceNames(queue, label)) { - sumCapacity += queue.getConfiguredCapacityVector(label).getResource( - resourceName).getResourceValue() / queue.getParent() - .getQueueResourceQuotas().getEffectiveMinResource(label) - .getResourceValue(resourceName); - sumAbsoluteCapacity += updateContext.getAbsoluteMinCapacity( - queue.getQueuePath(), label).getValue(resourceName); - } - - queue.getQueueCapacities().setCapacity(label, sumCapacity); - queue.getQueueCapacities().setAbsoluteCapacity(label, sumAbsoluteCapacity); + public ResourceUnitCapacityType getCapacityType() { + return ResourceUnitCapacityType.ABSOLUTE; } } 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 d6772fa9b68b0..83f52dfd8bb49 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 @@ -196,6 +196,20 @@ public float getAbsoluteCapacity() { return queueCapacities.getAbsoluteCapacity(); } + @Override + public ResourceVector getOrCreateAbsoluteMinCapacityVector(String label) { + usageTracker.getAbsoluteMinCapacityVector().putIfAbsent(label, ResourceVector.newInstance()); + + return usageTracker.getAbsoluteMinCapacityVector().get(label); + } + + @Override + public ResourceVector getOrCreateAbsoluteMaxCapacityVector(String label) { + usageTracker.getAbsoluteMaxCapacityVector().putIfAbsent(label, ResourceVector.newInstance()); + + return usageTracker.getAbsoluteMaxCapacityVector().get(label); + } + @Override public float getAbsoluteMaximumCapacity() { return queueCapacities.getAbsoluteMaximumCapacity(); @@ -368,11 +382,11 @@ protected void setupQueueConfigs(Resource clusterResource, this.reservationsContinueLooking = configuration.getReservationContinueLook(); - this.configuredCapacityVectors = csContext.getConfiguration() + this.configuredCapacityVectors = configuration .parseConfiguredResourceVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels()); this.configuredMaximumCapacityVectors = configuration - .parseConfiguredMaximumCapacityVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels()); + .parseConfiguredMaximumCapacityVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels(), QueueCapacityVector.newInstance()); // Update metrics CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, this, labelManager, null); @@ -567,7 +581,17 @@ public QueueCapacityVector getConfiguredCapacityVector(String label) { @Override public QueueCapacityVector getConfiguredMaximumCapacityVector(String label) { - return configuredCapacityVectors.get(label); + return configuredMaximumCapacityVectors.get(label); + } + + @Override + public void setConfiguredMinCapacityVector(String label, QueueCapacityVector minCapacityVector) { + configuredCapacityVectors.put(label, minCapacityVector); + } + + @Override + public void setConfiguredMaxCapacityVector(String label, QueueCapacityVector maxCapacityVector) { + configuredMaximumCapacityVectors.put(label, maxCapacityVector); } private void initializeQueueState(CapacitySchedulerConfiguration 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 2979a096b13da..61b57a7860dac 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -18,16 +18,21 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; import org.apache.hadoop.yarn.util.UnitsConversionUtil; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; -import java.util.HashMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedList; import java.util.Map; import java.util.Set; import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; /** * A strategy class to encapsulate queue capacity setup and resource calculation @@ -36,78 +41,62 @@ public abstract class AbstractQueueCapacityCalculator { /** - * Sets all field of the queue based on its configurations. - * - * @param queue queue to setup - * @param label node label - */ - public abstract void setup(CSQueue queue, String label); - - /** - * Sets the metrics and statistics after effective resource calculation. + * Sets the metrics and statistics after effective resource values calculation. * * @param updateContext context of the current update phase * @param queue queue to update * @param label node label */ - public abstract void setMetrics(QueueHierarchyUpdateContext updateContext, CSQueue queue, - String label); - - /** - * Calculate the effective resource for a specific resource. - * - * @param updateContext context of the current update phase - * @param parentQueue the parent whose children will be updated - */ - public void calculateChildQueueResources(QueueHierarchyUpdateContext updateContext, - CSQueue parentQueue) { - calculateResourcePrerequisites(updateContext, parentQueue); - - Map aggregatedResources = new HashMap<>(); - for (CSQueue childQueue : parentQueue.getChildQueues()) { - childQueue.getWriteLock().lock(); - try { - for (String label : childQueue.getConfiguredNodeLabels()) { - ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault(label, - ResourceVector.newInstance()); - calculateResources(updateContext, childQueue, label, aggregatedUsedResource); - aggregatedResources.put(label, aggregatedUsedResource); - } - } finally { - childQueue.getWriteLock().unlock(); - } - } + public abstract void updateCapacitiesAfterCalculation( + QueueCapacityUpdateContext updateContext, CSQueue queue, String label); - for (Map.Entry entry : aggregatedResources.entrySet()) { - updateContext.getQueueBranchContext(parentQueue.getQueuePath()).getRemainingResource( - entry.getKey()).subtract(entry.getValue()); - } - } /** * Returns the capacity type the calculator could handle. * * @return capacity type */ - protected abstract QueueCapacityType getCapacityType(); + public abstract ResourceUnitCapacityType getCapacityType(); - protected abstract float calculateMinimumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + /** + * Calculates the minimum effective resource. + * + * @param updateContext context of the current update phase + * @param childQueue queue to update + * @param label node label + * @param capacityVectorEntry resource unit for which the calculation is executed + * @return minimum effective resource + */ + public abstract float calculateMinimumResource( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry); - protected abstract float calculateMaximumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + /** + * Calculates the maximum effective resource. + * + * @param updateContext context of the current update phase + * @param childQueue queue to update + * @param label node label + * @param capacityVectorEntry resource unit for which the calculation is executed + * @return minimum effective resource + */ + public abstract float calculateMaximumResource( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry); - protected void calculateResourcePrerequisites(QueueHierarchyUpdateContext updateContext, - CSQueue parentQueue) { + /** + * Executes all logic that must be called prior to the effective resource value calculations. + * + * @param updateContext context of the current update phase + * @param parentQueue parent for which the prerequisite actions must be executed + */ + public void calculateResourcePrerequisites(QueueCapacityUpdateContext updateContext, + CSQueue parentQueue) { for (String label : parentQueue.getConfiguredNodeLabels()) { // We need to set normalized resource ratio only once per parent - if (!updateContext.getQueueBranchContext(parentQueue.getQueuePath()) - .isParentAlreadyUpdated()) { + if (updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + .getNormalizedResourceRatios().isEmpty()) { setNormalizedResourceRatio(updateContext, parentQueue, label); - updateContext.getQueueBranchContext(parentQueue.getQueuePath()) - .setUpdateFlag(); } } } @@ -133,22 +122,70 @@ protected Set getResourceNames(CSQueue queue, String label) { * @return resource names */ protected Set getResourceNames(CSQueue queue, String label, - QueueCapacityType capacityType) { + ResourceUnitCapacityType capacityType) { return queue.getConfiguredCapacityVector(label) .getResourceNamesByCapacityType(capacityType); } - protected float sumCapacityValues(CSQueue queue, String label) { - float sumValue = 0f; - QueueCapacityVector capacityVector = queue.getConfiguredCapacityVector(label); - for (String resourceName : getResourceNames(queue, label)) { - sumValue += capacityVector.getResource(resourceName).getResourceValue(); + /** + * Sets capacity and absolute capacity values based on minimum and maximum effective resources. + * + * @param clusterResource cluster resource for the corresponding label + * @param queue child queue for which the capacities are set + * @param label node label + */ + public static void setQueueCapacities( + Resource clusterResource, CSQueue queue, String label) { + if (!(queue instanceof AbstractCSQueue)) { + return; + } + + AbstractCSQueue csQueue = (AbstractCSQueue) queue; + ResourceCalculator resourceCalculator = csQueue.resourceCalculator; + + CSQueue parent = queue.getParent(); + if (parent == null) { + return; } - return sumValue; + // Update capacity with a float calculated from the parent's minResources + // and the recently changed queue minResources. + // capacity = effectiveMinResource / {parent's effectiveMinResource} + float result = resourceCalculator.divide(clusterResource, + queue.getQueueResourceQuotas().getEffectiveMinResource(label), + parent.getQueueResourceQuotas().getEffectiveMinResource(label)); + queue.getQueueCapacities().setCapacity(label, + Float.isInfinite(result) ? 0 : result); + + // Update maxCapacity with a float calculated from the parent's maxResources + // and the recently changed queue maxResources. + // maxCapacity = effectiveMaxResource / parent's effectiveMaxResource + result = resourceCalculator.divide(clusterResource, + queue.getQueueResourceQuotas().getEffectiveMaxResource(label), + parent.getQueueResourceQuotas().getEffectiveMaxResource(label)); + queue.getQueueCapacities().setMaximumCapacity(label, + Float.isInfinite(result) ? 0 : result); + + csQueue.updateAbsoluteCapacities(); } + /** + * Calculates the normalized resource ratio of a parent queue, under which children are defined + * with absolute capacity type. If the effective resource of the parent is less, than the + * aggregated configured absolute resource of its children, the resource ratio will be less, + * than 1. + * + * @param updateContext context of the current update phase + * @param parentQueue parent for which the normalized ratio is defined + * @param label node label + */ private void setNormalizedResourceRatio( - QueueHierarchyUpdateContext updateContext, CSQueue parentQueue, String label) { + QueueCapacityUpdateContext updateContext, CSQueue parentQueue, String label) { + // ManagedParents assign zero capacity to queues in case of overutilization, downscaling is + // turned off for their children + if (parentQueue instanceof ManagedParentQueue) { + return; + } + for (QueueCapacityVectorEntry capacityVectorEntry : parentQueue.getConfiguredCapacityVector( label)) { String resourceName = capacityVectorEntry.getResourceName(); @@ -159,13 +196,15 @@ private void setNormalizedResourceRatio( // Total configured min resources of direct children of this given parent // queue for (CSQueue childQueue : parentQueue.getChildQueues()) { + if (!childQueue.getConfiguredNodeLabels().contains(label)) { + continue; + } QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); - if (capacityVector.isResourceOfType(resourceName, QueueCapacityType.ABSOLUTE)) { + if (capacityVector.isResourceOfType(resourceName, ResourceUnitCapacityType.ABSOLUTE)) { childrenConfiguredResource += capacityVector.getResource(resourceName).getResourceValue(); } } - // If no children is using ABSOLUTE capacity type, normalization is - // not needed + // If no children is using ABSOLUTE capacity type, normalization is not needed if (childrenConfiguredResource == 0) { continue; } @@ -186,69 +225,12 @@ private void setNormalizedResourceRatio( .getUnits(), childrenConfiguredResource); if (convertedValue != 0) { - updateContext.getNormalizedMinResourceRatio(parentQueue.getQueuePath(), label).setValue( - resourceName, numeratorForMinRatio / convertedValue); + Map normalizedResourceRatios = updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).getNormalizedResourceRatios(); + normalizedResourceRatios.putIfAbsent(label, ResourceVector.newInstance()); + normalizedResourceRatios.get(label).setValue(resourceName, numeratorForMinRatio / + convertedValue); } } } - - private void calculateResources( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, - ResourceVector aggregatedUsedResource) { - CSQueue parentQueue = childQueue.getParent(); - for (String resourceName : getResourceNames(childQueue, label)) { - long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( - resourceName); - float minimumResource = calculateMinimumResource(updateContext, childQueue, label, - childQueue.getConfiguredCapacityVector(label).getResource(resourceName)); - long parentMinimumResource = parentQueue.getEffectiveCapacity(label).getResourceValue( - resourceName); - - long parentMaximumResource = parentQueue.getEffectiveMaxCapacity(label).getResourceValue( - resourceName); - float maximumResource = calculateMaximumResource(updateContext, childQueue, label, - childQueue.getConfiguredMaximumCapacityVector(label).getResource(resourceName)); - - if (maximumResource != 0 && maximumResource > parentMaximumResource) { - updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( - childQueue.getQueuePath())); - } - maximumResource = maximumResource == 0 ? parentMaximumResource - : Math.min(maximumResource, parentMaximumResource); - - if (maximumResource < minimumResource) { - updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( - childQueue.getQueuePath())); - minimumResource = maximumResource; - } - - if (minimumResource > parentMinimumResource) { - updateContext.addUpdateWarning( - QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath()).withInfo( - "Resource name: " + resourceName + " resource value: " + minimumResource)); - minimumResource = parentMinimumResource; - } - - if (minimumResource == 0) { - updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( - childQueue.getQueuePath()).withInfo("Resource name: " + resourceName)); - } - - float absoluteMinCapacity = minimumResource / clusterResource; - float absoluteMaxCapacity = maximumResource / clusterResource; - updateContext.getAbsoluteMinCapacity(childQueue.getQueuePath(), label).setValue(resourceName, - absoluteMinCapacity); - updateContext.getAbsoluteMaxCapacity(childQueue.getQueuePath(), label).setValue(resourceName, - absoluteMaxCapacity); - - long roundedMinResource = (long) Math.floor(minimumResource); - long roundedMaxResource = (long) Math.floor(maximumResource); - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( - resourceName, roundedMinResource); - childQueue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( - resourceName, roundedMaxResource); - - aggregatedUsedResource.increment(resourceName, roundedMinResource); - } - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index 084ac81c8641e..4ff4637552246 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -108,7 +108,11 @@ public interface CSQueue extends SchedulerQueue { * cumulative capacity in the cluster */ public float getAbsoluteCapacity(); - + + public ResourceVector getOrCreateAbsoluteMinCapacityVector(String label); + + public ResourceVector getOrCreateAbsoluteMaxCapacityVector(String label); + /** * Get the configured maximum-capacity of the queue. * @return the configured maximum-capacity of the queue @@ -257,6 +261,9 @@ public void completedContainer(Resource clusterResource, public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) throws IOException; + public void refreshAfterResourceCalculation( + Resource clusterResource, ResourceLimits resourceLimits); + /** * Update the cluster resource for queues as we add/remove nodes * @param clusterResource the current cluster resource @@ -441,6 +448,19 @@ public void validateSubmitApplication(ApplicationId applicationId, */ QueueCapacityVector getConfiguredMaximumCapacityVector(String label); + /** + * Sets the configured minimum capacity vector to a specific value + * @param label node label (partition) + * @param minCapacityVector capacity vector + */ + void setConfiguredMinCapacityVector(String label, QueueCapacityVector minCapacityVector); + + /** + * Sets the configured maximum capacity vector to a specific value + * @param label node label (partition) + * @param maxCapacityVector capacity vector + */ + void setConfiguredMaxCapacityVector(String label, QueueCapacityVector maxCapacityVector); Set getConfiguredNodeLabels(); 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/CSQueueUsageTracker.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/CSQueueUsageTracker.java index 0f18e944e9a38..326a74477920f 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/CSQueueUsageTracker.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/CSQueueUsageTracker.java @@ -21,6 +21,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; +import java.util.HashMap; +import java.util.Map; + public class CSQueueUsageTracker { private final CSQueueMetrics metrics; private int numContainers; @@ -38,6 +41,9 @@ public class CSQueueUsageTracker { private final QueueResourceQuotas queueResourceQuotas; + private final Map absoluteMinCapacityVector = new HashMap<>(); + private final Map absoluteMaxCapacityVector = new HashMap<>(); + public CSQueueUsageTracker(CSQueueMetrics metrics) { this.metrics = metrics; this.queueUsage = new ResourceUsage(); @@ -75,4 +81,12 @@ public ResourceUsage getQueueUsage() { public QueueResourceQuotas getQueueResourceQuotas() { return queueResourceQuotas; } + + public Map getAbsoluteMinCapacityVector() { + return absoluteMinCapacityVector; + } + + public Map getAbsoluteMaxCapacityVector() { + return absoluteMaxCapacityVector; + } } 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 21cbe2c94b3b8..11ae4c3034e9c 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 @@ -608,7 +608,8 @@ public void setCapacity(String queue, String absoluteResourceCapacity) { public float getNonLabeledQueueMaximumCapacity(String queue) { String configuredCapacity = get(getQueuePrefix(queue) + MAXIMUM_CAPACITY); boolean matcher = (configuredCapacity != null) - && RESOURCE_PATTERN.matcher(configuredCapacity).find(); + && RESOURCE_PATTERN.matcher(configuredCapacity).find() + || queueCapacityConfigParser.isCapacityVectorFormat(configuredCapacity); if (matcher) { // Return capacity in percentage as 0 for non-root queues and 100 for // root.From AbstractCSQueue, absolute resource will be parsed and @@ -809,6 +810,11 @@ public void setCapacityVector(String queuePath, String label, String capacityVec set(capacityPropertyName, capacityVector); } + public void setMaximumCapacityVector(String queuePath, String label, String capacityVector) { + String capacityPropertyName = getNodeLabelPrefix(queuePath, label) + MAXIMUM_CAPACITY; + set(capacityPropertyName, capacityVector); + } + private boolean configuredWeightAsCapacity(String configureValue) { if (configureValue == null) { return false; @@ -2599,13 +2605,17 @@ public Map parseConfiguredResourceVector( } public Map parseConfiguredMaximumCapacityVector( - String queuePath, Set labels) { + String queuePath, Set labels, QueueCapacityVector defaultVector) { Map queueResourceVectors = new HashMap<>(); for (String label : labels) { String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( queuePath, label) + CapacitySchedulerConfiguration.MAXIMUM_CAPACITY; String capacityString = get(propertyName); - queueResourceVectors.put(label, queueCapacityConfigParser.parse(capacityString, queuePath)); + QueueCapacityVector capacityVector = queueCapacityConfigParser.parse(capacityString, queuePath); + if (capacityVector.isEmpty()) { + capacityVector = defaultVector; + } + queueResourceVectors.put(label, capacityVector); } return queueResourceVectors; 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index 95dfbbeafb191..f1a3fa8cebfb0 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -19,147 +19,290 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; +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.QueueUpdateWarning.QueueUpdateWarningType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; /** * Controls how capacity and resource values are set and calculated for a queue. - * Resources are set for each label and resource separately. + * Effective minimum and maximum resource values are set for each label and resource separately. */ public class CapacitySchedulerQueueCapacityHandler { private static final Logger LOG = LoggerFactory.getLogger(CapacitySchedulerQueueCapacityHandler.class); - private static final Set CALCULATOR_PRECEDENCE = + private static final Set CALCULATOR_PRECEDENCE = ImmutableSet.of( - QueueCapacityType.ABSOLUTE, - QueueCapacityType.PERCENTAGE, - QueueCapacityType.WEIGHT); + ResourceUnitCapacityType.ABSOLUTE, + ResourceUnitCapacityType.PERCENTAGE, + ResourceUnitCapacityType.WEIGHT); + public static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0); - private final Map + private final Map calculators; private final AbstractQueueCapacityCalculator rootCalculator = new RootQueueCapacityCalculator(); - private QueueHierarchyUpdateContext lastUpdateContext; + private final RMNodeLabelsManager labelsManager; + private QueueResourceRoundingStrategy roundingStrategy = + new DefaultQueueResourceRoundingStrategy(); public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) { this.calculators = new HashMap<>(); - this.lastUpdateContext = new QueueHierarchyUpdateContext( - Resource.newInstance(0, 0), labelsManager); + this.labelsManager = labelsManager; - this.calculators.put(QueueCapacityType.ABSOLUTE, + this.calculators.put(ResourceUnitCapacityType.ABSOLUTE, new AbsoluteResourceCapacityCalculator()); - this.calculators.put(QueueCapacityType.PERCENTAGE, + this.calculators.put(ResourceUnitCapacityType.PERCENTAGE, new PercentageQueueCapacityCalculator()); - this.calculators.put(QueueCapacityType.WEIGHT, + this.calculators.put(ResourceUnitCapacityType.WEIGHT, new WeightQueueCapacityCalculator()); } /** - * Set static capacity config values (mostly due to backward compatibility). - * These values are not calculated but defined at configuration time. - * - * @param queue queue to set capacity config values to - */ - public void setup(CSQueue queue) { - for (String label : queue.getConfiguredNodeLabels()) { - for (QueueCapacityType capacityType : - queue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { - AbstractQueueCapacityCalculator calculator = - calculators.get(capacityType); - calculator.setup(queue, label); - } - } - } - - /** - * Updates the resource and metrics values for a queue and its descendants - * (and siblings if needed). These values are calculated at runtime. + * Updates the resource and metrics values for a queue, its siblings and descendants. + * These values are calculated at runtime. * * @param clusterResource resource of the cluster * @param queue queue to update */ - public QueueHierarchyUpdateContext update(Resource clusterResource, CSQueue queue) { - QueueHierarchyUpdateContext newContext = - new QueueHierarchyUpdateContext(clusterResource, lastUpdateContext); - this.lastUpdateContext = newContext; + public QueueCapacityUpdateContext update(Resource clusterResource, CSQueue queue) { + ResourceLimits resourceLimits = new ResourceLimits(clusterResource); + QueueCapacityUpdateContext newContext = + new QueueCapacityUpdateContext(clusterResource, labelsManager); if (queue.getQueuePath().equals(ROOT)) { - calculateResources(newContext, queue, ImmutableSet.of(rootCalculator)); + updateRoot(queue, newContext, resourceLimits); + updateChildren(queue, newContext, resourceLimits); + } else { + updateChildren(queue.getParent(), newContext, resourceLimits); } - update(queue, newContext); - return newContext; } - private void update( - CSQueue parent, QueueHierarchyUpdateContext queueHierarchyContext) { + private void updateRoot( + CSQueue queue, QueueCapacityUpdateContext newContext, ResourceLimits resourceLimits) { + for (String label : queue.getConfiguredNodeLabels()) { + for (QueueCapacityVectorEntry capacityVectorEntry : queue.getConfiguredCapacityVector(label)) { + queue.getOrCreateAbsoluteMinCapacityVector(label).setValue( + capacityVectorEntry.getResourceName(), 1); + queue.getOrCreateAbsoluteMaxCapacityVector(label).setValue( + capacityVectorEntry.getResourceName(), 1); + + float minimumResource = rootCalculator.calculateMinimumResource(newContext, queue, label, + capacityVectorEntry); + float maximumResource = rootCalculator.calculateMaximumResource(newContext, queue, label, + capacityVectorEntry); + long roundedMinResource = (long) Math.floor(minimumResource); + long roundedMaxResource = (long) Math.floor(maximumResource); + queue.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(capacityVectorEntry.getResourceName(), roundedMinResource); + queue.getQueueResourceQuotas().getEffectiveMaxResource(label) + .setResourceValue(capacityVectorEntry.getResourceName(), roundedMaxResource); + } + rootCalculator.updateCapacitiesAfterCalculation(newContext, queue, label); + } + + rootCalculator.calculateResourcePrerequisites(newContext, queue); + queue.refreshAfterResourceCalculation(newContext.getUpdatedClusterResource(), resourceLimits); + } + + + private void updateChildren( + CSQueue parent, QueueCapacityUpdateContext updateContext, + ResourceLimits resourceLimits) { if (parent == null || CollectionUtils.isEmpty(parent.getChildQueues())) { return; } - calculateResources(queueHierarchyContext, parent, - CALCULATOR_PRECEDENCE.stream().map((calculators::get)) - .collect(Collectors.toList())); + for (String label : parent.getConfiguredNodeLabels()) { + updateContext.getQueueBranchContext(parent.getQueuePath()).setBatchRemainingResource(label, + ResourceVector.of(parent.getEffectiveCapacity(label))); + } + + calculateChildrenResources(updateContext, parent); for (String label : parent.getConfiguredNodeLabels()) { - if (!queueHierarchyContext.getQueueBranchContext(parent.getQueuePath()) - .getRemainingResource(label).equals(ResourceVector.newInstance())) { - queueHierarchyContext.addUpdateWarning( - QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( - parent.getQueuePath())); + if (!updateContext.getQueueBranchContext(parent.getQueuePath()).getBatchRemainingResources( + label).equals(ResourceVector.newInstance())) { + updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( + parent.getQueuePath())); } } - updateChildren(queueHierarchyContext, parent); + for (CSQueue childQueue : parent.getChildQueues()) { + for (String label : childQueue.getConfiguredNodeLabels()) { + updateChildCapacities(updateContext, childQueue, label); + } + + ResourceLimits childLimit = ((ParentQueue) parent).getResourceLimitsOfChild( + childQueue, updateContext.getUpdatedClusterResource(), resourceLimits, NO_LABEL, false); + childQueue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), + childLimit); + updateChildren(childQueue, updateContext, childLimit); + } } - private void calculateResources( - QueueHierarchyUpdateContext queueHierarchyContext, CSQueue parent, - Collection usableCalculators) { - for (String label : parent.getConfiguredNodeLabels()) { - queueHierarchyContext.getQueueBranchContext(parent.getQueuePath()) - .setRemainingResource(label, ResourceVector.of(parent.getEffectiveCapacity(label))); - for (AbstractQueueCapacityCalculator calculator : usableCalculators) { - calculator.calculateChildQueueResources(queueHierarchyContext, parent); + private void updateChildCapacities( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label) { + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + if (capacityVector.isMixedCapacityVector()) { + // Post update capacities based on the calculated effective resource values + AbstractQueueCapacityCalculator.setQueueCapacities(updateContext.getUpdatedClusterResource( + label), childQueue, label); + } else { + // Update capacities according to the legacy logic + for (ResourceUnitCapacityType capacityType : + childQueue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); + calculator.updateCapacitiesAfterCalculation(updateContext, childQueue, label); } - setMetrics(parent, label); + } + + // If memory is zero, all other resource units should be considered zero as well. + if (childQueue.getEffectiveCapacity(label).getMemorySize() == 0) { + childQueue.getQueueResourceQuotas().setEffectiveMinResource(label, ZERO_RESOURCE); + } + + if (childQueue.getEffectiveMaxCapacity(label).getMemorySize() == 0) { + childQueue.getQueueResourceQuotas().setEffectiveMaxResource(label, ZERO_RESOURCE); } } - private void updateChildren( - QueueHierarchyUpdateContext queueHierarchyContext, CSQueue parent) { - if (parent.getChildQueues() != null) { + private void calculateChildrenResources( + QueueCapacityUpdateContext updateContext, CSQueue parent) { + for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { + Map aggregatedResources = new HashMap<>(); + AbstractQueueCapacityCalculator capacityCalculator = calculators.get(capacityType); + capacityCalculator.calculateResourcePrerequisites(updateContext, parent); + for (CSQueue childQueue : parent.getChildQueues()) { - update(childQueue, queueHierarchyContext); + childQueue.getWriteLock().lock(); + try { + for (String label : childQueue.getConfiguredNodeLabels()) { + ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault(label, + ResourceVector.newInstance()); + setChildResources(updateContext, childQueue, label, aggregatedUsedResource, capacityCalculator); + aggregatedResources.put(label, aggregatedUsedResource); + } + } finally { + childQueue.getWriteLock().unlock(); + } + } + + for (Map.Entry entry : aggregatedResources.entrySet()) { + updateContext.getQueueBranchContext(parent.getQueuePath()).getBatchRemainingResources( + entry.getKey()).subtract(entry.getValue()); } } } - private void setMetrics(CSQueue parent, String label) { - for (CSQueue childQueue : parent.getChildQueues()) { - for (QueueCapacityType capacityType : - childQueue.getConfiguredCapacityVector(label) - .getDefinedCapacityTypes()) { - AbstractQueueCapacityCalculator calculator = - calculators.get(capacityType); - calculator.setMetrics(lastUpdateContext, childQueue, label); + private void setChildResources( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, + ResourceVector usedResourcesOfCalculator, AbstractQueueCapacityCalculator capacityCalculator) { + for (String resourceName : capacityCalculator.getResourceNames(childQueue, label)) { + long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( + resourceName); + QueueCapacityVectorEntry capacityVectorEntry = childQueue.getConfiguredCapacityVector(label) + .getResource(resourceName); + QueueCapacityVectorEntry maximumCapacityVectorEntry = childQueue + .getConfiguredMaximumCapacityVector(label).getResource(resourceName); + AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( + maximumCapacityVectorEntry.getVectorResourceType()); + + float minimumResource = capacityCalculator.calculateMinimumResource( + updateContext, childQueue, label, capacityVectorEntry); + float maximumResource = maximumCapacityCalculator.calculateMaximumResource( + updateContext, childQueue, label, maximumCapacityVectorEntry); + + minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); + maximumResource = roundingStrategy.getRoundedResource(maximumResource, + maximumCapacityVectorEntry); + Pair resources = validateCalculatedResources(updateContext, childQueue, resourceName, label, + usedResourcesOfCalculator, new ImmutablePair<>(minimumResource, maximumResource)); + minimumResource = resources.getLeft(); + maximumResource = resources.getRight(); + + float absoluteMinCapacity = minimumResource / clusterResource; + float absoluteMaxCapacity = maximumResource / clusterResource; + childQueue.getOrCreateAbsoluteMinCapacityVector(label).setValue( + resourceName, absoluteMinCapacity); + childQueue.getOrCreateAbsoluteMaxCapacityVector(label).setValue( + resourceName, absoluteMaxCapacity); + + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + resourceName, (long) minimumResource); + childQueue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + resourceName, (long) maximumResource); + + usedResourcesOfCalculator.increment(resourceName, minimumResource); + } + } + + private Pair validateCalculatedResources( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String resourceName, + String label, ResourceVector usedResourcesOfCalculator, Pair calculatedResources) { + CSQueue parentQueue = childQueue.getParent(); + + float minimumResource = calculatedResources.getLeft(); + long minimumMemoryResource = childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) + .getMemorySize(); + + float remainingResourceUnderParent = updateContext.getQueueBranchContext( + parentQueue.getQueuePath()).getBatchRemainingResources(label).getValue(resourceName) + - usedResourcesOfCalculator.getValue(resourceName); + + long parentMaximumResource = parentQueue.getEffectiveMaxCapacity(label).getResourceValue( + resourceName); + float maximumResource = calculatedResources.getRight(); + + if (maximumResource != 0 && maximumResource > parentMaximumResource) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( + childQueue.getQueuePath())); + } + maximumResource = maximumResource == 0 ? parentMaximumResource + : Math.min(maximumResource, parentMaximumResource); + + if (maximumResource < minimumResource) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( + childQueue.getQueuePath())); + minimumResource = maximumResource; + } + + if (minimumResource > remainingResourceUnderParent) { + // Legacy auto queues are assigned a zero resource if not enough resource is left + if (parentQueue instanceof ManagedParentQueue) { + minimumResource = 0; + } else { + updateContext.addUpdateWarning( + QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath()).withInfo( + "Resource name: " + resourceName + " resource value: " + minimumResource)); + minimumResource = remainingResourceUnderParent; } } + + if (minimumResource == 0) { + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( + childQueue.getQueuePath()).withInfo("Resource name: " + resourceName)); + } + + return new ImmutablePair<>(minimumResource, maximumResource); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/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 f0c8a27f95493..f95d88fcac5de 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 @@ -80,6 +80,7 @@ public CSQueue hook(CSQueue queue) { private CSQueue root; private final RMNodeLabelsManager labelManager; private AppPriorityACLsManager appPriorityACLManager; + private CapacitySchedulerQueueCapacityHandler queueCapacityHandler; private QueueStateManager queueStateManager; @@ -99,6 +100,7 @@ public CapacitySchedulerQueueManager(Configuration conf, this.queueStateManager = new QueueStateManager<>(); this.appPriorityACLManager = appPriorityACLManager; this.configuredNodeLabels = new ConfiguredNodeLabels(); + this.queueCapacityHandler = new CapacitySchedulerQueueCapacityHandler(labelManager); } @Override @@ -444,6 +446,10 @@ private Map> getQueueToLabels() { return this.queueStateManager; } + public CapacitySchedulerQueueCapacityHandler getQueueCapacityHandler() { + return queueCapacityHandler; + } + /** * Removes an {@code AutoCreatedLeafQueue} from the manager collection and * from its parent children collection. 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java new file mode 100644 index 0000000000000..855d927d69f00 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/DefaultQueueResourceRoundingStrategy.java @@ -0,0 +1,38 @@ +/** + * 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.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.WEIGHT; + +/** + * The default rounding strategy for resource calculation. Uses floor for all types except WEIGHT, + * which is always the last type to consider, therefore it is safe to round up. + */ +public class DefaultQueueResourceRoundingStrategy implements QueueResourceRoundingStrategy { + @Override + public float getRoundedResource(float resourceValue, QueueCapacityVectorEntry capacityVectorEntry) { + if (capacityVectorEntry.getVectorResourceType().equals(WEIGHT)) { + return Math.round(resourceValue); + } else { + return (float) Math.floor(resourceValue); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index a4e2a822ead50..1c6822bccfd84 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -78,6 +78,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; + @Private @Unstable public class LeafQueue extends AbstractCSQueue { @@ -1937,6 +1940,48 @@ private void updateCurrentResourceLimits( currentResourceLimits.getLimit())); } + @Override + public void refreshAfterResourceCalculation(Resource clusterResource, ResourceLimits resourceLimits) { + lastClusterResource = clusterResource; + // Update maximum applications for the queue and for users + updateMaximumApplications(csContext.getConfiguration()); + + updateCurrentResourceLimits(resourceLimits, clusterResource); + + // Update headroom info based on new cluster resource value + // absoluteMaxCapacity now, will be replaced with absoluteMaxAvailCapacity + // during allocation + setQueueResourceLimitsInfo(clusterResource); + + // Update user consumedRatios + recalculateQueueUsageRatio(clusterResource, null); + + // Update metrics + CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, + this, labelManager, null); + // Update configured capacity/max-capacity for default partition only + CSQueueUtils.updateConfiguredCapacityMetrics(resourceCalculator, + labelManager.getResourceByLabel(null, clusterResource), + NO_LABEL, this); + + // queue metrics are updated, more resource may be available + // activate the pending applications if possible + activateApplications(); + + // In case of any resource change, invalidate recalculateULCount to clear + // the computed user-limit. + usersManager.userLimitNeedsRecompute(); + + // Update application properties + for (FiCaSchedulerApp application : orderingPolicy + .getSchedulableEntities()) { + computeUserLimitAndSetHeadroom(application, clusterResource, + NO_LABEL, + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY, null); + + } + } + @Override public void updateClusterResource(Resource clusterResource, ResourceLimits currentResourceLimits) { @@ -2217,10 +2262,12 @@ public void detachContainer(Resource clusterResource, } public void setCapacity(float capacity) { + configuredCapacityVectors.put(NO_LABEL, QueueCapacityVector.of(capacity * 100, PERCENTAGE)); queueCapacities.setCapacity(capacity); } public void setCapacity(String nodeLabel, float capacity) { + configuredCapacityVectors.put(nodeLabel, QueueCapacityVector.of(capacity * 100, PERCENTAGE)); queueCapacities.setCapacity(nodeLabel, capacity); } 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 8e9698b018768..b1ded6d3588ea 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 @@ -298,96 +298,98 @@ private enum QueueCapacityType { */ void setChildQueues(Collection childQueues) throws IOException { writeLock.lock(); + boolean isLegacyQueueMode = csContext.getConfiguration().isLegacyQueueMode(); try { - QueueCapacityType childrenCapacityType = - getCapacityConfigurationTypeForQueues(childQueues); - QueueCapacityType parentCapacityType = - getCapacityConfigurationTypeForQueues(ImmutableList.of(this)); - - if (childrenCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE - || parentCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE) { - // We don't allow any mixed absolute + {weight, percentage} between - // children and parent - if (childrenCapacityType != parentCapacityType && !this.getQueuePath() - .equals(CapacitySchedulerConfiguration.ROOT) - && csContext.getConfiguration().isLegacyQueueMode()) { - throw new IOException("Parent=" + this.getQueuePath() - + ": When absolute minResource is used, we must make sure both " - + "parent and child all use absolute minResource"); - } - - // Ensure that for each parent queue: parent.min-resource >= - // Σ(child.min-resource). - for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { - Resource minRes = Resources.createResource(0, 0); - for (CSQueue queue : childQueues) { - // Accumulate all min/max resource configured for all child queues. - Resources.addTo(minRes, queue.getQueueResourceQuotas() - .getConfiguredMinResource(nodeLabel)); + if (isLegacyQueueMode) { + QueueCapacityType childrenCapacityType = + getCapacityConfigurationTypeForQueues(childQueues); + QueueCapacityType parentCapacityType = + getCapacityConfigurationTypeForQueues(ImmutableList.of(this)); + + if (childrenCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE + || parentCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE) { + // We don't allow any mixed absolute + {weight, percentage} between + // children and parent + if (childrenCapacityType != parentCapacityType && !this.getQueuePath() + .equals(CapacitySchedulerConfiguration.ROOT)) { + throw new IOException("Parent=" + this.getQueuePath() + + ": When absolute minResource is used, we must make sure both " + + "parent and child all use absolute minResource"); } - Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel, - scheduler.getClusterResource()); - Resource parentMinResource = - usageTracker.getQueueResourceQuotas().getConfiguredMinResource(nodeLabel); - if (!parentMinResource.equals(Resources.none()) && Resources.lessThan( - resourceCalculator, resourceByLabel, parentMinResource, minRes)) { - throw new IOException( - "Parent Queues" + " capacity: " + parentMinResource - + " is less than" + " to its children:" + minRes - + " for queue:" + queueName); + + // Ensure that for each parent queue: parent.min-resource >= + // Σ(child.min-resource). + for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { + Resource minRes = Resources.createResource(0, 0); + for (CSQueue queue : childQueues) { + // Accumulate all min/max resource configured for all child queues. + Resources.addTo(minRes, queue.getQueueResourceQuotas() + .getConfiguredMinResource(nodeLabel)); + } + Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel, + scheduler.getClusterResource()); + Resource parentMinResource = + usageTracker.getQueueResourceQuotas().getConfiguredMinResource(nodeLabel); + if (!parentMinResource.equals(Resources.none()) && Resources.lessThan( + resourceCalculator, resourceByLabel, parentMinResource, minRes)) { + throw new IOException( + "Parent Queues" + " capacity: " + parentMinResource + + " is less than" + " to its children:" + minRes + + " for queue:" + queueName); + } } } - } - // When child uses percent - if (childrenCapacityType == QueueCapacityType.PERCENT) { - float childrenPctSum = 0; - // check label capacities - for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { - // check children's labels - childrenPctSum = 0; - for (CSQueue queue : childQueues) { - childrenPctSum += queue.getQueueCapacities().getCapacity(nodeLabel); - } + // When child uses percent + if (childrenCapacityType == QueueCapacityType.PERCENT) { + float childrenPctSum = 0; + // check label capacities + for (String nodeLabel : queueCapacities.getExistingNodeLabels()) { + // check children's labels + childrenPctSum = 0; + for (CSQueue queue : childQueues) { + childrenPctSum += queue.getQueueCapacities().getCapacity(nodeLabel); + } - if (Math.abs(1 - childrenPctSum) > PRECISION) { - // When children's percent sum != 100% - if (Math.abs(childrenPctSum) > PRECISION) { - // It is wrong when percent sum != {0, 1} - throw new IOException( - "Illegal" + " capacity sum of " + childrenPctSum - + " for children of queue " + queueName + " for label=" - + nodeLabel + ". It should be either 0 or 1.0"); - } else{ - // We also allow children's percent sum = 0 under the following - // conditions - // - Parent uses weight mode - // - Parent uses percent mode, and parent has - // (capacity=0 OR allowZero) - if (parentCapacityType == QueueCapacityType.PERCENT) { - if ((Math.abs(queueCapacities.getCapacity(nodeLabel)) - > PRECISION) && (!allowZeroCapacitySum)) { - throw new IOException( - "Illegal" + " capacity sum of " + childrenPctSum - + " for children of queue " + queueName - + " for label=" + nodeLabel - + ". It is set to 0, but parent percent != 0, and " - + "doesn't allow children capacity to set to 0"); + if (Math.abs(1 - childrenPctSum) > PRECISION) { + // When children's percent sum != 100% + if (Math.abs(childrenPctSum) > PRECISION) { + // It is wrong when percent sum != {0, 1} + throw new IOException( + "Illegal" + " capacity sum of " + childrenPctSum + + " for children of queue " + queueName + " for label=" + + nodeLabel + ". It should be either 0 or 1.0"); + } else { + // We also allow children's percent sum = 0 under the following + // conditions + // - Parent uses weight mode + // - Parent uses percent mode, and parent has + // (capacity=0 OR allowZero) + if (parentCapacityType == QueueCapacityType.PERCENT) { + if ((Math.abs(queueCapacities.getCapacity(nodeLabel)) + > PRECISION) && (!allowZeroCapacitySum)) { + throw new IOException( + "Illegal" + " capacity sum of " + childrenPctSum + + " for children of queue " + queueName + + " for label=" + nodeLabel + + ". It is set to 0, but parent percent != 0, and " + + "doesn't allow children capacity to set to 0"); + } } } - } - } else { - // Even if child pct sum == 1.0, we will make sure parent has - // positive percent. - if (parentCapacityType == QueueCapacityType.PERCENT && Math.abs( - queueCapacities.getCapacity(nodeLabel)) <= 0f - && !allowZeroCapacitySum) { - throw new IOException( - "Illegal" + " capacity sum of " + childrenPctSum - + " for children of queue " + queueName + " for label=" - + nodeLabel + ". queue=" + queueName - + " has zero capacity, but child" - + "queues have positive capacities"); + } else { + // Even if child pct sum == 1.0, we will make sure parent has + // positive percent. + if (parentCapacityType == QueueCapacityType.PERCENT && Math.abs( + queueCapacities.getCapacity(nodeLabel)) <= 0f + && !allowZeroCapacitySum) { + throw new IOException( + "Illegal" + " capacity sum of " + childrenPctSum + + " for children of queue " + queueName + " for label=" + + nodeLabel + ". queue=" + queueName + + " has zero capacity, but child" + + "queues have positive capacities"); + } } } } @@ -1056,7 +1058,7 @@ private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) { return accept; } - private ResourceLimits getResourceLimitsOfChild(CSQueue child, + public ResourceLimits getResourceLimitsOfChild(CSQueue child, Resource clusterResource, ResourceLimits parentLimits, String nodePartition, boolean netLimit) { // Set resource-limit of a given child, child.limit = @@ -1207,6 +1209,16 @@ public void completedContainer(Resource clusterResource, } } + @Override + public void refreshAfterResourceCalculation(Resource clusterResource, ResourceLimits resourceLimits) { + CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, + this, labelManager, null); + // Update configured capacity/max-capacity for default partition only + CSQueueUtils.updateConfiguredCapacityMetrics(resourceCalculator, + labelManager.getResourceByLabel(null, clusterResource), + RMNodeLabelsManager.NO_LABEL, this); + } + @Override public void updateClusterResource(Resource clusterResource, ResourceLimits resourceLimits) { 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index 33addfe19d946..a27899c6625b3 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -1,33 +1,39 @@ +/** + * 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.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import java.util.Set; - public class PercentageQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public void setup(CSQueue queue, String label) { - float sumCapacity = 0f; - QueueCapacityVector capacityVector = - queue.getConfiguredCapacityVector(label); - for (String resourceName : getResourceNames(queue, label)) { - sumCapacity += capacityVector.getResource(resourceName).getResourceValue(); - } - } - - @Override - protected float calculateMinimumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + public float calculateMinimumResource( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { CSQueue parentQueue = childQueue.getParent(); String resourceName = capacityVectorEntry.getResourceName(); - float parentAbsoluteCapacity = updateContext.getAbsoluteMinCapacity( - parentQueue.getQueuePath(), label).getValue(resourceName); + float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label).getValue( + resourceName); float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).getRemainingResource(label) + parentQueue.getQueuePath()).getBatchRemainingResources(label) .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) .getResourceValue(resourceName); float absoluteCapacity = parentAbsoluteCapacity * @@ -39,14 +45,14 @@ protected float calculateMinimumResource( } @Override - protected float calculateMaximumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + public float calculateMaximumResource( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { CSQueue parentQueue = childQueue.getParent(); String resourceName = capacityVectorEntry.getResourceName(); - float parentAbsoluteMaxCapacity = updateContext.getAbsoluteMaxCapacity( - parentQueue.getQueuePath(), label).getValue(resourceName); + float parentAbsoluteMaxCapacity = parentQueue.getOrCreateAbsoluteMaxCapacityVector(label) + .getValue(resourceName); float absoluteMaxCapacity = parentAbsoluteMaxCapacity * capacityVectorEntry.getResourceValue() / 100; @@ -55,21 +61,12 @@ protected float calculateMaximumResource( } @Override - public void setMetrics( - QueueHierarchyUpdateContext updateContext, CSQueue queue, String label) { - float sumAbsoluteCapacity = 0f; - Set resources = getResourceNames(queue, label); - for (String resourceName : resources) { - sumAbsoluteCapacity += updateContext.getAbsoluteMinCapacity( - queue.getQueuePath(), label).getValue(resourceName); - } - - queue.getQueueCapacities().setAbsoluteCapacity(sumAbsoluteCapacity - / resources.size()); + public void updateCapacitiesAfterCalculation(QueueCapacityUpdateContext updateContext, CSQueue queue, String label) { + ((AbstractCSQueue)queue).updateAbsoluteCapacities(); } @Override - protected QueueCapacityType getCapacityType() { - return QueueCapacityType.PERCENTAGE; + public ResourceUnitCapacityType getCapacityType() { + return 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/capacity/QueueBranchContext.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/QueueBranchContext.java index 207cbf6a19e1d..4e1e3d7fb97e8 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/QueueBranchContext.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/QueueBranchContext.java @@ -18,10 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; - -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -31,49 +27,54 @@ */ public class QueueBranchContext { private final Map remainingResourceByLabel = new HashMap<>(); + private final Map normalizedResourceRatio = new HashMap<>(); private final Map> sumWeightsPerLabel = new HashMap<>(); - private final Map> sumMaxWeightsPerLabel = new HashMap<>(); - private boolean isUpdated = false; + /** + * Increments the aggregated weight. + * @param label node label + * @param resourceName resource unit name + * @param value weight value + */ public void incrementWeight(String label, String resourceName, float value) { sumWeightsPerLabel.putIfAbsent(label, new HashMap<>()); sumWeightsPerLabel.get(label).put(resourceName, sumWeightsPerLabel.get(label).getOrDefault(resourceName, 0f) + value); } + /** + * Returns the aggregated children weights. + * @param label node label + * @param resourceName resource unit name + * @return aggregated weights of children + */ public float getSumWeightsByResource(String label, String resourceName) { return sumWeightsPerLabel.get(label).get(resourceName); } - public void incrementMaxWeight(String label, String resourceName, float value) { - sumMaxWeightsPerLabel.putIfAbsent(label, new HashMap<>()); - sumMaxWeightsPerLabel.get(label).put(resourceName, - sumMaxWeightsPerLabel.get(label).getOrDefault(resourceName, 0f) + value); - } - - public float getSumMaxWeightsByResource(String label, String resourceName) { - return sumMaxWeightsPerLabel.get(label).get(resourceName); + /** + * Sets the overall remaining resource under a parent that is available for its children to + * occupy. + * + * @param label node label + * @param resource resource vector + */ + public void setBatchRemainingResource(String label, ResourceVector resource) { + remainingResourceByLabel.put(label, resource); } - public void setRemainingResource(String label, ResourceVector resource) { - remainingResourceByLabel.put(label, resource); + public Map getNormalizedResourceRatios() { + return normalizedResourceRatio; } /** * Returns the remaining resources of a parent that is still available for its - * children. + * children. Decremented only after the calculator is finished its work on the corresponding + * resources. * @param label node label * @return remaining resources */ - public ResourceVector getRemainingResource(String label) { - return remainingResourceByLabel.get(label); - } - - public void setUpdateFlag() { - isUpdated = true; - } - - public boolean isParentAlreadyUpdated() { - return isUpdated; + public ResourceVector getBatchRemainingResources(String label) { + return remainingResourceByLabel.getOrDefault(label, ResourceVector.newInstance()); } } 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/QueueHierarchyUpdateContext.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/QueueCapacityUpdateContext.java similarity index 52% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueHierarchyUpdateContext.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacityUpdateContext.java index f62279eed1bc4..21de811f1c298 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/QueueHierarchyUpdateContext.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/QueueCapacityUpdateContext.java @@ -29,49 +29,24 @@ /** * A storage that encapsulates intermediate calculation values throughout a - * full queue update phase. + * full queue capacity update phase. */ -public class QueueHierarchyUpdateContext { +public class QueueCapacityUpdateContext { private final Resource updatedClusterResource; private final Map queueBranchContext = LazyMap.decorate(new HashMap(), QueueBranchContext::new); private final RMNodeLabelsManager labelsManager; - private Map> normalizedResourceRatios = - createLazyResourceVector(); - private Map> relativeResourceRatio = - createLazyResourceVector(); - private Map> relativeMaxResourceRatio = - createLazyResourceVector(); - private List warnings = new ArrayList(); - public QueueHierarchyUpdateContext( - Resource updatedClusterResource, - QueueHierarchyUpdateContext queueHierarchyUpdateContext) { - this.updatedClusterResource = updatedClusterResource; - this.normalizedResourceRatios = queueHierarchyUpdateContext - .normalizedResourceRatios; - this.relativeResourceRatio = queueHierarchyUpdateContext - .relativeResourceRatio; - this.labelsManager = queueHierarchyUpdateContext.labelsManager; - } + private List warnings = new ArrayList(); - public QueueHierarchyUpdateContext(Resource updatedClusterResource, - RMNodeLabelsManager labelsManager) { + public QueueCapacityUpdateContext(Resource updatedClusterResource, + RMNodeLabelsManager labelsManager) { this.updatedClusterResource = updatedClusterResource; this.labelsManager = labelsManager; } - private static Map> - createLazyResourceVector() { - return LazyMap.decorate( - new HashMap>(), - () -> LazyMap.decorate( - new HashMap(), - ResourceVector::newInstance)); - } - /** * Returns the overall cluster resource available for the update phase. * @return cluster resource @@ -80,6 +55,14 @@ public Resource getUpdatedClusterResource(String label) { return labelsManager.getResourceByLabel(label, updatedClusterResource); } + /** + * Returns the overall cluster resource available for the update phase. + * @return cluster resource + */ + public Resource getUpdatedClusterResource() { + return updatedClusterResource; + } + /** * Returns the context for a queue branch, which is identified by the path of * the parent. @@ -91,40 +74,17 @@ public QueueBranchContext getQueueBranchContext(String queuePath) { } /** - * Returns the normalized resource ratio calculated for a queue. - * @param queuePath queue path - * @param label node label - * @return normalized resource ratio + * Adds an update warning to the context. + * @param warning warning during update phase */ - public ResourceVector getNormalizedMinResourceRatio( - String queuePath, String label) { - return normalizedResourceRatios.get(queuePath).get(label); - } - - /** - * Returns the ratio of a child queue and its parent's resource. - * @param queuePath queue path - * @param label node label - * @return resource ratio - */ - public ResourceVector getAbsoluteMinCapacity(String queuePath, String label) { - return relativeResourceRatio.get(queuePath).get(label); - } - - /** - * Returns the ratio of a child queue and its parent's maximum resource. - * @param queuePath queue path - * @param label node label - * @return resource ratio - */ - public ResourceVector getAbsoluteMaxCapacity(String queuePath, String label) { - return relativeMaxResourceRatio.get(queuePath).get(label); - } - public void addUpdateWarning(QueueUpdateWarning warning) { warnings.add(warning); } + /** + * Returns all update warnings occurred in this update phase. + * @return update warnings + */ public List getUpdateWarnings() { return warnings; } 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/QueueCapacityVector.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/QueueCapacityVector.java index 9f6e0e264a320..c585efd9892aa 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/QueueCapacityVector.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/QueueCapacityVector.java @@ -39,9 +39,9 @@ public class QueueCapacityVector implements private static final String VALUE_DELIMITER = "="; private final ResourceVector resource; - private final Map capacityTypes + private final Map capacityTypes = new HashMap<>(); - private final Map> capacityTypePerResource + private final Map> capacityTypePerResource = new HashMap<>(); public QueueCapacityVector() { @@ -63,7 +63,7 @@ public static QueueCapacityVector newInstance() { new QueueCapacityVector(ResourceVector.newInstance()); for (Map.Entry resourceEntry : newCapacityVector.resource) { newCapacityVector.storeResourceType(resourceEntry.getKey(), - QueueCapacityType.ABSOLUTE); + ResourceUnitCapacityType.ABSOLUTE); } return newCapacityVector; @@ -78,7 +78,7 @@ public static QueueCapacityVector newInstance() { * @return uniform capacity vector */ public static QueueCapacityVector of( - float value, QueueCapacityType capacityType) { + float value, ResourceUnitCapacityType capacityType) { QueueCapacityVector newCapacityVector = new QueueCapacityVector(ResourceVector.of(value)); for (Map.Entry resourceEntry : newCapacityVector.resource) { @@ -110,7 +110,7 @@ public int getResourceCount() { * @param capacityType type of the resource */ public void setResource(String resourceName, float value, - QueueCapacityType capacityType) { + ResourceUnitCapacityType capacityType) { // Necessary due to backward compatibility (memory = memory-mb) String convertedResourceName = resourceName; if (resourceName.equals("memory")) { @@ -129,6 +129,10 @@ public float getMemory() { return resource.getValue(ResourceInformation.MEMORY_URI); } + public boolean isEmpty() { + return resource.isEmpty() && capacityTypePerResource.isEmpty() && capacityTypes.isEmpty(); + } + /** * Returns the name of all resources that are defined in the given capacity * type. @@ -137,13 +141,20 @@ public float getMemory() { * @return all resource names for the given capacity type */ public Set getResourceNamesByCapacityType( - QueueCapacityType capacityType) { - return capacityTypePerResource.getOrDefault(capacityType, - Collections.emptySet()); + ResourceUnitCapacityType capacityType) { + return new HashSet<>(capacityTypePerResource.getOrDefault(capacityType, + Collections.emptySet())); } + /** + * Checks whether a resource unit is defined as a specific type. + * + * @param resourceName resource unit name + * @param capacityType capacity type + * @return true, if resource unit is defined as the given type + */ public boolean isResourceOfType( - String resourceName, QueueCapacityType capacityType) { + String resourceName, ResourceUnitCapacityType capacityType) { return capacityTypes.containsKey(resourceName) && capacityTypes.get(resourceName).equals(capacityType); } @@ -172,16 +183,29 @@ public QueueCapacityVectorEntry next() { } /** - * Returns a set of all capacity type defined for this vector. + * Returns a set of all capacity types defined for this vector. * * @return capacity types */ - public Set getDefinedCapacityTypes() { + public Set getDefinedCapacityTypes() { return capacityTypePerResource.keySet(); } + /** + * Checks whether the vector is a mixed capacity vector (more than one capacity type is used, + * therefore it is not uniform). + * @return true, if the vector is mixed + */ + public boolean isMixedCapacityVector() { + return getDefinedCapacityTypes().size() > 1; + } + + public Set getResourceNames() { + return resource.getResourceNames(); + } + private void storeResourceType( - String resourceName, QueueCapacityType resourceType) { + String resourceName, ResourceUnitCapacityType resourceType) { if (capacityTypes.get(resourceName) != null && !capacityTypes.get(resourceName).equals(resourceType)) { capacityTypePerResource.get(capacityTypes.get(resourceName)) @@ -218,11 +242,11 @@ public String toString() { /** * Represents a capacity type associated with its syntax postfix. */ - public enum QueueCapacityType { + public enum ResourceUnitCapacityType { PERCENTAGE("%"), ABSOLUTE(""), WEIGHT("w"); private final String postfix; - QueueCapacityType(String postfix) { + ResourceUnitCapacityType(String postfix) { this.postfix = postfix; } @@ -232,18 +256,18 @@ public String getPostfix() { } public static class QueueCapacityVectorEntry { - private final QueueCapacityType vectorResourceType; + private final ResourceUnitCapacityType vectorResourceType; private final float resourceValue; private final String resourceName; - public QueueCapacityVectorEntry(QueueCapacityType vectorResourceType, + public QueueCapacityVectorEntry(ResourceUnitCapacityType vectorResourceType, String resourceName, float resourceValue) { this.vectorResourceType = vectorResourceType; this.resourceValue = resourceValue; this.resourceName = resourceName; } - public QueueCapacityType getVectorResourceType() { + public ResourceUnitCapacityType getVectorResourceType() { return vectorResourceType; } 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/QueueResourceRoundingStrategy.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/QueueResourceRoundingStrategy.java new file mode 100644 index 0000000000000..e81cef8abc306 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueResourceRoundingStrategy.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; + +/** + * Represents an approach on how to convert a calculated resource from floating point to a whole + * number. + */ +public interface QueueResourceRoundingStrategy { + + /** + * Returns a whole number converted from the calculated resource value. + * @param resourceValue calculated resource value + * @param capacityVectorEntry configured capacity entry + * @return rounded resource value + */ + float getRoundedResource(float resourceValue, QueueCapacityVectorEntry capacityVectorEntry); +} 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/QueueUpdateWarning.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/QueueUpdateWarning.java index f26d4649b9628..826de78c2822b 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/QueueUpdateWarning.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/QueueUpdateWarning.java @@ -1,5 +1,26 @@ +/** + * 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; +/** + * Represents a warning event that occurred during a queue capacity update phase. + */ public class QueueUpdateWarning { private final String queue; private final QueueUpdateWarningType warningType; @@ -12,7 +33,7 @@ public QueueUpdateWarning(QueueUpdateWarningType queueUpdateWarningType, String public enum QueueUpdateWarningType { BRANCH_UNDERUTILIZED("Remaining resource found in branch under parent queue '%s'. %s"), - QUEUE_OVERUTILIZED("Queue '%' is configured to use more resources than what is available under its parent. %s"), + QUEUE_OVERUTILIZED("Queue '%s' is configured to use more resources than what is available under its parent. %s"), QUEUE_ZERO_RESOURCE("Queue '%s' is assigned zero resource. %s"), BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are downscaled due to insufficient cluster resource. %s"), QUEUE_EXCEEDS_MAX_RESOURCE("Queue '%s' exceeds its maximum available resources. %s"), 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/ResourceVector.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/ResourceVector.java index 88c09af6b0911..5f6e769b2dfaa 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/ResourceVector.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/ResourceVector.java @@ -22,9 +22,11 @@ import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Set; /** * Represents a simple resource floating point value storage @@ -97,7 +99,20 @@ public void increment(String resourceName, float value) { setValue(resourceName, getValue(resourceName) + value); } - public Float getValue(String resourceName) { + /** + * Gets the average of all resource unit values. + * @return average of resource unit values + */ + public float getAverageValue() { + return (float) resourcesByName.values().stream().mapToDouble(value -> value).average() + .orElse(0); + } + + public float getMaxValue() { + return resourcesByName.values().stream().max(Comparator.naturalOrder()).orElse(0f); + } + + public float getValue(String resourceName) { return resourcesByName.get(resourceName); } @@ -105,6 +120,14 @@ public void setValue(String resourceName, float value) { resourcesByName.put(resourceName, value); } + public boolean isEmpty() { + return resourcesByName.isEmpty(); + } + + public Set getResourceNames() { + return resourcesByName.keySet(); + } + @Override public Iterator> iterator() { return resourcesByName.entrySet().iterator(); 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index 051ff7acfffe3..3c0758127bca3 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -20,57 +20,32 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.PERCENTAGE; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; public class RootQueueCapacityCalculator extends AbstractQueueCapacityCalculator { - @Override - public void setup(CSQueue queue, String label) { - queue.getQueueCapacities().setCapacity(label, 100f); - queue.getQueueCapacities().setMaximumCapacity(label, 100f); - } @Override - public void calculateChildQueueResources( - QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { - for (String label : parentQueue.getConfiguredNodeLabels()) { - for (QueueCapacityVectorEntry capacityVectorEntry : parentQueue.getConfiguredCapacityVector(label)) { - updateContext.getAbsoluteMinCapacity(ROOT, label).setValue( - capacityVectorEntry.getResourceName(), 1); - - float minimumResource = calculateMinimumResource(updateContext, parentQueue, label, capacityVectorEntry); - float maximumResource = calculateMinimumResource(updateContext, parentQueue, label, capacityVectorEntry); - long roundedMinResource = (long) Math.floor(minimumResource); - long roundedMaxResource = (long) Math.floor(maximumResource); - parentQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), roundedMinResource); - parentQueue.getQueueResourceQuotas().getEffectiveMaxResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), roundedMaxResource); - } - } - - calculateResourcePrerequisites(updateContext, parentQueue); - } - - @Override - protected float calculateMinimumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + public float calculateMinimumResource(QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { return updateContext.getUpdatedClusterResource(label).getResourceValue(capacityVectorEntry.getResourceName()); } @Override - protected float calculateMaximumResource(QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { + public float calculateMaximumResource(QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { return updateContext.getUpdatedClusterResource(label).getResourceValue(capacityVectorEntry.getResourceName()); } @Override - public void setMetrics( - QueueHierarchyUpdateContext updateContext, CSQueue queue, String label) { + public void updateCapacitiesAfterCalculation( + QueueCapacityUpdateContext updateContext, CSQueue queue, String label) { queue.getQueueCapacities().setAbsoluteCapacity(label, 1); + if (queue.getQueueCapacities().getWeight(label) == 1) { + queue.getQueueCapacities().setNormalizedWeight(label, 1); + } } @Override - protected QueueCapacityVector.QueueCapacityType getCapacityType() { + public QueueCapacityVector.ResourceUnitCapacityType getCapacityType() { return 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/capacity/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index c7d95999c01f0..3294c69903a2c 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -1,14 +1,35 @@ +/** + * 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.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType.WEIGHT; +import java.util.Collection; + +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.WEIGHT; public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - protected void calculateResourcePrerequisites(QueueHierarchyUpdateContext updateContext, CSQueue parentQueue) { + public void calculateResourcePrerequisites( + QueueCapacityUpdateContext updateContext, CSQueue parentQueue) { super.calculateResourcePrerequisites(updateContext, parentQueue); for (CSQueue childQueue : parentQueue.getChildQueues()) { @@ -18,17 +39,14 @@ protected void calculateResourcePrerequisites(QueueHierarchyUpdateContext update updateContext.getQueueBranchContext(parentQueue.getQueuePath()) .incrementWeight(label, resourceName, childQueue.getConfiguredCapacityVector(label) .getResource(resourceName).getResourceValue()); - updateContext.getQueueBranchContext(parentQueue.getQueuePath()).incrementMaxWeight( - label, resourceName, childQueue.getConfiguredMaximumCapacityVector(label) - .getResource(resourceName).getResourceValue()); } } } } @Override - protected float calculateMinimumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + public float calculateMinimumResource( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { CSQueue parentQueue = childQueue.getParent(); String resourceName = capacityVectorEntry.getResourceName(); @@ -37,64 +55,61 @@ protected float calculateMinimumResource( .getSumWeightsByResource(label, resourceName); float remainingResource = updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).getRemainingResource(label) + parentQueue.getQueuePath()).getBatchRemainingResources(label) .getValue(resourceName); + + // Due to rounding loss it is better to use all remaining resources if no other resource uses + // weight + if (normalizedWeight == 1) { + return remainingResource; + } + float remainingPerEffectiveResourceRatio = remainingResource / parentQueue.getEffectiveCapacity( label).getResourceValue(resourceName); - float parentAbsoluteCapacity = updateContext.getAbsoluteMinCapacity( - parentQueue.getQueuePath(), label).getValue(resourceName); + float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label) + .getValue(resourceName); float queueAbsoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio * normalizedWeight; - // Due to rounding loss it is better to use all remaining resources - // if no other resource uses weight - if (normalizedWeight == 1) { - return remainingResource; - } - - return updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName) + // Weight capacity types are the last to consider, therefore it is safe to assign all remaining + // effective resources between queues. The strategy is to round values to the closest whole + // number. + float resource = updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName) * queueAbsoluteCapacity; + + return Math.round(resource); } @Override - protected float calculateMaximumResource( - QueueHierarchyUpdateContext updateContext, CSQueue childQueue, String label, + public float calculateMaximumResource( + QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { - CSQueue parentQueue = childQueue.getParent(); - String resourceName = capacityVectorEntry.getResourceName(); - float normalizedMaxWeight = capacityVectorEntry.getResourceValue() - / updateContext.getQueueBranchContext(parentQueue.getQueuePath()) - .getSumMaxWeightsByResource(label, resourceName); - - float parentAbsoluteMaxCapacity = updateContext.getAbsoluteMaxCapacity( - parentQueue.getQueuePath(), label).getValue(resourceName); - float absoluteMaxCapacity = parentAbsoluteMaxCapacity * normalizedMaxWeight; - - return updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName) - * absoluteMaxCapacity; + throw new IllegalStateException("Resource " + capacityVectorEntry.getResourceName() + " has " + + "WEIGHT maximum capacity type, which is not supported"); } @Override - protected QueueCapacityType getCapacityType() { + public ResourceUnitCapacityType getCapacityType() { return WEIGHT; } @Override - public void setup(CSQueue queue, String label) { - queue.getQueueCapacities().setWeight(label, sumCapacityValues(queue, label)); - } - - @Override - public void setMetrics(QueueHierarchyUpdateContext updateContext, - CSQueue queue, String label) { - float sumNormalizedWeight = 0; - for (String resourceName : getResourceNames(queue, label)) { - sumNormalizedWeight += updateContext.getAbsoluteMinCapacity( - queue.getQueuePath(), label).getValue(resourceName); + public void updateCapacitiesAfterCalculation( + QueueCapacityUpdateContext updateContext, CSQueue queue, String label) { + float sumCapacityPerResource = 0f; + + Collection resourceNames = getResourceNames(queue, label); + for (String resourceName : resourceNames) { + float sumBranchWeight = updateContext.getQueueBranchContext(queue.getParent().getQueuePath()) + .getSumWeightsByResource(label, resourceName); + float capacity = queue.getConfiguredCapacityVector(label).getResource( + resourceName).getResourceValue() / sumBranchWeight; + sumCapacityPerResource += capacity; } - queue.getQueueCapacities().setNormalizedWeight(label, sumNormalizedWeight); + queue.getQueueCapacities().setNormalizedWeight(label, sumCapacityPerResource + / resourceNames.size()); + ((AbstractCSQueue) queue).updateAbsoluteCapacities(); } - } 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 ff0fed28927b1..79786a11b3cb3 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 @@ -20,7 +20,7 @@ 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.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.util.UnitsConversionUtil; import java.util.ArrayList; @@ -68,7 +68,7 @@ public QueueCapacityConfigParser() { public QueueCapacityVector parse(String capacityString, String queuePath) { if (queuePath.equals(CapacitySchedulerConfiguration.ROOT)) { - return QueueCapacityVector.of(100f, QueueCapacityType.PERCENTAGE); + return QueueCapacityVector.of(100f, ResourceUnitCapacityType.PERCENTAGE); } if (capacityString == null) { @@ -95,13 +95,13 @@ public QueueCapacityVector parse(String capacityString, String queuePath) { * @return a parsed capacity vector */ private QueueCapacityVector uniformParser(Matcher matcher) { - QueueCapacityType capacityType = null; + ResourceUnitCapacityType capacityType = null; String value = matcher.group(1); if (matcher.groupCount() == 2) { String matchedSuffix = matcher.group(2); - for (QueueCapacityType suffix : QueueCapacityType.values()) { + for (ResourceUnitCapacityType suffix : ResourceUnitCapacityType.values()) { // Absolute uniform syntax is not supported - if (suffix.equals(QueueCapacityType.ABSOLUTE)) { + if (suffix.equals(ResourceUnitCapacityType.ABSOLUTE)) { continue; } // when capacity is given in percentage, we do not need % symbol @@ -158,7 +158,7 @@ private QueueCapacityVector heterogeneousParser(Matcher matcher) { private void setCapacityVector( QueueCapacityVector resource, String resourceName, String resourceValue) { - QueueCapacityType capacityType = QueueCapacityType.ABSOLUTE; + ResourceUnitCapacityType capacityType = ResourceUnitCapacityType.ABSOLUTE; // Extract suffix from a value e.g. for 6w extract w String suffix = resourceValue.replaceAll(FLOAT_DIGIT_REGEX, ""); @@ -174,7 +174,7 @@ private void setCapacityVector( // Convert all incoming units to MB if units is configured. convertedValue = UnitsConversionUtil.convert(suffix, "Mi", (long) parsedResourceValue); } else { - for (QueueCapacityType capacityTypeSuffix : QueueCapacityType.values()) { + for (ResourceUnitCapacityType capacityTypeSuffix : ResourceUnitCapacityType.values()) { if (capacityTypeSuffix.getPostfix().equals(suffix)) { capacityType = capacityTypeSuffix; } @@ -192,8 +192,12 @@ private void setCapacityVector( * false otherwise */ public boolean isCapacityVectorFormat(String configuredCapacity) { - return configuredCapacity != null - && RESOURCE_PATTERN.matcher(configuredCapacity).find(); + if (configuredCapacity == null) { + return false; + } + + String formattedCapacityString = configuredCapacity.replaceAll(" ", ""); + return RESOURCE_PATTERN.matcher(formattedCapacityString).find(); } private static class Parser { 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 a4436127e7f11..3b561ef771bc8 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 @@ -8,12 +8,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.junit.Assert; import org.junit.Before; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -24,8 +24,24 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; public class CapacitySchedulerQueueCalculationTestBase { + 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"; + + protected static final String CAPACITY_VECTOR_TEMPLATE = "[memory=%s, vcores=%s]"; + + protected ResourceCalculator resourceCalculator; protected static class QueueAssertionBuilder { + public static final String EFFECTIVE_MAX_RES_INFO = "Effective Maximum Resource"; + public static final Function EFFECTIVE_MAX_RES = + QueueResourceQuotas::getEffectiveMaxResource; + public static final String EFFECTIVE_MIN_RES_INFO = "Effective Minimum Resource"; public static final Function EFFECTIVE_MIN_RES = QueueResourceQuotas::getEffectiveMinResource; @@ -65,6 +81,10 @@ public class ValueAssertion { this.expectedResource = expectedResource; } + public QueueAssertion assertEffectiveMaxResource() { + return withResourceSupplier(EFFECTIVE_MAX_RES, EFFECTIVE_MAX_RES_INFO); + } + public QueueAssertion assertEffectiveMinResource() { return withResourceSupplier(EFFECTIVE_MIN_RES, EFFECTIVE_MIN_RES_INFO); } @@ -181,14 +201,13 @@ public void setUp() throws Exception { ResourceScheduler.class); csConf.setQueues("root", new String[]{"a", "b"}); - csConf.setNonLabeledQueueWeight("root", 1f); - csConf.setNonLabeledQueueWeight("root.a", 6f); - csConf.setNonLabeledQueueWeight("root.b", 4f); + csConf.setCapacity("root.a", 50f); + csConf.setCapacity("root.b", 50f); csConf.setQueues("root.a", new String[]{"a1", "a2"}); - csConf.setNonLabeledQueueWeight("root.a.a1", 1f); + csConf.setCapacity("root.a.a1", 100f); csConf.setQueues("root.a.a1", new String[]{"a11", "a12"}); - csConf.setNonLabeledQueueWeight("root.a.a1.a11", 1f); - csConf.setNonLabeledQueueWeight("root.a.a1.a12", 1f); + csConf.setCapacity("root.a.a1.a11", 50f); + csConf.setCapacity("root.a.a1.a12", 50f); mgr = new NullRMNodeLabelsManager(); mgr.init(csConf); @@ -203,21 +222,18 @@ protected RMNodeLabelsManager createNodeLabelManager() { mockRM.start(); cs.start(); mockRM.registerNode("h1:1234", 10 * GB); // label = x + resourceCalculator = cs.getResourceCalculator(); } - protected QueueHierarchyUpdateContext update( + protected QueueCapacityUpdateContext update( QueueAssertionBuilder assertions, Resource resource) throws IOException { cs.reinitialize(csConf, mockRM.getRMContext()); CapacitySchedulerQueueCapacityHandler queueController = new CapacitySchedulerQueueCapacityHandler(mgr); mgr.setResourceForLabel(CommonNodeLabelsManager.NO_LABEL, resource); - for (String queueToAssert : assertions.getQueues()) { - CSQueue queue = cs.getQueue(queueToAssert); - queueController.setup(queue); - } - QueueHierarchyUpdateContext updateContext = + QueueCapacityUpdateContext updateContext = queueController.update(resource, cs.getQueue("root")); assertions.finishAssertion(); @@ -228,4 +244,24 @@ protected QueueHierarchyUpdateContext update( protected QueueAssertionBuilder createAssertionBuilder() { return new QueueAssertionBuilder(cs); } + + protected String createMemoryVcoresVector(Object memory, Object vcores) { + return String.format(CAPACITY_VECTOR_TEMPLATE, memory, vcores); + } + + protected static String absolute(double value) { + return String.valueOf((long) value); + } + + protected static String weight(float value) { + return value + "w"; + } + + protected static String percentage(float value) { + return value + "%"; + } + + protected static Resource createResource(double memory, double vcores) { + return Resource.newInstance((int) memory, (int) vcores); + } } 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 02b721d4255a5..d42f9a614f9a3 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 @@ -1,8 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; import org.junit.Test; @@ -10,64 +29,144 @@ import java.util.Collection; import java.util.Optional; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; + public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { + private static final long MEMORY = 16384; + private static final long VCORES = 16; private static final Resource UPDATE_RESOURCE = Resource.newInstance(16384, 16); + private static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0); + + public static final Resource A_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(2486, 9); + public static final Resource A1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(621, 4); + public static final Resource A11_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(217, 1); + public static final Resource A12_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(403, 3); + public static final Resource A2_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(1865, 5); + public static final Resource B_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); + public static final Resource B1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); + public static final Resource C_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(5803, 4); + + public static final Resource B_WARNING_RESOURCE = Resource.newInstance(8096, 3); + public static final Resource B1_WARNING_RESOURCE = Resource.newInstance(8096, 3); + public static final Resource A_WARNING_RESOURCE = Resource.newInstance(8288, 9); + public static final Resource A1_WARNING_RESOURCE = Resource.newInstance(2048, 4); + public static final Resource A2_WARNING_RESOURCE = Resource.newInstance(2048, 5); + public static final Resource A12_WARNING_RESOURCE = Resource.newInstance(2048, 4); + + @Override + public void setUp() throws Exception { + super.setUp(); + csConf.setLegacyQueueModeEnabled(false); + } @Test - public void testMixedCapacitiesWithoutRemainingResource() throws IOException { + public void testComplexHierarchyWithoutRemainingResource() throws IOException { setupQueueHierarchyWithoutRemainingResource(); QueueAssertionBuilder assertionBuilder = createAssertionBuilder() - .withQueue("root.a") - .toExpect(Resource.newInstance(2486, 9)) + .withQueue(A) + .toExpect(A_COMPLEX_NO_REMAINING_RESOURCE) .assertEffectiveMinResource() - .withQueue("root.a.a1") - .toExpect(Resource.newInstance(621, 4)) + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(A1) + .toExpect(A1_COMPLEX_NO_REMAINING_RESOURCE) .assertEffectiveMinResource() - .withQueue("root.a.a1.a11") - .toExpect(Resource.newInstance(217, 1)) + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(A11) + .toExpect(A11_COMPLEX_NO_REMAINING_RESOURCE) .assertEffectiveMinResource() - .withQueue("root.a.a1.a12") - .toExpect(Resource.newInstance(404, 3)) + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A11_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(A12) + .toExpect(A12_COMPLEX_NO_REMAINING_RESOURCE) .assertEffectiveMinResource() - .withQueue("root.a.a2") - .toExpect(Resource.newInstance(1865, 5)) + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A12_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(A2) + .toExpect(A2_COMPLEX_NO_REMAINING_RESOURCE) .assertEffectiveMinResource() - .withQueue("root.b") - .toExpect(Resource.newInstance(8095, 3)) + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A2_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(B) + .toExpect(B_COMPLEX_NO_REMAINING_RESOURCE) .assertEffectiveMinResource() - .withQueue("root.b.b1") - .toExpect(Resource.newInstance(8095, 3)) + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, B_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(B1) + .toExpect(B1_COMPLEX_NO_REMAINING_RESOURCE) .assertEffectiveMinResource() - .withQueue("root.c") - .toExpect(Resource.newInstance(5803, 4)) + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, B1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(C) + .toExpect(C_COMPLEX_NO_REMAINING_RESOURCE) .assertEffectiveMinResource() + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, C_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() .build(); - QueueHierarchyUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); - - Assert.assertEquals("No warning should occur", 0, - updateContext.getUpdateWarnings().size()); + update(assertionBuilder, UPDATE_RESOURCE); } @Test - public void testMixedCapacitiesWithWarnings() throws IOException { - csConf.setLegacyQueueModeEnabled(false); + public void testComplexHierarchyWithWarnings() throws IOException { setupQueueHierarchyWithWarnings(); - QueueAssertionBuilder assertionBuilder = createAssertionBuilder(); + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .toExpect(A_WARNING_RESOURCE) + .assertEffectiveMinResource() + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(A1) + .toExpect(A1_WARNING_RESOURCE) + .assertEffectiveMinResource() + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A1_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(A2) + .toExpect(A2_WARNING_RESOURCE) + .assertEffectiveMinResource() + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A2_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(A11) + .toExpect(ZERO_RESOURCE) + .assertEffectiveMinResource() + .toExpect(0) + .assertAbsoluteCapacity() + .withQueue(A12) + .toExpect(A12_WARNING_RESOURCE) + .assertEffectiveMinResource() + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A12_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(B) + .toExpect(B_WARNING_RESOURCE) + .assertEffectiveMinResource() + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, B_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(B1) + .toExpect(B1_WARNING_RESOURCE) + .assertEffectiveMinResource() + .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, B1_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity() + .withQueue(C) + .toExpect(ZERO_RESOURCE) + .assertEffectiveMinResource() + .toExpect(0) + .assertAbsoluteCapacity() + .build(); - QueueHierarchyUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); Optional queueCZeroResourceWarning = getSpecificWarning( - updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, "root.c"); + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, C); Optional queueARemainingResourceWarning = getSpecificWarning( - updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_UNDERUTILIZED, "root.a"); + updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_UNDERUTILIZED, A); Optional queueBDownscalingWarning = getSpecificWarning( - updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_DOWNSCALED, "root.b"); + updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_DOWNSCALED, B); Optional queueA11ZeroResourceWarning = getSpecificWarning( - updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, "root.a.a1.a11"); + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, A11); Optional queueA12ZeroResourceWarning = getSpecificWarning( - updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, "root.a.a1.a12"); + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, A12); Assert.assertTrue(queueCZeroResourceWarning.isPresent()); Assert.assertTrue(queueARemainingResourceWarning.isPresent()); @@ -76,48 +175,136 @@ public void testMixedCapacitiesWithWarnings() throws IOException { Assert.assertTrue(queueA12ZeroResourceWarning.isPresent()); } + @Test + public void testZeroResourceIfNoMemory() throws IOException { + csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(100), weight(6))); + csConf.setCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), absolute(VCORES * 0.5))); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .toExpect(ZERO_RESOURCE) + .assertEffectiveMinResource() + .withQueue(B) + .toExpect(createResource(MEMORY, VCORES * 0.5)) + .assertEffectiveMinResource() + .build(); + + QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + Optional queueAZeroResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, A); + Optional rootUnderUtilizedWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_UNDERUTILIZED, ROOT); + Assert.assertTrue(queueAZeroResourceWarning.isPresent()); + Assert.assertTrue(rootUnderUtilizedWarning.isPresent()); + } + + @Test + public void testDifferentMinimumAndMaximumCapacityTypes() throws IOException { + csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(50), absolute(VCORES * 0.5))); + csConf.setMaximumCapacityVector(A, "", createMemoryVcoresVector(absolute(MEMORY), percentage(80))); + csConf.setCapacityVector(B, "", createMemoryVcoresVector(weight(6), percentage(100))); + csConf.setMaximumCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), absolute(VCORES * 0.5))); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .toExpect(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) + .assertEffectiveMinResource() + .toExpect(Resource.newInstance(MEMORY, (int) (VCORES * 0.8))) + .assertEffectiveMaxResource() + .withQueue(B) + .toExpect(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) + .assertEffectiveMinResource() + .toExpect(Resource.newInstance(MEMORY, (int) (VCORES * 0.5))) + .assertEffectiveMaxResource() + .build(); + + QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + Assert.assertEquals(0, updateContext.getUpdateWarnings().size()); + + // WEIGHT capacity type for maximum capacity is not supported + csConf.setMaximumCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), weight(10))); + try { + cs.reinitialize(csConf, mockRM.getRMContext()); + update(assertionBuilder, UPDATE_RESOURCE); + Assert.fail("WEIGHT maximum capacity type is not supported, an error should be thrown when set up"); + } catch (IllegalStateException ignored) { + } + } + + @Test + public void testMaximumResourceWarnings() throws IOException { + csConf.setMaximumCapacityVector(A1, "", createMemoryVcoresVector(absolute(MEMORY * 0.5), percentage(100))); + csConf.setCapacityVector(A11, "", createMemoryVcoresVector(percentage(50), percentage(100))); + csConf.setCapacityVector(A12, "", createMemoryVcoresVector(percentage(50), percentage(0))); + csConf.setMaximumCapacityVector(A11, "", createMemoryVcoresVector(absolute(MEMORY), percentage(10))); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A11) + .toExpect(createResource(0.5 * 0.5 * MEMORY, 0.1 * VCORES)) + .assertEffectiveMinResource() + .toExpect(createResource(MEMORY * 0.5, 0.1 * VCORES)) + .assertEffectiveMaxResource() + .withQueue(A12) + .toExpect(createResource(0.5 * 0.5 * MEMORY, 0)) + .assertEffectiveMinResource() + .toExpect(createResource(MEMORY * 0.5, VCORES)) + .assertEffectiveMaxResource() + .build(); + + QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); + Optional queueA11ExceedsParentMaxResourceWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT, A11); + Optional queueA11MinExceedsMaxWarning = getSpecificWarning( + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE, A11); + Assert.assertTrue(queueA11ExceedsParentMaxResourceWarning.isPresent()); + Assert.assertTrue(queueA11MinExceedsMaxWarning.isPresent()); + } + private void setupQueueHierarchyWithoutRemainingResource() throws IOException { - csConf.setState("root.b", QueueState.STOPPED); + csConf.setState(B, QueueState.STOPPED); cs.reinitialize(csConf, mockRM.getRMContext()); + setQueues(); - csConf.setQueues("root", new String[]{"a", "b", "c"}); - csConf.setQueues("root.a", new String[]{"a1", "a2"}); - csConf.setQueues("root.b", new String[]{"b1"}); - - csConf.setState("root.b", QueueState.RUNNING); - csConf.setCapacityVector("root.a", "", "[memory=30%,vcores=6w]"); - csConf.setCapacityVector("root.a.a1", "", "[memory=1w,vcores=4]"); - csConf.setCapacityVector("root.a.a1.a11", "", "[memory=35%,vcores=25%]"); - csConf.setCapacityVector("root.a.a1.a12", "", "[memory=65%,vcores=75%]"); - csConf.setCapacityVector("root.a.a2", "", "[memory=3w,vcores=100%]"); - csConf.setCapacityVector("root.b", "", "[memory=8095,vcores=30%]"); - csConf.setCapacityVector("root.b.b1", "", "[memory=5w,vcores=3]"); - csConf.setCapacityVector("root.c", "", "[memory=3w,vcores=4]"); + csConf.setState(B, QueueState.RUNNING); + csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(30), weight(6))); + csConf.setCapacityVector(A1, "", createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A11, "", createMemoryVcoresVector(percentage(35), percentage(25))); + csConf.setCapacityVector(A12, "", createMemoryVcoresVector(percentage(65), percentage(75))); + csConf.setCapacityVector(A2, "", createMemoryVcoresVector(weight(3), percentage(100))); + csConf.setCapacityVector(B, "", createMemoryVcoresVector(absolute(8095), percentage(30))); + csConf.setCapacityVector(B1, "", createMemoryVcoresVector(weight(5), absolute(3))); + csConf.setCapacityVector(C, "", createMemoryVcoresVector(weight(3), absolute(VCORES * 0.25))); cs.reinitialize(csConf, mockRM.getRMContext()); } private void setupQueueHierarchyWithWarnings() throws IOException { - csConf.setState("root.b", QueueState.STOPPED); + csConf.setState(B, QueueState.STOPPED); cs.reinitialize(csConf, mockRM.getRMContext()); + setQueues(); - csConf.setQueues("root", new String[]{"a", "b", "c"}); - csConf.setQueues("root.a", new String[]{"a1", "a2"}); - csConf.setQueues("root.b", new String[]{"b1"}); - - csConf.setState("root.b", QueueState.RUNNING); - csConf.setCapacityVector("root.a", "", "[memory=100%,vcores=6w]"); - csConf.setCapacityVector("root.a.a1", "", "[memory=2048,vcores=4]"); - csConf.setCapacityVector("root.a.a1.a11", "", "[memory=1w,vcores=4]"); - csConf.setCapacityVector("root.a.a1.a12", "", "[memory=100%,vcores=100%]"); - csConf.setCapacityVector("root.a.a2", "", "[memory=2048,vcores=100%]"); - csConf.setCapacityVector("root.b", "", "[memory=8096,vcores=30%]"); - csConf.setCapacityVector("root.b.b1", "", "[memory=10256,vcores=3]"); - csConf.setCapacityVector("root.c", "", "[memory=3w,vcores=4]"); + Resource.newInstance(0, 0); // C + Resource.newInstance(0, 0); // A12 + + csConf.setState(B, QueueState.RUNNING); + csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(100), weight(6))); + csConf.setCapacityVector(A1, "", createMemoryVcoresVector(absolute(2048), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A11, "", createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A12, "", createMemoryVcoresVector(percentage(100), percentage(100))); + csConf.setCapacityVector(A2, "", createMemoryVcoresVector(absolute(2048), percentage(100))); + csConf.setCapacityVector(B, "", createMemoryVcoresVector(absolute(8096), percentage(30))); + csConf.setCapacityVector(B1, "", createMemoryVcoresVector(absolute(10256), absolute(3))); + csConf.setCapacityVector(C, "", createMemoryVcoresVector(weight(3), absolute(VCORES * 0.25))); cs.reinitialize(csConf, mockRM.getRMContext()); } + private void setQueues() { + 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, String queue) { 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/TestQueueCapacityVector.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/TestQueueCapacityVector.java index 058e14bfaf272..7b5ab6e225cda 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/TestQueueCapacityVector.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/TestQueueCapacityVector.java @@ -20,7 +20,7 @@ import org.apache.hadoop.util.Lists; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; @@ -50,21 +50,21 @@ public void setUp() { public void getResourceNamesByCapacityType() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.PERCENTAGE); - capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); + capacityVector.setResource(MEMORY_URI, 10, ResourceUnitCapacityType.PERCENTAGE); + capacityVector.setResource(VCORES_URI, 6, ResourceUnitCapacityType.PERCENTAGE); // custom is not set, defaults to 0 Assert.assertEquals(1, capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.ABSOLUTE).size()); + ResourceUnitCapacityType.ABSOLUTE).size()); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.ABSOLUTE).contains(CUSTOM_RESOURCE)); + ResourceUnitCapacityType.ABSOLUTE).contains(CUSTOM_RESOURCE)); Assert.assertEquals(2, capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.PERCENTAGE).size()); + ResourceUnitCapacityType.PERCENTAGE).size()); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.PERCENTAGE).contains(VCORES_URI)); + ResourceUnitCapacityType.PERCENTAGE).contains(VCORES_URI)); Assert.assertTrue(capacityVector.getResourceNamesByCapacityType( - QueueCapacityType.PERCENTAGE).contains(MEMORY_URI)); + ResourceUnitCapacityType.PERCENTAGE).contains(MEMORY_URI)); Assert.assertEquals(10, capacityVector.getResource(MEMORY_URI).getResourceValue(), EPSILON); Assert.assertEquals(6, capacityVector.getResource(VCORES_URI).getResourceValue(), EPSILON); } @@ -73,13 +73,13 @@ public void getResourceNamesByCapacityType() { public void isResourceOfType() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.WEIGHT); - capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); - capacityVector.setResource(CUSTOM_RESOURCE, 3, QueueCapacityType.ABSOLUTE); + capacityVector.setResource(MEMORY_URI, 10, ResourceUnitCapacityType.WEIGHT); + capacityVector.setResource(VCORES_URI, 6, ResourceUnitCapacityType.PERCENTAGE); + capacityVector.setResource(CUSTOM_RESOURCE, 3, ResourceUnitCapacityType.ABSOLUTE); - Assert.assertTrue(capacityVector.isResourceOfType(MEMORY_URI, QueueCapacityType.WEIGHT)); - Assert.assertTrue(capacityVector.isResourceOfType(VCORES_URI, QueueCapacityType.PERCENTAGE)); - Assert.assertTrue(capacityVector.isResourceOfType(CUSTOM_RESOURCE, QueueCapacityType.ABSOLUTE)); + Assert.assertTrue(capacityVector.isResourceOfType(MEMORY_URI, ResourceUnitCapacityType.WEIGHT)); + Assert.assertTrue(capacityVector.isResourceOfType(VCORES_URI, ResourceUnitCapacityType.PERCENTAGE)); + Assert.assertTrue(capacityVector.isResourceOfType(CUSTOM_RESOURCE, ResourceUnitCapacityType.ABSOLUTE)); } @Test @@ -99,9 +99,9 @@ public void testIterator() { public void testToString() { QueueCapacityVector capacityVector = QueueCapacityVector.newInstance(); - capacityVector.setResource(MEMORY_URI, 10, QueueCapacityType.WEIGHT); - capacityVector.setResource(VCORES_URI, 6, QueueCapacityType.PERCENTAGE); - capacityVector.setResource(CUSTOM_RESOURCE, 3, QueueCapacityType.ABSOLUTE); + capacityVector.setResource(MEMORY_URI, 10, ResourceUnitCapacityType.WEIGHT); + capacityVector.setResource(VCORES_URI, 6, ResourceUnitCapacityType.PERCENTAGE); + capacityVector.setResource(CUSTOM_RESOURCE, 3, ResourceUnitCapacityType.ABSOLUTE); Assert.assertEquals(MIXED_CAPACITY_VECTOR_STRING, capacityVector.toString()); 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 8ea9f56690707..de36b090be617 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 @@ -42,75 +42,124 @@ public class TestUniformQueueResourceCalculation extends CapacitySchedulerQueueC 2); private static final Resource UPDATE_RES = Resource.newInstance(250 * GB, 40); private static final Resource PERCENTAGE_ALL_RES = Resource.newInstance(10 * GB, 20); - private static final Resource WEIGHT_ALL_RES = Resource.newInstance(10 * GB, 20); + + public static final float A_CAPACITY = 0.3f; + public static final float B_CAPACITY = 0.7f; + public static final float A1_CAPACITY = 0.17f; + public static final float A11_CAPACITY = 0.25f; + public static final float A12_CAPACITY = 0.75f; + public static final float A2_CAPACITY = 0.83f; + + public static final float A_WEIGHT = 3; + public static final float B_WEIGHT = 6; + public static final float A1_WEIGHT = 2; + public static final float A11_WEIGHT = 5; + public static final float A12_WEIGHT = 8; + public static final float A2_WEIGHT = 3; + + public static final float A_NORMALIZED_WEIGHT = A_WEIGHT / (A_WEIGHT + B_WEIGHT); + public static final float B_NORMALIZED_WEIGHT = B_WEIGHT / (A_WEIGHT + B_WEIGHT); + public static final float A1_NORMALIZED_WEIGHT = A1_WEIGHT / (A1_WEIGHT + A2_WEIGHT); + public static final float A2_NORMALIZED_WEIGHT = A2_WEIGHT / (A1_WEIGHT + A2_WEIGHT); + public static final float A11_NORMALIZED_WEIGHT = A11_WEIGHT / (A11_WEIGHT + A12_WEIGHT); + public static final float A12_NORMALIZED_WEIGHT = A12_WEIGHT / (A11_WEIGHT + A12_WEIGHT); @Test public void testWeightResourceCalculation() throws IOException { -// CapacitySchedulerQueueCapacityHandler queueController = -// new CapacitySchedulerQueueCapacityHandler(mgr); -// update(WEIGHT_ALL_RES); -// queueController.update(WEIGHT_ALL_RES, cs.getQueue("root.a")); -// CSQueue a = cs.getQueue("root.a"); -// -// Assert.assertEquals(6 * GB, a.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize(), 1e-6); + csConf.setNonLabeledQueueWeight(A, A_WEIGHT); + csConf.setNonLabeledQueueWeight(B, B_WEIGHT); + csConf.setNonLabeledQueueWeight(A1, A1_WEIGHT); + csConf.setNonLabeledQueueWeight(A11, A11_WEIGHT); + csConf.setNonLabeledQueueWeight(A12, A12_WEIGHT); + csConf.setNonLabeledQueueWeight(A2, A2_WEIGHT); + + QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() + .withQueue(A) + .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource() + .toExpect(A_NORMALIZED_WEIGHT) + .assertAbsoluteCapacity() + .withQueue(B) + .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, B_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource() + .toExpect(B_NORMALIZED_WEIGHT) + .assertAbsoluteCapacity() + .withQueue(A1) + .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource() + .toExpect(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT) + .assertAbsoluteCapacity() + .withQueue(A2) + .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource() + .toExpect(A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT) + .assertAbsoluteCapacity() + .withQueue(A11) + .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource() + .toExpect(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT) + .assertAbsoluteCapacity() + .withQueue(A12) + .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource() + .toExpect(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT) + .assertAbsoluteCapacity() + .build(); + + update(queueAssertionBuilder, UPDATE_RES); } @Test public void testPercentageResourceCalculation() throws IOException { - csConf.setCapacity("root.a", 30); - csConf.setCapacity("root.b", 70); - csConf.setCapacity("root.a.a1", 17); - csConf.setCapacity("root.a.a1.a11", 25); - csConf.setCapacity("root.a.a1.a12", 75); - csConf.setCapacity("root.a.a2", 83); + csConf.setCapacity(A, A_CAPACITY * 100); + csConf.setCapacity(B, B_CAPACITY * 100); + csConf.setCapacity(A1, A1_CAPACITY * 100); + csConf.setCapacity(A11, A11_CAPACITY * 100); + csConf.setCapacity(A12, A12_CAPACITY * 100); + csConf.setCapacity(A2, A2_CAPACITY * 100); QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() - .withQueue("root.a") - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.3f)) + .withQueue(A) + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY)) .assertEffectiveMinResource() - .toExpect(0.3f) + .toExpect(A_CAPACITY) .assertCapacity() - .toExpect(0.3f) + .toExpect(A_CAPACITY) .assertAbsoluteCapacity() - - .withQueue("root.b") - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.7f)) + .withQueue(B) + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, B_CAPACITY)) .assertEffectiveMinResource() - .toExpect(0.7f) + .toExpect(B_CAPACITY) .assertCapacity() - .toExpect(0.7f) + .toExpect(B_CAPACITY) .assertAbsoluteCapacity() - - .withQueue("root.a.a1") - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.3f * 0.17f)) + .withQueue(A1) + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY * A1_CAPACITY)) .assertEffectiveMinResource() - .toExpect(0.17f) + .toExpect(A1_CAPACITY) .assertCapacity() - .toExpect(0.3f * 0.17f) + .toExpect(A_CAPACITY * A1_CAPACITY) .assertAbsoluteCapacity() - - .withQueue("root.a.a2") - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.3f * 0.83f)) + .withQueue(A2) + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY * A2_CAPACITY)) .assertEffectiveMinResource() - .toExpect(0.83f) + .toExpect(A2_CAPACITY) .assertCapacity() - .toExpect(0.3f * 0.83f) + .toExpect(A_CAPACITY * A2_CAPACITY) .assertAbsoluteCapacity() - - .withQueue("root.a.a1.a11") - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.25f * 0.3f * 0.17f)) + .withQueue(A11) + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A11_CAPACITY * A_CAPACITY * A1_CAPACITY)) .assertEffectiveMinResource() - .toExpect(0.25f) + .toExpect(A11_CAPACITY) .assertCapacity() - .toExpect(0.25f * 0.3f * 0.17f) + .toExpect(A11_CAPACITY * A_CAPACITY * A1_CAPACITY) .assertAbsoluteCapacity() - - .withQueue("root.a.a1.a12") - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, 0.75f * 0.3f * 0.17f)) + .withQueue(A12) + .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A12_CAPACITY * A_CAPACITY * A1_CAPACITY)) .assertEffectiveMinResource() - .toExpect(0.75f) + .toExpect(A12_CAPACITY) .assertCapacity() - .toExpect(0.75f * 0.3f * 0.17f) + .toExpect(A12_CAPACITY * A_CAPACITY * A1_CAPACITY) .assertAbsoluteCapacity() .build(); @@ -119,30 +168,30 @@ public void testPercentageResourceCalculation() throws IOException { @Test public void testAbsoluteResourceCalculation() throws IOException { - csConf.setMinimumResourceRequirement("", "root.a", QUEUE_A_RES); - csConf.setMinimumResourceRequirement("", "root.b", QUEUE_B_RES); - csConf.setMinimumResourceRequirement("", "root.a.a1", QUEUE_A1_RES); - csConf.setMinimumResourceRequirement("", "root.a.a2", QUEUE_A2_RES); - csConf.setMinimumResourceRequirement("", "root.a.a1.a11", QUEUE_A11_RES); - csConf.setMinimumResourceRequirement("", "root.a.a1.a12", QUEUE_A12_RES); + 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); QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() - .withQueue("root.a") + .withQueue(A) .toExpect(QUEUE_A_RES) .assertEffectiveMinResource() - .withQueue("root.b") + .withQueue(B) .toExpect(QUEUE_B_RES) .assertEffectiveMinResource() - .withQueue("root.a.a1") + .withQueue(A1) .toExpect(QUEUE_A1_RES) .assertEffectiveMinResource() - .withQueue("root.a.a2") + .withQueue(A2) .toExpect(QUEUE_A2_RES) .assertEffectiveMinResource() - .withQueue("root.a.a1.a11") + .withQueue(A11) .toExpect(QUEUE_A11_RES) .assertEffectiveMinResource() - .withQueue("root.a.a1.a12") + .withQueue(A12) .toExpect(QUEUE_A12_RES) .assertEffectiveMinResource() .build(); @@ -150,22 +199,22 @@ public void testAbsoluteResourceCalculation() throws IOException { update(queueAssertionBuilder, UPDATE_RES); QueueAssertionBuilder queueAssertionHalfClusterResource = createAssertionBuilder() - .withQueue("root.a") + .withQueue(A) .toExpect(ResourceUtils.multiply(QUEUE_A_RES, 0.5f)) .assertEffectiveMinResource() - .withQueue("root.b") + .withQueue(B) .toExpect(ResourceUtils.multiply(QUEUE_B_RES, 0.5f)) .assertEffectiveMinResource() - .withQueue("root.a.a1") + .withQueue(A1) .toExpect(ResourceUtils.multiply(QUEUE_A1_RES, 0.5f)) .assertEffectiveMinResource() - .withQueue("root.a.a2") + .withQueue(A2) .toExpect(ResourceUtils.multiply(QUEUE_A2_RES, 0.5f)) .assertEffectiveMinResource() - .withQueue("root.a.a1.a11") + .withQueue(A11) .toExpect(ResourceUtils.multiply(QUEUE_A11_RES, 0.5f)) .assertEffectiveMinResource() - .withQueue("root.a.a1.a12") + .withQueue(A12) .toExpect(ResourceUtils.multiply(QUEUE_A12_RES, 0.5f)) .assertEffectiveMinResource() .build(); 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 7a3d4e9048d0c..c126da407ed7f 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 @@ -23,7 +23,7 @@ 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.QueueCapacityVectorEntry; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; import org.junit.Test; @@ -73,10 +73,10 @@ public void testPercentageCapacityConfig() { QueueCapacityVectorEntry memory = percentageCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcore = percentageCapacityVector.getResource(VCORES_URI); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, memory.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, memory.getVectorResourceType()); Assert.assertEquals(PERCENTAGE_VALUE, memory.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, vcore.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, vcore.getVectorResourceType()); Assert.assertEquals(PERCENTAGE_VALUE, vcore.getResourceValue(), EPSILON); QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), @@ -85,10 +85,10 @@ public void testPercentageCapacityConfig() { QueueCapacityVectorEntry memoryRoot = rootCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcoreRoot = rootCapacityVector.getResource(VCORES_URI); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, memoryRoot.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, memoryRoot.getVectorResourceType()); Assert.assertEquals(100f, memoryRoot.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, vcoreRoot.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, vcoreRoot.getVectorResourceType()); Assert.assertEquals(100f, vcoreRoot.getResourceValue(), EPSILON); } @@ -99,10 +99,10 @@ public void testWeightCapacityConfig() { QueueCapacityVectorEntry memory = weightCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcore = weightCapacityVector.getResource(VCORES_URI); - Assert.assertEquals(QueueCapacityType.WEIGHT, memory.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.WEIGHT, memory.getVectorResourceType()); Assert.assertEquals(WEIGHT_VALUE, memory.getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.WEIGHT, vcore.getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.WEIGHT, vcore.getVectorResourceType()); Assert.assertEquals(WEIGHT_VALUE, vcore.getResourceValue(), EPSILON); } @@ -116,17 +116,17 @@ public void testAbsoluteCapacityVectorConfig() { QueueCapacityVector absoluteCapacityVector = capacityConfigParser.parse(ABSOLUTE_RESOURCE, QUEUE); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(MEMORY_URI) + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(MEMORY_URI) .getVectorResourceType()); Assert.assertEquals(12 * GB, absoluteCapacityVector.getResource(MEMORY_URI) .getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) .getVectorResourceType()); Assert.assertEquals(VCORE_ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) .getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) .getVectorResourceType()); Assert.assertEquals(GPU_ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) .getResourceValue(), EPSILON); @@ -146,17 +146,17 @@ public void testMixedCapacityConfig() { QueueCapacityVector mixedCapacityVector = capacityConfigParser.parse(MIXED_RESOURCE, QUEUE); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, mixedCapacityVector.getResource(MEMORY_URI).getVectorResourceType()); Assert.assertEquals(MEMORY_MIXED, mixedCapacityVector.getResource(MEMORY_URI) .getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.PERCENTAGE, + Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, mixedCapacityVector.getResource(VCORES_URI).getVectorResourceType()); Assert.assertEquals(PERCENTAGE_VALUE, mixedCapacityVector.getResource(VCORES_URI).getResourceValue(), EPSILON); - Assert.assertEquals(QueueCapacityType.WEIGHT, + Assert.assertEquals(ResourceUnitCapacityType.WEIGHT, mixedCapacityVector.getResource(GPU_URI).getVectorResourceType()); Assert.assertEquals(WEIGHT_VALUE, mixedCapacityVector.getResource(GPU_URI).getResourceValue(), EPSILON); @@ -164,7 +164,7 @@ public void testMixedCapacityConfig() { // Test undefined capacity type default value QueueCapacityVector mixedCapacityVectorWithGpuUndefined = capacityConfigParser.parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); - Assert.assertEquals(QueueCapacityType.ABSOLUTE, + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, mixedCapacityVectorWithGpuUndefined.getResource(MEMORY_URI).getVectorResourceType()); Assert.assertEquals(0, mixedCapacityVectorWithGpuUndefined.getResource(GPU_URI) .getResourceValue(), EPSILON); From 3dae1dcdc119dc2183b489233b3fec8d56d3a631 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 16 Nov 2021 08:31:56 +0100 Subject: [PATCH 11/27] YARN-10965. Implement strict resource iteration order --- .../AbsoluteResourceCapacityCalculator.java | 2 +- .../AbstractQueueCapacityCalculator.java | 8 +- ...CapacitySchedulerQueueCapacityHandler.java | 247 ++++++++++-------- .../PercentageQueueCapacityCalculator.java | 4 +- .../capacity/QueueBranchContext.java | 33 ++- .../capacity/QueueCapacityUpdateContext.java | 10 +- .../scheduler/capacity/ResourceVector.java | 22 +- .../WeightQueueCapacityCalculator.java | 10 +- .../TestMixedQueueResourceCalculation.java | 40 +-- 9 files changed, 215 insertions(+), 161 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index 25357e06095a2..3560c9a12589f 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -28,7 +28,7 @@ public float calculateMinimumResource( QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { String resourceName = capacityVectorEntry.getResourceName(); - ResourceVector ratio = updateContext.getQueueBranchContext(childQueue.getParent().getQueuePath()) + ResourceVector ratio = updateContext.getOrCreateQueueBranchContext(childQueue.getParent().getQueuePath()) .getNormalizedResourceRatios().getOrDefault(label, ResourceVector.of(1)); return ratio.getValue(resourceName) * capacityVectorEntry.getResourceValue(); 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 61b57a7860dac..87c514b2621d1 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -25,14 +25,10 @@ import org.apache.hadoop.yarn.util.UnitsConversionUtil; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; -import java.util.ArrayList; -import java.util.Collection; -import java.util.LinkedList; import java.util.Map; import java.util.Set; import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; -import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; /** * A strategy class to encapsulate queue capacity setup and resource calculation @@ -94,7 +90,7 @@ public void calculateResourcePrerequisites(QueueCapacityUpdateContext updateCont CSQueue parentQueue) { for (String label : parentQueue.getConfiguredNodeLabels()) { // We need to set normalized resource ratio only once per parent - if (updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + if (updateContext.getOrCreateQueueBranchContext(parentQueue.getQueuePath()) .getNormalizedResourceRatios().isEmpty()) { setNormalizedResourceRatio(updateContext, parentQueue, label); } @@ -225,7 +221,7 @@ private void setNormalizedResourceRatio( .getUnits(), childrenConfiguredResource); if (convertedValue != 0) { - Map normalizedResourceRatios = updateContext.getQueueBranchContext( + Map normalizedResourceRatios = updateContext.getOrCreateQueueBranchContext( parentQueue.getQueuePath()).getNormalizedResourceRatios(); normalizedResourceRatios.putIfAbsent(label, ResourceVector.newInstance()); normalizedResourceRatios.get(label).setValue(resourceName, numeratorForMinRatio / 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index f1a3fa8cebfb0..cceff569d5278 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -28,14 +28,19 @@ 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.QueueUpdateWarning.QueueUpdateWarningType; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; @@ -53,14 +58,13 @@ public class CapacitySchedulerQueueCapacityHandler { ResourceUnitCapacityType.ABSOLUTE, ResourceUnitCapacityType.PERCENTAGE, ResourceUnitCapacityType.WEIGHT); - public static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0); - private final Map calculators; private final AbstractQueueCapacityCalculator rootCalculator = new RootQueueCapacityCalculator(); private final RMNodeLabelsManager labelsManager; - private QueueResourceRoundingStrategy roundingStrategy = + private final Collection definedResources = new LinkedHashSet<>(); + private final QueueResourceRoundingStrategy roundingStrategy = new DefaultQueueResourceRoundingStrategy(); public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) { @@ -73,6 +77,8 @@ public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) new PercentageQueueCapacityCalculator()); this.calculators.put(ResourceUnitCapacityType.WEIGHT, new WeightQueueCapacityCalculator()); + + loadResourceNames(); } /** @@ -124,7 +130,6 @@ private void updateRoot( queue.refreshAfterResourceCalculation(newContext.getUpdatedClusterResource(), resourceLimits); } - private void updateChildren( CSQueue parent, QueueCapacityUpdateContext updateContext, ResourceLimits resourceLimits) { @@ -133,152 +138,134 @@ private void updateChildren( } for (String label : parent.getConfiguredNodeLabels()) { - updateContext.getQueueBranchContext(parent.getQueuePath()).setBatchRemainingResource(label, + updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()).setPostCalculatorRemainingResource(label, + ResourceVector.of(parent.getEffectiveCapacity(label))); + updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()).setOverallRemainingResource(label, ResourceVector.of(parent.getEffectiveCapacity(label))); } + for (AbstractQueueCapacityCalculator capacityCalculator : calculators.values()) { + capacityCalculator.calculateResourcePrerequisites(updateContext, parent); + } + calculateChildrenResources(updateContext, parent); for (String label : parent.getConfiguredNodeLabels()) { - if (!updateContext.getQueueBranchContext(parent.getQueuePath()).getBatchRemainingResources( + if (!updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()).getPostCalculatorRemainingResource( label).equals(ResourceVector.newInstance())) { updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( parent.getQueuePath())); } } - for (CSQueue childQueue : parent.getChildQueues()) { - for (String label : childQueue.getConfiguredNodeLabels()) { - updateChildCapacities(updateContext, childQueue, label); - } - - ResourceLimits childLimit = ((ParentQueue) parent).getResourceLimitsOfChild( - childQueue, updateContext.getUpdatedClusterResource(), resourceLimits, NO_LABEL, false); - childQueue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), - childLimit); - updateChildren(childQueue, updateContext, childLimit); - } - } - - private void updateChildCapacities( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label) { - QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); - if (capacityVector.isMixedCapacityVector()) { - // Post update capacities based on the calculated effective resource values - AbstractQueueCapacityCalculator.setQueueCapacities(updateContext.getUpdatedClusterResource( - label), childQueue, label); - } else { - // Update capacities according to the legacy logic - for (ResourceUnitCapacityType capacityType : - childQueue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { - AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); - calculator.updateCapacitiesAfterCalculation(updateContext, childQueue, label); - } - } - - // If memory is zero, all other resource units should be considered zero as well. - if (childQueue.getEffectiveCapacity(label).getMemorySize() == 0) { - childQueue.getQueueResourceQuotas().setEffectiveMinResource(label, ZERO_RESOURCE); - } - - if (childQueue.getEffectiveMaxCapacity(label).getMemorySize() == 0) { - childQueue.getQueueResourceQuotas().setEffectiveMaxResource(label, ZERO_RESOURCE); - } + updateChildrenAfterCalculation(parent, updateContext, resourceLimits); } private void calculateChildrenResources( QueueCapacityUpdateContext updateContext, CSQueue parent) { - for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { - Map aggregatedResources = new HashMap<>(); - AbstractQueueCapacityCalculator capacityCalculator = calculators.get(capacityType); - capacityCalculator.calculateResourcePrerequisites(updateContext, parent); - - for (CSQueue childQueue : parent.getChildQueues()) { - childQueue.getWriteLock().lock(); - try { - for (String label : childQueue.getConfiguredNodeLabels()) { - ResourceVector aggregatedUsedResource = aggregatedResources.getOrDefault(label, - ResourceVector.newInstance()); - setChildResources(updateContext, childQueue, label, aggregatedUsedResource, capacityCalculator); - aggregatedResources.put(label, aggregatedUsedResource); + for (String resourceName : definedResources) { + for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { + Map effectiveResourceUsedByLabel = new HashMap<>(); + for (CSQueue childQueue : parent.getChildQueues()) { + childQueue.getWriteLock().lock(); + try { + for (String label : childQueue.getConfiguredNodeLabels()) { + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + if (!childQueue.getConfiguredCapacityVector(label).isResourceOfType( + resourceName, capacityType)) { + continue; + } + float aggregatedUsedResource = effectiveResourceUsedByLabel.getOrDefault(label, + 0f); + float usedResourceByChild = setChildResources(updateContext, childQueue, label, + capacityVector.getResource(resourceName)); + float resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; + updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()) + .getOverallRemainingResource(label).subtract(resourceName, usedResourceByChild); + effectiveResourceUsedByLabel.put(label, resourceUsedByLabel); + } + } finally { + childQueue.getWriteLock().unlock(); } - } finally { - childQueue.getWriteLock().unlock(); } - } - for (Map.Entry entry : aggregatedResources.entrySet()) { - updateContext.getQueueBranchContext(parent.getQueuePath()).getBatchRemainingResources( - entry.getKey()).subtract(entry.getValue()); + // Only decrement post calculator remaining resource at the end of each calculator phase + for (Map.Entry entry : effectiveResourceUsedByLabel.entrySet()) { + updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()).getPostCalculatorRemainingResource( + entry.getKey()).subtract(resourceName, entry.getValue()); + } } } } - private void setChildResources( + private float setChildResources( QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - ResourceVector usedResourcesOfCalculator, AbstractQueueCapacityCalculator capacityCalculator) { - for (String resourceName : capacityCalculator.getResourceNames(childQueue, label)) { - long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( - resourceName); - QueueCapacityVectorEntry capacityVectorEntry = childQueue.getConfiguredCapacityVector(label) - .getResource(resourceName); - QueueCapacityVectorEntry maximumCapacityVectorEntry = childQueue - .getConfiguredMaximumCapacityVector(label).getResource(resourceName); - AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( - maximumCapacityVectorEntry.getVectorResourceType()); - - float minimumResource = capacityCalculator.calculateMinimumResource( - updateContext, childQueue, label, capacityVectorEntry); - float maximumResource = maximumCapacityCalculator.calculateMaximumResource( - updateContext, childQueue, label, maximumCapacityVectorEntry); - - minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); - maximumResource = roundingStrategy.getRoundedResource(maximumResource, - maximumCapacityVectorEntry); - Pair resources = validateCalculatedResources(updateContext, childQueue, resourceName, label, - usedResourcesOfCalculator, new ImmutablePair<>(minimumResource, maximumResource)); - minimumResource = resources.getLeft(); - maximumResource = resources.getRight(); - - float absoluteMinCapacity = minimumResource / clusterResource; - float absoluteMaxCapacity = maximumResource / clusterResource; - childQueue.getOrCreateAbsoluteMinCapacityVector(label).setValue( - resourceName, absoluteMinCapacity); - childQueue.getOrCreateAbsoluteMaxCapacityVector(label).setValue( - resourceName, absoluteMaxCapacity); - - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( - resourceName, (long) minimumResource); - childQueue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( - resourceName, (long) maximumResource); - - usedResourcesOfCalculator.increment(resourceName, minimumResource); - } + QueueCapacityVectorEntry capacityVectorEntry) { + AbstractQueueCapacityCalculator capacityCalculator = calculators.get( + capacityVectorEntry.getVectorResourceType()); + String resourceName = capacityVectorEntry.getResourceName(); + long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( + resourceName); + QueueCapacityVectorEntry maximumCapacityVectorEntry = childQueue + .getConfiguredMaximumCapacityVector(label).getResource(resourceName); + AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( + maximumCapacityVectorEntry.getVectorResourceType()); + + float minimumResource = capacityCalculator.calculateMinimumResource( + updateContext, childQueue, label, capacityVectorEntry); + float maximumResource = maximumCapacityCalculator.calculateMaximumResource( + updateContext, childQueue, label, maximumCapacityVectorEntry); + + minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); + maximumResource = roundingStrategy.getRoundedResource(maximumResource, + maximumCapacityVectorEntry); + Pair resources = validateCalculatedResources(updateContext, childQueue, + resourceName, label, new ImmutablePair<>(minimumResource, maximumResource)); + minimumResource = resources.getLeft(); + maximumResource = resources.getRight(); + + float absoluteMinCapacity = minimumResource / clusterResource; + float absoluteMaxCapacity = maximumResource / clusterResource; + childQueue.getOrCreateAbsoluteMinCapacityVector(label).setValue( + resourceName, absoluteMinCapacity); + childQueue.getOrCreateAbsoluteMaxCapacityVector(label).setValue( + resourceName, absoluteMaxCapacity); + + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + resourceName, (long) minimumResource); + childQueue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + resourceName, (long) maximumResource); + + return minimumResource; } private Pair validateCalculatedResources( QueueCapacityUpdateContext updateContext, CSQueue childQueue, String resourceName, - String label, ResourceVector usedResourcesOfCalculator, Pair calculatedResources) { + String label, Pair calculatedResources) { CSQueue parentQueue = childQueue.getParent(); float minimumResource = calculatedResources.getLeft(); long minimumMemoryResource = childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) .getMemorySize(); - float remainingResourceUnderParent = updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).getBatchRemainingResources(label).getValue(resourceName) - - usedResourcesOfCalculator.getValue(resourceName); + float remainingResourceUnderParent = updateContext.getOrCreateQueueBranchContext( + parentQueue.getQueuePath()).getOverallRemainingResource(label).getValue(resourceName); long parentMaximumResource = parentQueue.getEffectiveMaxCapacity(label).getResourceValue( resourceName); float maximumResource = calculatedResources.getRight(); + // Memory is the primary resource, if its zero, all other resource units are zero as well. + if (!resourceName.equals(MEMORY_URI) && minimumMemoryResource == 0) { + minimumResource = 0; + } + if (maximumResource != 0 && maximumResource > parentMaximumResource) { updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( childQueue.getQueuePath())); } - maximumResource = maximumResource == 0 ? parentMaximumResource - : Math.min(maximumResource, parentMaximumResource); + maximumResource = maximumResource == 0 ? parentMaximumResource : Math.min(maximumResource, + parentMaximumResource); if (maximumResource < minimumResource) { updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( @@ -305,4 +292,50 @@ private Pair validateCalculatedResources( return new ImmutablePair<>(minimumResource, maximumResource); } + + private void updateChildrenAfterCalculation( + CSQueue parent, QueueCapacityUpdateContext updateContext, ResourceLimits resourceLimits) { + for (CSQueue childQueue : parent.getChildQueues()) { + updateChildCapacities(updateContext, childQueue); + ResourceLimits childLimit = ((ParentQueue) parent).getResourceLimitsOfChild( + childQueue, updateContext.getUpdatedClusterResource(), resourceLimits, NO_LABEL, false); + childQueue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), + childLimit); + updateChildren(childQueue, updateContext, childLimit); + } + } + + private void updateChildCapacities( + QueueCapacityUpdateContext updateContext, CSQueue childQueue) { + for (String label : childQueue.getConfiguredNodeLabels()) { + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + if (capacityVector.isMixedCapacityVector()) { + // Post update capacities based on the calculated effective resource values + AbstractQueueCapacityCalculator.setQueueCapacities(updateContext.getUpdatedClusterResource( + label), childQueue, label); + } else { + // Update capacities according to the legacy logic + for (ResourceUnitCapacityType capacityType : + childQueue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); + calculator.updateCapacitiesAfterCalculation(updateContext, childQueue, label); + } + } + } + } + + private void loadResourceNames() { + Set resources = new HashSet<>(ResourceUtils.getResourceTypes().keySet()); + if (resources.contains(MEMORY_URI)) { + resources.remove(MEMORY_URI); + definedResources.add(MEMORY_URI); + } + + if (resources.contains(VCORES_URI)) { + resources.remove(VCORES_URI); + definedResources.add(VCORES_URI); + } + + definedResources.addAll(resources); + } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index a27899c6625b3..832f6872019d9 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -32,8 +32,8 @@ public float calculateMinimumResource( float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label).getValue( resourceName); - float remainingPerEffectiveResourceRatio = updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).getBatchRemainingResources(label) + float remainingPerEffectiveResourceRatio = updateContext.getOrCreateQueueBranchContext( + parentQueue.getQueuePath()).getPostCalculatorRemainingResource(label) .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) .getResourceValue(resourceName); float absoluteCapacity = parentAbsoluteCapacity * 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/QueueBranchContext.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/QueueBranchContext.java index 4e1e3d7fb97e8..70c6a0d331b0f 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/QueueBranchContext.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/QueueBranchContext.java @@ -28,6 +28,7 @@ public class QueueBranchContext { private final Map remainingResourceByLabel = new HashMap<>(); private final Map normalizedResourceRatio = new HashMap<>(); + private final Map overallRemainingResource = new HashMap<>(); private final Map> sumWeightsPerLabel = new HashMap<>(); /** @@ -53,13 +54,13 @@ public float getSumWeightsByResource(String label, String resourceName) { } /** - * Sets the overall remaining resource under a parent that is available for its children to + * Sets the remaining resource under a parent that is available for its children to * occupy. * * @param label node label * @param resource resource vector */ - public void setBatchRemainingResource(String label, ResourceVector resource) { + public void setPostCalculatorRemainingResource(String label, ResourceVector resource) { remainingResourceByLabel.put(label, resource); } @@ -67,14 +68,38 @@ public Map getNormalizedResourceRatios() { return normalizedResourceRatio; } + /** + * Returns the remaining resources of a parent that is still available for its + * children. + * + * @param label node label + * @return remaining resources + */ + public ResourceVector getOverallRemainingResource(String label) { + overallRemainingResource.putIfAbsent(label, ResourceVector.newInstance()); + return overallRemainingResource.get(label); + } + + /** + * Sets the remaining resources of a parent that is still available for its children. + * + * @param label node label + * @param resourceVector resource vector + */ + public void setOverallRemainingResource(String label, ResourceVector resourceVector) { + overallRemainingResource.put(label, resourceVector); + } + /** * Returns the remaining resources of a parent that is still available for its * children. Decremented only after the calculator is finished its work on the corresponding * resources. + * * @param label node label * @return remaining resources */ - public ResourceVector getBatchRemainingResources(String label) { - return remainingResourceByLabel.getOrDefault(label, ResourceVector.newInstance()); + public ResourceVector getPostCalculatorRemainingResource(String label) { + remainingResourceByLabel.putIfAbsent(label, ResourceVector.newInstance()); + return remainingResourceByLabel.get(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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java index 21de811f1c298..ee3c4e49e29ee 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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java @@ -33,13 +33,10 @@ */ public class QueueCapacityUpdateContext { private final Resource updatedClusterResource; - - private final Map queueBranchContext - = LazyMap.decorate(new HashMap(), - QueueBranchContext::new); private final RMNodeLabelsManager labelsManager; - private List warnings = new ArrayList(); + private final Map queueBranchContext = new HashMap<>(); + private final List warnings = new ArrayList(); public QueueCapacityUpdateContext(Resource updatedClusterResource, RMNodeLabelsManager labelsManager) { @@ -69,7 +66,8 @@ public Resource getUpdatedClusterResource() { * @param queuePath queue path of the parent * @return queue branch context */ - public QueueBranchContext getQueueBranchContext(String queuePath) { + public QueueBranchContext getOrCreateQueueBranchContext(String queuePath) { + queueBranchContext.putIfAbsent(queuePath, new QueueBranchContext()); return queueBranchContext.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/ResourceVector.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/ResourceVector.java index 5f6e769b2dfaa..6011d4d686de5 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/ResourceVector.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/ResourceVector.java @@ -91,25 +91,21 @@ public void subtract(ResourceVector otherResourceVector) { } /** - * Increments the given resource by the specified value. + * Subtracts the given resource by the specified value. * @param resourceName name of the resource - * @param value value to be added to the resource's current value + * @param value value to be subtracted from the resource's current value */ - public void increment(String resourceName, float value) { - setValue(resourceName, getValue(resourceName) + value); + public void subtract(String resourceName, float value) { + setValue(resourceName, getValue(resourceName) - value); } /** - * Gets the average of all resource unit values. - * @return average of resource unit values + * Increments the given resource by the specified value. + * @param resourceName name of the resource + * @param value value to be added to the resource's current value */ - public float getAverageValue() { - return (float) resourcesByName.values().stream().mapToDouble(value -> value).average() - .orElse(0); - } - - public float getMaxValue() { - return resourcesByName.values().stream().max(Comparator.naturalOrder()).orElse(0f); + public void increment(String resourceName, float value) { + setValue(resourceName, getValue(resourceName) + value); } public float getValue(String resourceName) { 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 3294c69903a2c..3e77a28637bbd 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -36,7 +36,7 @@ public void calculateResourcePrerequisites( for (String label : childQueue.getConfiguredNodeLabels()) { for (String resourceName : childQueue.getConfiguredCapacityVector(label) .getResourceNamesByCapacityType(getCapacityType())) { - updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + updateContext.getOrCreateQueueBranchContext(parentQueue.getQueuePath()) .incrementWeight(label, resourceName, childQueue.getConfiguredCapacityVector(label) .getResource(resourceName).getResourceValue()); } @@ -51,11 +51,11 @@ public float calculateMinimumResource( CSQueue parentQueue = childQueue.getParent(); String resourceName = capacityVectorEntry.getResourceName(); float normalizedWeight = capacityVectorEntry.getResourceValue() - / updateContext.getQueueBranchContext(parentQueue.getQueuePath()) + / updateContext.getOrCreateQueueBranchContext(parentQueue.getQueuePath()) .getSumWeightsByResource(label, resourceName); - float remainingResource = updateContext.getQueueBranchContext( - parentQueue.getQueuePath()).getBatchRemainingResources(label) + float remainingResource = updateContext.getOrCreateQueueBranchContext( + parentQueue.getQueuePath()).getPostCalculatorRemainingResource(label) .getValue(resourceName); // Due to rounding loss it is better to use all remaining resources if no other resource uses @@ -101,7 +101,7 @@ public void updateCapacitiesAfterCalculation( Collection resourceNames = getResourceNames(queue, label); for (String resourceName : resourceNames) { - float sumBranchWeight = updateContext.getQueueBranchContext(queue.getParent().getQueuePath()) + float sumBranchWeight = updateContext.getOrCreateQueueBranchContext(queue.getParent().getQueuePath()) .getSumWeightsByResource(label, resourceName); float capacity = queue.getConfiguredCapacityVector(label).getResource( resourceName).getResourceValue() / sumBranchWeight; 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 d42f9a614f9a3..cd5d3babc38c9 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 @@ -47,11 +47,11 @@ public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCal public static final Resource B1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); public static final Resource C_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(5803, 4); - public static final Resource B_WARNING_RESOURCE = Resource.newInstance(8096, 3); + public static final Resource B_WARNING_RESOURCE = Resource.newInstance(8096, 4); public static final Resource B1_WARNING_RESOURCE = Resource.newInstance(8096, 3); - public static final Resource A_WARNING_RESOURCE = Resource.newInstance(8288, 9); + public static final Resource A_WARNING_RESOURCE = Resource.newInstance(8288, 12); public static final Resource A1_WARNING_RESOURCE = Resource.newInstance(2048, 4); - public static final Resource A2_WARNING_RESOURCE = Resource.newInstance(2048, 5); + public static final Resource A2_WARNING_RESOURCE = Resource.newInstance(2048, 8); public static final Resource A12_WARNING_RESOURCE = Resource.newInstance(2048, 4); @Override @@ -165,20 +165,18 @@ public void testComplexHierarchyWithWarnings() throws IOException { updateContext.getUpdateWarnings(), QueueUpdateWarningType.BRANCH_DOWNSCALED, B); Optional queueA11ZeroResourceWarning = getSpecificWarning( updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, A11); - Optional queueA12ZeroResourceWarning = getSpecificWarning( - updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_ZERO_RESOURCE, A12); Assert.assertTrue(queueCZeroResourceWarning.isPresent()); Assert.assertTrue(queueARemainingResourceWarning.isPresent()); Assert.assertTrue(queueBDownscalingWarning.isPresent()); Assert.assertTrue(queueA11ZeroResourceWarning.isPresent()); - Assert.assertTrue(queueA12ZeroResourceWarning.isPresent()); } @Test public void testZeroResourceIfNoMemory() throws IOException { csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(100), weight(6))); - csConf.setCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), absolute(VCORES * 0.5))); + csConf.setCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), + absolute(VCORES * 0.5))); QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) @@ -200,10 +198,13 @@ public void testZeroResourceIfNoMemory() throws IOException { @Test public void testDifferentMinimumAndMaximumCapacityTypes() throws IOException { - csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(50), absolute(VCORES * 0.5))); - csConf.setMaximumCapacityVector(A, "", createMemoryVcoresVector(absolute(MEMORY), percentage(80))); + csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(50), + absolute(VCORES * 0.5))); + csConf.setMaximumCapacityVector(A, "", createMemoryVcoresVector(absolute(MEMORY), + percentage(80))); csConf.setCapacityVector(B, "", createMemoryVcoresVector(weight(6), percentage(100))); - csConf.setMaximumCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), absolute(VCORES * 0.5))); + csConf.setMaximumCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), + absolute(VCORES * 0.5))); QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) @@ -226,17 +227,20 @@ public void testDifferentMinimumAndMaximumCapacityTypes() throws IOException { try { cs.reinitialize(csConf, mockRM.getRMContext()); update(assertionBuilder, UPDATE_RESOURCE); - Assert.fail("WEIGHT maximum capacity type is not supported, an error should be thrown when set up"); + Assert.fail("WEIGHT maximum capacity type is not supported, an error should be thrown when " + + "set up"); } catch (IllegalStateException ignored) { } } @Test public void testMaximumResourceWarnings() throws IOException { - csConf.setMaximumCapacityVector(A1, "", createMemoryVcoresVector(absolute(MEMORY * 0.5), percentage(100))); + csConf.setMaximumCapacityVector(A1, "", createMemoryVcoresVector(absolute(MEMORY * 0.5), + percentage(100))); csConf.setCapacityVector(A11, "", createMemoryVcoresVector(percentage(50), percentage(100))); csConf.setCapacityVector(A12, "", createMemoryVcoresVector(percentage(50), percentage(0))); - csConf.setMaximumCapacityVector(A11, "", createMemoryVcoresVector(absolute(MEMORY), percentage(10))); + csConf.setMaximumCapacityVector(A11, "", createMemoryVcoresVector(absolute(MEMORY), + percentage(10))); QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A11) @@ -253,7 +257,8 @@ public void testMaximumResourceWarnings() throws IOException { QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); Optional queueA11ExceedsParentMaxResourceWarning = getSpecificWarning( - updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT, A11); + updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT, + A11); Optional queueA11MinExceedsMaxWarning = getSpecificWarning( updateContext.getUpdateWarnings(), QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE, A11); Assert.assertTrue(queueA11ExceedsParentMaxResourceWarning.isPresent()); @@ -288,7 +293,8 @@ private void setupQueueHierarchyWithWarnings() throws IOException { csConf.setState(B, QueueState.RUNNING); csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(100), weight(6))); - csConf.setCapacityVector(A1, "", createMemoryVcoresVector(absolute(2048), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A1, "", createMemoryVcoresVector(absolute(2048), + absolute(VCORES * 0.25))); csConf.setCapacityVector(A11, "", createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); csConf.setCapacityVector(A12, "", createMemoryVcoresVector(percentage(100), percentage(100))); csConf.setCapacityVector(A2, "", createMemoryVcoresVector(absolute(2048), percentage(100))); @@ -308,7 +314,7 @@ private void setQueues() { private Optional getSpecificWarning( Collection warnings, QueueUpdateWarningType warningTypeToSelect, String queue) { - return warnings.stream().filter((w) -> w.getWarningType().equals(warningTypeToSelect) && w.getQueue().equals( - queue)).findFirst(); + return warnings.stream().filter((w) -> w.getWarningType().equals(warningTypeToSelect) + && w.getQueue().equals(queue)).findFirst(); } } From e549acc7261d3a9e426326d25fea1d3721b72974 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 18 Nov 2021 09:42:40 +0100 Subject: [PATCH 12/27] YARN-10965. Introduce driver concept to simplify the logic and encapsulate the intermediate values that belong together --- .../AbsoluteResourceCapacityCalculator.java | 23 +- .../AbstractQueueCapacityCalculator.java | 52 ++- ...CapacitySchedulerQueueCapacityHandler.java | 244 ++----------- .../PercentageQueueCapacityCalculator.java | 41 +-- .../capacity/QueueBranchContext.java | 105 ------ .../capacity/QueueCapacityUpdateContext.java | 12 - .../capacity/ResourceCalculationDriver.java | 334 ++++++++++++++++++ .../capacity/RootCalculationDriver.java | 47 +++ .../capacity/RootQueueCapacityCalculator.java | 18 +- .../WeightQueueCapacityCalculator.java | 64 ++-- 10 files changed, 492 insertions(+), 448 deletions(-) delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueBranchContext.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ResourceCalculationDriver.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootCalculationDriver.java 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index 3560c9a12589f..3e62cd9356671 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -19,32 +19,31 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { @Override public float calculateMinimumResource( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry) { - String resourceName = capacityVectorEntry.getResourceName(); - ResourceVector ratio = updateContext.getOrCreateQueueBranchContext(childQueue.getParent().getQueuePath()) - .getNormalizedResourceRatios().getOrDefault(label, ResourceVector.of(1)); + ResourceCalculationDriver resourceCalculationDriver, String label) { + String resourceName = resourceCalculationDriver.getCurrentResourceName(); + ResourceVector ratio = resourceCalculationDriver.getNormalizedResourceRatios().getOrDefault( + label, ResourceVector.of(1)); - return ratio.getValue(resourceName) * capacityVectorEntry.getResourceValue(); + return ratio.getValue(resourceName) * resourceCalculationDriver.getCurrentMinimumCapacityEntry(label) + .getResourceValue(); } @Override public float calculateMaximumResource( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry) { - return capacityVectorEntry.getResourceValue(); + ResourceCalculationDriver resourceCalculationDriver, String label) { + return resourceCalculationDriver.getCurrentMaximumCapacityEntry(label).getResourceValue(); } @Override public void updateCapacitiesAfterCalculation( - QueueCapacityUpdateContext updateContext, CSQueue queue, String label) { - setQueueCapacities(updateContext.getUpdatedClusterResource(label), queue, label); + ResourceCalculationDriver resourceCalculationDriver, String label) { + setQueueCapacities(resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource( + label), resourceCalculationDriver.getCurrentChild(), label); } @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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 87c514b2621d1..856941ac77e85 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -35,16 +35,15 @@ * logic. */ public abstract class AbstractQueueCapacityCalculator { + private static final String MB_UNIT = "Mi"; /** * Sets the metrics and statistics after effective resource values calculation. * - * @param updateContext context of the current update phase - * @param queue queue to update * @param label node label */ public abstract void updateCapacitiesAfterCalculation( - QueueCapacityUpdateContext updateContext, CSQueue queue, String label); + ResourceCalculationDriver resourceCalculationDriver, String label); /** @@ -57,42 +56,36 @@ public abstract void updateCapacitiesAfterCalculation( /** * Calculates the minimum effective resource. * - * @param updateContext context of the current update phase - * @param childQueue queue to update + * @param resourceCalculationDriver driver that contains the current resource unit and child to + * process * @param label node label - * @param capacityVectorEntry resource unit for which the calculation is executed * @return minimum effective resource */ - public abstract float calculateMinimumResource( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry); + public abstract float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + String label); /** * Calculates the maximum effective resource. * - * @param updateContext context of the current update phase - * @param childQueue queue to update + * @param resourceCalculationDriver driver that contains the current resource unit and child to + * process * @param label node label - * @param capacityVectorEntry resource unit for which the calculation is executed * @return minimum effective resource */ - public abstract float calculateMaximumResource( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry); + public abstract float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + String label); /** * Executes all logic that must be called prior to the effective resource value calculations. * - * @param updateContext context of the current update phase - * @param parentQueue parent for which the prerequisite actions must be executed + * @param resourceCalculationDriver driver that contains the parent queue on which the prerequisite + * calculation should be made */ - public void calculateResourcePrerequisites(QueueCapacityUpdateContext updateContext, - CSQueue parentQueue) { - for (String label : parentQueue.getConfiguredNodeLabels()) { + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + for (String label : resourceCalculationDriver.getParent().getConfiguredNodeLabels()) { // We need to set normalized resource ratio only once per parent - if (updateContext.getOrCreateQueueBranchContext(parentQueue.getQueuePath()) - .getNormalizedResourceRatios().isEmpty()) { - setNormalizedResourceRatio(updateContext, parentQueue, label); + if (resourceCalculationDriver.getNormalizedResourceRatios().isEmpty()) { + setNormalizedResourceRatio(resourceCalculationDriver, label); } } } @@ -170,14 +163,15 @@ public static void setQueueCapacities( * aggregated configured absolute resource of its children, the resource ratio will be less, * than 1. * - * @param updateContext context of the current update phase - * @param parentQueue parent for which the normalized ratio is defined * @param label node label */ private void setNormalizedResourceRatio( - QueueCapacityUpdateContext updateContext, CSQueue parentQueue, String label) { + ResourceCalculationDriver resourceCalculationDriver, String label) { // ManagedParents assign zero capacity to queues in case of overutilization, downscaling is // turned off for their children + CSQueue parentQueue = resourceCalculationDriver.getParent(); + QueueCapacityUpdateContext updateContext = resourceCalculationDriver.getUpdateContext(); + if (parentQueue instanceof ManagedParentQueue) { return; } @@ -215,14 +209,14 @@ private void setNormalizedResourceRatio( parentQueue.getQueuePath())); } - String unit = resourceName.equals(MEMORY_URI) ? "Mi" : ""; + String unit = resourceName.equals(MEMORY_URI) ? MB_UNIT : ""; long convertedValue = UnitsConversionUtil.convert(unit, updateContext.getUpdatedClusterResource(label).getResourceInformation(resourceName) .getUnits(), childrenConfiguredResource); if (convertedValue != 0) { - Map normalizedResourceRatios = updateContext.getOrCreateQueueBranchContext( - parentQueue.getQueuePath()).getNormalizedResourceRatios(); + Map normalizedResourceRatios = resourceCalculationDriver + .getNormalizedResourceRatios(); normalizedResourceRatios.putIfAbsent(label, ResourceVector.newInstance()); normalizedResourceRatios.get(label).setValue(resourceName, numeratorForMinRatio / convertedValue); 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index cceff569d5278..f326c53c02915 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -19,15 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; -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.QueueUpdateWarning.QueueUpdateWarningType; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,19 +48,12 @@ public class CapacitySchedulerQueueCapacityHandler { private static final Logger LOG = LoggerFactory.getLogger(CapacitySchedulerQueueCapacityHandler.class); - private static final Set CALCULATOR_PRECEDENCE = - ImmutableSet.of( - ResourceUnitCapacityType.ABSOLUTE, - ResourceUnitCapacityType.PERCENTAGE, - ResourceUnitCapacityType.WEIGHT); private final Map calculators; private final AbstractQueueCapacityCalculator rootCalculator = new RootQueueCapacityCalculator(); private final RMNodeLabelsManager labelsManager; private final Collection definedResources = new LinkedHashSet<>(); - private final QueueResourceRoundingStrategy roundingStrategy = - new DefaultQueueResourceRoundingStrategy(); public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) { this.calculators = new HashMap<>(); @@ -90,44 +78,25 @@ public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) */ public QueueCapacityUpdateContext update(Resource clusterResource, CSQueue queue) { ResourceLimits resourceLimits = new ResourceLimits(clusterResource); - QueueCapacityUpdateContext newContext = + QueueCapacityUpdateContext updateContext = new QueueCapacityUpdateContext(clusterResource, labelsManager); if (queue.getQueuePath().equals(ROOT)) { - updateRoot(queue, newContext, resourceLimits); - updateChildren(queue, newContext, resourceLimits); + updateRoot(queue, updateContext, resourceLimits); + updateChildren(queue, updateContext, resourceLimits); } else { - updateChildren(queue.getParent(), newContext, resourceLimits); + updateChildren(queue.getParent(), updateContext, resourceLimits); } - return newContext; + return updateContext; } private void updateRoot( - CSQueue queue, QueueCapacityUpdateContext newContext, ResourceLimits resourceLimits) { - for (String label : queue.getConfiguredNodeLabels()) { - for (QueueCapacityVectorEntry capacityVectorEntry : queue.getConfiguredCapacityVector(label)) { - queue.getOrCreateAbsoluteMinCapacityVector(label).setValue( - capacityVectorEntry.getResourceName(), 1); - queue.getOrCreateAbsoluteMaxCapacityVector(label).setValue( - capacityVectorEntry.getResourceName(), 1); - - float minimumResource = rootCalculator.calculateMinimumResource(newContext, queue, label, - capacityVectorEntry); - float maximumResource = rootCalculator.calculateMaximumResource(newContext, queue, label, - capacityVectorEntry); - long roundedMinResource = (long) Math.floor(minimumResource); - long roundedMaxResource = (long) Math.floor(maximumResource); - queue.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), roundedMinResource); - queue.getQueueResourceQuotas().getEffectiveMaxResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), roundedMaxResource); - } - rootCalculator.updateCapacitiesAfterCalculation(newContext, queue, label); - } - - rootCalculator.calculateResourcePrerequisites(newContext, queue); - queue.refreshAfterResourceCalculation(newContext.getUpdatedClusterResource(), resourceLimits); + CSQueue queue, QueueCapacityUpdateContext updateContext, ResourceLimits resourceLimits) { + RootCalculationDriver rootCalculationDriver = new RootCalculationDriver(queue, updateContext, + rootCalculator, definedResources); + rootCalculationDriver.calculateResources(); + queue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), resourceLimits); } private void updateChildren( @@ -137,190 +106,23 @@ private void updateChildren( return; } - for (String label : parent.getConfiguredNodeLabels()) { - updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()).setPostCalculatorRemainingResource(label, - ResourceVector.of(parent.getEffectiveCapacity(label))); - updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()).setOverallRemainingResource(label, - ResourceVector.of(parent.getEffectiveCapacity(label))); - } - - for (AbstractQueueCapacityCalculator capacityCalculator : calculators.values()) { - capacityCalculator.calculateResourcePrerequisites(updateContext, parent); - } - - calculateChildrenResources(updateContext, parent); - - for (String label : parent.getConfiguredNodeLabels()) { - if (!updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()).getPostCalculatorRemainingResource( - label).equals(ResourceVector.newInstance())) { - updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( - parent.getQueuePath())); - } - } - - updateChildrenAfterCalculation(parent, updateContext, resourceLimits); - } - - private void calculateChildrenResources( - QueueCapacityUpdateContext updateContext, CSQueue parent) { - for (String resourceName : definedResources) { - for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { - Map effectiveResourceUsedByLabel = new HashMap<>(); - for (CSQueue childQueue : parent.getChildQueues()) { - childQueue.getWriteLock().lock(); - try { - for (String label : childQueue.getConfiguredNodeLabels()) { - QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); - if (!childQueue.getConfiguredCapacityVector(label).isResourceOfType( - resourceName, capacityType)) { - continue; - } - float aggregatedUsedResource = effectiveResourceUsedByLabel.getOrDefault(label, - 0f); - float usedResourceByChild = setChildResources(updateContext, childQueue, label, - capacityVector.getResource(resourceName)); - float resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; - updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()) - .getOverallRemainingResource(label).subtract(resourceName, usedResourceByChild); - effectiveResourceUsedByLabel.put(label, resourceUsedByLabel); - } - } finally { - childQueue.getWriteLock().unlock(); - } - } - - // Only decrement post calculator remaining resource at the end of each calculator phase - for (Map.Entry entry : effectiveResourceUsedByLabel.entrySet()) { - updateContext.getOrCreateQueueBranchContext(parent.getQueuePath()).getPostCalculatorRemainingResource( - entry.getKey()).subtract(resourceName, entry.getValue()); - } - } - } - } - - private float setChildResources( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry) { - AbstractQueueCapacityCalculator capacityCalculator = calculators.get( - capacityVectorEntry.getVectorResourceType()); - String resourceName = capacityVectorEntry.getResourceName(); - long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( - resourceName); - QueueCapacityVectorEntry maximumCapacityVectorEntry = childQueue - .getConfiguredMaximumCapacityVector(label).getResource(resourceName); - AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( - maximumCapacityVectorEntry.getVectorResourceType()); - - float minimumResource = capacityCalculator.calculateMinimumResource( - updateContext, childQueue, label, capacityVectorEntry); - float maximumResource = maximumCapacityCalculator.calculateMaximumResource( - updateContext, childQueue, label, maximumCapacityVectorEntry); - - minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); - maximumResource = roundingStrategy.getRoundedResource(maximumResource, - maximumCapacityVectorEntry); - Pair resources = validateCalculatedResources(updateContext, childQueue, - resourceName, label, new ImmutablePair<>(minimumResource, maximumResource)); - minimumResource = resources.getLeft(); - maximumResource = resources.getRight(); - - float absoluteMinCapacity = minimumResource / clusterResource; - float absoluteMaxCapacity = maximumResource / clusterResource; - childQueue.getOrCreateAbsoluteMinCapacityVector(label).setValue( - resourceName, absoluteMinCapacity); - childQueue.getOrCreateAbsoluteMaxCapacityVector(label).setValue( - resourceName, absoluteMaxCapacity); - - childQueue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( - resourceName, (long) minimumResource); - childQueue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( - resourceName, (long) maximumResource); + ResourceCalculationDriver resourceCalculationDriver = new ResourceCalculationDriver( + parent, updateContext, calculators, definedResources); + resourceCalculationDriver.calculateResources(); - return minimumResource; - } - - private Pair validateCalculatedResources( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String resourceName, - String label, Pair calculatedResources) { - CSQueue parentQueue = childQueue.getParent(); - - float minimumResource = calculatedResources.getLeft(); - long minimumMemoryResource = childQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .getMemorySize(); - - float remainingResourceUnderParent = updateContext.getOrCreateQueueBranchContext( - parentQueue.getQueuePath()).getOverallRemainingResource(label).getValue(resourceName); - - long parentMaximumResource = parentQueue.getEffectiveMaxCapacity(label).getResourceValue( - resourceName); - float maximumResource = calculatedResources.getRight(); - - // Memory is the primary resource, if its zero, all other resource units are zero as well. - if (!resourceName.equals(MEMORY_URI) && minimumMemoryResource == 0) { - minimumResource = 0; - } - - if (maximumResource != 0 && maximumResource > parentMaximumResource) { - updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( - childQueue.getQueuePath())); - } - maximumResource = maximumResource == 0 ? parentMaximumResource : Math.min(maximumResource, - parentMaximumResource); - - if (maximumResource < minimumResource) { - updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( - childQueue.getQueuePath())); - minimumResource = maximumResource; - } - - if (minimumResource > remainingResourceUnderParent) { - // Legacy auto queues are assigned a zero resource if not enough resource is left - if (parentQueue instanceof ManagedParentQueue) { - minimumResource = 0; - } else { - updateContext.addUpdateWarning( - QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(childQueue.getQueuePath()).withInfo( - "Resource name: " + resourceName + " resource value: " + minimumResource)); - minimumResource = remainingResourceUnderParent; - } - } - - if (minimumResource == 0) { - updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( - childQueue.getQueuePath()).withInfo("Resource name: " + resourceName)); - } - - return new ImmutablePair<>(minimumResource, maximumResource); + updateChildrenAfterCalculation(resourceCalculationDriver, resourceLimits); } private void updateChildrenAfterCalculation( - CSQueue parent, QueueCapacityUpdateContext updateContext, ResourceLimits resourceLimits) { - for (CSQueue childQueue : parent.getChildQueues()) { - updateChildCapacities(updateContext, childQueue); - ResourceLimits childLimit = ((ParentQueue) parent).getResourceLimitsOfChild( - childQueue, updateContext.getUpdatedClusterResource(), resourceLimits, NO_LABEL, false); - childQueue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), - childLimit); - updateChildren(childQueue, updateContext, childLimit); - } - } - - private void updateChildCapacities( - QueueCapacityUpdateContext updateContext, CSQueue childQueue) { - for (String label : childQueue.getConfiguredNodeLabels()) { - QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); - if (capacityVector.isMixedCapacityVector()) { - // Post update capacities based on the calculated effective resource values - AbstractQueueCapacityCalculator.setQueueCapacities(updateContext.getUpdatedClusterResource( - label), childQueue, label); - } else { - // Update capacities according to the legacy logic - for (ResourceUnitCapacityType capacityType : - childQueue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { - AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); - calculator.updateCapacitiesAfterCalculation(updateContext, childQueue, label); - } - } + ResourceCalculationDriver resourceCalculationDriver, ResourceLimits resourceLimits) { + for (CSQueue childQueue : resourceCalculationDriver.getParent().getChildQueues()) { + resourceCalculationDriver.setCurrentChild(childQueue); + resourceCalculationDriver.updateChildCapacities(); + ResourceLimits childLimit = ((ParentQueue) resourceCalculationDriver.getParent()).getResourceLimitsOfChild( + childQueue, resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(), resourceLimits, NO_LABEL, false); + childQueue.refreshAfterResourceCalculation(resourceCalculationDriver.getUpdateContext() + .getUpdatedClusterResource(), childLimit); + updateChildren(childQueue, resourceCalculationDriver.getUpdateContext(), childLimit); } } 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index 832f6872019d9..77765a126fc14 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -19,50 +19,45 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; public class PercentageQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override public float calculateMinimumResource( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry) { - CSQueue parentQueue = childQueue.getParent(); - String resourceName = capacityVectorEntry.getResourceName(); + ResourceCalculationDriver resourceCalculationDriver, String label) { + CSQueue parentQueue = resourceCalculationDriver.getParent(); + String resourceName = resourceCalculationDriver.getCurrentMinimumCapacityEntry(label).getResourceName(); float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label).getValue( resourceName); - float remainingPerEffectiveResourceRatio = updateContext.getOrCreateQueueBranchContext( - parentQueue.getQueuePath()).getPostCalculatorRemainingResource(label) - .getValue(resourceName) / parentQueue.getEffectiveCapacity(label) - .getResourceValue(resourceName); - float absoluteCapacity = parentAbsoluteCapacity * - remainingPerEffectiveResourceRatio - * capacityVectorEntry.getResourceValue() / 100; + float remainingPerEffectiveResourceRatio = resourceCalculationDriver.getBatchRemainingResource( + label).getValue(resourceName) / parentQueue.getEffectiveCapacity(label).getResourceValue( + resourceName); + float absoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio + * resourceCalculationDriver.getCurrentMinimumCapacityEntry(label).getResourceValue() / 100; - return updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName) - * absoluteCapacity; + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) + .getResourceValue(resourceName) * absoluteCapacity; } @Override public float calculateMaximumResource( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry) { - CSQueue parentQueue = childQueue.getParent(); - String resourceName = capacityVectorEntry.getResourceName(); + ResourceCalculationDriver resourceCalculationDriver, String label) { + CSQueue parentQueue = resourceCalculationDriver.getParent(); + String resourceName = resourceCalculationDriver.getCurrentResourceName(); float parentAbsoluteMaxCapacity = parentQueue.getOrCreateAbsoluteMaxCapacityVector(label) .getValue(resourceName); float absoluteMaxCapacity = parentAbsoluteMaxCapacity - * capacityVectorEntry.getResourceValue() / 100; + * resourceCalculationDriver.getCurrentMaximumCapacityEntry(label).getResourceValue() / 100; - return updateContext.getUpdatedClusterResource(label).getResourceValue( - capacityVectorEntry.getResourceName()) * absoluteMaxCapacity; + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) + .getResourceValue(resourceName) * absoluteMaxCapacity; } @Override - public void updateCapacitiesAfterCalculation(QueueCapacityUpdateContext updateContext, CSQueue queue, String label) { - ((AbstractCSQueue)queue).updateAbsoluteCapacities(); + public void updateCapacitiesAfterCalculation(ResourceCalculationDriver resourceCalculationDriver, String label) { + ((AbstractCSQueue)resourceCalculationDriver.getCurrentChild()).updateAbsoluteCapacities(); } @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/QueueBranchContext.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/QueueBranchContext.java deleted file mode 100644 index 70c6a0d331b0f..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/QueueBranchContext.java +++ /dev/null @@ -1,105 +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 java.util.HashMap; -import java.util.Map; - -/** - * Contains all intermediate calculation values that are common for a queue - * branch (all siblings that have a common parent). - */ -public class QueueBranchContext { - private final Map remainingResourceByLabel = new HashMap<>(); - private final Map normalizedResourceRatio = new HashMap<>(); - private final Map overallRemainingResource = new HashMap<>(); - private final Map> sumWeightsPerLabel = new HashMap<>(); - - /** - * Increments the aggregated weight. - * @param label node label - * @param resourceName resource unit name - * @param value weight value - */ - public void incrementWeight(String label, String resourceName, float value) { - sumWeightsPerLabel.putIfAbsent(label, new HashMap<>()); - sumWeightsPerLabel.get(label).put(resourceName, - sumWeightsPerLabel.get(label).getOrDefault(resourceName, 0f) + value); - } - - /** - * Returns the aggregated children weights. - * @param label node label - * @param resourceName resource unit name - * @return aggregated weights of children - */ - public float getSumWeightsByResource(String label, String resourceName) { - return sumWeightsPerLabel.get(label).get(resourceName); - } - - /** - * Sets the remaining resource under a parent that is available for its children to - * occupy. - * - * @param label node label - * @param resource resource vector - */ - public void setPostCalculatorRemainingResource(String label, ResourceVector resource) { - remainingResourceByLabel.put(label, resource); - } - - public Map getNormalizedResourceRatios() { - return normalizedResourceRatio; - } - - /** - * Returns the remaining resources of a parent that is still available for its - * children. - * - * @param label node label - * @return remaining resources - */ - public ResourceVector getOverallRemainingResource(String label) { - overallRemainingResource.putIfAbsent(label, ResourceVector.newInstance()); - return overallRemainingResource.get(label); - } - - /** - * Sets the remaining resources of a parent that is still available for its children. - * - * @param label node label - * @param resourceVector resource vector - */ - public void setOverallRemainingResource(String label, ResourceVector resourceVector) { - overallRemainingResource.put(label, resourceVector); - } - - /** - * Returns the remaining resources of a parent that is still available for its - * children. Decremented only after the calculator is finished its work on the corresponding - * resources. - * - * @param label node label - * @return remaining resources - */ - public ResourceVector getPostCalculatorRemainingResource(String label) { - remainingResourceByLabel.putIfAbsent(label, ResourceVector.newInstance()); - return remainingResourceByLabel.get(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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java index ee3c4e49e29ee..67f682609ba02 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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java @@ -35,7 +35,6 @@ public class QueueCapacityUpdateContext { private final Resource updatedClusterResource; private final RMNodeLabelsManager labelsManager; - private final Map queueBranchContext = new HashMap<>(); private final List warnings = new ArrayList(); public QueueCapacityUpdateContext(Resource updatedClusterResource, @@ -60,17 +59,6 @@ public Resource getUpdatedClusterResource() { return updatedClusterResource; } - /** - * Returns the context for a queue branch, which is identified by the path of - * the parent. - * @param queuePath queue path of the parent - * @return queue branch context - */ - public QueueBranchContext getOrCreateQueueBranchContext(String queuePath) { - queueBranchContext.putIfAbsent(queuePath, new QueueBranchContext()); - return queueBranchContext.get(queuePath); - } - /** * Adds an update warning to the context. * @param warning warning during update phase 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java new file mode 100644 index 0000000000000..da91431f220a3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ResourceCalculationDriver.java @@ -0,0 +1,334 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; + +/** + * Drives the main logic of resource calculation for all children under a parent queue. Acts as a + * bookkeeper of disposable update information that is used by all children under a common parent. + */ +public class ResourceCalculationDriver { + protected static final Set CALCULATOR_PRECEDENCE = + ImmutableSet.of( + ResourceUnitCapacityType.ABSOLUTE, + ResourceUnitCapacityType.PERCENTAGE, + ResourceUnitCapacityType.WEIGHT); + + protected final QueueResourceRoundingStrategy roundingStrategy = + new DefaultQueueResourceRoundingStrategy(); + protected final CSQueue parent; + protected final QueueCapacityUpdateContext updateContext; + protected final Map calculators; + protected final Collection definedResources; + + protected final Map overallRemainingResource = new HashMap<>(); + protected final Map batchRemainingResource = new HashMap<>(); + + protected String currentResourceName; + protected AbstractQueueCapacityCalculator currentCalculator; + protected CSQueue currentChild; + protected Map usedResourceByCurrentCalculator = new HashMap<>(); + + // Used by ABSOLUTE capacity types + protected final Map normalizedResourceRatio = new HashMap<>(); + // Used by WEIGHT capacity types + protected final Map> sumWeightsPerLabel = new HashMap<>(); + + public ResourceCalculationDriver( + CSQueue parent, QueueCapacityUpdateContext updateContext, + Map calculators, + Collection definedResources) { + this.parent = parent; + this.updateContext = updateContext; + this.calculators = calculators; + this.definedResources = definedResources; + } + + /** + * Returns the parent that is driving the calculation. + * @return a common parent queue + */ + public CSQueue getParent() { + return parent; + } + + /** + * Returns the context that is used throughout the whole update phase. + * @return update context + */ + public QueueCapacityUpdateContext getUpdateContext() { + return updateContext; + } + + /** + * Returns the name of the resource that is currently processed. + * @return resource name + */ + public String getCurrentResourceName() { + return currentResourceName; + } + + /** + * Returns the child that is currently processed. + * @return child queue + */ + public CSQueue getCurrentChild() { + return currentChild; + } + + /** + * Sets the currently evaluated child to a specific queue. + * @param currentChild a child queue + */ + public void setCurrentChild(CSQueue currentChild) { + if (currentChild.getParent() != parent) { + throw new IllegalArgumentException("Child queue " + currentChild.getQueuePath() + " is not " + + "a child of " + parent.getQueuePath()); + } + + this.currentChild = currentChild; + } + + /** + * A shorthand to return the minimum capacity vector entry for the currently evaluated child and + * resource name. + * + * @param label node label + * @return capacity vector entry + */ + public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { + return currentChild.getConfiguredCapacityVector(label).getResource(currentResourceName); + } + + /** + * A shorthand to return the maximum capacity vector entry for the currently evaluated child and + * resource name. + * + * @param label node label + * @return capacity vector entry + */ + public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { + return currentChild.getConfiguredMaximumCapacityVector(label).getResource(currentResourceName); + } + + /** + * Increments the aggregated weight. + * @param label node label + * @param resourceName resource unit name + * @param value weight value + */ + public void incrementWeight(String label, String resourceName, float value) { + sumWeightsPerLabel.putIfAbsent(label, new HashMap<>()); + sumWeightsPerLabel.get(label).put(resourceName, + sumWeightsPerLabel.get(label).getOrDefault(resourceName, 0f) + value); + } + + /** + * Returns the aggregated children weights. + * @param label node label + * @param resourceName resource unit name + * @return aggregated weights of children + */ + public float getSumWeightsByResource(String label, String resourceName) { + return sumWeightsPerLabel.get(label).get(resourceName); + } + + public Map getNormalizedResourceRatios() { + return normalizedResourceRatio; + } + + /** + * Returns the remaining resources of a parent that is still available for its + * children. Decremented only after the calculator is finished its work on the corresponding + * resources. + * + * @param label node label + * @return remaining resources + */ + public ResourceVector getBatchRemainingResource(String label) { + batchRemainingResource.putIfAbsent(label, ResourceVector.newInstance()); + return batchRemainingResource.get(label); + } + + /** + * Calculates and sets the minimum and maximum effective resources for all children under the + * parent queue with which this driver was initialized. + */ + public void calculateResources() { + // Reset both remaining resource storage to the parent's available resource + for (String label : parent.getConfiguredNodeLabels()) { + overallRemainingResource.put(label, ResourceVector.of(parent.getEffectiveCapacity(label))); + batchRemainingResource.put(label, ResourceVector.of(parent.getEffectiveCapacity(label))); + } + + for (AbstractQueueCapacityCalculator capacityCalculator : calculators.values()) { + capacityCalculator.calculateResourcePrerequisites(this); + } + + for (String resourceName : definedResources) { + currentResourceName = resourceName; + for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { + currentCalculator = calculators.get(capacityType); + for (CSQueue childQueue : parent.getChildQueues()) { + currentChild = childQueue; + calculateResourceOnChild(capacityType); + } + // Flush aggregated used resource by labels at the end of a calculator phase + for (Map.Entry entry : usedResourceByCurrentCalculator.entrySet()) { + batchRemainingResource.get(entry.getKey()).subtract(resourceName, entry.getValue()); + } + usedResourceByCurrentCalculator = new HashMap<>(); + } + } + + validateRemainingResource(); + } + + /** + * Updates the capacity values of the currently evaluated child. + */ + public void updateChildCapacities() { + for (String label : currentChild.getConfiguredNodeLabels()) { + QueueCapacityVector capacityVector = currentChild.getConfiguredCapacityVector(label); + if (capacityVector.isMixedCapacityVector()) { + // Post update capacities based on the calculated effective resource values + AbstractQueueCapacityCalculator.setQueueCapacities(updateContext.getUpdatedClusterResource( + label), currentChild, label); + } else { + // Update capacities according to the legacy logic + for (ResourceUnitCapacityType capacityType : + currentChild.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); + calculator.updateCapacitiesAfterCalculation(this, label); + } + } + } + } + + private void validateRemainingResource() { + for (String label : parent.getConfiguredNodeLabels()) { + if (!batchRemainingResource.get(label).equals(ResourceVector.newInstance())) { + updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( + parent.getQueuePath())); + } + } + } + + private void calculateResourceOnChild(ResourceUnitCapacityType capacityType) { + currentChild.getWriteLock().lock(); + try { + for (String label : currentChild.getConfiguredNodeLabels()) { + if (!currentChild.getConfiguredCapacityVector(label).isResourceOfType(currentResourceName, + capacityType)) { + return; + } + float usedResourceByChild = setChildResources(label); + float aggregatedUsedResource = usedResourceByCurrentCalculator.getOrDefault(label, + 0f); + float resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; + + overallRemainingResource.get(label).subtract(currentResourceName, usedResourceByChild); + usedResourceByCurrentCalculator.put(label, resourceUsedByLabel); + } + } finally { + currentChild.getWriteLock().unlock(); + } + } + + private float setChildResources(String label) { + QueueCapacityVectorEntry capacityVectorEntry = currentChild.getConfiguredCapacityVector( + label).getResource(currentResourceName); + long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( + currentResourceName); + QueueCapacityVectorEntry maximumCapacityVectorEntry = currentChild + .getConfiguredMaximumCapacityVector(label).getResource(currentResourceName); + AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( + maximumCapacityVectorEntry.getVectorResourceType()); + + float minimumResource = currentCalculator.calculateMinimumResource(this, label); + float maximumResource = maximumCapacityCalculator.calculateMaximumResource(this, label); + + minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); + maximumResource = roundingStrategy.getRoundedResource(maximumResource, + maximumCapacityVectorEntry); + Pair resources = validateCalculatedResources(label, new ImmutablePair<>( + minimumResource, maximumResource)); + minimumResource = resources.getLeft(); + maximumResource = resources.getRight(); + + float absoluteMinCapacity = minimumResource / clusterResource; + float absoluteMaxCapacity = maximumResource / clusterResource; + currentChild.getOrCreateAbsoluteMinCapacityVector(label).setValue( + currentResourceName, absoluteMinCapacity); + currentChild.getOrCreateAbsoluteMaxCapacityVector(label).setValue( + currentResourceName, absoluteMaxCapacity); + + currentChild.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + currentResourceName, (long) minimumResource); + currentChild.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + currentResourceName, (long) maximumResource); + + return minimumResource; + } + + private Pair validateCalculatedResources( + String label, Pair calculatedResources) { + float minimumResource = calculatedResources.getLeft(); + long minimumMemoryResource = currentChild.getQueueResourceQuotas().getEffectiveMinResource(label) + .getMemorySize(); + + float remainingResourceUnderParent = overallRemainingResource.get(label).getValue( + currentResourceName); + + long parentMaximumResource = parent.getEffectiveMaxCapacity(label).getResourceValue( + currentResourceName); + float maximumResource = calculatedResources.getRight(); + + // Memory is the primary resource, if its zero, all other resource units are zero as well. + if (!currentResourceName.equals(MEMORY_URI) && minimumMemoryResource == 0) { + minimumResource = 0; + } + + if (maximumResource != 0 && maximumResource > parentMaximumResource) { + updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( + currentChild.getQueuePath())); + } + maximumResource = maximumResource == 0 ? parentMaximumResource : Math.min(maximumResource, + parentMaximumResource); + + if (maximumResource < minimumResource) { + updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( + currentChild.getQueuePath())); + minimumResource = maximumResource; + } + + if (minimumResource > remainingResourceUnderParent) { + // Legacy auto queues are assigned a zero resource if not enough resource is left + if (parent instanceof ManagedParentQueue) { + minimumResource = 0; + } else { + updateContext.addUpdateWarning( + QueueUpdateWarning.QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(currentChild.getQueuePath()).withInfo( + "Resource name: " + currentResourceName + " resource value: " + minimumResource)); + minimumResource = remainingResourceUnderParent; + } + } + + if (minimumResource == 0) { + updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( + currentChild.getQueuePath()).withInfo("Resource name: " + currentResourceName)); + } + + return new ImmutablePair<>(minimumResource, maximumResource); + } +} 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/RootCalculationDriver.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/RootCalculationDriver.java new file mode 100644 index 0000000000000..175b55d1b1b91 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/RootCalculationDriver.java @@ -0,0 +1,47 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; + +/** + * A special case that contains the resource calculation of the root queue. + */ +public final class RootCalculationDriver extends ResourceCalculationDriver { + private final AbstractQueueCapacityCalculator rootCalculator; + + public RootCalculationDriver(CSQueue rootQueue, QueueCapacityUpdateContext updateContext, + AbstractQueueCapacityCalculator rootCalculator, + Collection definedResources) { + super(rootQueue, updateContext, Collections.emptyMap(), definedResources); + this.rootCalculator = rootCalculator; + } + + @Override + public void calculateResources() { + setCurrentChild(parent); + + for (String label : parent.getConfiguredNodeLabels()) { + for (QueueCapacityVector.QueueCapacityVectorEntry capacityVectorEntry : parent.getConfiguredCapacityVector(label)) { + currentResourceName = capacityVectorEntry.getResourceName(); + parent.getOrCreateAbsoluteMinCapacityVector(label).setValue( + capacityVectorEntry.getResourceName(), 1); + parent.getOrCreateAbsoluteMaxCapacityVector(label).setValue( + capacityVectorEntry.getResourceName(), 1); + + float minimumResource = rootCalculator.calculateMinimumResource(this, label); + float maximumResource = rootCalculator.calculateMaximumResource(this, label); + long roundedMinResource = (long) Math.floor(minimumResource); + long roundedMaxResource = (long) Math.floor(maximumResource); + parent.getQueueResourceQuotas().getEffectiveMinResource(label) + .setResourceValue(capacityVectorEntry.getResourceName(), roundedMinResource); + parent.getQueueResourceQuotas().getEffectiveMaxResource(label) + .setResourceValue(capacityVectorEntry.getResourceName(), roundedMaxResource); + } + rootCalculator.updateCapacitiesAfterCalculation(this, label); + } + + + rootCalculator.calculateResourcePrerequisites(this); + } +} 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index 3c0758127bca3..84f18f7691ac8 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -18,29 +18,27 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; - import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; public class RootQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public float calculateMinimumResource(QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { - return updateContext.getUpdatedClusterResource(label).getResourceValue(capacityVectorEntry.getResourceName()); + public float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, String label) { + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(resourceCalculationDriver.getCurrentResourceName()); } @Override - public float calculateMaximumResource(QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, QueueCapacityVectorEntry capacityVectorEntry) { - return updateContext.getUpdatedClusterResource(label).getResourceValue(capacityVectorEntry.getResourceName()); + public float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, String label) { + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(resourceCalculationDriver.getCurrentResourceName()); } @Override public void updateCapacitiesAfterCalculation( - QueueCapacityUpdateContext updateContext, CSQueue queue, String label) { - queue.getQueueCapacities().setAbsoluteCapacity(label, 1); - if (queue.getQueueCapacities().getWeight(label) == 1) { - queue.getQueueCapacities().setNormalizedWeight(label, 1); + ResourceCalculationDriver resourceCalculationDriver, String label) { + resourceCalculationDriver.getCurrentChild().getQueueCapacities().setAbsoluteCapacity(label, 1); + if (resourceCalculationDriver.getCurrentChild().getQueueCapacities().getWeight(label) == 1) { + resourceCalculationDriver.getCurrentChild().getQueueCapacities().setNormalizedWeight(label, 1); } } 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 3e77a28637bbd..c3aee324f0f68 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import java.util.Collection; @@ -28,35 +27,30 @@ public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public void calculateResourcePrerequisites( - QueueCapacityUpdateContext updateContext, CSQueue parentQueue) { - super.calculateResourcePrerequisites(updateContext, parentQueue); + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + super.calculateResourcePrerequisites(resourceCalculationDriver); - for (CSQueue childQueue : parentQueue.getChildQueues()) { + for (CSQueue childQueue : resourceCalculationDriver.getParent().getChildQueues()) { for (String label : childQueue.getConfiguredNodeLabels()) { for (String resourceName : childQueue.getConfiguredCapacityVector(label) .getResourceNamesByCapacityType(getCapacityType())) { - updateContext.getOrCreateQueueBranchContext(parentQueue.getQueuePath()) - .incrementWeight(label, resourceName, childQueue.getConfiguredCapacityVector(label) - .getResource(resourceName).getResourceValue()); + resourceCalculationDriver.incrementWeight(label, resourceName, childQueue + .getConfiguredCapacityVector(label).getResource(resourceName).getResourceValue()); } } } } @Override - public float calculateMinimumResource( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry) { - CSQueue parentQueue = childQueue.getParent(); - String resourceName = capacityVectorEntry.getResourceName(); - float normalizedWeight = capacityVectorEntry.getResourceValue() - / updateContext.getOrCreateQueueBranchContext(parentQueue.getQueuePath()) - .getSumWeightsByResource(label, resourceName); - - float remainingResource = updateContext.getOrCreateQueueBranchContext( - parentQueue.getQueuePath()).getPostCalculatorRemainingResource(label) - .getValue(resourceName); + public float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + String label) { + CSQueue parentQueue = resourceCalculationDriver.getParent(); + String resourceName = resourceCalculationDriver.getCurrentResourceName(); + float normalizedWeight = resourceCalculationDriver.getCurrentMinimumCapacityEntry(label) + .getResourceValue() / resourceCalculationDriver.getSumWeightsByResource(label, resourceName); + + float remainingResource = resourceCalculationDriver.getBatchRemainingResource(label).getValue( + resourceName); // Due to rounding loss it is better to use all remaining resources if no other resource uses // weight @@ -75,18 +69,17 @@ public float calculateMinimumResource( // Weight capacity types are the last to consider, therefore it is safe to assign all remaining // effective resources between queues. The strategy is to round values to the closest whole // number. - float resource = updateContext.getUpdatedClusterResource(label).getResourceValue(resourceName) - * queueAbsoluteCapacity; + float resource = resourceCalculationDriver.getUpdateContext() + .getUpdatedClusterResource(label).getResourceValue(resourceName) * queueAbsoluteCapacity; return Math.round(resource); } @Override - public float calculateMaximumResource( - QueueCapacityUpdateContext updateContext, CSQueue childQueue, String label, - QueueCapacityVectorEntry capacityVectorEntry) { - throw new IllegalStateException("Resource " + capacityVectorEntry.getResourceName() + " has " + - "WEIGHT maximum capacity type, which is not supported"); + public float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + String label) { + throw new IllegalStateException("Resource " + resourceCalculationDriver.getCurrentMinimumCapacityEntry( + label).getResourceName() + " has " + "WEIGHT maximum capacity type, which is not supported"); } @Override @@ -96,20 +89,19 @@ public ResourceUnitCapacityType getCapacityType() { @Override public void updateCapacitiesAfterCalculation( - QueueCapacityUpdateContext updateContext, CSQueue queue, String label) { + ResourceCalculationDriver resourceCalculationDriver, String label) { float sumCapacityPerResource = 0f; - Collection resourceNames = getResourceNames(queue, label); + Collection resourceNames = getResourceNames(resourceCalculationDriver.getCurrentChild(), label); for (String resourceName : resourceNames) { - float sumBranchWeight = updateContext.getOrCreateQueueBranchContext(queue.getParent().getQueuePath()) - .getSumWeightsByResource(label, resourceName); - float capacity = queue.getConfiguredCapacityVector(label).getResource( - resourceName).getResourceValue() / sumBranchWeight; + float sumBranchWeight = resourceCalculationDriver.getSumWeightsByResource(label, resourceName); + float capacity = resourceCalculationDriver.getCurrentChild().getConfiguredCapacityVector( + label).getResource(resourceName).getResourceValue() / sumBranchWeight; sumCapacityPerResource += capacity; } - queue.getQueueCapacities().setNormalizedWeight(label, sumCapacityPerResource - / resourceNames.size()); - ((AbstractCSQueue) queue).updateAbsoluteCapacities(); + resourceCalculationDriver.getCurrentChild().getQueueCapacities().setNormalizedWeight(label, + sumCapacityPerResource / resourceNames.size()); + ((AbstractCSQueue) resourceCalculationDriver.getCurrentChild()).updateAbsoluteCapacities(); } } From 6c629ea83a22f84cb129c63ac15373e021d59a40 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 18 Nov 2021 14:30:46 +0100 Subject: [PATCH 13/27] YARN-10965. Fix root driver child queue setting --- .../capacity/RootCalculationDriver.java | 17 ++++++----------- .../capacity/RootQueueCapacityCalculator.java | 6 +++--- 2 files changed, 9 insertions(+), 14 deletions(-) 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/RootCalculationDriver.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/RootCalculationDriver.java index 175b55d1b1b91..fd5ebca629f19 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/RootCalculationDriver.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/RootCalculationDriver.java @@ -19,29 +19,24 @@ public RootCalculationDriver(CSQueue rootQueue, QueueCapacityUpdateContext updat @Override public void calculateResources() { - setCurrentChild(parent); - for (String label : parent.getConfiguredNodeLabels()) { for (QueueCapacityVector.QueueCapacityVectorEntry capacityVectorEntry : parent.getConfiguredCapacityVector(label)) { currentResourceName = capacityVectorEntry.getResourceName(); - parent.getOrCreateAbsoluteMinCapacityVector(label).setValue( - capacityVectorEntry.getResourceName(), 1); - parent.getOrCreateAbsoluteMaxCapacityVector(label).setValue( - capacityVectorEntry.getResourceName(), 1); + parent.getOrCreateAbsoluteMinCapacityVector(label).setValue(currentResourceName, 1); + parent.getOrCreateAbsoluteMaxCapacityVector(label).setValue(currentResourceName, 1); float minimumResource = rootCalculator.calculateMinimumResource(this, label); float maximumResource = rootCalculator.calculateMaximumResource(this, label); long roundedMinResource = (long) Math.floor(minimumResource); long roundedMaxResource = (long) Math.floor(maximumResource); - parent.getQueueResourceQuotas().getEffectiveMinResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), roundedMinResource); - parent.getQueueResourceQuotas().getEffectiveMaxResource(label) - .setResourceValue(capacityVectorEntry.getResourceName(), roundedMaxResource); + parent.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + currentResourceName, roundedMinResource); + parent.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + currentResourceName, roundedMaxResource); } rootCalculator.updateCapacitiesAfterCalculation(this, label); } - rootCalculator.calculateResourcePrerequisites(this); } } 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index 84f18f7691ac8..5966d832becbe 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -36,9 +36,9 @@ public float calculateMaximumResource(ResourceCalculationDriver resourceCalculat @Override public void updateCapacitiesAfterCalculation( ResourceCalculationDriver resourceCalculationDriver, String label) { - resourceCalculationDriver.getCurrentChild().getQueueCapacities().setAbsoluteCapacity(label, 1); - if (resourceCalculationDriver.getCurrentChild().getQueueCapacities().getWeight(label) == 1) { - resourceCalculationDriver.getCurrentChild().getQueueCapacities().setNormalizedWeight(label, 1); + resourceCalculationDriver.getParent().getQueueCapacities().setAbsoluteCapacity(label, 1); + if (resourceCalculationDriver.getParent().getQueueCapacities().getWeight(label) == 1) { + resourceCalculationDriver.getParent().getQueueCapacities().setNormalizedWeight(label, 1); } } From 27e6344b91cef81ca1d5515c90aabb28956fbc4e Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Mon, 22 Nov 2021 08:32:40 +0100 Subject: [PATCH 14/27] YARN-10965. Fix warnings and nits --- .../AbstractQueueCapacityCalculator.java | 2 + ...CapacitySchedulerQueueCapacityHandler.java | 1 + .../scheduler/capacity/LeafQueue.java | 2 +- .../scheduler/capacity/ParentQueue.java | 2 +- .../capacity/QueueCapacityUpdateContext.java | 4 +- .../capacity/ResourceCalculationDriver.java | 87 ++++++++++++------- .../capacity/RootCalculationDriver.java | 19 +++- ...citySchedulerQueueCalculationTestBase.java | 18 ++++ 8 files changed, 102 insertions(+), 33 deletions(-) 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 856941ac77e85..b52f28f5b93d2 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -40,6 +40,8 @@ public abstract class AbstractQueueCapacityCalculator { /** * Sets the metrics and statistics after effective resource values calculation. * + * @param resourceCalculationDriver driver that contains the current resource unit and child to + * process * @param label node label */ public abstract void updateCapacitiesAfterCalculation( 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index f326c53c02915..3d1060bda7b2d 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -75,6 +75,7 @@ public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) * * @param clusterResource resource of the cluster * @param queue queue to update + * @return update context that contains information about the update phase */ public QueueCapacityUpdateContext update(Resource clusterResource, CSQueue queue) { ResourceLimits resourceLimits = new ResourceLimits(clusterResource); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index 1c6822bccfd84..7e0660ef06e1d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -167,7 +167,7 @@ public LeafQueue(CapacitySchedulerContext cs, resourceCalculator); // One time initialization is enough since it is static ordering policy - this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps(); + this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps<>(); LOG.debug("LeafQueue: name={}, fullname={}", queueName, 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/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 b1ded6d3588ea..2286071cd69b5 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 @@ -298,8 +298,8 @@ private enum QueueCapacityType { */ void setChildQueues(Collection childQueues) throws IOException { writeLock.lock(); - boolean isLegacyQueueMode = csContext.getConfiguration().isLegacyQueueMode(); try { + boolean isLegacyQueueMode = csContext.getConfiguration().isLegacyQueueMode(); if (isLegacyQueueMode) { QueueCapacityType childrenCapacityType = getCapacityConfigurationTypeForQueues(childQueues); 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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java index 67f682609ba02..6bd8d5cb361f6 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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java @@ -45,6 +45,8 @@ public QueueCapacityUpdateContext(Resource updatedClusterResource, /** * Returns the overall cluster resource available for the update phase. + * + * @param label node label * @return cluster resource */ public Resource getUpdatedClusterResource(String label) { @@ -52,7 +54,7 @@ public Resource getUpdatedClusterResource(String label) { } /** - * Returns the overall cluster resource available for the update phase. + * Returns the overall cluster resource available for the update phase of empty label. * @return cluster resource */ public Resource getUpdatedClusterResource() { 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index da91431f220a3..363c6b1a448f1 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -1,3 +1,21 @@ +/** + * 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.commons.lang3.tuple.ImmutablePair; @@ -33,17 +51,16 @@ public class ResourceCalculationDriver { protected final Map overallRemainingResource = new HashMap<>(); protected final Map batchRemainingResource = new HashMap<>(); + // Used by ABSOLUTE capacity types + protected final Map normalizedResourceRatio = new HashMap<>(); + // Used by WEIGHT capacity types + protected final Map> sumWeightsPerLabel = new HashMap<>(); protected String currentResourceName; protected AbstractQueueCapacityCalculator currentCalculator; protected CSQueue currentChild; protected Map usedResourceByCurrentCalculator = new HashMap<>(); - // Used by ABSOLUTE capacity types - protected final Map normalizedResourceRatio = new HashMap<>(); - // Used by WEIGHT capacity types - protected final Map> sumWeightsPerLabel = new HashMap<>(); - public ResourceCalculationDriver( CSQueue parent, QueueCapacityUpdateContext updateContext, Map calculators, @@ -56,6 +73,7 @@ public ResourceCalculationDriver( /** * Returns the parent that is driving the calculation. + * * @return a common parent queue */ public CSQueue getParent() { @@ -64,6 +82,7 @@ public CSQueue getParent() { /** * Returns the context that is used throughout the whole update phase. + * * @return update context */ public QueueCapacityUpdateContext getUpdateContext() { @@ -72,6 +91,7 @@ public QueueCapacityUpdateContext getUpdateContext() { /** * Returns the name of the resource that is currently processed. + * * @return resource name */ public String getCurrentResourceName() { @@ -80,6 +100,7 @@ public String getCurrentResourceName() { /** * Returns the child that is currently processed. + * * @return child queue */ public CSQueue getCurrentChild() { @@ -88,6 +109,7 @@ public CSQueue getCurrentChild() { /** * Sets the currently evaluated child to a specific queue. + * * @param currentChild a child queue */ public void setCurrentChild(CSQueue currentChild) { @@ -123,9 +145,10 @@ public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { /** * Increments the aggregated weight. - * @param label node label + * + * @param label node label * @param resourceName resource unit name - * @param value weight value + * @param value weight value */ public void incrementWeight(String label, String resourceName, float value) { sumWeightsPerLabel.putIfAbsent(label, new HashMap<>()); @@ -135,7 +158,8 @@ public void incrementWeight(String label, String resourceName, float value) { /** * Returns the aggregated children weights. - * @param label node label + * + * @param label node label * @param resourceName resource unit name * @return aggregated weights of children */ @@ -198,29 +222,25 @@ public void calculateResources() { * Updates the capacity values of the currently evaluated child. */ public void updateChildCapacities() { - for (String label : currentChild.getConfiguredNodeLabels()) { - QueueCapacityVector capacityVector = currentChild.getConfiguredCapacityVector(label); - if (capacityVector.isMixedCapacityVector()) { - // Post update capacities based on the calculated effective resource values - AbstractQueueCapacityCalculator.setQueueCapacities(updateContext.getUpdatedClusterResource( - label), currentChild, label); - } else { - // Update capacities according to the legacy logic - for (ResourceUnitCapacityType capacityType : - currentChild.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { - AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); - calculator.updateCapacitiesAfterCalculation(this, label); + currentChild.getWriteLock().lock(); + try { + for (String label : currentChild.getConfiguredNodeLabels()) { + QueueCapacityVector capacityVector = currentChild.getConfiguredCapacityVector(label); + if (capacityVector.isMixedCapacityVector()) { + // Post update capacities based on the calculated effective resource values + AbstractQueueCapacityCalculator.setQueueCapacities(updateContext.getUpdatedClusterResource( + label), currentChild, label); + } else { + // Update capacities according to the legacy logic + for (ResourceUnitCapacityType capacityType : + currentChild.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); + calculator.updateCapacitiesAfterCalculation(this, label); + } } } - } - } - - private void validateRemainingResource() { - for (String label : parent.getConfiguredNodeLabels()) { - if (!batchRemainingResource.get(label).equals(ResourceVector.newInstance())) { - updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( - parent.getQueuePath())); - } + } finally { + currentChild.getWriteLock().unlock(); } } @@ -331,4 +351,13 @@ private Pair validateCalculatedResources( return new ImmutablePair<>(minimumResource, maximumResource); } + + private void validateRemainingResource() { + for (String label : parent.getConfiguredNodeLabels()) { + if (!batchRemainingResource.get(label).equals(ResourceVector.newInstance())) { + updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( + parent.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/RootCalculationDriver.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/RootCalculationDriver.java index fd5ebca629f19..c49e9326e1bde 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/RootCalculationDriver.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/RootCalculationDriver.java @@ -1,8 +1,25 @@ +/** + * 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 java.util.Collection; import java.util.Collections; -import java.util.Map; /** * A special case that contains the resource calculation of the root queue. 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 3b561ef771bc8..73c1af5c7f021 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 @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.api.records.Resource; From e6609f2b93c2e2eef474cebd96cecf31dddb8ab8 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Mon, 22 Nov 2021 11:16:20 +0100 Subject: [PATCH 15/27] YARN-10965. Make Absolute calculator use remaining resource ratio as well --- .../AbsoluteResourceCapacityCalculator.java | 10 ++++++---- .../capacity/PercentageQueueCapacityCalculator.java | 5 ++--- .../capacity/ResourceCalculationDriver.java | 5 +++++ .../scheduler/capacity/ResourceVector.java | 1 - .../capacity/WeightQueueCapacityCalculator.java | 13 ++++++------- .../CapacitySchedulerQueueCalculationTestBase.java | 13 +++++-------- 6 files changed, 24 insertions(+), 23 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index 3e62cd9356671..d4135ca1b2403 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -26,11 +26,13 @@ public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCal public float calculateMinimumResource( ResourceCalculationDriver resourceCalculationDriver, String label) { String resourceName = resourceCalculationDriver.getCurrentResourceName(); - ResourceVector ratio = resourceCalculationDriver.getNormalizedResourceRatios().getOrDefault( - label, ResourceVector.of(1)); + float normalizedRatio = resourceCalculationDriver.getNormalizedResourceRatios().getOrDefault( + label, ResourceVector.of(1)).getValue(resourceName); + float remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( + label, resourceName); - return ratio.getValue(resourceName) * resourceCalculationDriver.getCurrentMinimumCapacityEntry(label) - .getResourceValue(); + return normalizedRatio * remainingResourceRatio * resourceCalculationDriver + .getCurrentMinimumCapacityEntry(label).getResourceValue(); } @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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index 77765a126fc14..a1a40f93510e5 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -30,9 +30,8 @@ public float calculateMinimumResource( float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label).getValue( resourceName); - float remainingPerEffectiveResourceRatio = resourceCalculationDriver.getBatchRemainingResource( - label).getValue(resourceName) / parentQueue.getEffectiveCapacity(label).getResourceValue( - resourceName); + float remainingPerEffectiveResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( + label, resourceName); float absoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio * resourceCalculationDriver.getCurrentMinimumCapacityEntry(label).getResourceValue() / 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index 363c6b1a448f1..0de40fcb09012 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -171,6 +171,11 @@ public Map getNormalizedResourceRatios() { return normalizedResourceRatio; } + public float getRemainingRatioOfResource(String label, String resourceName) { + return batchRemainingResource.get(label).getValue(resourceName) + / parent.getEffectiveCapacity(label).getResourceValue(resourceName); + } + /** * Returns the remaining resources of a parent that is still available for its * children. Decremented only after the calculator is finished its work on the corresponding 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/ResourceVector.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/ResourceVector.java index 6011d4d686de5..be9a89d6db189 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/ResourceVector.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/ResourceVector.java @@ -22,7 +22,6 @@ import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.Map; 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index c3aee324f0f68..02d211195d54e 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -58,21 +58,20 @@ public float calculateMinimumResource(ResourceCalculationDriver resourceCalculat return remainingResource; } - float remainingPerEffectiveResourceRatio = remainingResource / parentQueue.getEffectiveCapacity( - label).getResourceValue(resourceName); + float remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( + label, resourceName); float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label) .getValue(resourceName); - float queueAbsoluteCapacity = parentAbsoluteCapacity * - remainingPerEffectiveResourceRatio * normalizedWeight; + float queueAbsoluteCapacity = parentAbsoluteCapacity * remainingResourceRatio + * normalizedWeight; // Weight capacity types are the last to consider, therefore it is safe to assign all remaining // effective resources between queues. The strategy is to round values to the closest whole // number. - float resource = resourceCalculationDriver.getUpdateContext() - .getUpdatedClusterResource(label).getResourceValue(resourceName) * queueAbsoluteCapacity; - return Math.round(resource); + return resourceCalculationDriver.getUpdateContext() + .getUpdatedClusterResource(label).getResourceValue(resourceName) * queueAbsoluteCapacity; } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/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 73c1af5c7f021..6a768362900cc 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 @@ -39,6 +39,7 @@ import java.util.function.Function; import java.util.function.Supplier; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; public class CapacitySchedulerQueueCalculationTestBase { @@ -186,17 +187,13 @@ public void finishAssertion() { String errorMessage = String.format(RESOURCE_ASSERTION_ERROR_MESSAGE, assertion.assertionType, assertionEntry.getKey(), assertion.expectedResource.toString()); - Assert.assertEquals(errorMessage, - assertion.expectedResource, + Assert.assertEquals(errorMessage, assertion.expectedResource, assertion.resourceSupplier.get()); } else { String errorMessage = String.format(ASSERTION_ERROR_MESSAGE, - assertion.assertionType, assertionEntry.getKey(), - assertion.expectedValue); - Assert.assertEquals(errorMessage, - assertion.expectedValue, - assertion.valueSupplier.get(), - 1e-6); + assertion.assertionType, assertionEntry.getKey(), assertion.expectedValue); + Assert.assertEquals(errorMessage, assertion.expectedValue, + assertion.valueSupplier.get(), EPSILON); } } } From 784c1be80aa18fc52033da193888838d5a5dbf08 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 23 Nov 2021 08:18:01 +0100 Subject: [PATCH 16/27] YARN-10965. Create node label test --- .../scheduler/capacity/AbstractCSQueue.java | 3 +- .../CapacitySchedulerConfiguration.java | 2 +- .../capacity/ResourceCalculationDriver.java | 6 +- ...citySchedulerQueueCalculationTestBase.java | 44 +++++--- .../TestMixedQueueResourceCalculation.java | 104 ++++++++++++++++++ 5 files changed, 138 insertions(+), 21 deletions(-) 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 83f52dfd8bb49..e6915ac510696 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 @@ -536,7 +536,8 @@ private void validateMinResourceIsNotGreaterThanMaxResource(Resource minResource private void validateAbsoluteVsPercentageCapacityConfig( CapacityConfigType localType) { if (!getQueuePath().equals("root") - && !this.capacityConfigType.equals(localType)) { + && !this.capacityConfigType.equals(localType) && + csContext.getConfiguration().isLegacyQueueMode()) { throw new IllegalArgumentException("Queue '" + getQueuePath() + "' should use either percentage based capacity" + " configuration or absolute resource."); 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 11ae4c3034e9c..844613e2b1db3 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 @@ -839,7 +839,7 @@ private float internalGetLabeledQueueCapacity(String queue, String label, (configuredCapacity != null) && RESOURCE_PATTERN.matcher( configuredCapacity).find(); if (absoluteResourceConfigured || configuredWeightAsCapacity( - configuredCapacity)) { + configuredCapacity) || queueCapacityConfigParser.isCapacityVectorFormat(configuredCapacity)) { // Return capacity in percentage as 0 for non-root queues and 100 for // root.From AbstractCSQueue, absolute resource, and weight will be parsed // and updated separately. Once nodes are added/removed in cluster, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ResourceCalculationDriver.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/ResourceCalculationDriver.java index 0de40fcb09012..b05cb290214c1 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -53,7 +53,7 @@ public class ResourceCalculationDriver { protected final Map batchRemainingResource = new HashMap<>(); // Used by ABSOLUTE capacity types protected final Map normalizedResourceRatio = new HashMap<>(); - // Used by WEIGHT capacity types + // Used by WEIGHT capacity typet js protected final Map> sumWeightsPerLabel = new HashMap<>(); protected String currentResourceName; @@ -255,7 +255,7 @@ private void calculateResourceOnChild(ResourceUnitCapacityType capacityType) { for (String label : currentChild.getConfiguredNodeLabels()) { if (!currentChild.getConfiguredCapacityVector(label).isResourceOfType(currentResourceName, capacityType)) { - return; + continue; } float usedResourceByChild = setChildResources(label); float aggregatedUsedResource = usedResourceByCurrentCalculator.getOrDefault(label, @@ -361,7 +361,7 @@ private void validateRemainingResource() { for (String label : parent.getConfiguredNodeLabels()) { if (!batchRemainingResource.get(label).equals(ResourceVector.newInstance())) { updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( - parent.getQueuePath())); + parent.getQueuePath()).withInfo("Label: " + 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/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 6a768362900cc..64f2e823e3c0e 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 @@ -36,7 +36,7 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.Function; +import java.util.function.BiFunction; import java.util.function.Supplier; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; @@ -58,25 +58,25 @@ public class CapacitySchedulerQueueCalculationTestBase { protected static class QueueAssertionBuilder { public static final String EFFECTIVE_MAX_RES_INFO = "Effective Maximum Resource"; - public static final Function EFFECTIVE_MAX_RES = + public static final BiFunction EFFECTIVE_MAX_RES = QueueResourceQuotas::getEffectiveMaxResource; public static final String EFFECTIVE_MIN_RES_INFO = "Effective Minimum Resource"; - public static final Function EFFECTIVE_MIN_RES = + public static final BiFunction EFFECTIVE_MIN_RES = QueueResourceQuotas::getEffectiveMinResource; public static final String CAPACITY_INFO = "Capacity"; - public static final Function CAPACITY = + public static final BiFunction CAPACITY = QueueCapacities::getCapacity; public static final String ABS_CAPACITY_INFO = "Absolute Capacity"; - public static final Function ABS_CAPACITY = + public static final BiFunction ABS_CAPACITY = QueueCapacities::getAbsoluteCapacity; private static final String ASSERTION_ERROR_MESSAGE = - "'%s' of queue '%s' does not match %f"; + "'%s' of queue '%s' does not match %f for label %s"; private static final String RESOURCE_ASSERTION_ERROR_MESSAGE = - "'%s' of queue '%s' does not match %s"; + "'%s' of queue '%s' does not match %s for label %s"; private final CapacityScheduler cs; QueueAssertionBuilder(CapacityScheduler cs) { @@ -91,6 +91,7 @@ public class ValueAssertion { private String assertionType; private Supplier valueSupplier; private Supplier resourceSupplier; + private String label = ""; ValueAssertion(float expectedValue) { this.expectedValue = expectedValue; @@ -103,6 +104,10 @@ public class ValueAssertion { public QueueAssertion assertEffectiveMaxResource() { return withResourceSupplier(EFFECTIVE_MAX_RES, EFFECTIVE_MAX_RES_INFO); } + public QueueAssertion assertEffectiveMinResource(String label) { + this.label = label; + return assertEffectiveMinResource(); + } public QueueAssertion assertEffectiveMinResource() { return withResourceSupplier(EFFECTIVE_MIN_RES, EFFECTIVE_MIN_RES_INFO); @@ -117,27 +122,27 @@ public QueueAssertion assertAbsoluteCapacity() { } public QueueAssertion withResourceSupplier( - Function assertion, String messageInfo) { + BiFunction assertion, String messageInfo) { CSQueue queue = cs.getQueue(queuePath); if (queue == null) { Assert.fail("Queue " + queuePath + " is not found"); } assertionType = messageInfo; - resourceSupplier = () -> assertion.apply(queue.getQueueResourceQuotas()); + resourceSupplier = () -> assertion.apply(queue.getQueueResourceQuotas(), label); QueueAssertion.this.assertions.add(this); return QueueAssertion.this; } public QueueAssertion withCapacitySupplier( - Function assertion, String messageInfo) { + BiFunction assertion, String messageInfo) { CSQueue queue = cs.getQueue(queuePath); if (queue == null) { Assert.fail("Queue " + queuePath + " is not found"); } assertionType = messageInfo; - valueSupplier = () -> assertion.apply(queue.getQueueCapacities()); + valueSupplier = () -> assertion.apply(queue.getQueueCapacities(), label); QueueAssertion.this.assertions.add(this); return QueueAssertion.this; @@ -186,12 +191,13 @@ public void finishAssertion() { if (assertion.resourceSupplier != null) { String errorMessage = String.format(RESOURCE_ASSERTION_ERROR_MESSAGE, assertion.assertionType, assertionEntry.getKey(), - assertion.expectedResource.toString()); + assertion.expectedResource.toString(), assertion.label); Assert.assertEquals(errorMessage, assertion.expectedResource, assertion.resourceSupplier.get()); } else { String errorMessage = String.format(ASSERTION_ERROR_MESSAGE, - assertion.assertionType, assertionEntry.getKey(), assertion.expectedValue); + assertion.assertionType, assertionEntry.getKey(), assertion.expectedValue, + assertion.label); Assert.assertEquals(errorMessage, assertion.expectedValue, assertion.valueSupplier.get(), EPSILON); } @@ -239,17 +245,23 @@ protected RMNodeLabelsManager createNodeLabelManager() { mockRM.registerNode("h1:1234", 10 * GB); // label = x resourceCalculator = cs.getResourceCalculator(); } + protected QueueCapacityUpdateContext update( + QueueAssertionBuilder assertions, Resource clusterResource) + throws IOException { + return update(assertions, clusterResource, clusterResource); + } protected QueueCapacityUpdateContext update( - QueueAssertionBuilder assertions, Resource resource) throws IOException { + QueueAssertionBuilder assertions, Resource clusterResource, Resource emptyLabelResource) + throws IOException { cs.reinitialize(csConf, mockRM.getRMContext()); CapacitySchedulerQueueCapacityHandler queueController = new CapacitySchedulerQueueCapacityHandler(mgr); - mgr.setResourceForLabel(CommonNodeLabelsManager.NO_LABEL, resource); + mgr.setResourceForLabel(CommonNodeLabelsManager.NO_LABEL, emptyLabelResource); QueueCapacityUpdateContext updateContext = - queueController.update(resource, cs.getQueue("root")); + queueController.update(clusterResource, cs.getQueue("root")); assertions.finishAssertion(); 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 cd5d3babc38c9..20feda37f60e9 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 @@ -18,8 +18,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.junit.Assert; @@ -29,7 +33,9 @@ import java.util.Collection; import java.util.Optional; +import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { private static final long MEMORY = 16384; @@ -53,6 +59,9 @@ public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCal public static final Resource A1_WARNING_RESOURCE = Resource.newInstance(2048, 4); public static final Resource A2_WARNING_RESOURCE = Resource.newInstance(2048, 8); public static final Resource A12_WARNING_RESOURCE = Resource.newInstance(2048, 4); + public static final String X_LABEL = "x"; + public static final String Y_LABEL = "y"; + public static final String Z_LABEL = "z"; @Override public void setUp() throws Exception { @@ -265,6 +274,101 @@ public void testMaximumResourceWarnings() throws IOException { Assert.assertTrue(queueA11MinExceedsMaxWarning.isPresent()); } + @Test + public void testNodeLabels() throws Exception { + setLabeledQueueConfigs(); + + QueueAssertionBuilder assertionBuilder = createAssertionBuilder() + .withQueue(A) + .toExpect(createResource(2048, 8)) + .assertEffectiveMinResource(NO_LABEL) + .withQueue(A1) + .toExpect(createResource(1024, 5)) + .assertEffectiveMinResource(NO_LABEL) + .withQueue(A2) + .toExpect(createResource(1024, 2)) + .assertEffectiveMinResource(NO_LABEL) + .withQueue(B) + .toExpect(createResource(3072, 8)) + .assertEffectiveMinResource(NO_LABEL) + .withQueue(A) + .toExpect(createResource(30720, 30)) + .assertEffectiveMinResource(X_LABEL) + .withQueue(A1) + .toExpect(createResource(20480, 0)) + .assertEffectiveMinResource(X_LABEL) + .withQueue(A2) + .toExpect(createResource(10240, 30)) + .assertEffectiveMinResource(X_LABEL) + .withQueue(B) + .toExpect(createResource(30720, 30)) + .assertEffectiveMinResource(X_LABEL) + .withQueue(A) + .toExpect(createResource(8096, 42)) + .assertEffectiveMinResource(Y_LABEL) + .withQueue(A1) + .toExpect(createResource(6186, 21)) + .assertEffectiveMinResource(Y_LABEL) + .withQueue(A2) + .toExpect(createResource(1910, 21)) + .assertEffectiveMinResource(Y_LABEL) + .withQueue(B) + .toExpect(createResource(12384, 18)) + .assertEffectiveMinResource(Y_LABEL) + .withQueue(A) + .toExpect(createResource(7168, 11)) + .assertEffectiveMinResource(Z_LABEL) + .withQueue(A1) + .toExpect(createResource(6451, 4)) + .assertEffectiveMinResource(Z_LABEL) + .withQueue(A2) + .toExpect(createResource(716, 7)) + .assertEffectiveMinResource(Z_LABEL) + .withQueue(B) + .toExpect(createResource(3072, 4)) + .assertEffectiveMinResource(Z_LABEL) + .build(); + + update(assertionBuilder, UPDATE_RESOURCE, Resource.newInstance(5 * GB, 16)); + } + + private void setLabeledQueueConfigs() throws Exception { + mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of(X_LABEL, Y_LABEL, Z_LABEL)); + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), + TestUtils.toSet(X_LABEL), NodeId.newInstance("h2", 0), + TestUtils.toSet(Y_LABEL), NodeId.newInstance("h3", 0), + TestUtils.toSet(Y_LABEL), NodeId.newInstance("h4", 0), + TestUtils.toSet(Z_LABEL), NodeId.newInstance("h5", 0), + RMNodeLabelsManager.EMPTY_STRING_SET)); + + mockRM.registerNode("h1:1234", 60 * GB, 60); // label = x + mockRM.registerNode("h2:1234", 10 * GB, 25); // label = y + mockRM.registerNode("h3:1234", 10 * GB, 35); // label = y + mockRM.registerNode("h4:1234", 10 * GB, 15); // label = z + + csConf.setCapacityVector(A, "", createMemoryVcoresVector(absolute(2048), percentage(50))); + csConf.setCapacityVector(A1, "", createMemoryVcoresVector(absolute(1024), percentage(70))); + csConf.setCapacityVector(A2, "", createMemoryVcoresVector(absolute(1024), percentage(30))); + csConf.setCapacityVector(B, "", createMemoryVcoresVector(weight(3), percentage(50))); + + csConf.setCapacityVector(A, X_LABEL, createMemoryVcoresVector(percentage(50), weight(3))); + csConf.setCapacityVector(A1, X_LABEL, createMemoryVcoresVector(absolute(20480), percentage(10))); + csConf.setCapacityVector(A2, X_LABEL, createMemoryVcoresVector(absolute(10240), absolute(30))); + csConf.setCapacityVector(B, X_LABEL, createMemoryVcoresVector(percentage(50), percentage(50))); + + csConf.setCapacityVector(A, Y_LABEL, createMemoryVcoresVector(absolute(8096), weight(1))); + csConf.setCapacityVector(A1, Y_LABEL, createMemoryVcoresVector(absolute(6186), weight(3))); + csConf.setCapacityVector(A2, Y_LABEL, createMemoryVcoresVector(weight(3), weight(3))); + csConf.setCapacityVector(B, Y_LABEL, createMemoryVcoresVector(percentage(100), percentage(30))); + + csConf.setCapacityVector(A, Z_LABEL, createMemoryVcoresVector(percentage(70), absolute(11))); + csConf.setCapacityVector(A1, Z_LABEL, createMemoryVcoresVector(percentage(90), percentage(40))); + csConf.setCapacityVector(A2, Z_LABEL, createMemoryVcoresVector(percentage(10), weight(4))); + csConf.setCapacityVector(B, Z_LABEL, createMemoryVcoresVector(percentage(30), absolute(4))); + + cs.reinitialize(csConf, mockRM.getRMContext()); + } + private void setupQueueHierarchyWithoutRemainingResource() throws IOException { csConf.setState(B, QueueState.STOPPED); cs.reinitialize(csConf, mockRM.getRMContext()); From 07933227b000867097e7928d79558ac6f443cc27 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 30 Nov 2021 12:51:58 +0100 Subject: [PATCH 17/27] YARN-10965. Make rounding strategies more flexible with regards to capacity types --- .../DefaultQueueResourceRoundingStrategy.java | 17 +++++++- .../capacity/ResourceCalculationDriver.java | 10 ++--- ...citySchedulerQueueCalculationTestBase.java | 4 +- .../TestMixedQueueResourceCalculation.java | 43 +++++++++---------- 4 files changed, 43 insertions(+), 31 deletions(-) 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java index 855d927d69f00..aad6fa2238e92 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java @@ -19,6 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; + +import java.util.Collection; +import java.util.SortedSet; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.WEIGHT; @@ -27,9 +31,20 @@ * which is always the last type to consider, therefore it is safe to round up. */ public class DefaultQueueResourceRoundingStrategy implements QueueResourceRoundingStrategy { + private final ResourceUnitCapacityType lastCapacityType; + + public DefaultQueueResourceRoundingStrategy( + SortedSet capacityTypePrecedence) { + if (capacityTypePrecedence.isEmpty()) { + throw new IllegalArgumentException("Capacity type precedence collection is empty"); + } + + lastCapacityType = capacityTypePrecedence.last(); + } + @Override public float getRoundedResource(float resourceValue, QueueCapacityVectorEntry capacityVectorEntry) { - if (capacityVectorEntry.getVectorResourceType().equals(WEIGHT)) { + if (capacityVectorEntry.getVectorResourceType().equals(lastCapacityType)) { return Math.round(resourceValue); } else { return (float) Math.floor(resourceValue); 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index b05cb290214c1..2a88135fb47a9 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -20,14 +20,14 @@ import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSortedSet; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import java.util.Collection; import java.util.HashMap; import java.util.Map; -import java.util.Set; +import java.util.SortedSet; import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; @@ -36,14 +36,14 @@ * bookkeeper of disposable update information that is used by all children under a common parent. */ public class ResourceCalculationDriver { - protected static final Set CALCULATOR_PRECEDENCE = - ImmutableSet.of( + protected static final SortedSet CALCULATOR_PRECEDENCE = + ImmutableSortedSet.of( ResourceUnitCapacityType.ABSOLUTE, ResourceUnitCapacityType.PERCENTAGE, ResourceUnitCapacityType.WEIGHT); protected final QueueResourceRoundingStrategy roundingStrategy = - new DefaultQueueResourceRoundingStrategy(); + new DefaultQueueResourceRoundingStrategy(CALCULATOR_PRECEDENCE); protected final CSQueue parent; protected final QueueCapacityUpdateContext updateContext; protected final Map calculators; 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 64f2e823e3c0e..6d04716a81341 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 @@ -52,7 +52,7 @@ public class CapacitySchedulerQueueCalculationTestBase { protected static final String B1 = "root.b.b1"; protected static final String C = "root.c"; - protected static final String CAPACITY_VECTOR_TEMPLATE = "[memory=%s, vcores=%s]"; + private static final String CAPACITY_VECTOR_TEMPLATE = "[memory=%s, vcores=%s]"; protected ResourceCalculator resourceCalculator; @@ -272,7 +272,7 @@ protected QueueAssertionBuilder createAssertionBuilder() { return new QueueAssertionBuilder(cs); } - protected String createMemoryVcoresVector(Object memory, Object vcores) { + protected static String createMemoryVcoresVector(Object memory, Object vcores) { return String.format(CAPACITY_VECTOR_TEMPLATE, memory, vcores); } 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 20feda37f60e9..fffcd1da8c8fb 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 @@ -346,10 +346,10 @@ private void setLabeledQueueConfigs() throws Exception { mockRM.registerNode("h3:1234", 10 * GB, 35); // label = y mockRM.registerNode("h4:1234", 10 * GB, 15); // label = z - csConf.setCapacityVector(A, "", createMemoryVcoresVector(absolute(2048), percentage(50))); - csConf.setCapacityVector(A1, "", createMemoryVcoresVector(absolute(1024), percentage(70))); - csConf.setCapacityVector(A2, "", createMemoryVcoresVector(absolute(1024), percentage(30))); - csConf.setCapacityVector(B, "", createMemoryVcoresVector(weight(3), percentage(50))); + csConf.setCapacityVector(A, NO_LABEL, createMemoryVcoresVector(absolute(2048), percentage(50))); + csConf.setCapacityVector(A1, NO_LABEL, createMemoryVcoresVector(absolute(1024), percentage(70))); + csConf.setCapacityVector(A2, NO_LABEL, createMemoryVcoresVector(absolute(1024), percentage(30))); + csConf.setCapacityVector(B, NO_LABEL, createMemoryVcoresVector(weight(3), percentage(50))); csConf.setCapacityVector(A, X_LABEL, createMemoryVcoresVector(percentage(50), weight(3))); csConf.setCapacityVector(A1, X_LABEL, createMemoryVcoresVector(absolute(20480), percentage(10))); @@ -375,14 +375,14 @@ private void setupQueueHierarchyWithoutRemainingResource() throws IOException { setQueues(); csConf.setState(B, QueueState.RUNNING); - csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(30), weight(6))); - csConf.setCapacityVector(A1, "", createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); - csConf.setCapacityVector(A11, "", createMemoryVcoresVector(percentage(35), percentage(25))); - csConf.setCapacityVector(A12, "", createMemoryVcoresVector(percentage(65), percentage(75))); - csConf.setCapacityVector(A2, "", createMemoryVcoresVector(weight(3), percentage(100))); - csConf.setCapacityVector(B, "", createMemoryVcoresVector(absolute(8095), percentage(30))); - csConf.setCapacityVector(B1, "", createMemoryVcoresVector(weight(5), absolute(3))); - csConf.setCapacityVector(C, "", createMemoryVcoresVector(weight(3), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A, NO_LABEL, createMemoryVcoresVector(percentage(30), weight(6))); + csConf.setCapacityVector(A1, NO_LABEL, createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A11, NO_LABEL, createMemoryVcoresVector(percentage(35), percentage(25))); + csConf.setCapacityVector(A12, NO_LABEL, createMemoryVcoresVector(percentage(65), percentage(75))); + csConf.setCapacityVector(A2, NO_LABEL, createMemoryVcoresVector(weight(3), percentage(100))); + csConf.setCapacityVector(B, NO_LABEL, createMemoryVcoresVector(absolute(8095), percentage(30))); + csConf.setCapacityVector(B1, NO_LABEL, createMemoryVcoresVector(weight(5), absolute(3))); + csConf.setCapacityVector(C, NO_LABEL, createMemoryVcoresVector(weight(3), absolute(VCORES * 0.25))); cs.reinitialize(csConf, mockRM.getRMContext()); } @@ -392,19 +392,16 @@ private void setupQueueHierarchyWithWarnings() throws IOException { cs.reinitialize(csConf, mockRM.getRMContext()); setQueues(); - Resource.newInstance(0, 0); // C - Resource.newInstance(0, 0); // A12 - csConf.setState(B, QueueState.RUNNING); - csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(100), weight(6))); - csConf.setCapacityVector(A1, "", createMemoryVcoresVector(absolute(2048), + csConf.setCapacityVector(A, NO_LABEL, createMemoryVcoresVector(percentage(100), weight(6))); + csConf.setCapacityVector(A1, NO_LABEL, createMemoryVcoresVector(absolute(2048), absolute(VCORES * 0.25))); - csConf.setCapacityVector(A11, "", createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); - csConf.setCapacityVector(A12, "", createMemoryVcoresVector(percentage(100), percentage(100))); - csConf.setCapacityVector(A2, "", createMemoryVcoresVector(absolute(2048), percentage(100))); - csConf.setCapacityVector(B, "", createMemoryVcoresVector(absolute(8096), percentage(30))); - csConf.setCapacityVector(B1, "", createMemoryVcoresVector(absolute(10256), absolute(3))); - csConf.setCapacityVector(C, "", createMemoryVcoresVector(weight(3), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A11, NO_LABEL, createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A12, NO_LABEL, createMemoryVcoresVector(percentage(100), percentage(100))); + csConf.setCapacityVector(A2, NO_LABEL, createMemoryVcoresVector(absolute(2048), percentage(100))); + csConf.setCapacityVector(B, NO_LABEL, createMemoryVcoresVector(absolute(8096), percentage(30))); + csConf.setCapacityVector(B1, NO_LABEL, createMemoryVcoresVector(absolute(10256), absolute(3))); + csConf.setCapacityVector(C, NO_LABEL, createMemoryVcoresVector(weight(3), absolute(VCORES * 0.25))); cs.reinitialize(csConf, mockRM.getRMContext()); } From 75bca46bc7f94e1367a6c8c2db834ab7fbdeac61 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 9 Dec 2021 08:43:40 +0100 Subject: [PATCH 18/27] YARN-10965. Simplify test API --- ...citySchedulerQueueCalculationTestBase.java | 164 ---------------- .../capacity/QueueAssertionBuilder.java | 181 ++++++++++++++++++ .../TestMixedQueueResourceCalculation.java | 174 ++++++----------- .../TestUniformQueueResourceCalculation.java | 126 ++++-------- 4 files changed, 281 insertions(+), 364 deletions(-) create mode 100644 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 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 6d04716a81341..0fc5dfd92ba95 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 @@ -24,22 +24,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; -import org.junit.Assert; import org.junit.Before; import java.io.IOException; -import java.util.ArrayList; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.BiFunction; -import java.util.function.Supplier; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.GB; public class CapacitySchedulerQueueCalculationTestBase { @@ -56,160 +46,6 @@ public class CapacitySchedulerQueueCalculationTestBase { protected ResourceCalculator resourceCalculator; - protected static class QueueAssertionBuilder { - public static final String EFFECTIVE_MAX_RES_INFO = "Effective Maximum Resource"; - public static final BiFunction EFFECTIVE_MAX_RES = - QueueResourceQuotas::getEffectiveMaxResource; - - public static final String EFFECTIVE_MIN_RES_INFO = "Effective Minimum Resource"; - public static final BiFunction EFFECTIVE_MIN_RES = - QueueResourceQuotas::getEffectiveMinResource; - - public static final String CAPACITY_INFO = "Capacity"; - public static final BiFunction CAPACITY = - QueueCapacities::getCapacity; - - public static final String ABS_CAPACITY_INFO = "Absolute Capacity"; - public static final BiFunction ABS_CAPACITY = - QueueCapacities::getAbsoluteCapacity; - - private static final String ASSERTION_ERROR_MESSAGE = - "'%s' of queue '%s' does not match %f for label %s"; - private static final String RESOURCE_ASSERTION_ERROR_MESSAGE = - "'%s' of queue '%s' does not match %s for label %s"; - private final CapacityScheduler cs; - - QueueAssertionBuilder(CapacityScheduler cs) { - this.cs = cs; - } - - public class QueueAssertion { - - public class ValueAssertion { - private float expectedValue = 0; - private Resource expectedResource = null; - private String assertionType; - private Supplier valueSupplier; - private Supplier resourceSupplier; - private String label = ""; - - ValueAssertion(float expectedValue) { - this.expectedValue = expectedValue; - } - - ValueAssertion(Resource expectedResource) { - this.expectedResource = expectedResource; - } - - public QueueAssertion assertEffectiveMaxResource() { - return withResourceSupplier(EFFECTIVE_MAX_RES, EFFECTIVE_MAX_RES_INFO); - } - public QueueAssertion assertEffectiveMinResource(String label) { - this.label = label; - return assertEffectiveMinResource(); - } - - public QueueAssertion assertEffectiveMinResource() { - return withResourceSupplier(EFFECTIVE_MIN_RES, EFFECTIVE_MIN_RES_INFO); - } - - public QueueAssertion assertCapacity() { - return withCapacitySupplier(CAPACITY, CAPACITY_INFO); - } - - public QueueAssertion assertAbsoluteCapacity() { - return withCapacitySupplier(ABS_CAPACITY, ABS_CAPACITY_INFO); - } - - public QueueAssertion withResourceSupplier( - BiFunction assertion, String messageInfo) { - CSQueue queue = cs.getQueue(queuePath); - if (queue == null) { - Assert.fail("Queue " + queuePath + " is not found"); - } - - assertionType = messageInfo; - resourceSupplier = () -> assertion.apply(queue.getQueueResourceQuotas(), label); - QueueAssertion.this.assertions.add(this); - - return QueueAssertion.this; - } - - public QueueAssertion withCapacitySupplier( - BiFunction assertion, String messageInfo) { - CSQueue queue = cs.getQueue(queuePath); - if (queue == null) { - Assert.fail("Queue " + queuePath + " is not found"); - } - assertionType = messageInfo; - valueSupplier = () -> assertion.apply(queue.getQueueCapacities(), label); - QueueAssertion.this.assertions.add(this); - - return QueueAssertion.this; - } - } - - private final String queuePath; - private final List assertions = new ArrayList<>(); - - QueueAssertion(String queuePath) { - this.queuePath = queuePath; - } - - public ValueAssertion toExpect(float expected) { - return new ValueAssertion(expected); - } - - public ValueAssertion toExpect(Resource expected) { - return new ValueAssertion(expected); - } - - - public QueueAssertion withQueue(String queuePath) { - return QueueAssertionBuilder.this.withQueue(queuePath); - } - - public QueueAssertionBuilder build() { - return QueueAssertionBuilder.this.build(); - } - } - - private final Map assertions = new LinkedHashMap<>(); - - public QueueAssertionBuilder build() { - return this; - } - - public QueueAssertion withQueue(String queuePath) { - assertions.putIfAbsent(queuePath, new QueueAssertion(queuePath)); - return assertions.get(queuePath); - } - - public void finishAssertion() { - 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, - assertion.assertionType, assertionEntry.getKey(), - assertion.expectedResource.toString(), assertion.label); - Assert.assertEquals(errorMessage, assertion.expectedResource, - assertion.resourceSupplier.get()); - } else { - String errorMessage = String.format(ASSERTION_ERROR_MESSAGE, - assertion.assertionType, assertionEntry.getKey(), assertion.expectedValue, - assertion.label); - Assert.assertEquals(errorMessage, assertion.expectedValue, - assertion.valueSupplier.get(), EPSILON); - } - } - } - } - - public Set getQueues() { - return assertions.keySet(); - } - } - protected MockRM mockRM; protected CapacityScheduler cs; protected CapacitySchedulerConfiguration 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/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 new file mode 100644 index 0000000000000..57f3110cb0e43 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAssertionBuilder.java @@ -0,0 +1,181 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; +import org.junit.Assert; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.Supplier; + +import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON; + +/** + * Provides a fluent API to assert resource and capacity attributes of queues. + */ +class QueueAssertionBuilder { + private static final String EFFECTIVE_MAX_RES_INFO = "Effective Maximum Resource"; + private static final BiFunction EFFECTIVE_MAX_RES = + QueueResourceQuotas::getEffectiveMaxResource; + + private static final String EFFECTIVE_MIN_RES_INFO = "Effective Minimum Resource"; + private static final BiFunction EFFECTIVE_MIN_RES = + QueueResourceQuotas::getEffectiveMinResource; + + private static final String CAPACITY_INFO = "Capacity"; + private static final BiFunction CAPACITY = + QueueCapacities::getCapacity; + + private static final String ABS_CAPACITY_INFO = "Absolute Capacity"; + private static final BiFunction ABS_CAPACITY = + QueueCapacities::getAbsoluteCapacity; + + private static final String ASSERTION_ERROR_MESSAGE = + "'%s' of queue '%s' does not match %f for label %s"; + private static final String RESOURCE_ASSERTION_ERROR_MESSAGE = + "'%s' of queue '%s' does not match %s for label %s"; + private final CapacityScheduler cs; + + QueueAssertionBuilder(CapacityScheduler cs) { + this.cs = cs; + } + + public class QueueAssertion { + private final String queuePath; + private final List assertions = new ArrayList<>(); + + QueueAssertion(String queuePath) { + this.queuePath = queuePath; + } + + + public QueueAssertion withQueue(String queuePath) { + return QueueAssertionBuilder.this.withQueue(queuePath); + } + + public QueueAssertionBuilder build() { + return QueueAssertionBuilder.this.build(); + } + + public QueueAssertion assertEffectiveMaxResource(Resource expected) { + ValueAssertion valueAssertion = new ValueAssertion(expected); + valueAssertion.withResourceSupplier(EFFECTIVE_MAX_RES, EFFECTIVE_MAX_RES_INFO); + assertions.add(valueAssertion); + + return this; + } + + public QueueAssertion assertEffectiveMinResource(Resource expected, String label) { + ValueAssertion valueAssertion = new ValueAssertion(expected); + valueAssertion.withResourceSupplier(EFFECTIVE_MIN_RES, EFFECTIVE_MIN_RES_INFO); + assertions.add(valueAssertion); + valueAssertion.label = label; + + return this; + } + + public QueueAssertion assertEffectiveMinResource(Resource expected) { + return assertEffectiveMinResource(expected, NO_LABEL); + } + + public QueueAssertion assertCapacity(float expected) { + ValueAssertion valueAssertion = new ValueAssertion(expected); + valueAssertion.withCapacitySupplier(CAPACITY, CAPACITY_INFO); + assertions.add(valueAssertion); + + return this; + } + + public QueueAssertion assertAbsoluteCapacity(float expected) { + ValueAssertion valueAssertion = new ValueAssertion(expected); + valueAssertion.withCapacitySupplier(ABS_CAPACITY, ABS_CAPACITY_INFO); + assertions.add(valueAssertion); + + return this; + } + + public class ValueAssertion { + private float expectedValue = 0; + private Resource expectedResource = null; + private String assertionType; + private Supplier valueSupplier; + private Supplier resourceSupplier; + private String label = ""; + + ValueAssertion(float expectedValue) { + this.expectedValue = expectedValue; + } + + ValueAssertion(Resource expectedResource) { + this.expectedResource = expectedResource; + } + + public void setLabel(String label) { + this.label = label; + } + + public void withResourceSupplier( + BiFunction assertion, String messageInfo) { + CSQueue queue = cs.getQueue(queuePath); + if (queue == null) { + Assert.fail("Queue " + queuePath + " is not found"); + } + + assertionType = messageInfo; + resourceSupplier = () -> assertion.apply(queue.getQueueResourceQuotas(), label); + } + + public void withCapacitySupplier( + BiFunction assertion, String messageInfo) { + CSQueue queue = cs.getQueue(queuePath); + if (queue == null) { + Assert.fail("Queue " + queuePath + " is not found"); + } + assertionType = messageInfo; + valueSupplier = () -> assertion.apply(queue.getQueueCapacities(), label); + } + } + + } + + private final Map assertions = new LinkedHashMap<>(); + + public QueueAssertionBuilder build() { + return this; + } + + public QueueAssertion withQueue(String queuePath) { + assertions.putIfAbsent(queuePath, new QueueAssertion(queuePath)); + return assertions.get(queuePath); + } + + public void finishAssertion() { + 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, + assertion.assertionType, assertionEntry.getKey(), + assertion.expectedResource.toString(), assertion.label); + Assert.assertEquals(errorMessage, assertion.expectedResource, + assertion.resourceSupplier.get()); + } else { + String errorMessage = String.format(ASSERTION_ERROR_MESSAGE, + assertion.assertionType, assertionEntry.getKey(), assertion.expectedValue, + assertion.label); + Assert.assertEquals(errorMessage, assertion.expectedValue, + assertion.valueSupplier.get(), EPSILON); + } + } + } + } + + 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/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 fffcd1da8c8fb..0ae10afabc10f 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 @@ -75,45 +75,29 @@ public void testComplexHierarchyWithoutRemainingResource() throws IOException { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) - .toExpect(A_COMPLEX_NO_REMAINING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A1) - .toExpect(A1_COMPLEX_NO_REMAINING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A1_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A11) - .toExpect(A11_COMPLEX_NO_REMAINING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A11_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A11_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A11_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A12) - .toExpect(A12_COMPLEX_NO_REMAINING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A12_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A12_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A12_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A2) - .toExpect(A2_COMPLEX_NO_REMAINING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A2_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A2_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A2_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(B) - .toExpect(B_COMPLEX_NO_REMAINING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, B_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(B_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, B_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(B1) - .toExpect(B1_COMPLEX_NO_REMAINING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, B1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(B1_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, B1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(C) - .toExpect(C_COMPLEX_NO_REMAINING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, C_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(C_COMPLEX_NO_REMAINING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, C_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .build(); update(assertionBuilder, UPDATE_RESOURCE); @@ -124,45 +108,29 @@ public void testComplexHierarchyWithWarnings() throws IOException { setupQueueHierarchyWithWarnings(); QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) - .toExpect(A_WARNING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A_WARNING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A1) - .toExpect(A1_WARNING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A1_WARNING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A1_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A1_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A2) - .toExpect(A2_WARNING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A2_WARNING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A2_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A2_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A11) - .toExpect(ZERO_RESOURCE) - .assertEffectiveMinResource() - .toExpect(0) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ZERO_RESOURCE) + .assertAbsoluteCapacity(0) .withQueue(A12) - .toExpect(A12_WARNING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, A12_WARNING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(A12_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A12_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(B) - .toExpect(B_WARNING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, B_WARNING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(B_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, B_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(B1) - .toExpect(B1_WARNING_RESOURCE) - .assertEffectiveMinResource() - .toExpect(resourceCalculator.divide(UPDATE_RESOURCE, B1_WARNING_RESOURCE, UPDATE_RESOURCE)) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(B1_WARNING_RESOURCE) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, B1_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(C) - .toExpect(ZERO_RESOURCE) - .assertEffectiveMinResource() - .toExpect(0) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ZERO_RESOURCE) + .assertAbsoluteCapacity(0) .build(); QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); @@ -189,11 +157,9 @@ public void testZeroResourceIfNoMemory() throws IOException { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) - .toExpect(ZERO_RESOURCE) - .assertEffectiveMinResource() + .assertEffectiveMinResource(ZERO_RESOURCE) .withQueue(B) - .toExpect(createResource(MEMORY, VCORES * 0.5)) - .assertEffectiveMinResource() + .assertEffectiveMinResource(createResource(MEMORY, VCORES * 0.5)) .build(); QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); @@ -217,15 +183,11 @@ public void testDifferentMinimumAndMaximumCapacityTypes() throws IOException { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) - .toExpect(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) - .assertEffectiveMinResource() - .toExpect(Resource.newInstance(MEMORY, (int) (VCORES * 0.8))) - .assertEffectiveMaxResource() + .assertEffectiveMinResource(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) + .assertEffectiveMaxResource(Resource.newInstance(MEMORY, (int) (VCORES * 0.8))) .withQueue(B) - .toExpect(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) - .assertEffectiveMinResource() - .toExpect(Resource.newInstance(MEMORY, (int) (VCORES * 0.5))) - .assertEffectiveMaxResource() + .assertEffectiveMinResource(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) + .assertEffectiveMaxResource(Resource.newInstance(MEMORY, (int) (VCORES * 0.5))) .build(); QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); @@ -253,15 +215,11 @@ public void testMaximumResourceWarnings() throws IOException { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A11) - .toExpect(createResource(0.5 * 0.5 * MEMORY, 0.1 * VCORES)) - .assertEffectiveMinResource() - .toExpect(createResource(MEMORY * 0.5, 0.1 * VCORES)) - .assertEffectiveMaxResource() + .assertEffectiveMinResource(createResource(0.5 * 0.5 * MEMORY, 0.1 * VCORES)) + .assertEffectiveMaxResource(createResource(MEMORY * 0.5, 0.1 * VCORES)) .withQueue(A12) - .toExpect(createResource(0.5 * 0.5 * MEMORY, 0)) - .assertEffectiveMinResource() - .toExpect(createResource(MEMORY * 0.5, VCORES)) - .assertEffectiveMaxResource() + .assertEffectiveMinResource(createResource(0.5 * 0.5 * MEMORY, 0)) + .assertEffectiveMaxResource(createResource(MEMORY * 0.5, VCORES)) .build(); QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); @@ -280,53 +238,37 @@ public void testNodeLabels() throws Exception { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) - .toExpect(createResource(2048, 8)) - .assertEffectiveMinResource(NO_LABEL) + .assertEffectiveMinResource(createResource(2048, 8), NO_LABEL) .withQueue(A1) - .toExpect(createResource(1024, 5)) - .assertEffectiveMinResource(NO_LABEL) + .assertEffectiveMinResource(createResource(1024, 5), NO_LABEL) .withQueue(A2) - .toExpect(createResource(1024, 2)) - .assertEffectiveMinResource(NO_LABEL) + .assertEffectiveMinResource(createResource(1024, 2), NO_LABEL) .withQueue(B) - .toExpect(createResource(3072, 8)) - .assertEffectiveMinResource(NO_LABEL) + .assertEffectiveMinResource(createResource(3072, 8), NO_LABEL) .withQueue(A) - .toExpect(createResource(30720, 30)) - .assertEffectiveMinResource(X_LABEL) + .assertEffectiveMinResource(createResource(30720, 30), X_LABEL) .withQueue(A1) - .toExpect(createResource(20480, 0)) - .assertEffectiveMinResource(X_LABEL) + .assertEffectiveMinResource(createResource(20480, 0), X_LABEL) .withQueue(A2) - .toExpect(createResource(10240, 30)) - .assertEffectiveMinResource(X_LABEL) + .assertEffectiveMinResource(createResource(10240, 30), X_LABEL) .withQueue(B) - .toExpect(createResource(30720, 30)) - .assertEffectiveMinResource(X_LABEL) + .assertEffectiveMinResource(createResource(30720, 30), X_LABEL) .withQueue(A) - .toExpect(createResource(8096, 42)) - .assertEffectiveMinResource(Y_LABEL) + .assertEffectiveMinResource(createResource(8096, 42), Y_LABEL) .withQueue(A1) - .toExpect(createResource(6186, 21)) - .assertEffectiveMinResource(Y_LABEL) + .assertEffectiveMinResource(createResource(6186, 21), Y_LABEL) .withQueue(A2) - .toExpect(createResource(1910, 21)) - .assertEffectiveMinResource(Y_LABEL) + .assertEffectiveMinResource(createResource(1910, 21), Y_LABEL) .withQueue(B) - .toExpect(createResource(12384, 18)) - .assertEffectiveMinResource(Y_LABEL) + .assertEffectiveMinResource(createResource(12384, 18), Y_LABEL) .withQueue(A) - .toExpect(createResource(7168, 11)) - .assertEffectiveMinResource(Z_LABEL) + .assertEffectiveMinResource(createResource(7168, 11), Z_LABEL) .withQueue(A1) - .toExpect(createResource(6451, 4)) - .assertEffectiveMinResource(Z_LABEL) + .assertEffectiveMinResource(createResource(6451, 4), Z_LABEL) .withQueue(A2) - .toExpect(createResource(716, 7)) - .assertEffectiveMinResource(Z_LABEL) + .assertEffectiveMinResource(createResource(716, 7), Z_LABEL) .withQueue(B) - .toExpect(createResource(3072, 4)) - .assertEffectiveMinResource(Z_LABEL) + .assertEffectiveMinResource(createResource(3072, 4), Z_LABEL) .build(); update(assertionBuilder, UPDATE_RESOURCE, Resource.newInstance(5 * GB, 16)); 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 de36b090be617..bbab4a8041eed 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 @@ -75,35 +75,23 @@ public void testWeightResourceCalculation() throws IOException { QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() .withQueue(A) - .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT)) - .assertEffectiveMinResource() - .toExpect(A_NORMALIZED_WEIGHT) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT) .withQueue(B) - .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, B_NORMALIZED_WEIGHT)) - .assertEffectiveMinResource() - .toExpect(B_NORMALIZED_WEIGHT) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, B_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(B_NORMALIZED_WEIGHT) .withQueue(A1) - .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT)) - .assertEffectiveMinResource() - .toExpect(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT) .withQueue(A2) - .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT)) - .assertEffectiveMinResource() - .toExpect(A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT) .withQueue(A11) - .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT)) - .assertEffectiveMinResource() - .toExpect(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT) .withQueue(A12) - .toExpect(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT)) - .assertEffectiveMinResource() - .toExpect(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT)) + .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT) .build(); update(queueAssertionBuilder, UPDATE_RES); @@ -120,47 +108,29 @@ public void testPercentageResourceCalculation() throws IOException { QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() .withQueue(A) - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY)) - .assertEffectiveMinResource() - .toExpect(A_CAPACITY) - .assertCapacity() - .toExpect(A_CAPACITY) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY)) + .assertCapacity(A_CAPACITY) + .assertAbsoluteCapacity(A_CAPACITY) .withQueue(B) - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, B_CAPACITY)) - .assertEffectiveMinResource() - .toExpect(B_CAPACITY) - .assertCapacity() - .toExpect(B_CAPACITY) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, B_CAPACITY)) + .assertCapacity(B_CAPACITY) + .assertAbsoluteCapacity(B_CAPACITY) .withQueue(A1) - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY * A1_CAPACITY)) - .assertEffectiveMinResource() - .toExpect(A1_CAPACITY) - .assertCapacity() - .toExpect(A_CAPACITY * A1_CAPACITY) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY * A1_CAPACITY)) + .assertCapacity(A1_CAPACITY) + .assertAbsoluteCapacity(A_CAPACITY * A1_CAPACITY) .withQueue(A2) - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY * A2_CAPACITY)) - .assertEffectiveMinResource() - .toExpect(A2_CAPACITY) - .assertCapacity() - .toExpect(A_CAPACITY * A2_CAPACITY) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY * A2_CAPACITY)) + .assertCapacity(A2_CAPACITY) + .assertAbsoluteCapacity(A_CAPACITY * A2_CAPACITY) .withQueue(A11) - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A11_CAPACITY * A_CAPACITY * A1_CAPACITY)) - .assertEffectiveMinResource() - .toExpect(A11_CAPACITY) - .assertCapacity() - .toExpect(A11_CAPACITY * A_CAPACITY * A1_CAPACITY) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A11_CAPACITY * A_CAPACITY * A1_CAPACITY)) + .assertCapacity(A11_CAPACITY) + .assertAbsoluteCapacity(A11_CAPACITY * A_CAPACITY * A1_CAPACITY) .withQueue(A12) - .toExpect(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A12_CAPACITY * A_CAPACITY * A1_CAPACITY)) - .assertEffectiveMinResource() - .toExpect(A12_CAPACITY) - .assertCapacity() - .toExpect(A12_CAPACITY * A_CAPACITY * A1_CAPACITY) - .assertAbsoluteCapacity() + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A12_CAPACITY * A_CAPACITY * A1_CAPACITY)) + .assertCapacity(A12_CAPACITY) + .assertAbsoluteCapacity(A12_CAPACITY * A_CAPACITY * A1_CAPACITY) .build(); update(queueAssertionBuilder, PERCENTAGE_ALL_RES); @@ -177,46 +147,34 @@ public void testAbsoluteResourceCalculation() throws IOException { QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() .withQueue(A) - .toExpect(QUEUE_A_RES) - .assertEffectiveMinResource() + .assertEffectiveMinResource(QUEUE_A_RES) .withQueue(B) - .toExpect(QUEUE_B_RES) - .assertEffectiveMinResource() + .assertEffectiveMinResource(QUEUE_B_RES) .withQueue(A1) - .toExpect(QUEUE_A1_RES) - .assertEffectiveMinResource() + .assertEffectiveMinResource(QUEUE_A1_RES) .withQueue(A2) - .toExpect(QUEUE_A2_RES) - .assertEffectiveMinResource() + .assertEffectiveMinResource(QUEUE_A2_RES) .withQueue(A11) - .toExpect(QUEUE_A11_RES) - .assertEffectiveMinResource() + .assertEffectiveMinResource(QUEUE_A11_RES) .withQueue(A12) - .toExpect(QUEUE_A12_RES) - .assertEffectiveMinResource() + .assertEffectiveMinResource(QUEUE_A12_RES) .build(); update(queueAssertionBuilder, UPDATE_RES); QueueAssertionBuilder queueAssertionHalfClusterResource = createAssertionBuilder() .withQueue(A) - .toExpect(ResourceUtils.multiply(QUEUE_A_RES, 0.5f)) - .assertEffectiveMinResource() + .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A_RES, 0.5f)) .withQueue(B) - .toExpect(ResourceUtils.multiply(QUEUE_B_RES, 0.5f)) - .assertEffectiveMinResource() + .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_B_RES, 0.5f)) .withQueue(A1) - .toExpect(ResourceUtils.multiply(QUEUE_A1_RES, 0.5f)) - .assertEffectiveMinResource() + .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A1_RES, 0.5f)) .withQueue(A2) - .toExpect(ResourceUtils.multiply(QUEUE_A2_RES, 0.5f)) - .assertEffectiveMinResource() + .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A2_RES, 0.5f)) .withQueue(A11) - .toExpect(ResourceUtils.multiply(QUEUE_A11_RES, 0.5f)) - .assertEffectiveMinResource() + .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A11_RES, 0.5f)) .withQueue(A12) - .toExpect(ResourceUtils.multiply(QUEUE_A12_RES, 0.5f)) - .assertEffectiveMinResource() + .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A12_RES, 0.5f)) .build(); update(queueAssertionHalfClusterResource, ResourceUtils.multiply(UPDATE_RES, 0.5f)); From 513555ccc245aa52633808fafc7faae291bbc891 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 9 Dec 2021 12:59:18 +0100 Subject: [PATCH 19/27] YARN-10965. Fix review feedbacks --- .../yarn/util/resource/ResourceUtils.java | 12 +- .../scheduler/capacity/AbstractCSQueue.java | 10 +- .../scheduler/capacity/CSQueue.java | 2 +- .../CapacitySchedulerConfiguration.java | 2 +- .../DefaultQueueResourceRoundingStrategy.java | 12 +- .../capacity/ResourceCalculationDriver.java | 16 +- .../scheduler/capacity/ResourceVector.java | 8 +- ...citySchedulerQueueCalculationTestBase.java | 2 +- .../TestMixedQueueResourceCalculation.java | 316 ++++++++++++------ .../capacity/TestResourceVector.java | 6 +- 10 files changed, 251 insertions(+), 135 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java index 454bdf6438768..f1cf7f73aa5b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java @@ -819,23 +819,23 @@ public static Resource createResourceWithSameValue(long value) { return res; } - public static Resource multiply(Resource lhs, float rhs) { + public static Resource multiply(Resource resource, float multiplier) { Resource newResource = Resource.newInstance(0, 0); - for (ResourceInformation resourceInformation : lhs.getResources()) { + for (ResourceInformation resourceInformation : resource.getResources()) { newResource.setResourceValue(resourceInformation.getName(), - (long) Math.floor(resourceInformation.getValue() * rhs)); + (long) Math.floor(resourceInformation.getValue() * multiplier)); } return newResource; } - public static Resource multiplyRound(Resource lhs, float rhs) { + public static Resource multiplyRound(Resource resource, float multiplier) { Resource newResource = Resource.newInstance(0, 0); - for (ResourceInformation resourceInformation : lhs.getResources()) { + for (ResourceInformation resourceInformation : resource.getResources()) { newResource.setResourceValue(resourceInformation.getName(), - Math.round(resourceInformation.getValue() * rhs)); + Math.round(resourceInformation.getValue() * multiplier)); } return newResource; 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 e6915ac510696..0d2a8aff28eaf 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 @@ -115,7 +115,7 @@ public enum CapacityConfigType { CapacityConfigType.NONE; protected Map configuredCapacityVectors; - protected Map configuredMaximumCapacityVectors; + protected Map configuredMaxCapacityVectors; private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); @@ -385,7 +385,7 @@ protected void setupQueueConfigs(Resource clusterResource, this.configuredCapacityVectors = configuration .parseConfiguredResourceVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels()); - this.configuredMaximumCapacityVectors = configuration + this.configuredMaxCapacityVectors = configuration .parseConfiguredMaximumCapacityVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels(), QueueCapacityVector.newInstance()); // Update metrics CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, @@ -581,8 +581,8 @@ public QueueCapacityVector getConfiguredCapacityVector(String label) { } @Override - public QueueCapacityVector getConfiguredMaximumCapacityVector(String label) { - return configuredMaximumCapacityVectors.get(label); + public QueueCapacityVector getConfiguredMaxCapacityVector(String label) { + return configuredMaxCapacityVectors.get(label); } @Override @@ -592,7 +592,7 @@ public void setConfiguredMinCapacityVector(String label, QueueCapacityVector min @Override public void setConfiguredMaxCapacityVector(String label, QueueCapacityVector maxCapacityVector) { - configuredMaximumCapacityVectors.put(label, maxCapacityVector); + configuredMaxCapacityVectors.put(label, maxCapacityVector); } private void initializeQueueState(CapacitySchedulerConfiguration 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/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index 4ff4637552246..323fa1ba6a473 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -446,7 +446,7 @@ public void validateSubmitApplication(ApplicationId applicationId, * @param label node label (partition) * @return capacity resource vector */ - QueueCapacityVector getConfiguredMaximumCapacityVector(String label); + QueueCapacityVector getConfiguredMaxCapacityVector(String label); /** * Sets the configured minimum capacity vector to a specific value 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 844613e2b1db3..c1fd135856c90 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 @@ -2812,7 +2812,7 @@ public boolean isLegacyQueueMode() { } public void setLegacyQueueModeEnabled(boolean value) { - setBoolean(LEGACY_QUEUE_MODE_ENABLED, false); + setBoolean(LEGACY_QUEUE_MODE_ENABLED, value); } public boolean getMultiNodePlacementEnabled() { 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java index aad6fa2238e92..12bc381052948 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java @@ -21,7 +21,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import java.util.ArrayList; import java.util.Collection; +import java.util.LinkedHashSet; +import java.util.Set; import java.util.SortedSet; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.WEIGHT; @@ -34,12 +37,9 @@ public class DefaultQueueResourceRoundingStrategy implements QueueResourceRoundi private final ResourceUnitCapacityType lastCapacityType; public DefaultQueueResourceRoundingStrategy( - SortedSet capacityTypePrecedence) { - if (capacityTypePrecedence.isEmpty()) { - throw new IllegalArgumentException("Capacity type precedence collection is empty"); - } - - lastCapacityType = capacityTypePrecedence.last(); + Set capacityTypePrecedence) { + lastCapacityType = capacityTypePrecedence.stream().reduce((c1, c2) -> c2).orElseThrow(() + -> new IllegalArgumentException("Capacity type precedence collection is empty")); } @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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index 2a88135fb47a9..3df373d7a52a6 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -20,14 +20,14 @@ import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSortedSet; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import java.util.Collection; import java.util.HashMap; import java.util.Map; -import java.util.SortedSet; +import java.util.Set; import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; @@ -36,8 +36,8 @@ * bookkeeper of disposable update information that is used by all children under a common parent. */ public class ResourceCalculationDriver { - protected static final SortedSet CALCULATOR_PRECEDENCE = - ImmutableSortedSet.of( + protected static final Set CALCULATOR_PRECEDENCE = + ImmutableSet.of( ResourceUnitCapacityType.ABSOLUTE, ResourceUnitCapacityType.PERCENTAGE, ResourceUnitCapacityType.WEIGHT); @@ -140,7 +140,7 @@ public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { * @return capacity vector entry */ public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { - return currentChild.getConfiguredMaximumCapacityVector(label).getResource(currentResourceName); + return currentChild.getConfiguredMaxCapacityVector(label).getResource(currentResourceName); } /** @@ -214,7 +214,7 @@ public void calculateResources() { } // Flush aggregated used resource by labels at the end of a calculator phase for (Map.Entry entry : usedResourceByCurrentCalculator.entrySet()) { - batchRemainingResource.get(entry.getKey()).subtract(resourceName, entry.getValue()); + batchRemainingResource.get(entry.getKey()).decrement(resourceName, entry.getValue()); } usedResourceByCurrentCalculator = new HashMap<>(); } @@ -262,7 +262,7 @@ private void calculateResourceOnChild(ResourceUnitCapacityType capacityType) { 0f); float resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; - overallRemainingResource.get(label).subtract(currentResourceName, usedResourceByChild); + overallRemainingResource.get(label).decrement(currentResourceName, usedResourceByChild); usedResourceByCurrentCalculator.put(label, resourceUsedByLabel); } } finally { @@ -276,7 +276,7 @@ private float setChildResources(String label) { long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( currentResourceName); QueueCapacityVectorEntry maximumCapacityVectorEntry = currentChild - .getConfiguredMaximumCapacityVector(label).getResource(currentResourceName); + .getConfiguredMaxCapacityVector(label).getResource(currentResourceName); AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( maximumCapacityVectorEntry.getVectorResourceType()); 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/ResourceVector.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/ResourceVector.java index be9a89d6db189..df1eac8aa3c4f 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/ResourceVector.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/ResourceVector.java @@ -80,21 +80,21 @@ public static ResourceVector of(Resource resource) { } /** - * Subtract values for each resource defined in the given resource vector. + * Decrements values for each resource defined in the given resource vector. * @param otherResourceVector rhs resource vector of the subtraction */ - public void subtract(ResourceVector otherResourceVector) { + public void decrement(ResourceVector otherResourceVector) { for (Map.Entry resource : otherResourceVector) { setValue(resource.getKey(), getValue(resource.getKey()) - resource.getValue()); } } /** - * Subtracts the given resource by the specified value. + * Decrements the given resource by the specified value. * @param resourceName name of the resource * @param value value to be subtracted from the resource's current value */ - public void subtract(String resourceName, float value) { + public void decrement(String resourceName, float value) { setValue(resourceName, getValue(resourceName) - 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/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 0fc5dfd92ba95..e760b0586539e 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 @@ -108,7 +108,7 @@ protected QueueAssertionBuilder createAssertionBuilder() { return new QueueAssertionBuilder(cs); } - protected static String createMemoryVcoresVector(Object memory, Object vcores) { + protected static String createCapacityVector(Object memory, Object vcores) { return String.format(CAPACITY_VECTOR_TEMPLATE, memory, vcores); } 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 0ae10afabc10f..ef356fa74b14b 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 @@ -40,28 +40,123 @@ public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { private static final long MEMORY = 16384; private static final long VCORES = 16; + private static final String C_VECTOR_WITH_WARNING = createCapacityVector(weight(3), absolute(VCORES * 0.25)); + private static final String A11_VECTOR_WITH_WARNING = createCapacityVector(weight(1), absolute(VCORES * 0.25)); + private static final String A1_VECTOR_WITH_WARNING = createCapacityVector(absolute(2048), + absolute(VCORES * 0.25)); + private static final String C_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(3), absolute(VCORES * 0.25)); + private static final String A1_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(1), absolute(VCORES * 0.25)); + + private static final Resource A12_EXPECTED_MAX_RESOURCE_MAX_WARNINGS = createResource(MEMORY * 0.5, VCORES); + private static final Resource A11_EXPECTED_MAX_RESOURCE_MAX_WARNINGS = createResource(MEMORY * 0.5, 0.1 * VCORES); + private static final Resource A11_EXPECTED_MIN_RESOURCE_MAX_WARNINGS = createResource(0.5 * 0.5 * MEMORY, 0.1 * VCORES); + private static final Resource A12_EXPECTED_MIN_RESOURCE_MAX_WARNINGS = createResource(0.5 * 0.5 * MEMORY, 0); + private static final String A11_MAX_VECTOR_MAX_WARNINGS = createCapacityVector(absolute(MEMORY), + percentage(10)); + private static final String A1_MAX_VECTOR_MAX_WARNINGS = createCapacityVector(absolute(MEMORY * 0.5), + percentage(100)); private static final Resource UPDATE_RESOURCE = Resource.newInstance(16384, 16); private static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0); - public static final Resource A_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(2486, 9); - public static final Resource A1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(621, 4); - public static final Resource A11_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(217, 1); - public static final Resource A12_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(403, 3); - public static final Resource A2_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(1865, 5); - public static final Resource B_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); - public static final Resource B1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); - public static final Resource C_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(5803, 4); - - public static final Resource B_WARNING_RESOURCE = Resource.newInstance(8096, 4); - public static final Resource B1_WARNING_RESOURCE = Resource.newInstance(8096, 3); - public static final Resource A_WARNING_RESOURCE = Resource.newInstance(8288, 12); - public static final Resource A1_WARNING_RESOURCE = Resource.newInstance(2048, 4); - public static final Resource A2_WARNING_RESOURCE = Resource.newInstance(2048, 8); - public static final Resource A12_WARNING_RESOURCE = Resource.newInstance(2048, 4); - public static final String X_LABEL = "x"; - public static final String Y_LABEL = "y"; - public static final String Z_LABEL = "z"; + private static final Resource A_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(2486, 9); + private static final Resource A1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(621, 4); + private static final Resource A11_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(217, 1); + private static final Resource A12_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(403, 3); + private static final Resource A2_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(1865, 5); + private static final Resource B_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); + private static final Resource B1_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(8095, 3); + private static final Resource C_COMPLEX_NO_REMAINING_RESOURCE = Resource.newInstance(5803, 4); + + private static final Resource B_WARNING_RESOURCE = Resource.newInstance(8096, 4); + private static final Resource B1_WARNING_RESOURCE = Resource.newInstance(8096, 3); + private static final Resource A_WARNING_RESOURCE = Resource.newInstance(8288, 12); + private static final Resource A1_WARNING_RESOURCE = Resource.newInstance(2048, 4); + private static final Resource A2_WARNING_RESOURCE = Resource.newInstance(2048, 8); + private static final Resource A12_WARNING_RESOURCE = Resource.newInstance(2048, 4); + + private static final String A_VECTOR_ZERO_RESOURCE = createCapacityVector(percentage(100), weight(6)); + private static final String B_VECTOR_ZERO_RESOURCE = createCapacityVector(absolute(MEMORY), + absolute(VCORES * 0.5)); + + private static final String A_MAX_VECTOR_DIFFERENT_MIN_MAX = createCapacityVector(absolute(MEMORY), + percentage(80)); + private static final Resource B_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX = Resource.newInstance(MEMORY, (int) (VCORES * 0.5)); + private static final Resource A_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX = Resource.newInstance(MEMORY, (int) (VCORES * 0.8)); + private static final String B_MAX_VECTOR_DIFFERENT_MIN_MAX = createCapacityVector(absolute(MEMORY), + absolute(VCORES * 0.5)); + private static final String A_MIN_VECTOR_DIFFERENT_MIN_MAX = createCapacityVector(percentage(50), + absolute(VCORES * 0.5)); + private static final String B_MIN_VECTOR_DIFFERENT_MIN_MAX = createCapacityVector(weight(6), percentage(100)); + private static final String B_INVALID_MAX_VECTOR = createCapacityVector(absolute(MEMORY), weight(10)); + + private static final String X_LABEL = "x"; + private static final String Y_LABEL = "y"; + private static final String Z_LABEL = "z"; + + private static final String H1_NODE = "h1"; + private static final String H2_NODE = "h2"; + private static final String H3_NODE = "h3"; + private static final String H4_NODE = "h4"; + private static final String H5_NODE = "h5"; + private static final int H1_MEMORY = 60 * GB; + private static final int H1_VCORES = 60; + private static final int H2_MEMORY = 10 * GB; + private static final int H2_VCORES = 25; + private static final int H3_VCORES = 35; + private static final int H3_MEMORY = 10 * GB; + private static final int H4_MEMORY = 10 * GB; + private static final int H4_VCORES = 15; + + private static final String A11_MIN_VECTOR_MAX_WARNINGS = createCapacityVector(percentage(50), percentage(100)); + private static final String A12_MIN_VECTOR_MAX_WARNINGS = createCapacityVector(percentage(50), percentage(0)); + + private static final Resource A_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(2048, 8); + private static final Resource A1_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(1024, 5); + private static final Resource A2_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(1024, 2); + private static final Resource B_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(3072, 8); + private static final Resource A_EXPECTED_MIN_RESOURCE_X_LABEL = createResource(30720, 30); + private static final Resource A1_EXPECTED_MIN_RESOURCE_X_LABEL = createResource(20480, 0); + private static final Resource A2_EXPECTED_MIN_RESOURCE_X_LABEL = createResource(10240, 30); + private static final Resource B_EXPECTED_MIN_RESOURCE_X_LABEL = createResource(30720, 30); + private static final Resource A_EXPECTED_MIN_RESOURCE_Y_LABEL = createResource(8096, 42); + private static final Resource A1_EXPECTED_MIN_RESOURCE_Y_LABEL = createResource(6186, 21); + private static final Resource A2_EXPECTED_MIN_RESOURCE_Y_LABEL = createResource(1910, 21); + private static final Resource B_EXPECTED_MIN_RESOURCE_Y_LABEL = createResource(12384, 18); + private static final Resource A_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(7168, 11); + private static final Resource A1_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(6451, 4); + private static final Resource A2_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(716, 7); + private static final Resource B_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(3072, 4); + private static final Resource EMPTY_LABEL_RESOURCE = Resource.newInstance(5 * GB, 16); + + private static final String A_VECTOR_NO_LABEL = createCapacityVector(absolute(2048), percentage(50)); + private static final String A1_VECTOR_NO_LABEL = createCapacityVector(absolute(1024), percentage(70)); + private static final String A2_VECTOR_NO_LABEL = createCapacityVector(absolute(1024), percentage(30)); + private static final String B_VECTOR_NO_LABEL = createCapacityVector(weight(3), percentage(50)); + private static final String A_VECTOR_X_LABEL = createCapacityVector(percentage(50), weight(3)); + private static final String A1_VECTOR_X_LABEL = createCapacityVector(absolute(20480), percentage(10)); + private static final String A2_VECTOR_X_LABEL = createCapacityVector(absolute(10240), absolute(30)); + private static final String B_VECTOR_X_LABEL = createCapacityVector(percentage(50), percentage(50)); + private static final String A_VECTOR_Y_LABEL = createCapacityVector(absolute(8096), weight(1)); + private static final String A1_VECTOR_Y_LABEL = createCapacityVector(absolute(6186), weight(3)); + private static final String A2_VECTOR_Y_LABEL = createCapacityVector(weight(3), weight(3)); + private static final String B_VECTOR_Y_LABEL = createCapacityVector(percentage(100), percentage(30)); + private static final String A_VECTOR_Z_LABEL = createCapacityVector(percentage(70), absolute(11)); + private static final String A1_VECTOR_Z_LABEL = createCapacityVector(percentage(90), percentage(40)); + private static final String A2_VECTOR_Z_LABEL = createCapacityVector(percentage(10), weight(4)); + private static final String B_VECTOR_Z_LABEL = createCapacityVector(percentage(30), absolute(4)); + + private static final String A_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(percentage(30), weight(6)); + private static final String A11_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(percentage(35), percentage(25)); + private static final String A12_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(percentage(65), percentage(75)); + private static final String A2_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(3), percentage(100)); + private static final String B_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(absolute(8095), percentage(30)); + private static final String B1_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(5), absolute(3)); + private static final String A_VECTOR_WITH_WARNINGS = createCapacityVector(percentage(100), weight(6)); + private static final String A12_VECTOR_WITH_WARNING = createCapacityVector(percentage(100), percentage(100)); + private static final String A2_VECTOR_WITH_WARNING = createCapacityVector(absolute(2048), percentage(100)); + private static final String B_VECTOR_WITH_WARNING = createCapacityVector(absolute(8096), percentage(30)); + private static final String B1_VECTOR_WITH_WARNING = createCapacityVector(absolute(10256), absolute(3)); @Override public void setUp() throws Exception { @@ -69,6 +164,20 @@ public void setUp() throws Exception { csConf.setLegacyQueueModeEnabled(false); } + /** + * Tests a complex scenario in which no warning or remaining resource is generated during the + * update phase (except for rounding leftovers, eg. 1 memory or 1 vcores). + * + * -root- + * / \ \ + * A B C + * / \ | + * A1 A2 B1 + * / \ + * A11 A12 + * + * @throws IOException if update is failed + */ @Test public void testComplexHierarchyWithoutRemainingResource() throws IOException { setupQueueHierarchyWithoutRemainingResource(); @@ -103,6 +212,20 @@ public void testComplexHierarchyWithoutRemainingResource() throws IOException { update(assertionBuilder, UPDATE_RESOURCE); } + /** + * Tests a complex scenario in which several validation warnings are generated during the update + * phase. + * + * -root- + * / \ \ + * A B C + * / \ | + * A1 A2 B1 + * / \ + * A11 A12 + * + * @throws IOException if update is failed + */ @Test public void testComplexHierarchyWithWarnings() throws IOException { setupQueueHierarchyWithWarnings(); @@ -151,9 +274,8 @@ public void testComplexHierarchyWithWarnings() throws IOException { @Test public void testZeroResourceIfNoMemory() throws IOException { - csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(100), weight(6))); - csConf.setCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), - absolute(VCORES * 0.5))); + csConf.setCapacityVector(A, NO_LABEL, A_VECTOR_ZERO_RESOURCE); + csConf.setCapacityVector(B, NO_LABEL, B_VECTOR_ZERO_RESOURCE); QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) @@ -173,28 +295,25 @@ public void testZeroResourceIfNoMemory() throws IOException { @Test public void testDifferentMinimumAndMaximumCapacityTypes() throws IOException { - csConf.setCapacityVector(A, "", createMemoryVcoresVector(percentage(50), - absolute(VCORES * 0.5))); - csConf.setMaximumCapacityVector(A, "", createMemoryVcoresVector(absolute(MEMORY), - percentage(80))); - csConf.setCapacityVector(B, "", createMemoryVcoresVector(weight(6), percentage(100))); - csConf.setMaximumCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), - absolute(VCORES * 0.5))); + csConf.setCapacityVector(A, NO_LABEL, A_MIN_VECTOR_DIFFERENT_MIN_MAX); + csConf.setMaximumCapacityVector(A, NO_LABEL, A_MAX_VECTOR_DIFFERENT_MIN_MAX); + csConf.setCapacityVector(B, NO_LABEL, B_MIN_VECTOR_DIFFERENT_MIN_MAX); + csConf.setMaximumCapacityVector(B, NO_LABEL, B_MAX_VECTOR_DIFFERENT_MIN_MAX); QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) .assertEffectiveMinResource(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) - .assertEffectiveMaxResource(Resource.newInstance(MEMORY, (int) (VCORES * 0.8))) + .assertEffectiveMaxResource(A_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX) .withQueue(B) .assertEffectiveMinResource(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) - .assertEffectiveMaxResource(Resource.newInstance(MEMORY, (int) (VCORES * 0.5))) + .assertEffectiveMaxResource(B_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX) .build(); QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); Assert.assertEquals(0, updateContext.getUpdateWarnings().size()); // WEIGHT capacity type for maximum capacity is not supported - csConf.setMaximumCapacityVector(B, "", createMemoryVcoresVector(absolute(MEMORY), weight(10))); + csConf.setMaximumCapacityVector(B, NO_LABEL, B_INVALID_MAX_VECTOR); try { cs.reinitialize(csConf, mockRM.getRMContext()); update(assertionBuilder, UPDATE_RESOURCE); @@ -206,20 +325,18 @@ public void testDifferentMinimumAndMaximumCapacityTypes() throws IOException { @Test public void testMaximumResourceWarnings() throws IOException { - csConf.setMaximumCapacityVector(A1, "", createMemoryVcoresVector(absolute(MEMORY * 0.5), - percentage(100))); - csConf.setCapacityVector(A11, "", createMemoryVcoresVector(percentage(50), percentage(100))); - csConf.setCapacityVector(A12, "", createMemoryVcoresVector(percentage(50), percentage(0))); - csConf.setMaximumCapacityVector(A11, "", createMemoryVcoresVector(absolute(MEMORY), - percentage(10))); + csConf.setMaximumCapacityVector(A1, NO_LABEL, A1_MAX_VECTOR_MAX_WARNINGS); + csConf.setCapacityVector(A11, NO_LABEL, A11_MIN_VECTOR_MAX_WARNINGS); + csConf.setCapacityVector(A12, NO_LABEL, A12_MIN_VECTOR_MAX_WARNINGS); + csConf.setMaximumCapacityVector(A11, NO_LABEL, A11_MAX_VECTOR_MAX_WARNINGS); QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A11) - .assertEffectiveMinResource(createResource(0.5 * 0.5 * MEMORY, 0.1 * VCORES)) - .assertEffectiveMaxResource(createResource(MEMORY * 0.5, 0.1 * VCORES)) + .assertEffectiveMinResource(A11_EXPECTED_MIN_RESOURCE_MAX_WARNINGS) + .assertEffectiveMaxResource(A11_EXPECTED_MAX_RESOURCE_MAX_WARNINGS) .withQueue(A12) - .assertEffectiveMinResource(createResource(0.5 * 0.5 * MEMORY, 0)) - .assertEffectiveMaxResource(createResource(MEMORY * 0.5, VCORES)) + .assertEffectiveMinResource(A12_EXPECTED_MIN_RESOURCE_MAX_WARNINGS) + .assertEffectiveMaxResource(A12_EXPECTED_MAX_RESOURCE_MAX_WARNINGS) .build(); QueueCapacityUpdateContext updateContext = update(assertionBuilder, UPDATE_RESOURCE); @@ -238,75 +355,75 @@ public void testNodeLabels() throws Exception { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) - .assertEffectiveMinResource(createResource(2048, 8), NO_LABEL) + .assertEffectiveMinResource(A_EXPECTED_MIN_RESOURCE_NO_LABEL, NO_LABEL) .withQueue(A1) - .assertEffectiveMinResource(createResource(1024, 5), NO_LABEL) + .assertEffectiveMinResource(A1_EXPECTED_MIN_RESOURCE_NO_LABEL, NO_LABEL) .withQueue(A2) - .assertEffectiveMinResource(createResource(1024, 2), NO_LABEL) + .assertEffectiveMinResource(A2_EXPECTED_MIN_RESOURCE_NO_LABEL, NO_LABEL) .withQueue(B) - .assertEffectiveMinResource(createResource(3072, 8), NO_LABEL) + .assertEffectiveMinResource(B_EXPECTED_MIN_RESOURCE_NO_LABEL, NO_LABEL) .withQueue(A) - .assertEffectiveMinResource(createResource(30720, 30), X_LABEL) + .assertEffectiveMinResource(A_EXPECTED_MIN_RESOURCE_X_LABEL, X_LABEL) .withQueue(A1) - .assertEffectiveMinResource(createResource(20480, 0), X_LABEL) + .assertEffectiveMinResource(A1_EXPECTED_MIN_RESOURCE_X_LABEL, X_LABEL) .withQueue(A2) - .assertEffectiveMinResource(createResource(10240, 30), X_LABEL) + .assertEffectiveMinResource(A2_EXPECTED_MIN_RESOURCE_X_LABEL, X_LABEL) .withQueue(B) - .assertEffectiveMinResource(createResource(30720, 30), X_LABEL) + .assertEffectiveMinResource(B_EXPECTED_MIN_RESOURCE_X_LABEL, X_LABEL) .withQueue(A) - .assertEffectiveMinResource(createResource(8096, 42), Y_LABEL) + .assertEffectiveMinResource(A_EXPECTED_MIN_RESOURCE_Y_LABEL, Y_LABEL) .withQueue(A1) - .assertEffectiveMinResource(createResource(6186, 21), Y_LABEL) + .assertEffectiveMinResource(A1_EXPECTED_MIN_RESOURCE_Y_LABEL, Y_LABEL) .withQueue(A2) - .assertEffectiveMinResource(createResource(1910, 21), Y_LABEL) + .assertEffectiveMinResource(A2_EXPECTED_MIN_RESOURCE_Y_LABEL, Y_LABEL) .withQueue(B) - .assertEffectiveMinResource(createResource(12384, 18), Y_LABEL) + .assertEffectiveMinResource(B_EXPECTED_MIN_RESOURCE_Y_LABEL, Y_LABEL) .withQueue(A) - .assertEffectiveMinResource(createResource(7168, 11), Z_LABEL) + .assertEffectiveMinResource(A_EXPECTED_MIN_RESOURCE_Z_LABEL, Z_LABEL) .withQueue(A1) - .assertEffectiveMinResource(createResource(6451, 4), Z_LABEL) + .assertEffectiveMinResource(A1_EXPECTED_MIN_RESOURCE_Z_LABEL, Z_LABEL) .withQueue(A2) - .assertEffectiveMinResource(createResource(716, 7), Z_LABEL) + .assertEffectiveMinResource(A2_EXPECTED_MIN_RESOURCE_Z_LABEL, Z_LABEL) .withQueue(B) - .assertEffectiveMinResource(createResource(3072, 4), Z_LABEL) + .assertEffectiveMinResource(B_EXPECTED_MIN_RESOURCE_Z_LABEL, Z_LABEL) .build(); - update(assertionBuilder, UPDATE_RESOURCE, Resource.newInstance(5 * GB, 16)); + update(assertionBuilder, UPDATE_RESOURCE, EMPTY_LABEL_RESOURCE); } private void setLabeledQueueConfigs() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of(X_LABEL, Y_LABEL, Z_LABEL)); - mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), - TestUtils.toSet(X_LABEL), NodeId.newInstance("h2", 0), - TestUtils.toSet(Y_LABEL), NodeId.newInstance("h3", 0), - TestUtils.toSet(Y_LABEL), NodeId.newInstance("h4", 0), - TestUtils.toSet(Z_LABEL), NodeId.newInstance("h5", 0), + mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance(H1_NODE, 0), + TestUtils.toSet(X_LABEL), NodeId.newInstance(H2_NODE, 0), + TestUtils.toSet(Y_LABEL), NodeId.newInstance(H3_NODE, 0), + TestUtils.toSet(Y_LABEL), NodeId.newInstance(H4_NODE, 0), + TestUtils.toSet(Z_LABEL), NodeId.newInstance(H5_NODE, 0), RMNodeLabelsManager.EMPTY_STRING_SET)); - mockRM.registerNode("h1:1234", 60 * GB, 60); // label = x - mockRM.registerNode("h2:1234", 10 * GB, 25); // label = y - mockRM.registerNode("h3:1234", 10 * GB, 35); // label = y - mockRM.registerNode("h4:1234", 10 * GB, 15); // label = z + mockRM.registerNode("h1:1234", H1_MEMORY, H1_VCORES); // label = x + mockRM.registerNode("h2:1234", H2_MEMORY, H2_VCORES); // label = y + mockRM.registerNode("h3:1234", H3_MEMORY, H3_VCORES); // label = y + mockRM.registerNode("h4:1234", H4_MEMORY, H4_VCORES); // label = z - csConf.setCapacityVector(A, NO_LABEL, createMemoryVcoresVector(absolute(2048), percentage(50))); - csConf.setCapacityVector(A1, NO_LABEL, createMemoryVcoresVector(absolute(1024), percentage(70))); - csConf.setCapacityVector(A2, NO_LABEL, createMemoryVcoresVector(absolute(1024), percentage(30))); - csConf.setCapacityVector(B, NO_LABEL, createMemoryVcoresVector(weight(3), percentage(50))); + csConf.setCapacityVector(A, NO_LABEL, A_VECTOR_NO_LABEL); + csConf.setCapacityVector(A1, NO_LABEL, A1_VECTOR_NO_LABEL); + csConf.setCapacityVector(A2, NO_LABEL, A2_VECTOR_NO_LABEL); + csConf.setCapacityVector(B, NO_LABEL, B_VECTOR_NO_LABEL); - csConf.setCapacityVector(A, X_LABEL, createMemoryVcoresVector(percentage(50), weight(3))); - csConf.setCapacityVector(A1, X_LABEL, createMemoryVcoresVector(absolute(20480), percentage(10))); - csConf.setCapacityVector(A2, X_LABEL, createMemoryVcoresVector(absolute(10240), absolute(30))); - csConf.setCapacityVector(B, X_LABEL, createMemoryVcoresVector(percentage(50), percentage(50))); + csConf.setCapacityVector(A, X_LABEL, A_VECTOR_X_LABEL); + csConf.setCapacityVector(A1, X_LABEL, A1_VECTOR_X_LABEL); + csConf.setCapacityVector(A2, X_LABEL, A2_VECTOR_X_LABEL); + csConf.setCapacityVector(B, X_LABEL, B_VECTOR_X_LABEL); - csConf.setCapacityVector(A, Y_LABEL, createMemoryVcoresVector(absolute(8096), weight(1))); - csConf.setCapacityVector(A1, Y_LABEL, createMemoryVcoresVector(absolute(6186), weight(3))); - csConf.setCapacityVector(A2, Y_LABEL, createMemoryVcoresVector(weight(3), weight(3))); - csConf.setCapacityVector(B, Y_LABEL, createMemoryVcoresVector(percentage(100), percentage(30))); + csConf.setCapacityVector(A, Y_LABEL, A_VECTOR_Y_LABEL); + csConf.setCapacityVector(A1, Y_LABEL, A1_VECTOR_Y_LABEL); + csConf.setCapacityVector(A2, Y_LABEL, A2_VECTOR_Y_LABEL); + csConf.setCapacityVector(B, Y_LABEL, B_VECTOR_Y_LABEL); - csConf.setCapacityVector(A, Z_LABEL, createMemoryVcoresVector(percentage(70), absolute(11))); - csConf.setCapacityVector(A1, Z_LABEL, createMemoryVcoresVector(percentage(90), percentage(40))); - csConf.setCapacityVector(A2, Z_LABEL, createMemoryVcoresVector(percentage(10), weight(4))); - csConf.setCapacityVector(B, Z_LABEL, createMemoryVcoresVector(percentage(30), absolute(4))); + csConf.setCapacityVector(A, Z_LABEL, A_VECTOR_Z_LABEL); + csConf.setCapacityVector(A1, Z_LABEL, A1_VECTOR_Z_LABEL); + csConf.setCapacityVector(A2, Z_LABEL, A2_VECTOR_Z_LABEL); + csConf.setCapacityVector(B, Z_LABEL, B_VECTOR_Z_LABEL); cs.reinitialize(csConf, mockRM.getRMContext()); } @@ -317,14 +434,14 @@ private void setupQueueHierarchyWithoutRemainingResource() throws IOException { setQueues(); csConf.setState(B, QueueState.RUNNING); - csConf.setCapacityVector(A, NO_LABEL, createMemoryVcoresVector(percentage(30), weight(6))); - csConf.setCapacityVector(A1, NO_LABEL, createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); - csConf.setCapacityVector(A11, NO_LABEL, createMemoryVcoresVector(percentage(35), percentage(25))); - csConf.setCapacityVector(A12, NO_LABEL, createMemoryVcoresVector(percentage(65), percentage(75))); - csConf.setCapacityVector(A2, NO_LABEL, createMemoryVcoresVector(weight(3), percentage(100))); - csConf.setCapacityVector(B, NO_LABEL, createMemoryVcoresVector(absolute(8095), percentage(30))); - csConf.setCapacityVector(B1, NO_LABEL, createMemoryVcoresVector(weight(5), absolute(3))); - csConf.setCapacityVector(C, NO_LABEL, createMemoryVcoresVector(weight(3), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A, NO_LABEL, A_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(A1, NO_LABEL, A1_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(A11, NO_LABEL, A11_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(A12, NO_LABEL, A12_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(A2, NO_LABEL, A2_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(B, NO_LABEL, B_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(B1, NO_LABEL, B1_VECTOR_NO_REMAINING_RESOURCE); + csConf.setCapacityVector(C, NO_LABEL, C_VECTOR_NO_REMAINING_RESOURCE); cs.reinitialize(csConf, mockRM.getRMContext()); } @@ -335,15 +452,14 @@ private void setupQueueHierarchyWithWarnings() throws IOException { setQueues(); csConf.setState(B, QueueState.RUNNING); - csConf.setCapacityVector(A, NO_LABEL, createMemoryVcoresVector(percentage(100), weight(6))); - csConf.setCapacityVector(A1, NO_LABEL, createMemoryVcoresVector(absolute(2048), - absolute(VCORES * 0.25))); - csConf.setCapacityVector(A11, NO_LABEL, createMemoryVcoresVector(weight(1), absolute(VCORES * 0.25))); - csConf.setCapacityVector(A12, NO_LABEL, createMemoryVcoresVector(percentage(100), percentage(100))); - csConf.setCapacityVector(A2, NO_LABEL, createMemoryVcoresVector(absolute(2048), percentage(100))); - csConf.setCapacityVector(B, NO_LABEL, createMemoryVcoresVector(absolute(8096), percentage(30))); - csConf.setCapacityVector(B1, NO_LABEL, createMemoryVcoresVector(absolute(10256), absolute(3))); - csConf.setCapacityVector(C, NO_LABEL, createMemoryVcoresVector(weight(3), absolute(VCORES * 0.25))); + csConf.setCapacityVector(A, NO_LABEL, A_VECTOR_WITH_WARNINGS); + csConf.setCapacityVector(A1, NO_LABEL, A1_VECTOR_WITH_WARNING); + csConf.setCapacityVector(A11, NO_LABEL, A11_VECTOR_WITH_WARNING); + csConf.setCapacityVector(A12, NO_LABEL, A12_VECTOR_WITH_WARNING); + csConf.setCapacityVector(A2, NO_LABEL, A2_VECTOR_WITH_WARNING); + csConf.setCapacityVector(B, NO_LABEL, B_VECTOR_WITH_WARNING); + csConf.setCapacityVector(B1, NO_LABEL, B1_VECTOR_WITH_WARNING); + csConf.setCapacityVector(C, NO_LABEL, C_VECTOR_WITH_WARNING); cs.reinitialize(csConf, mockRM.getRMContext()); } 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/TestResourceVector.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/TestResourceVector.java index fd6edb1fa5d49..c56b37dc99081 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/TestResourceVector.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/TestResourceVector.java @@ -68,7 +68,7 @@ public void testCreation() { public void testSubtract() { ResourceVector lhsResourceVector = ResourceVector.of(13); ResourceVector rhsResourceVector = ResourceVector.of(5); - lhsResourceVector.subtract(rhsResourceVector); + lhsResourceVector.decrement(rhsResourceVector); Assert.assertEquals(8, lhsResourceVector.getValue(MEMORY_URI), EPSILON); Assert.assertEquals(8, lhsResourceVector.getValue(VCORES_URI), EPSILON); @@ -77,7 +77,7 @@ public void testSubtract() { ResourceVector negativeResourceVector = ResourceVector.of(-100); // Check whether overflow causes any issues - negativeResourceVector.subtract(ResourceVector.of(Float.MAX_VALUE)); + negativeResourceVector.decrement(ResourceVector.of(Float.MAX_VALUE)); Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(MEMORY_URI), EPSILON); Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(VCORES_URI), EPSILON); Assert.assertEquals(-Float.MAX_VALUE, negativeResourceVector.getValue(CUSTOM_RESOURCE), @@ -111,7 +111,7 @@ public void testEquals() { Assert.assertNotEquals(resource, resourceVector); ResourceVector resourceVectorOne = ResourceVector.of(1); - resourceVectorOther.subtract(resourceVectorOne); + resourceVectorOther.decrement(resourceVectorOne); Assert.assertEquals(resourceVectorOther, resourceVector); } From ec56a1cff40970773f9b9eedcab561b5132c2105 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Wed, 15 Dec 2021 12:53:02 +0100 Subject: [PATCH 20/27] YARN-10965. Fix checkstyle issues --- .../yarn/util/resource/ResourceUtils.java | 1 - .../scheduler/capacity/ParentQueue.java | 14 +- .../capacity/QueueAssertionBuilder.java | 33 ++- .../TestMixedQueueResourceCalculation.java | 197 +++++++++++------- .../TestUniformQueueResourceCalculation.java | 36 ++-- 5 files changed, 187 insertions(+), 94 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java index f1cf7f73aa5b7..b1a69dcf477e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java @@ -51,7 +51,6 @@ import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; 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 8e06c87f45786..b4e34a8ac9c6f 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 @@ -289,7 +289,7 @@ private enum QueueCapacityType { void setChildQueues(Collection childQueues) throws IOException { writeLock.lock(); try { - boolean isLegacyQueueMode = csContext.getConfiguration().isLegacyQueueMode(); + boolean isLegacyQueueMode = queueContext.getConfiguration().isLegacyQueueMode(); if (isLegacyQueueMode) { QueueCapacityType childrenCapacityType = getCapacityConfigurationTypeForQueues(childQueues); @@ -317,7 +317,7 @@ void setChildQueues(Collection childQueues) throws IOException { .getConfiguredMinResource(nodeLabel)); } Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel, - scheduler.getClusterResource()); + queueContext.getClusterResource()); Resource parentMinResource = usageTracker.getQueueResourceQuotas().getConfiguredMinResource(nodeLabel); if (!parentMinResource.equals(Resources.none()) && Resources.lessThan( @@ -325,7 +325,7 @@ void setChildQueues(Collection childQueues) throws IOException { throw new IOException( "Parent Queues" + " capacity: " + parentMinResource + " is less than" + " to its children:" + minRes - + " for queue:" + queueName); + + " for queue:" + getQueueName()); } } } @@ -347,7 +347,7 @@ void setChildQueues(Collection childQueues) throws IOException { // It is wrong when percent sum != {0, 1} throw new IOException( "Illegal" + " capacity sum of " + childrenPctSum - + " for children of queue " + queueName + " for label=" + + " for children of queue " + getQueueName() + " for label=" + nodeLabel + ". It should be either 0 or 1.0"); } else { // We also allow children's percent sum = 0 under the following @@ -360,7 +360,7 @@ void setChildQueues(Collection childQueues) throws IOException { > PRECISION) && (!allowZeroCapacitySum)) { throw new IOException( "Illegal" + " capacity sum of " + childrenPctSum - + " for children of queue " + queueName + + " for children of queue " + getQueueName() + " for label=" + nodeLabel + ". It is set to 0, but parent percent != 0, and " + "doesn't allow children capacity to set to 0"); @@ -375,8 +375,8 @@ void setChildQueues(Collection childQueues) throws IOException { && !allowZeroCapacitySum) { throw new IOException( "Illegal" + " capacity sum of " + childrenPctSum - + " for children of queue " + queueName + " for label=" - + nodeLabel + ". queue=" + queueName + + " for children of queue " + getQueueName() + " for label=" + + nodeLabel + ". queue=" + getQueueName() + " has zero capacity, but child" + "queues have positive 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 57f3110cb0e43..1a25d4bc8d680 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 @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.api.records.Resource; @@ -5,7 +23,6 @@ import org.junit.Assert; import java.util.ArrayList; -import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -100,7 +117,7 @@ public QueueAssertion assertAbsoluteCapacity(float expected) { return this; } - public class ValueAssertion { + private class ValueAssertion { private float expectedValue = 0; private Resource expectedResource = null; private String assertionType; @@ -150,11 +167,19 @@ public QueueAssertionBuilder build() { return this; } + /** + * Creates a new assertion group for a specific queue. + * @param queuePath path of the queue + * @return queue assertion group + */ public QueueAssertion withQueue(String queuePath) { assertions.putIfAbsent(queuePath, new QueueAssertion(queuePath)); return assertions.get(queuePath); } + /** + * Executes assertions created for all queues. + */ public void finishAssertion() { for (Map.Entry assertionEntry : assertions.entrySet()) { for (QueueAssertion.ValueAssertion assertion : assertionEntry.getValue().assertions) { @@ -175,6 +200,10 @@ public void finishAssertion() { } } + /** + * Returns all queues that have defined assertions. + * @return queue paths + */ 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/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 ef356fa74b14b..54210aae98270 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 @@ -40,20 +40,29 @@ public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCalculationTestBase { private static final long MEMORY = 16384; private static final long VCORES = 16; - private static final String C_VECTOR_WITH_WARNING = createCapacityVector(weight(3), absolute(VCORES * 0.25)); - private static final String A11_VECTOR_WITH_WARNING = createCapacityVector(weight(1), absolute(VCORES * 0.25)); + private static final String C_VECTOR_WITH_WARNING = createCapacityVector(weight(3), + absolute(VCORES * 0.25)); + private static final String A11_VECTOR_WITH_WARNING = createCapacityVector(weight(1), + absolute(VCORES * 0.25)); private static final String A1_VECTOR_WITH_WARNING = createCapacityVector(absolute(2048), absolute(VCORES * 0.25)); - private static final String C_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(3), absolute(VCORES * 0.25)); - private static final String A1_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(1), absolute(VCORES * 0.25)); - - private static final Resource A12_EXPECTED_MAX_RESOURCE_MAX_WARNINGS = createResource(MEMORY * 0.5, VCORES); - private static final Resource A11_EXPECTED_MAX_RESOURCE_MAX_WARNINGS = createResource(MEMORY * 0.5, 0.1 * VCORES); - private static final Resource A11_EXPECTED_MIN_RESOURCE_MAX_WARNINGS = createResource(0.5 * 0.5 * MEMORY, 0.1 * VCORES); - private static final Resource A12_EXPECTED_MIN_RESOURCE_MAX_WARNINGS = createResource(0.5 * 0.5 * MEMORY, 0); - private static final String A11_MAX_VECTOR_MAX_WARNINGS = createCapacityVector(absolute(MEMORY), - percentage(10)); - private static final String A1_MAX_VECTOR_MAX_WARNINGS = createCapacityVector(absolute(MEMORY * 0.5), + private static final String C_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(3), + absolute(VCORES * 0.25)); + private static final String A1_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(1), + absolute(VCORES * 0.25)); + + private static final Resource A12_EXPECTED_MAX_RESOURCE_MAX_WARNINGS = + createResource(MEMORY * 0.5, VCORES); + private static final Resource A11_EXPECTED_MAX_RESOURCE_MAX_WARNINGS = + createResource(MEMORY * 0.5, 0.1 * VCORES); + private static final Resource A11_EXPECTED_MIN_RESOURCE_MAX_WARNINGS = + createResource(0.5 * 0.5 * MEMORY, 0.1 * VCORES); + private static final Resource A12_EXPECTED_MIN_RESOURCE_MAX_WARNINGS = + createResource(0.5 * 0.5 * MEMORY, 0); + private static final String A11_MAX_VECTOR_MAX_WARNINGS = + createCapacityVector(absolute(MEMORY), percentage(10)); + private static final String A1_MAX_VECTOR_MAX_WARNINGS = + createCapacityVector(absolute(MEMORY * 0.5), percentage(100)); private static final Resource UPDATE_RESOURCE = Resource.newInstance(16384, 16); @@ -75,20 +84,25 @@ public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCal private static final Resource A2_WARNING_RESOURCE = Resource.newInstance(2048, 8); private static final Resource A12_WARNING_RESOURCE = Resource.newInstance(2048, 4); - private static final String A_VECTOR_ZERO_RESOURCE = createCapacityVector(percentage(100), weight(6)); - private static final String B_VECTOR_ZERO_RESOURCE = createCapacityVector(absolute(MEMORY), - absolute(VCORES * 0.5)); - - private static final String A_MAX_VECTOR_DIFFERENT_MIN_MAX = createCapacityVector(absolute(MEMORY), - percentage(80)); - private static final Resource B_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX = Resource.newInstance(MEMORY, (int) (VCORES * 0.5)); - private static final Resource A_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX = Resource.newInstance(MEMORY, (int) (VCORES * 0.8)); - private static final String B_MAX_VECTOR_DIFFERENT_MIN_MAX = createCapacityVector(absolute(MEMORY), - absolute(VCORES * 0.5)); - private static final String A_MIN_VECTOR_DIFFERENT_MIN_MAX = createCapacityVector(percentage(50), - absolute(VCORES * 0.5)); - private static final String B_MIN_VECTOR_DIFFERENT_MIN_MAX = createCapacityVector(weight(6), percentage(100)); - private static final String B_INVALID_MAX_VECTOR = createCapacityVector(absolute(MEMORY), weight(10)); + private static final String A_VECTOR_ZERO_RESOURCE = + createCapacityVector(percentage(100), weight(6)); + private static final String B_VECTOR_ZERO_RESOURCE = + createCapacityVector(absolute(MEMORY), absolute(VCORES * 0.5)); + + private static final String A_MAX_VECTOR_DIFFERENT_MIN_MAX = + createCapacityVector(absolute(MEMORY), percentage(80)); + private static final Resource B_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX = + Resource.newInstance(MEMORY, (int) (VCORES * 0.5)); + private static final Resource A_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX = + Resource.newInstance(MEMORY, (int) (VCORES * 0.8)); + private static final String B_MAX_VECTOR_DIFFERENT_MIN_MAX = + createCapacityVector(absolute(MEMORY), absolute(VCORES * 0.5)); + private static final String A_MIN_VECTOR_DIFFERENT_MIN_MAX = + createCapacityVector(percentage(50), absolute(VCORES * 0.5)); + private static final String B_MIN_VECTOR_DIFFERENT_MIN_MAX = + createCapacityVector(weight(6), percentage(100)); + private static final String B_INVALID_MAX_VECTOR = + createCapacityVector(absolute(MEMORY), weight(10)); private static final String X_LABEL = "x"; private static final String Y_LABEL = "y"; @@ -108,8 +122,10 @@ public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCal private static final int H4_MEMORY = 10 * GB; private static final int H4_VCORES = 15; - private static final String A11_MIN_VECTOR_MAX_WARNINGS = createCapacityVector(percentage(50), percentage(100)); - private static final String A12_MIN_VECTOR_MAX_WARNINGS = createCapacityVector(percentage(50), percentage(0)); + private static final String A11_MIN_VECTOR_MAX_WARNINGS = + createCapacityVector(percentage(50), percentage(100)); + private static final String A12_MIN_VECTOR_MAX_WARNINGS = + createCapacityVector(percentage(50), percentage(0)); private static final Resource A_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(2048, 8); private static final Resource A1_EXPECTED_MIN_RESOURCE_NO_LABEL = createResource(1024, 5); @@ -129,34 +145,61 @@ public class TestMixedQueueResourceCalculation extends CapacitySchedulerQueueCal private static final Resource B_EXPECTED_MIN_RESOURCE_Z_LABEL = createResource(3072, 4); private static final Resource EMPTY_LABEL_RESOURCE = Resource.newInstance(5 * GB, 16); - private static final String A_VECTOR_NO_LABEL = createCapacityVector(absolute(2048), percentage(50)); - private static final String A1_VECTOR_NO_LABEL = createCapacityVector(absolute(1024), percentage(70)); - private static final String A2_VECTOR_NO_LABEL = createCapacityVector(absolute(1024), percentage(30)); - private static final String B_VECTOR_NO_LABEL = createCapacityVector(weight(3), percentage(50)); - private static final String A_VECTOR_X_LABEL = createCapacityVector(percentage(50), weight(3)); - private static final String A1_VECTOR_X_LABEL = createCapacityVector(absolute(20480), percentage(10)); - private static final String A2_VECTOR_X_LABEL = createCapacityVector(absolute(10240), absolute(30)); - private static final String B_VECTOR_X_LABEL = createCapacityVector(percentage(50), percentage(50)); - private static final String A_VECTOR_Y_LABEL = createCapacityVector(absolute(8096), weight(1)); - private static final String A1_VECTOR_Y_LABEL = createCapacityVector(absolute(6186), weight(3)); - private static final String A2_VECTOR_Y_LABEL = createCapacityVector(weight(3), weight(3)); - private static final String B_VECTOR_Y_LABEL = createCapacityVector(percentage(100), percentage(30)); - private static final String A_VECTOR_Z_LABEL = createCapacityVector(percentage(70), absolute(11)); - private static final String A1_VECTOR_Z_LABEL = createCapacityVector(percentage(90), percentage(40)); - private static final String A2_VECTOR_Z_LABEL = createCapacityVector(percentage(10), weight(4)); - private static final String B_VECTOR_Z_LABEL = createCapacityVector(percentage(30), absolute(4)); - - private static final String A_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(percentage(30), weight(6)); - private static final String A11_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(percentage(35), percentage(25)); - private static final String A12_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(percentage(65), percentage(75)); - private static final String A2_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(3), percentage(100)); - private static final String B_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(absolute(8095), percentage(30)); - private static final String B1_VECTOR_NO_REMAINING_RESOURCE = createCapacityVector(weight(5), absolute(3)); - private static final String A_VECTOR_WITH_WARNINGS = createCapacityVector(percentage(100), weight(6)); - private static final String A12_VECTOR_WITH_WARNING = createCapacityVector(percentage(100), percentage(100)); - private static final String A2_VECTOR_WITH_WARNING = createCapacityVector(absolute(2048), percentage(100)); - private static final String B_VECTOR_WITH_WARNING = createCapacityVector(absolute(8096), percentage(30)); - private static final String B1_VECTOR_WITH_WARNING = createCapacityVector(absolute(10256), absolute(3)); + private static final String A_VECTOR_NO_LABEL = + createCapacityVector(absolute(2048), percentage(50)); + private static final String A1_VECTOR_NO_LABEL = + createCapacityVector(absolute(1024), percentage(70)); + private static final String A2_VECTOR_NO_LABEL = + createCapacityVector(absolute(1024), percentage(30)); + private static final String B_VECTOR_NO_LABEL = + createCapacityVector(weight(3), percentage(50)); + private static final String A_VECTOR_X_LABEL = + createCapacityVector(percentage(50), weight(3)); + private static final String A1_VECTOR_X_LABEL = + createCapacityVector(absolute(20480), percentage(10)); + private static final String A2_VECTOR_X_LABEL = + createCapacityVector(absolute(10240), absolute(30)); + private static final String B_VECTOR_X_LABEL = + createCapacityVector(percentage(50), percentage(50)); + private static final String A_VECTOR_Y_LABEL = + createCapacityVector(absolute(8096), weight(1)); + private static final String A1_VECTOR_Y_LABEL = + createCapacityVector(absolute(6186), weight(3)); + private static final String A2_VECTOR_Y_LABEL = + createCapacityVector(weight(3), weight(3)); + private static final String B_VECTOR_Y_LABEL = + createCapacityVector(percentage(100), percentage(30)); + private static final String A_VECTOR_Z_LABEL = + createCapacityVector(percentage(70), absolute(11)); + private static final String A1_VECTOR_Z_LABEL = + createCapacityVector(percentage(90), percentage(40)); + private static final String A2_VECTOR_Z_LABEL = + createCapacityVector(percentage(10), weight(4)); + private static final String B_VECTOR_Z_LABEL = + createCapacityVector(percentage(30), absolute(4)); + + private static final String A_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(percentage(30), weight(6)); + private static final String A11_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(percentage(35), percentage(25)); + private static final String A12_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(percentage(65), percentage(75)); + private static final String A2_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(weight(3), percentage(100)); + private static final String B_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(absolute(8095), percentage(30)); + private static final String B1_VECTOR_NO_REMAINING_RESOURCE = + createCapacityVector(weight(5), absolute(3)); + private static final String A_VECTOR_WITH_WARNINGS = + createCapacityVector(percentage(100), weight(6)); + private static final String A12_VECTOR_WITH_WARNING = + createCapacityVector(percentage(100), percentage(100)); + private static final String A2_VECTOR_WITH_WARNING = + createCapacityVector(absolute(2048), percentage(100)); + private static final String B_VECTOR_WITH_WARNING = + createCapacityVector(absolute(8096), percentage(30)); + private static final String B1_VECTOR_WITH_WARNING = + createCapacityVector(absolute(10256), absolute(3)); @Override public void setUp() throws Exception { @@ -185,28 +228,36 @@ public void testComplexHierarchyWithoutRemainingResource() throws IOException { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) .assertEffectiveMinResource(A_COMPLEX_NO_REMAINING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A1) .assertEffectiveMinResource(A1_COMPLEX_NO_REMAINING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A11) .assertEffectiveMinResource(A11_COMPLEX_NO_REMAINING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A11_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A11_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A12) .assertEffectiveMinResource(A12_COMPLEX_NO_REMAINING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A12_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A12_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A2) .assertEffectiveMinResource(A2_COMPLEX_NO_REMAINING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A2_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A2_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(B) .assertEffectiveMinResource(B_COMPLEX_NO_REMAINING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, B_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + B_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(B1) .assertEffectiveMinResource(B1_COMPLEX_NO_REMAINING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, B1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + B1_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .withQueue(C) .assertEffectiveMinResource(C_COMPLEX_NO_REMAINING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, C_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + C_COMPLEX_NO_REMAINING_RESOURCE, UPDATE_RESOURCE)) .build(); update(assertionBuilder, UPDATE_RESOURCE); @@ -232,25 +283,31 @@ public void testComplexHierarchyWithWarnings() throws IOException { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) .assertEffectiveMinResource(A_WARNING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A1) .assertEffectiveMinResource(A1_WARNING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A1_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A1_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A2) .assertEffectiveMinResource(A2_WARNING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A2_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A2_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(A11) .assertEffectiveMinResource(ZERO_RESOURCE) .assertAbsoluteCapacity(0) .withQueue(A12) .assertEffectiveMinResource(A12_WARNING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, A12_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + A12_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(B) .assertEffectiveMinResource(B_WARNING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, B_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + B_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(B1) .assertEffectiveMinResource(B1_WARNING_RESOURCE) - .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, B1_WARNING_RESOURCE, UPDATE_RESOURCE)) + .assertAbsoluteCapacity(resourceCalculator.divide(UPDATE_RESOURCE, + B1_WARNING_RESOURCE, UPDATE_RESOURCE)) .withQueue(C) .assertEffectiveMinResource(ZERO_RESOURCE) .assertAbsoluteCapacity(0) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/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 bbab4a8041eed..64c0cdd340ebd 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 @@ -81,16 +81,20 @@ public void testWeightResourceCalculation() throws IOException { .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, B_NORMALIZED_WEIGHT)) .assertAbsoluteCapacity(B_NORMALIZED_WEIGHT) .withQueue(A1) - .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, + A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT)) .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT) .withQueue(A2) - .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, + A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT)) .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A2_NORMALIZED_WEIGHT) .withQueue(A11) - .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, + A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT)) .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A11_NORMALIZED_WEIGHT) .withQueue(A12) - .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT)) + .assertEffectiveMinResource(ResourceUtils.multiplyRound(UPDATE_RES, + A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT)) .assertAbsoluteCapacity(A_NORMALIZED_WEIGHT * A1_NORMALIZED_WEIGHT * A12_NORMALIZED_WEIGHT) .build(); @@ -116,19 +120,23 @@ public void testPercentageResourceCalculation() throws IOException { .assertCapacity(B_CAPACITY) .assertAbsoluteCapacity(B_CAPACITY) .withQueue(A1) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY * A1_CAPACITY)) + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, + A_CAPACITY * A1_CAPACITY)) .assertCapacity(A1_CAPACITY) .assertAbsoluteCapacity(A_CAPACITY * A1_CAPACITY) .withQueue(A2) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY * A2_CAPACITY)) + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, + A_CAPACITY * A2_CAPACITY)) .assertCapacity(A2_CAPACITY) .assertAbsoluteCapacity(A_CAPACITY * A2_CAPACITY) .withQueue(A11) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A11_CAPACITY * A_CAPACITY * A1_CAPACITY)) + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, + A11_CAPACITY * A_CAPACITY * A1_CAPACITY)) .assertCapacity(A11_CAPACITY) .assertAbsoluteCapacity(A11_CAPACITY * A_CAPACITY * A1_CAPACITY) .withQueue(A12) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A12_CAPACITY * A_CAPACITY * A1_CAPACITY)) + .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, + A12_CAPACITY * A_CAPACITY * A1_CAPACITY)) .assertCapacity(A12_CAPACITY) .assertAbsoluteCapacity(A12_CAPACITY * A_CAPACITY * A1_CAPACITY) .build(); @@ -138,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) From 4deb72a583d51bcd836a27e263d8d1924bc3f581 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 1 Feb 2022 11:01:13 +0100 Subject: [PATCH 21/27] YARN-10965. Introduce calculation iteration context --- .../AbsoluteResourceCapacityCalculator.java | 18 +- .../AbstractQueueCapacityCalculator.java | 51 +--- ...CapacitySchedulerQueueCapacityHandler.java | 3 +- .../PercentageQueueCapacityCalculator.java | 17 +- .../capacity/ResourceCalculationDriver.java | 242 ++++++++++-------- .../capacity/RootCalculationDriver.java | 25 +- .../capacity/RootQueueCapacityCalculator.java | 18 +- .../WeightQueueCapacityCalculator.java | 20 +- 8 files changed, 194 insertions(+), 200 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index d4135ca1b2403..ac300bf81757b 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -19,33 +19,33 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { @Override public float calculateMinimumResource( - ResourceCalculationDriver resourceCalculationDriver, String label) { - String resourceName = resourceCalculationDriver.getCurrentResourceName(); + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + String resourceName = context.getResourceName(); float normalizedRatio = resourceCalculationDriver.getNormalizedResourceRatios().getOrDefault( label, ResourceVector.of(1)).getValue(resourceName); float remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( label, resourceName); - return normalizedRatio * remainingResourceRatio * resourceCalculationDriver - .getCurrentMinimumCapacityEntry(label).getResourceValue(); + return normalizedRatio * remainingResourceRatio * context.getCurrentMinimumCapacityEntry( + label).getResourceValue(); } @Override public float calculateMaximumResource( - ResourceCalculationDriver resourceCalculationDriver, String label) { - return resourceCalculationDriver.getCurrentMaximumCapacityEntry(label).getResourceValue(); + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + return context.getCurrentMaximumCapacityEntry(label).getResourceValue(); } @Override public void updateCapacitiesAfterCalculation( - ResourceCalculationDriver resourceCalculationDriver, String label) { - setQueueCapacities(resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource( - label), resourceCalculationDriver.getCurrentChild(), label); + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { + resourceCalculationDriver.setQueueCapacities(queue, label); } @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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index b52f28f5b93d2..4b4b952922654 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -18,12 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; import org.apache.hadoop.yarn.util.UnitsConversionUtil; -import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import java.util.Map; import java.util.Set; @@ -35,7 +34,6 @@ * logic. */ public abstract class AbstractQueueCapacityCalculator { - private static final String MB_UNIT = "Mi"; /** * Sets the metrics and statistics after effective resource values calculation. @@ -45,7 +43,7 @@ public abstract class AbstractQueueCapacityCalculator { * @param label node label */ public abstract void updateCapacitiesAfterCalculation( - ResourceCalculationDriver resourceCalculationDriver, String label); + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label); /** @@ -64,6 +62,7 @@ public abstract void updateCapacitiesAfterCalculation( * @return minimum effective resource */ public abstract float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, String label); /** @@ -75,6 +74,7 @@ public abstract float calculateMinimumResource(ResourceCalculationDriver resourc * @return minimum effective resource */ public abstract float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, String label); /** @@ -118,47 +118,6 @@ protected Set getResourceNames(CSQueue queue, String label, .getResourceNamesByCapacityType(capacityType); } - /** - * Sets capacity and absolute capacity values based on minimum and maximum effective resources. - * - * @param clusterResource cluster resource for the corresponding label - * @param queue child queue for which the capacities are set - * @param label node label - */ - public static void setQueueCapacities( - Resource clusterResource, CSQueue queue, String label) { - if (!(queue instanceof AbstractCSQueue)) { - return; - } - - AbstractCSQueue csQueue = (AbstractCSQueue) queue; - ResourceCalculator resourceCalculator = csQueue.resourceCalculator; - - CSQueue parent = queue.getParent(); - if (parent == null) { - return; - } - // Update capacity with a float calculated from the parent's minResources - // and the recently changed queue minResources. - // capacity = effectiveMinResource / {parent's effectiveMinResource} - float result = resourceCalculator.divide(clusterResource, - queue.getQueueResourceQuotas().getEffectiveMinResource(label), - parent.getQueueResourceQuotas().getEffectiveMinResource(label)); - queue.getQueueCapacities().setCapacity(label, - Float.isInfinite(result) ? 0 : result); - - // Update maxCapacity with a float calculated from the parent's maxResources - // and the recently changed queue maxResources. - // maxCapacity = effectiveMaxResource / parent's effectiveMaxResource - result = resourceCalculator.divide(clusterResource, - queue.getQueueResourceQuotas().getEffectiveMaxResource(label), - parent.getQueueResourceQuotas().getEffectiveMaxResource(label)); - queue.getQueueCapacities().setMaximumCapacity(label, - Float.isInfinite(result) ? 0 : result); - - csQueue.updateAbsoluteCapacities(); - } - /** * Calculates the normalized resource ratio of a parent queue, under which children are defined * with absolute capacity type. If the effective resource of the parent is less, than the @@ -211,7 +170,7 @@ private void setNormalizedResourceRatio( parentQueue.getQueuePath())); } - String unit = resourceName.equals(MEMORY_URI) ? MB_UNIT : ""; + String unit = resourceName.equals(MEMORY_URI) ? ResourceCalculationDriver.MB_UNIT : ""; long convertedValue = UnitsConversionUtil.convert(unit, updateContext.getUpdatedClusterResource(label).getResourceInformation(resourceName) .getUnits(), childrenConfiguredResource); 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index 3d1060bda7b2d..bd4c90689022b 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -117,8 +117,7 @@ private void updateChildren( private void updateChildrenAfterCalculation( ResourceCalculationDriver resourceCalculationDriver, ResourceLimits resourceLimits) { for (CSQueue childQueue : resourceCalculationDriver.getParent().getChildQueues()) { - resourceCalculationDriver.setCurrentChild(childQueue); - resourceCalculationDriver.updateChildCapacities(); + resourceCalculationDriver.updateChildCapacities(childQueue); ResourceLimits childLimit = ((ParentQueue) resourceCalculationDriver.getParent()).getResourceLimitsOfChild( childQueue, resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(), resourceLimits, NO_LABEL, false); childQueue.refreshAfterResourceCalculation(resourceCalculationDriver.getUpdateContext() 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index a1a40f93510e5..52c2e3bde1764 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -19,21 +19,22 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; public class PercentageQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override public float calculateMinimumResource( - ResourceCalculationDriver resourceCalculationDriver, String label) { + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { CSQueue parentQueue = resourceCalculationDriver.getParent(); - String resourceName = resourceCalculationDriver.getCurrentMinimumCapacityEntry(label).getResourceName(); + String resourceName = context.getResourceName(); float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label).getValue( resourceName); float remainingPerEffectiveResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( label, resourceName); float absoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio - * resourceCalculationDriver.getCurrentMinimumCapacityEntry(label).getResourceValue() / 100; + * context.getCurrentMinimumCapacityEntry(label).getResourceValue() / 100; return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) .getResourceValue(resourceName) * absoluteCapacity; @@ -41,22 +42,22 @@ public float calculateMinimumResource( @Override public float calculateMaximumResource( - ResourceCalculationDriver resourceCalculationDriver, String label) { + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { CSQueue parentQueue = resourceCalculationDriver.getParent(); - String resourceName = resourceCalculationDriver.getCurrentResourceName(); + String resourceName = context.getResourceName(); float parentAbsoluteMaxCapacity = parentQueue.getOrCreateAbsoluteMaxCapacityVector(label) .getValue(resourceName); float absoluteMaxCapacity = parentAbsoluteMaxCapacity - * resourceCalculationDriver.getCurrentMaximumCapacityEntry(label).getResourceValue() / 100; + * context.getCurrentMaximumCapacityEntry(label).getResourceValue() / 100; return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) .getResourceValue(resourceName) * absoluteMaxCapacity; } @Override - public void updateCapacitiesAfterCalculation(ResourceCalculationDriver resourceCalculationDriver, String label) { - ((AbstractCSQueue)resourceCalculationDriver.getCurrentChild()).updateAbsoluteCapacities(); + public void updateCapacitiesAfterCalculation(ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { + ((AbstractCSQueue) queue).updateAbsoluteCapacities(); } @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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index 3df373d7a52a6..bb939507e5ea4 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -21,8 +21,10 @@ import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.yarn.api.records.Resource; 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.util.resource.ResourceCalculator; import java.util.Collection; import java.util.HashMap; @@ -41,6 +43,7 @@ public class ResourceCalculationDriver { ResourceUnitCapacityType.ABSOLUTE, ResourceUnitCapacityType.PERCENTAGE, ResourceUnitCapacityType.WEIGHT); + static final String MB_UNIT = "Mi"; protected final QueueResourceRoundingStrategy roundingStrategy = new DefaultQueueResourceRoundingStrategy(CALCULATOR_PRECEDENCE); @@ -55,11 +58,7 @@ public class ResourceCalculationDriver { protected final Map normalizedResourceRatio = new HashMap<>(); // Used by WEIGHT capacity typet js protected final Map> sumWeightsPerLabel = new HashMap<>(); - - protected String currentResourceName; - protected AbstractQueueCapacityCalculator currentCalculator; - protected CSQueue currentChild; - protected Map usedResourceByCurrentCalculator = new HashMap<>(); + protected Map usedResourceByCurrentCalculatorPerLabel = new HashMap<>(); public ResourceCalculationDriver( CSQueue parent, QueueCapacityUpdateContext updateContext, @@ -71,76 +70,108 @@ public ResourceCalculationDriver( this.definedResources = definedResources; } - /** - * Returns the parent that is driving the calculation. - * - * @return a common parent queue - */ - public CSQueue getParent() { - return parent; - } + public static class CalculationContext { + private String resourceName; + private ResourceUnitCapacityType capacityType; + private CSQueue childQueue; - /** - * Returns the context that is used throughout the whole update phase. - * - * @return update context - */ - public QueueCapacityUpdateContext getUpdateContext() { - return updateContext; - } + public CalculationContext(String resourceName, ResourceUnitCapacityType capacityType, CSQueue childQueue) { + this.resourceName = resourceName; + this.capacityType = capacityType; + this.childQueue = childQueue; + } - /** - * Returns the name of the resource that is currently processed. - * - * @return resource name - */ - public String getCurrentResourceName() { - return currentResourceName; - } + public String getResourceName() { + return resourceName; + } - /** - * Returns the child that is currently processed. - * - * @return child queue - */ - public CSQueue getCurrentChild() { - return currentChild; + public ResourceUnitCapacityType getCapacityType() { + return capacityType; + } + + public CSQueue getChildQueue() { + return childQueue; + } + + /** + * A shorthand to return the minimum capacity vector entry for the currently evaluated child and + * resource name. + * + * @param label node label + * @return capacity vector entry + */ + public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { + return childQueue.getConfiguredCapacityVector(label).getResource(resourceName); + } + + /** + * A shorthand to return the maximum capacity vector entry for the currently evaluated child and + * resource name. + * + * @param label node label + * @return capacity vector entry + */ + public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { + return childQueue.getConfiguredMaxCapacityVector(label).getResource(resourceName); + } } /** - * Sets the currently evaluated child to a specific queue. + * Sets capacity and absolute capacity values based on minimum and maximum effective resources. * - * @param currentChild a child queue + * @param queue child queue for which the capacities are set + * @param label node label */ - public void setCurrentChild(CSQueue currentChild) { - if (currentChild.getParent() != parent) { - throw new IllegalArgumentException("Child queue " + currentChild.getQueuePath() + " is not " + - "a child of " + parent.getQueuePath()); + public void setQueueCapacities(CSQueue queue, String label) { + if (!(queue instanceof AbstractCSQueue)) { + return; } - this.currentChild = currentChild; + Resource clusterResource = updateContext.getUpdatedClusterResource(label); + AbstractCSQueue csQueue = (AbstractCSQueue) queue; + ResourceCalculator resourceCalculator = csQueue.resourceCalculator; + + CSQueue parent = queue.getParent(); + if (parent == null) { + return; + } + // Update capacity with a float calculated from the parent's minResources + // and the recently changed queue minResources. + // capacity = effectiveMinResource / {parent's effectiveMinResource} + float result = resourceCalculator.divide(clusterResource, + queue.getQueueResourceQuotas().getEffectiveMinResource(label), + parent.getQueueResourceQuotas().getEffectiveMinResource(label)); + queue.getQueueCapacities().setCapacity(label, + Float.isInfinite(result) ? 0 : result); + + // Update maxCapacity with a float calculated from the parent's maxResources + // and the recently changed queue maxResources. + // maxCapacity = effectiveMaxResource / parent's effectiveMaxResource + result = resourceCalculator.divide(clusterResource, + queue.getQueueResourceQuotas().getEffectiveMaxResource(label), + parent.getQueueResourceQuotas().getEffectiveMaxResource(label)); + queue.getQueueCapacities().setMaximumCapacity(label, + Float.isInfinite(result) ? 0 : result); + + csQueue.updateAbsoluteCapacities(); } /** - * A shorthand to return the minimum capacity vector entry for the currently evaluated child and - * resource name. + * Returns the parent that is driving the calculation. * - * @param label node label - * @return capacity vector entry + * @return a common parent queue */ - public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { - return currentChild.getConfiguredCapacityVector(label).getResource(currentResourceName); + public CSQueue getParent() { + return parent; } /** - * A shorthand to return the maximum capacity vector entry for the currently evaluated child and - * resource name. + * Returns the context that is used throughout the whole update phase. * - * @param label node label - * @return capacity vector entry + * @return update context */ - public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { - return currentChild.getConfiguredMaxCapacityVector(label).getResource(currentResourceName); + public QueueCapacityUpdateContext getUpdateContext() { + return updateContext; } /** @@ -205,18 +236,16 @@ public void calculateResources() { } for (String resourceName : definedResources) { - currentResourceName = resourceName; for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { - currentCalculator = calculators.get(capacityType); for (CSQueue childQueue : parent.getChildQueues()) { - currentChild = childQueue; - calculateResourceOnChild(capacityType); + CalculationContext context = new CalculationContext(resourceName, capacityType, childQueue); + calculateResourceOnChild(context); } // Flush aggregated used resource by labels at the end of a calculator phase - for (Map.Entry entry : usedResourceByCurrentCalculator.entrySet()) { + for (Map.Entry entry : usedResourceByCurrentCalculatorPerLabel.entrySet()) { batchRemainingResource.get(entry.getKey()).decrement(resourceName, entry.getValue()); } - usedResourceByCurrentCalculator = new HashMap<>(); + usedResourceByCurrentCalculatorPerLabel = new HashMap<>(); } } @@ -226,114 +255,113 @@ public void calculateResources() { /** * Updates the capacity values of the currently evaluated child. */ - public void updateChildCapacities() { - currentChild.getWriteLock().lock(); + public void updateChildCapacities(CSQueue childQueue) { + childQueue.getWriteLock().lock(); try { - for (String label : currentChild.getConfiguredNodeLabels()) { - QueueCapacityVector capacityVector = currentChild.getConfiguredCapacityVector(label); + for (String label : childQueue.getConfiguredNodeLabels()) { + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); if (capacityVector.isMixedCapacityVector()) { // Post update capacities based on the calculated effective resource values - AbstractQueueCapacityCalculator.setQueueCapacities(updateContext.getUpdatedClusterResource( - label), currentChild, label); + setQueueCapacities(childQueue, label); } else { // Update capacities according to the legacy logic for (ResourceUnitCapacityType capacityType : - currentChild.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + childQueue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); - calculator.updateCapacitiesAfterCalculation(this, label); + calculator.updateCapacitiesAfterCalculation(this, childQueue, label); } } } } finally { - currentChild.getWriteLock().unlock(); + childQueue.getWriteLock().unlock(); } } - private void calculateResourceOnChild(ResourceUnitCapacityType capacityType) { - currentChild.getWriteLock().lock(); + private void calculateResourceOnChild(CalculationContext context) { + context.getChildQueue().getWriteLock().lock(); try { - for (String label : currentChild.getConfiguredNodeLabels()) { - if (!currentChild.getConfiguredCapacityVector(label).isResourceOfType(currentResourceName, - capacityType)) { + for (String label : context.getChildQueue().getConfiguredNodeLabels()) { + if (!context.getChildQueue().getConfiguredCapacityVector(label).isResourceOfType(context.getResourceName(), + context.getCapacityType())) { continue; } - float usedResourceByChild = setChildResources(label); - float aggregatedUsedResource = usedResourceByCurrentCalculator.getOrDefault(label, + float usedResourceByChild = setChildResources(context, label); + float aggregatedUsedResource = usedResourceByCurrentCalculatorPerLabel.getOrDefault(label, 0f); float resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; - overallRemainingResource.get(label).decrement(currentResourceName, usedResourceByChild); - usedResourceByCurrentCalculator.put(label, resourceUsedByLabel); + overallRemainingResource.get(label).decrement(context.getResourceName(), usedResourceByChild); + usedResourceByCurrentCalculatorPerLabel.put(label, resourceUsedByLabel); } } finally { - currentChild.getWriteLock().unlock(); + context.getChildQueue().getWriteLock().unlock(); } } - private float setChildResources(String label) { - QueueCapacityVectorEntry capacityVectorEntry = currentChild.getConfiguredCapacityVector( - label).getResource(currentResourceName); + private float setChildResources(CalculationContext context, String label) { + QueueCapacityVectorEntry capacityVectorEntry = context.getChildQueue().getConfiguredCapacityVector( + label).getResource(context.getResourceName()); long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( - currentResourceName); - QueueCapacityVectorEntry maximumCapacityVectorEntry = currentChild - .getConfiguredMaxCapacityVector(label).getResource(currentResourceName); + context.getResourceName()); + QueueCapacityVectorEntry maximumCapacityVectorEntry = context.getChildQueue() + .getConfiguredMaxCapacityVector(label).getResource(context.getResourceName()); AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( maximumCapacityVectorEntry.getVectorResourceType()); - float minimumResource = currentCalculator.calculateMinimumResource(this, label); - float maximumResource = maximumCapacityCalculator.calculateMaximumResource(this, label); + float minimumResource = calculators.get(context.getCapacityType()).calculateMinimumResource(this, context, label); + float maximumResource = maximumCapacityCalculator.calculateMaximumResource(this, context, label); minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); maximumResource = roundingStrategy.getRoundedResource(maximumResource, maximumCapacityVectorEntry); - Pair resources = validateCalculatedResources(label, new ImmutablePair<>( + Pair resources = validateCalculatedResources(context, label, new ImmutablePair<>( minimumResource, maximumResource)); minimumResource = resources.getLeft(); maximumResource = resources.getRight(); float absoluteMinCapacity = minimumResource / clusterResource; float absoluteMaxCapacity = maximumResource / clusterResource; - currentChild.getOrCreateAbsoluteMinCapacityVector(label).setValue( - currentResourceName, absoluteMinCapacity); - currentChild.getOrCreateAbsoluteMaxCapacityVector(label).setValue( - currentResourceName, absoluteMaxCapacity); + context.getChildQueue().getOrCreateAbsoluteMinCapacityVector(label).setValue( + context.getResourceName(), absoluteMinCapacity); + context.getChildQueue().getOrCreateAbsoluteMaxCapacityVector(label).setValue( + context.getResourceName(), absoluteMaxCapacity); - currentChild.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( - currentResourceName, (long) minimumResource); - currentChild.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( - currentResourceName, (long) maximumResource); + context.getChildQueue().getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + context.getResourceName(), (long) minimumResource); + context.getChildQueue().getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + context.getResourceName(), (long) maximumResource); return minimumResource; } - private Pair validateCalculatedResources( + private Pair validateCalculatedResources(CalculationContext context, String label, Pair calculatedResources) { float minimumResource = calculatedResources.getLeft(); - long minimumMemoryResource = currentChild.getQueueResourceQuotas().getEffectiveMinResource(label) + long minimumMemoryResource = context.getChildQueue().getQueueResourceQuotas().getEffectiveMinResource(label) .getMemorySize(); float remainingResourceUnderParent = overallRemainingResource.get(label).getValue( - currentResourceName); + context.getResourceName()); long parentMaximumResource = parent.getEffectiveMaxCapacity(label).getResourceValue( - currentResourceName); + context.getResourceName()); float maximumResource = calculatedResources.getRight(); // Memory is the primary resource, if its zero, all other resource units are zero as well. - if (!currentResourceName.equals(MEMORY_URI) && minimumMemoryResource == 0) { + if (!context.getResourceName().equals(MEMORY_URI) && minimumMemoryResource == 0) { minimumResource = 0; } if (maximumResource != 0 && maximumResource > parentMaximumResource) { updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( - currentChild.getQueuePath())); + context.getChildQueue().getQueuePath())); } maximumResource = maximumResource == 0 ? parentMaximumResource : Math.min(maximumResource, parentMaximumResource); if (maximumResource < minimumResource) { updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( - currentChild.getQueuePath())); + context.getChildQueue().getQueuePath())); minimumResource = maximumResource; } @@ -343,15 +371,15 @@ private Pair validateCalculatedResources( minimumResource = 0; } else { updateContext.addUpdateWarning( - QueueUpdateWarning.QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(currentChild.getQueuePath()).withInfo( - "Resource name: " + currentResourceName + " resource value: " + minimumResource)); + QueueUpdateWarning.QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(context.getChildQueue().getQueuePath()).withInfo( + "Resource name: " + context.getResourceName() + " resource value: " + minimumResource)); minimumResource = remainingResourceUnderParent; } } if (minimumResource == 0) { updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( - currentChild.getQueuePath()).withInfo("Resource name: " + currentResourceName)); + context.getChildQueue().getQueuePath()).withInfo("Resource name: " + context.getResourceName())); } return new ImmutablePair<>(minimumResource, maximumResource); 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/RootCalculationDriver.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/RootCalculationDriver.java index c49e9326e1bde..9a7b03e385d51 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/RootCalculationDriver.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/RootCalculationDriver.java @@ -21,6 +21,8 @@ import java.util.Collection; import java.util.Collections; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; + /** * A special case that contains the resource calculation of the root queue. */ @@ -38,20 +40,21 @@ public RootCalculationDriver(CSQueue rootQueue, QueueCapacityUpdateContext updat public void calculateResources() { for (String label : parent.getConfiguredNodeLabels()) { for (QueueCapacityVector.QueueCapacityVectorEntry capacityVectorEntry : parent.getConfiguredCapacityVector(label)) { - currentResourceName = capacityVectorEntry.getResourceName(); - parent.getOrCreateAbsoluteMinCapacityVector(label).setValue(currentResourceName, 1); - parent.getOrCreateAbsoluteMaxCapacityVector(label).setValue(currentResourceName, 1); - - float minimumResource = rootCalculator.calculateMinimumResource(this, label); - float maximumResource = rootCalculator.calculateMaximumResource(this, label); - long roundedMinResource = (long) Math.floor(minimumResource); - long roundedMaxResource = (long) Math.floor(maximumResource); + String resourceName = capacityVectorEntry.getResourceName(); + parent.getOrCreateAbsoluteMinCapacityVector(label).setValue(resourceName, 1); + parent.getOrCreateAbsoluteMaxCapacityVector(label).setValue(resourceName, 1); + + CalculationContext context = new CalculationContext(resourceName, PERCENTAGE, parent); + float minimumResource = rootCalculator.calculateMinimumResource(this, context, label); + float maximumResource = rootCalculator.calculateMaximumResource(this, context, label); + long roundedMinResource = (long) roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); + long roundedMaxResource = (long) roundingStrategy.getRoundedResource(maximumResource, parent.getConfiguredMaxCapacityVector(label).getResource(resourceName)); parent.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( - currentResourceName, roundedMinResource); + resourceName, roundedMinResource); parent.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( - currentResourceName, roundedMaxResource); + resourceName, roundedMaxResource); } - rootCalculator.updateCapacitiesAfterCalculation(this, label); + rootCalculator.updateCapacitiesAfterCalculation(this, parent, label); } rootCalculator.calculateResourcePrerequisites(this); 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index 5966d832becbe..0c1547e0cef4c 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -18,27 +18,29 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; + import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; public class RootQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, String label) { - return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(resourceCalculationDriver.getCurrentResourceName()); + public float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(context.getResourceName()); } @Override - public float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, String label) { - return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(resourceCalculationDriver.getCurrentResourceName()); + public float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(context.getResourceName()); } @Override public void updateCapacitiesAfterCalculation( - ResourceCalculationDriver resourceCalculationDriver, String label) { - resourceCalculationDriver.getParent().getQueueCapacities().setAbsoluteCapacity(label, 1); - if (resourceCalculationDriver.getParent().getQueueCapacities().getWeight(label) == 1) { - resourceCalculationDriver.getParent().getQueueCapacities().setNormalizedWeight(label, 1); + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { + queue.getQueueCapacities().setAbsoluteCapacity(label, 1); + if (queue.getQueueCapacities().getWeight(label) == 1) { + queue.getQueueCapacities().setNormalizedWeight(label, 1); } } 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 02d211195d54e..1c03f73f5f7e9 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; import java.util.Collection; @@ -43,10 +44,11 @@ public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCal @Override public float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, String label) { CSQueue parentQueue = resourceCalculationDriver.getParent(); - String resourceName = resourceCalculationDriver.getCurrentResourceName(); - float normalizedWeight = resourceCalculationDriver.getCurrentMinimumCapacityEntry(label) + String resourceName = context.getResourceName(); + float normalizedWeight = context.getCurrentMinimumCapacityEntry(label) .getResourceValue() / resourceCalculationDriver.getSumWeightsByResource(label, resourceName); float remainingResource = resourceCalculationDriver.getBatchRemainingResource(label).getValue( @@ -76,8 +78,9 @@ public float calculateMinimumResource(ResourceCalculationDriver resourceCalculat @Override public float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, String label) { - throw new IllegalStateException("Resource " + resourceCalculationDriver.getCurrentMinimumCapacityEntry( + throw new IllegalStateException("Resource " + context.getCurrentMinimumCapacityEntry( label).getResourceName() + " has " + "WEIGHT maximum capacity type, which is not supported"); } @@ -88,19 +91,18 @@ public ResourceUnitCapacityType getCapacityType() { @Override public void updateCapacitiesAfterCalculation( - ResourceCalculationDriver resourceCalculationDriver, String label) { + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { float sumCapacityPerResource = 0f; - Collection resourceNames = getResourceNames(resourceCalculationDriver.getCurrentChild(), label); + Collection resourceNames = getResourceNames(queue, label); for (String resourceName : resourceNames) { float sumBranchWeight = resourceCalculationDriver.getSumWeightsByResource(label, resourceName); - float capacity = resourceCalculationDriver.getCurrentChild().getConfiguredCapacityVector( + float capacity = queue.getConfiguredCapacityVector( label).getResource(resourceName).getResourceValue() / sumBranchWeight; sumCapacityPerResource += capacity; } - resourceCalculationDriver.getCurrentChild().getQueueCapacities().setNormalizedWeight(label, - sumCapacityPerResource / resourceNames.size()); - ((AbstractCSQueue) resourceCalculationDriver.getCurrentChild()).updateAbsoluteCapacities(); + queue.getQueueCapacities().setNormalizedWeight(label, sumCapacityPerResource / resourceNames.size()); + ((AbstractCSQueue) queue).updateAbsoluteCapacities(); } } From 044cf3b72c87f84c3051c56c45a37016252e2a52 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Wed, 2 Feb 2022 08:55:43 +0100 Subject: [PATCH 22/27] YARN-10965. Fix review feedbacks --- .../yarn/util/resource/ResourceUtils.java | 4 +- .../AbsoluteResourceCapacityCalculator.java | 12 +- .../scheduler/capacity/AbstractCSQueue.java | 14 - .../AbstractQueueCapacityCalculator.java | 85 +---- .../scheduler/capacity/CSQueue.java | 4 - .../capacity/CSQueueUsageTracker.java | 10 - .../capacity/CalculationContext.java | 70 ++++ ...CapacitySchedulerQueueCapacityHandler.java | 123 +++++-- .../DefaultQueueResourceRoundingStrategy.java | 13 +- .../PercentageQueueCapacityCalculator.java | 19 +- .../capacity/QueueCapacityVector.java | 22 +- .../QueueResourceRoundingStrategy.java | 2 +- .../capacity/ResourceCalculationDriver.java | 341 +++++++++--------- .../scheduler/capacity/ResourceVector.java | 21 +- .../capacity/RootCalculationDriver.java | 20 +- .../capacity/RootQueueCapacityCalculator.java | 6 +- .../WeightQueueCapacityCalculator.java | 34 +- ...citySchedulerQueueCalculationTestBase.java | 3 +- .../capacity/QueueAssertionBuilder.java | 8 +- .../TestMixedQueueResourceCalculation.java | 4 +- .../TestUniformQueueResourceCalculation.java | 62 ++-- 21 files changed, 442 insertions(+), 435 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CalculationContext.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java index b1a69dcf477e7..5118d1288e5ac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java @@ -818,7 +818,7 @@ public static Resource createResourceWithSameValue(long value) { return res; } - public static Resource multiply(Resource resource, float multiplier) { + public static Resource multiplyFloor(Resource resource, double multiplier) { Resource newResource = Resource.newInstance(0, 0); for (ResourceInformation resourceInformation : resource.getResources()) { @@ -829,7 +829,7 @@ public static Resource multiply(Resource resource, float multiplier) { return newResource; } - public static Resource multiplyRound(Resource resource, float multiplier) { + public static Resource multiplyRound(Resource resource, double multiplier) { Resource newResource = Resource.newInstance(0, 0); for (ResourceInformation resourceInformation : resource.getResources()) { 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index ac300bf81757b..6d1bd5cffe796 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -19,17 +19,16 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public float calculateMinimumResource( + public double calculateMinimumResource( ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { String resourceName = context.getResourceName(); - float normalizedRatio = resourceCalculationDriver.getNormalizedResourceRatios().getOrDefault( + double normalizedRatio = resourceCalculationDriver.getNormalizedResourceRatios().getOrDefault( label, ResourceVector.of(1)).getValue(resourceName); - float remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( + double remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( label, resourceName); return normalizedRatio * remainingResourceRatio * context.getCurrentMinimumCapacityEntry( @@ -37,7 +36,7 @@ public float calculateMinimumResource( } @Override - public float calculateMaximumResource( + public double calculateMaximumResource( ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { return context.getCurrentMaximumCapacityEntry(label).getResourceValue(); } @@ -45,7 +44,8 @@ public float calculateMaximumResource( @Override public void updateCapacitiesAfterCalculation( ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { - resourceCalculationDriver.setQueueCapacities(queue, label); + CapacitySchedulerQueueCapacityHandler.setQueueCapacities( + resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label), queue, label); } @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/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 230c9aff477f6..34b907d45dbe5 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 @@ -200,20 +200,6 @@ public float getAbsoluteCapacity() { return queueCapacities.getAbsoluteCapacity(); } - @Override - public ResourceVector getOrCreateAbsoluteMinCapacityVector(String label) { - usageTracker.getAbsoluteMinCapacityVector().putIfAbsent(label, ResourceVector.newInstance()); - - return usageTracker.getAbsoluteMinCapacityVector().get(label); - } - - @Override - public ResourceVector getOrCreateAbsoluteMaxCapacityVector(String label) { - usageTracker.getAbsoluteMaxCapacityVector().putIfAbsent(label, ResourceVector.newInstance()); - - return usageTracker.getAbsoluteMaxCapacityVector().get(label); - } - @Override public float getAbsoluteMaximumCapacity() { return queueCapacities.getAbsoluteMaximumCapacity(); 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 4b4b952922654..3c1e9c0e69ceb 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -19,16 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; -import org.apache.hadoop.yarn.util.UnitsConversionUtil; - -import java.util.Map; import java.util.Set; -import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; - /** * A strategy class to encapsulate queue capacity setup and resource calculation * logic. @@ -61,7 +53,7 @@ public abstract void updateCapacitiesAfterCalculation( * @param label node label * @return minimum effective resource */ - public abstract float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + public abstract double calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label); @@ -73,7 +65,7 @@ public abstract float calculateMinimumResource(ResourceCalculationDriver resourc * @param label node label * @return minimum effective resource */ - public abstract float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + public abstract double calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label); @@ -84,12 +76,6 @@ public abstract float calculateMaximumResource(ResourceCalculationDriver resourc * calculation should be made */ public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { - for (String label : resourceCalculationDriver.getParent().getConfiguredNodeLabels()) { - // We need to set normalized resource ratio only once per parent - if (resourceCalculationDriver.getNormalizedResourceRatios().isEmpty()) { - setNormalizedResourceRatio(resourceCalculationDriver, label); - } - } } /** @@ -117,71 +103,4 @@ protected Set getResourceNames(CSQueue queue, String label, return queue.getConfiguredCapacityVector(label) .getResourceNamesByCapacityType(capacityType); } - - /** - * Calculates the normalized resource ratio of a parent queue, under which children are defined - * with absolute capacity type. If the effective resource of the parent is less, than the - * aggregated configured absolute resource of its children, the resource ratio will be less, - * than 1. - * - * @param label node label - */ - private void setNormalizedResourceRatio( - ResourceCalculationDriver resourceCalculationDriver, String label) { - // ManagedParents assign zero capacity to queues in case of overutilization, downscaling is - // turned off for their children - CSQueue parentQueue = resourceCalculationDriver.getParent(); - QueueCapacityUpdateContext updateContext = resourceCalculationDriver.getUpdateContext(); - - if (parentQueue instanceof ManagedParentQueue) { - return; - } - - for (QueueCapacityVectorEntry capacityVectorEntry : parentQueue.getConfiguredCapacityVector( - label)) { - String resourceName = capacityVectorEntry.getResourceName(); - long childrenConfiguredResource = 0; - long effectiveMinResource = parentQueue.getQueueResourceQuotas().getEffectiveMinResource( - label).getResourceValue(resourceName); - - // Total configured min resources of direct children of this given parent - // queue - for (CSQueue childQueue : parentQueue.getChildQueues()) { - if (!childQueue.getConfiguredNodeLabels().contains(label)) { - continue; - } - QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); - if (capacityVector.isResourceOfType(resourceName, ResourceUnitCapacityType.ABSOLUTE)) { - childrenConfiguredResource += capacityVector.getResource(resourceName).getResourceValue(); - } - } - // If no children is using ABSOLUTE capacity type, normalization is not needed - if (childrenConfiguredResource == 0) { - continue; - } - // Factor to scale down effective resource: When cluster has sufficient - // resources, effective_min_resources will be same as configured - // min_resources. - float numeratorForMinRatio = childrenConfiguredResource; - if (effectiveMinResource < childrenConfiguredResource) { - numeratorForMinRatio = parentQueue.getQueueResourceQuotas().getEffectiveMinResource(label) - .getResourceValue(resourceName); - updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_DOWNSCALED.ofQueue( - parentQueue.getQueuePath())); - } - - String unit = resourceName.equals(MEMORY_URI) ? ResourceCalculationDriver.MB_UNIT : ""; - long convertedValue = UnitsConversionUtil.convert(unit, - updateContext.getUpdatedClusterResource(label).getResourceInformation(resourceName) - .getUnits(), childrenConfiguredResource); - - if (convertedValue != 0) { - Map normalizedResourceRatios = resourceCalculationDriver - .getNormalizedResourceRatios(); - normalizedResourceRatios.putIfAbsent(label, ResourceVector.newInstance()); - normalizedResourceRatios.get(label).setValue(resourceName, numeratorForMinRatio / - convertedValue); - } - } - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index e661ef7408e1c..f06c775b3c778 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -115,10 +115,6 @@ public interface CSQueue extends SchedulerQueue { */ public float getAbsoluteCapacity(); - public ResourceVector getOrCreateAbsoluteMinCapacityVector(String label); - - public ResourceVector getOrCreateAbsoluteMaxCapacityVector(String label); - /** * Get the configured maximum-capacity of the queue. * @return the configured maximum-capacity of the queue diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUsageTracker.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/CSQueueUsageTracker.java index 326a74477920f..a983540f12600 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/CSQueueUsageTracker.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/CSQueueUsageTracker.java @@ -41,9 +41,6 @@ public class CSQueueUsageTracker { private final QueueResourceQuotas queueResourceQuotas; - private final Map absoluteMinCapacityVector = new HashMap<>(); - private final Map absoluteMaxCapacityVector = new HashMap<>(); - public CSQueueUsageTracker(CSQueueMetrics metrics) { this.metrics = metrics; this.queueUsage = new ResourceUsage(); @@ -82,11 +79,4 @@ public QueueResourceQuotas getQueueResourceQuotas() { return queueResourceQuotas; } - public Map getAbsoluteMinCapacityVector() { - return absoluteMinCapacityVector; - } - - public Map getAbsoluteMaxCapacityVector() { - return absoluteMaxCapacityVector; - } } 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/CalculationContext.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/CalculationContext.java new file mode 100644 index 0000000000000..71c4bd4641556 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CalculationContext.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; + +/** + * A storage class that wraps arguments used in a resource calculation iteration. + */ +public class CalculationContext { + private final String resourceName; + private final QueueCapacityVector.ResourceUnitCapacityType capacityType; + private final CSQueue childQueue; + + public CalculationContext(String resourceName, QueueCapacityVector.ResourceUnitCapacityType capacityType, CSQueue childQueue) { + this.resourceName = resourceName; + this.capacityType = capacityType; + this.childQueue = childQueue; + } + + public String getResourceName() { + return resourceName; + } + + public QueueCapacityVector.ResourceUnitCapacityType getCapacityType() { + return capacityType; + } + + public CSQueue getChildQueue() { + return childQueue; + } + + /** + * A shorthand to return the minimum capacity vector entry for the currently evaluated child and + * resource name. + * + * @param label node label + * @return capacity vector entry + */ + public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { + return childQueue.getConfiguredCapacityVector(label).getResource(resourceName); + } + + /** + * A shorthand to return the maximum capacity vector entry for the currently evaluated child and + * resource name. + * + * @param label node label + * @return capacity vector entry + */ + public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { + return childQueue.getConfiguredMaxCapacityVector(label).getResource(resourceName); + } +} 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index bd4c90689022b..d9844b724aad7 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,7 +38,6 @@ import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; import static org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI; import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; /** * Controls how capacity and resource values are set and calculated for a queue. @@ -70,45 +70,48 @@ public CapacitySchedulerQueueCapacityHandler(RMNodeLabelsManager labelsManager) } /** - * Updates the resource and metrics values for a queue, its siblings and descendants. + * Updates the resource and metrics values of all children under a specific queue. * These values are calculated at runtime. * * @param clusterResource resource of the cluster - * @param queue queue to update + * @param queue parent queue whose children will be updated * @return update context that contains information about the update phase */ - public QueueCapacityUpdateContext update(Resource clusterResource, CSQueue queue) { + public QueueCapacityUpdateContext updateChildren(Resource clusterResource, CSQueue queue) { ResourceLimits resourceLimits = new ResourceLimits(clusterResource); QueueCapacityUpdateContext updateContext = new QueueCapacityUpdateContext(clusterResource, labelsManager); - if (queue.getQueuePath().equals(ROOT)) { - updateRoot(queue, updateContext, resourceLimits); - updateChildren(queue, updateContext, resourceLimits); - } else { - updateChildren(queue.getParent(), updateContext, resourceLimits); - } - + update(queue, updateContext, resourceLimits); return updateContext; } - private void updateRoot( - CSQueue queue, QueueCapacityUpdateContext updateContext, ResourceLimits resourceLimits) { - RootCalculationDriver rootCalculationDriver = new RootCalculationDriver(queue, updateContext, + /** + * Updates the resource and metrics value of the root queue. Root queue always has percentage + * capacity type and is assigned the cluster resource as its minimum and maximum effective + * resource. + * @param rootQueue root queue + * @param clusterResource cluster resource + */ + public void updateRoot(CSQueue rootQueue, Resource clusterResource) { + ResourceLimits resourceLimits = new ResourceLimits(clusterResource); + QueueCapacityUpdateContext updateContext = + new QueueCapacityUpdateContext(clusterResource, labelsManager); + + RootCalculationDriver rootCalculationDriver = new RootCalculationDriver(rootQueue, updateContext, rootCalculator, definedResources); rootCalculationDriver.calculateResources(); - queue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), resourceLimits); + rootQueue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), resourceLimits); } - private void updateChildren( - CSQueue parent, QueueCapacityUpdateContext updateContext, - ResourceLimits resourceLimits) { - if (parent == null || CollectionUtils.isEmpty(parent.getChildQueues())) { + private void update( + CSQueue queue, QueueCapacityUpdateContext updateContext, ResourceLimits resourceLimits) { + if (queue == null || CollectionUtils.isEmpty(queue.getChildQueues())) { return; } ResourceCalculationDriver resourceCalculationDriver = new ResourceCalculationDriver( - parent, updateContext, calculators, definedResources); + queue, updateContext, calculators, definedResources); resourceCalculationDriver.calculateResources(); updateChildrenAfterCalculation(resourceCalculationDriver, resourceLimits); @@ -116,14 +119,84 @@ private void updateChildren( private void updateChildrenAfterCalculation( ResourceCalculationDriver resourceCalculationDriver, ResourceLimits resourceLimits) { - for (CSQueue childQueue : resourceCalculationDriver.getParent().getChildQueues()) { - resourceCalculationDriver.updateChildCapacities(childQueue); - ResourceLimits childLimit = ((ParentQueue) resourceCalculationDriver.getParent()).getResourceLimitsOfChild( - childQueue, resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(), resourceLimits, NO_LABEL, false); + ParentQueue parentQueue = (ParentQueue) resourceCalculationDriver.getQueue(); + for (CSQueue childQueue : parentQueue.getChildQueues()) { + updateChildCapacities(resourceCalculationDriver, childQueue); + + ResourceLimits childLimit = parentQueue.getResourceLimitsOfChild(childQueue, + resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(), + resourceLimits, NO_LABEL, false); childQueue.refreshAfterResourceCalculation(resourceCalculationDriver.getUpdateContext() .getUpdatedClusterResource(), childLimit); - updateChildren(childQueue, resourceCalculationDriver.getUpdateContext(), childLimit); + + update(childQueue, resourceCalculationDriver.getUpdateContext(), childLimit); + } + } + + /** + * Updates the capacity values of the currently evaluated child. + * @param childQueue child queue on which the capacities are set + */ + private void updateChildCapacities(ResourceCalculationDriver resourceCalculationDriver, CSQueue childQueue) { + childQueue.getWriteLock().lock(); + try { + for (String label : childQueue.getConfiguredNodeLabels()) { + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + if (capacityVector.isMixedCapacityVector()) { + // Post update capacities based on the calculated effective resource values + setQueueCapacities(resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label), childQueue, label); + } else { + // Update capacities according to the legacy logic + for (ResourceUnitCapacityType capacityType : + childQueue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); + calculator.updateCapacitiesAfterCalculation(resourceCalculationDriver, childQueue, label); + } + } + } + } finally { + childQueue.getWriteLock().unlock(); + } + } + + /** + * Sets capacity and absolute capacity values of a queue based on minimum and + * maximum effective resources. + * + * @param queue child queue for which the capacities are set + * @param label node label + */ + public static void setQueueCapacities(Resource clusterResource, CSQueue queue, String label) { + if (!(queue instanceof AbstractCSQueue)) { + return; + } + + AbstractCSQueue csQueue = (AbstractCSQueue) queue; + ResourceCalculator resourceCalculator = csQueue.resourceCalculator; + + CSQueue parent = queue.getParent(); + if (parent == null) { + return; } + // Update capacity with a double calculated from the parent's minResources + // and the recently changed queue minResources. + // capacity = effectiveMinResource / {parent's effectiveMinResource} + float result = resourceCalculator.divide(clusterResource, + queue.getQueueResourceQuotas().getEffectiveMinResource(label), + parent.getQueueResourceQuotas().getEffectiveMinResource(label)); + queue.getQueueCapacities().setCapacity(label, + Float.isInfinite(result) ? 0 : result); + + // Update maxCapacity with a double calculated from the parent's maxResources + // and the recently changed queue maxResources. + // maxCapacity = effectiveMaxResource / parent's effectiveMaxResource + result = resourceCalculator.divide(clusterResource, + queue.getQueueResourceQuotas().getEffectiveMaxResource(label), + parent.getQueueResourceQuotas().getEffectiveMaxResource(label)); + queue.getQueueCapacities().setMaximumCapacity(label, + Float.isInfinite(result) ? 0 : result); + + csQueue.updateAbsoluteCapacities(); } private void loadResourceNames() { 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java index 12bc381052948..a1ffcd5e225a4 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java @@ -37,17 +37,20 @@ public class DefaultQueueResourceRoundingStrategy implements QueueResourceRoundi private final ResourceUnitCapacityType lastCapacityType; public DefaultQueueResourceRoundingStrategy( - Set capacityTypePrecedence) { - lastCapacityType = capacityTypePrecedence.stream().reduce((c1, c2) -> c2).orElseThrow(() - -> new IllegalArgumentException("Capacity type precedence collection is empty")); + ResourceUnitCapacityType[] capacityTypePrecedence) { + if (capacityTypePrecedence.length == 0) { + throw new IllegalArgumentException("Capacity type precedence collection is empty"); + } + + lastCapacityType = capacityTypePrecedence[capacityTypePrecedence.length - 1]; } @Override - public float getRoundedResource(float resourceValue, QueueCapacityVectorEntry capacityVectorEntry) { + public double getRoundedResource(double resourceValue, QueueCapacityVectorEntry capacityVectorEntry) { if (capacityVectorEntry.getVectorResourceType().equals(lastCapacityType)) { return Math.round(resourceValue); } else { - return (float) Math.floor(resourceValue); + return Math.floor(resourceValue); } } } 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index 52c2e3bde1764..b9619aa59aabd 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -19,21 +19,18 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; public class PercentageQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public float calculateMinimumResource( + public double calculateMinimumResource( ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { - CSQueue parentQueue = resourceCalculationDriver.getParent(); String resourceName = context.getResourceName(); - float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label).getValue( - resourceName); - float remainingPerEffectiveResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( + double parentAbsoluteCapacity = resourceCalculationDriver.getParentAbsoluteMinCapacity(label, resourceName); + double remainingPerEffectiveResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( label, resourceName); - float absoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio + double absoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio * context.getCurrentMinimumCapacityEntry(label).getResourceValue() / 100; return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) @@ -41,14 +38,12 @@ public float calculateMinimumResource( } @Override - public float calculateMaximumResource( + public double calculateMaximumResource( ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { - CSQueue parentQueue = resourceCalculationDriver.getParent(); String resourceName = context.getResourceName(); - float parentAbsoluteMaxCapacity = parentQueue.getOrCreateAbsoluteMaxCapacityVector(label) - .getValue(resourceName); - float absoluteMaxCapacity = parentAbsoluteMaxCapacity + double parentAbsoluteMaxCapacity = resourceCalculationDriver.getParentAbsoluteMaxCapacity(label, resourceName); + double absoluteMaxCapacity = parentAbsoluteMaxCapacity * context.getCurrentMaximumCapacityEntry(label).getResourceValue() / 100; return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(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/QueueCapacityVector.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/QueueCapacityVector.java index c585efd9892aa..bcce996b27990 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/QueueCapacityVector.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/QueueCapacityVector.java @@ -61,7 +61,7 @@ private QueueCapacityVector(ResourceVector resource) { public static QueueCapacityVector newInstance() { QueueCapacityVector newCapacityVector = new QueueCapacityVector(ResourceVector.newInstance()); - for (Map.Entry resourceEntry : newCapacityVector.resource) { + for (Map.Entry resourceEntry : newCapacityVector.resource) { newCapacityVector.storeResourceType(resourceEntry.getKey(), ResourceUnitCapacityType.ABSOLUTE); } @@ -78,10 +78,10 @@ public static QueueCapacityVector newInstance() { * @return uniform capacity vector */ public static QueueCapacityVector of( - float value, ResourceUnitCapacityType capacityType) { + double value, ResourceUnitCapacityType capacityType) { QueueCapacityVector newCapacityVector = new QueueCapacityVector(ResourceVector.of(value)); - for (Map.Entry resourceEntry : newCapacityVector.resource) { + for (Map.Entry resourceEntry : newCapacityVector.resource) { newCapacityVector.storeResourceType(resourceEntry.getKey(), capacityType); } @@ -109,7 +109,7 @@ public int getResourceCount() { * @param value value of the resource * @param capacityType type of the resource */ - public void setResource(String resourceName, float value, + public void setResource(String resourceName, double value, ResourceUnitCapacityType capacityType) { // Necessary due to backward compatibility (memory = memory-mb) String convertedResourceName = resourceName; @@ -125,7 +125,7 @@ public void setResource(String resourceName, float value, * * @return value of memory resource */ - public float getMemory() { + public double getMemory() { return resource.getValue(ResourceInformation.MEMORY_URI); } @@ -162,7 +162,7 @@ public boolean isResourceOfType( @Override public Iterator iterator() { return new Iterator() { - private final Iterator> resources = + private final Iterator> resources = resource.iterator(); private int i = 0; @@ -173,7 +173,7 @@ public boolean hasNext() { @Override public QueueCapacityVectorEntry next() { - Map.Entry resourceInformation = resources.next(); + Map.Entry resourceInformation = resources.next(); i++; return new QueueCapacityVectorEntry( capacityTypes.get(resourceInformation.getKey()), @@ -223,7 +223,7 @@ public String toString() { stringVector.append(START_PARENTHESES); int resourceCount = 0; - for (Map.Entry resourceEntry : resource) { + for (Map.Entry resourceEntry : resource) { resourceCount++; stringVector.append(resourceEntry.getKey()) .append(VALUE_DELIMITER) @@ -257,11 +257,11 @@ public String getPostfix() { public static class QueueCapacityVectorEntry { private final ResourceUnitCapacityType vectorResourceType; - private final float resourceValue; + private final double resourceValue; private final String resourceName; public QueueCapacityVectorEntry(ResourceUnitCapacityType vectorResourceType, - String resourceName, float resourceValue) { + String resourceName, double resourceValue) { this.vectorResourceType = vectorResourceType; this.resourceValue = resourceValue; this.resourceName = resourceName; @@ -271,7 +271,7 @@ public ResourceUnitCapacityType getVectorResourceType() { return vectorResourceType; } - public float getResourceValue() { + public double getResourceValue() { return resourceValue; } 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/QueueResourceRoundingStrategy.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/QueueResourceRoundingStrategy.java index e81cef8abc306..ef753316e8435 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/QueueResourceRoundingStrategy.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/QueueResourceRoundingStrategy.java @@ -32,5 +32,5 @@ public interface QueueResourceRoundingStrategy { * @param capacityVectorEntry configured capacity entry * @return rounded resource value */ - float getRoundedResource(float resourceValue, QueueCapacityVectorEntry capacityVectorEntry); + double getRoundedResource(double resourceValue, QueueCapacityVectorEntry capacityVectorEntry); } 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index bb939507e5ea4..176539969e0b3 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -20,149 +20,63 @@ import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hadoop.yarn.api.records.Resource; 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.util.resource.ResourceCalculator; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType; +import org.apache.hadoop.yarn.util.UnitsConversionUtil; import java.util.Collection; import java.util.HashMap; import java.util.Map; -import java.util.Set; import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; /** - * Drives the main logic of resource calculation for all children under a parent queue. Acts as a - * bookkeeper of disposable update information that is used by all children under a common parent. + * Drives the main logic of resource calculation for all children under a queue. Acts as a + * bookkeeper of disposable update information that is used by all children under the common parent. */ public class ResourceCalculationDriver { - protected static final Set CALCULATOR_PRECEDENCE = - ImmutableSet.of( + protected static final ResourceUnitCapacityType[] CALCULATOR_PRECEDENCE = + new ResourceUnitCapacityType[] { ResourceUnitCapacityType.ABSOLUTE, ResourceUnitCapacityType.PERCENTAGE, - ResourceUnitCapacityType.WEIGHT); + ResourceUnitCapacityType.WEIGHT}; static final String MB_UNIT = "Mi"; protected final QueueResourceRoundingStrategy roundingStrategy = new DefaultQueueResourceRoundingStrategy(CALCULATOR_PRECEDENCE); - protected final CSQueue parent; + protected final CSQueue queue; protected final QueueCapacityUpdateContext updateContext; protected final Map calculators; protected final Collection definedResources; - protected final Map overallRemainingResource = new HashMap<>(); - protected final Map batchRemainingResource = new HashMap<>(); + protected final Map overallRemainingResourcePerLabel = new HashMap<>(); + protected final Map batchRemainingResourcePerLabel = new HashMap<>(); // Used by ABSOLUTE capacity types - protected final Map normalizedResourceRatio = new HashMap<>(); - // Used by WEIGHT capacity typet js - protected final Map> sumWeightsPerLabel = new HashMap<>(); - protected Map usedResourceByCurrentCalculatorPerLabel = new HashMap<>(); + protected final Map normalizedResourceRatioPerLabel = new HashMap<>(); + // Used by WEIGHT capacity types + protected final Map> sumWeightsPerLabel = new HashMap<>(); + protected Map usedResourceByCurrentCalculatorPerLabel = new HashMap<>(); public ResourceCalculationDriver( - CSQueue parent, QueueCapacityUpdateContext updateContext, + CSQueue queue, QueueCapacityUpdateContext updateContext, Map calculators, Collection definedResources) { - this.parent = parent; + this.queue = queue; this.updateContext = updateContext; this.calculators = calculators; this.definedResources = definedResources; + setNormalizedResourceRatio(); } - public static class CalculationContext { - private String resourceName; - private ResourceUnitCapacityType capacityType; - private CSQueue childQueue; - - public CalculationContext(String resourceName, ResourceUnitCapacityType capacityType, CSQueue childQueue) { - this.resourceName = resourceName; - this.capacityType = capacityType; - this.childQueue = childQueue; - } - - public String getResourceName() { - return resourceName; - } - - public ResourceUnitCapacityType getCapacityType() { - return capacityType; - } - - public CSQueue getChildQueue() { - return childQueue; - } - - /** - * A shorthand to return the minimum capacity vector entry for the currently evaluated child and - * resource name. - * - * @param label node label - * @return capacity vector entry - */ - public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { - return childQueue.getConfiguredCapacityVector(label).getResource(resourceName); - } - - /** - * A shorthand to return the maximum capacity vector entry for the currently evaluated child and - * resource name. - * - * @param label node label - * @return capacity vector entry - */ - public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { - return childQueue.getConfiguredMaxCapacityVector(label).getResource(resourceName); - } - } - - /** - * Sets capacity and absolute capacity values based on minimum and maximum effective resources. - * - * @param queue child queue for which the capacities are set - * @param label node label - */ - public void setQueueCapacities(CSQueue queue, String label) { - if (!(queue instanceof AbstractCSQueue)) { - return; - } - - Resource clusterResource = updateContext.getUpdatedClusterResource(label); - AbstractCSQueue csQueue = (AbstractCSQueue) queue; - ResourceCalculator resourceCalculator = csQueue.resourceCalculator; - - CSQueue parent = queue.getParent(); - if (parent == null) { - return; - } - // Update capacity with a float calculated from the parent's minResources - // and the recently changed queue minResources. - // capacity = effectiveMinResource / {parent's effectiveMinResource} - float result = resourceCalculator.divide(clusterResource, - queue.getQueueResourceQuotas().getEffectiveMinResource(label), - parent.getQueueResourceQuotas().getEffectiveMinResource(label)); - queue.getQueueCapacities().setCapacity(label, - Float.isInfinite(result) ? 0 : result); - - // Update maxCapacity with a float calculated from the parent's maxResources - // and the recently changed queue maxResources. - // maxCapacity = effectiveMaxResource / parent's effectiveMaxResource - result = resourceCalculator.divide(clusterResource, - queue.getQueueResourceQuotas().getEffectiveMaxResource(label), - parent.getQueueResourceQuotas().getEffectiveMaxResource(label)); - queue.getQueueCapacities().setMaximumCapacity(label, - Float.isInfinite(result) ? 0 : result); - - csQueue.updateAbsoluteCapacities(); - } /** * Returns the parent that is driving the calculation. * * @return a common parent queue */ - public CSQueue getParent() { - return parent; + public CSQueue getQueue() { + return queue; } /** @@ -181,10 +95,10 @@ public QueueCapacityUpdateContext getUpdateContext() { * @param resourceName resource unit name * @param value weight value */ - public void incrementWeight(String label, String resourceName, float value) { + public void incrementWeight(String label, String resourceName, double value) { sumWeightsPerLabel.putIfAbsent(label, new HashMap<>()); sumWeightsPerLabel.get(label).put(resourceName, - sumWeightsPerLabel.get(label).getOrDefault(resourceName, 0f) + value); + sumWeightsPerLabel.get(label).getOrDefault(resourceName, 0d) + value); } /** @@ -194,17 +108,57 @@ public void incrementWeight(String label, String resourceName, float value) { * @param resourceName resource unit name * @return aggregated weights of children */ - public float getSumWeightsByResource(String label, String resourceName) { + public double getSumWeightsByResource(String label, String resourceName) { return sumWeightsPerLabel.get(label).get(resourceName); } + /** + * Returns the ratio of the summary of children absolute configured resources and the parent's + * effective minimum resource. + * + * @return normalized resource ratio for all labels + */ public Map getNormalizedResourceRatios() { - return normalizedResourceRatio; + return normalizedResourceRatioPerLabel; + } + + /** + * Returns the remaining resource ratio under the parent queue. The remaining resource is only + * decremented after a capacity type is fully evaluated. + * + * @param label node label + * @param resourceName name of resource unit + * @return resource ratio + */ + public double getRemainingRatioOfResource(String label, String resourceName) { + return batchRemainingResourcePerLabel.get(label).getValue(resourceName) + / queue.getEffectiveCapacity(label).getResourceValue(resourceName); + } + + /** + * Returns the ratio of the parent queue's effective minimum resource relative to the full cluster + * resource. + * + * @param label node label + * @param resourceName name of resource unit + * @return absolute minimum capacity + */ + public double getParentAbsoluteMinCapacity(String label, String resourceName) { + return (double) queue.getEffectiveCapacity(label).getResourceValue(resourceName) + / getUpdateContext().getUpdatedClusterResource(label).getResourceValue(resourceName); } - public float getRemainingRatioOfResource(String label, String resourceName) { - return batchRemainingResource.get(label).getValue(resourceName) - / parent.getEffectiveCapacity(label).getResourceValue(resourceName); + /** + * Returns the ratio of the parent queue's effective maximum resource relative to the full cluster + * resource. + * + * @param label node label + * @param resourceName name of resource unit + * @return absolute maximum capacity + */ + public double getParentAbsoluteMaxCapacity(String label, String resourceName) { + return (double) queue.getEffectiveMaxCapacity(label).getResourceValue(resourceName) + / getUpdateContext().getUpdatedClusterResource(label).getResourceValue(resourceName); } /** @@ -216,8 +170,8 @@ public float getRemainingRatioOfResource(String label, String resourceName) { * @return remaining resources */ public ResourceVector getBatchRemainingResource(String label) { - batchRemainingResource.putIfAbsent(label, ResourceVector.newInstance()); - return batchRemainingResource.get(label); + batchRemainingResourcePerLabel.putIfAbsent(label, ResourceVector.newInstance()); + return batchRemainingResourcePerLabel.get(label); } /** @@ -226,9 +180,9 @@ public ResourceVector getBatchRemainingResource(String label) { */ public void calculateResources() { // Reset both remaining resource storage to the parent's available resource - for (String label : parent.getConfiguredNodeLabels()) { - overallRemainingResource.put(label, ResourceVector.of(parent.getEffectiveCapacity(label))); - batchRemainingResource.put(label, ResourceVector.of(parent.getEffectiveCapacity(label))); + for (String label : queue.getConfiguredNodeLabels()) { + overallRemainingResourcePerLabel.put(label, ResourceVector.of(queue.getEffectiveCapacity(label))); + batchRemainingResourcePerLabel.put(label, ResourceVector.of(queue.getEffectiveCapacity(label))); } for (AbstractQueueCapacityCalculator capacityCalculator : calculators.values()) { @@ -237,14 +191,16 @@ public void calculateResources() { for (String resourceName : definedResources) { for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { - for (CSQueue childQueue : parent.getChildQueues()) { + for (CSQueue childQueue : queue.getChildQueues()) { CalculationContext context = new CalculationContext(resourceName, capacityType, childQueue); calculateResourceOnChild(context); } + // Flush aggregated used resource by labels at the end of a calculator phase - for (Map.Entry entry : usedResourceByCurrentCalculatorPerLabel.entrySet()) { - batchRemainingResource.get(entry.getKey()).decrement(resourceName, entry.getValue()); + for (Map.Entry entry : usedResourceByCurrentCalculatorPerLabel.entrySet()) { + batchRemainingResourcePerLabel.get(entry.getKey()).decrement(resourceName, entry.getValue()); } + usedResourceByCurrentCalculatorPerLabel = new HashMap<>(); } } @@ -252,31 +208,6 @@ public void calculateResources() { validateRemainingResource(); } - /** - * Updates the capacity values of the currently evaluated child. - */ - public void updateChildCapacities(CSQueue childQueue) { - childQueue.getWriteLock().lock(); - try { - for (String label : childQueue.getConfiguredNodeLabels()) { - QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); - if (capacityVector.isMixedCapacityVector()) { - // Post update capacities based on the calculated effective resource values - setQueueCapacities(childQueue, label); - } else { - // Update capacities according to the legacy logic - for (ResourceUnitCapacityType capacityType : - childQueue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { - AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); - calculator.updateCapacitiesAfterCalculation(this, childQueue, label); - } - } - } - } finally { - childQueue.getWriteLock().unlock(); - } - } - private void calculateResourceOnChild(CalculationContext context) { context.getChildQueue().getWriteLock().lock(); try { @@ -285,12 +216,12 @@ private void calculateResourceOnChild(CalculationContext context) { context.getCapacityType())) { continue; } - float usedResourceByChild = setChildResources(context, label); - float aggregatedUsedResource = usedResourceByCurrentCalculatorPerLabel.getOrDefault(label, - 0f); - float resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; + double usedResourceByChild = setChildResources(context, label); + double aggregatedUsedResource = usedResourceByCurrentCalculatorPerLabel.getOrDefault(label, + 0d); + double resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; - overallRemainingResource.get(label).decrement(context.getResourceName(), usedResourceByChild); + overallRemainingResourcePerLabel.get(label).decrement(context.getResourceName(), usedResourceByChild); usedResourceByCurrentCalculatorPerLabel.put(label, resourceUsedByLabel); } } finally { @@ -298,34 +229,25 @@ private void calculateResourceOnChild(CalculationContext context) { } } - private float setChildResources(CalculationContext context, String label) { + private double setChildResources(CalculationContext context, String label) { QueueCapacityVectorEntry capacityVectorEntry = context.getChildQueue().getConfiguredCapacityVector( label).getResource(context.getResourceName()); - long clusterResource = updateContext.getUpdatedClusterResource(label).getResourceValue( - context.getResourceName()); QueueCapacityVectorEntry maximumCapacityVectorEntry = context.getChildQueue() .getConfiguredMaxCapacityVector(label).getResource(context.getResourceName()); AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( maximumCapacityVectorEntry.getVectorResourceType()); - float minimumResource = calculators.get(context.getCapacityType()).calculateMinimumResource(this, context, label); - float maximumResource = maximumCapacityCalculator.calculateMaximumResource(this, context, label); + double minimumResource = calculators.get(context.getCapacityType()).calculateMinimumResource(this, context, label); + double maximumResource = maximumCapacityCalculator.calculateMaximumResource(this, context, label); minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); maximumResource = roundingStrategy.getRoundedResource(maximumResource, maximumCapacityVectorEntry); - Pair resources = validateCalculatedResources(context, label, new ImmutablePair<>( + Pair resources = validateCalculatedResources(context, label, new ImmutablePair<>( minimumResource, maximumResource)); minimumResource = resources.getLeft(); maximumResource = resources.getRight(); - float absoluteMinCapacity = minimumResource / clusterResource; - float absoluteMaxCapacity = maximumResource / clusterResource; - context.getChildQueue().getOrCreateAbsoluteMinCapacityVector(label).setValue( - context.getResourceName(), absoluteMinCapacity); - context.getChildQueue().getOrCreateAbsoluteMaxCapacityVector(label).setValue( - context.getResourceName(), absoluteMaxCapacity); - context.getChildQueue().getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( context.getResourceName(), (long) minimumResource); context.getChildQueue().getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( @@ -334,18 +256,18 @@ private float setChildResources(CalculationContext context, String label) { return minimumResource; } - private Pair validateCalculatedResources(CalculationContext context, - String label, Pair calculatedResources) { - float minimumResource = calculatedResources.getLeft(); + private Pair validateCalculatedResources(CalculationContext context, + String label, Pair calculatedResources) { + double minimumResource = calculatedResources.getLeft(); long minimumMemoryResource = context.getChildQueue().getQueueResourceQuotas().getEffectiveMinResource(label) .getMemorySize(); - float remainingResourceUnderParent = overallRemainingResource.get(label).getValue( + double remainingResourceUnderParent = overallRemainingResourcePerLabel.get(label).getValue( context.getResourceName()); - long parentMaximumResource = parent.getEffectiveMaxCapacity(label).getResourceValue( + long parentMaximumResource = queue.getEffectiveMaxCapacity(label).getResourceValue( context.getResourceName()); - float maximumResource = calculatedResources.getRight(); + double maximumResource = calculatedResources.getRight(); // Memory is the primary resource, if its zero, all other resource units are zero as well. if (!context.getResourceName().equals(MEMORY_URI) && minimumMemoryResource == 0) { @@ -353,32 +275,32 @@ private Pair validateCalculatedResources(CalculationContext contex } if (maximumResource != 0 && maximumResource > parentMaximumResource) { - updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( context.getChildQueue().getQueuePath())); } maximumResource = maximumResource == 0 ? parentMaximumResource : Math.min(maximumResource, parentMaximumResource); if (maximumResource < minimumResource) { - updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( context.getChildQueue().getQueuePath())); minimumResource = maximumResource; } if (minimumResource > remainingResourceUnderParent) { // Legacy auto queues are assigned a zero resource if not enough resource is left - if (parent instanceof ManagedParentQueue) { + if (queue instanceof ManagedParentQueue) { minimumResource = 0; } else { updateContext.addUpdateWarning( - QueueUpdateWarning.QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(context.getChildQueue().getQueuePath()).withInfo( + QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(context.getChildQueue().getQueuePath()).withInfo( "Resource name: " + context.getResourceName() + " resource value: " + minimumResource)); minimumResource = remainingResourceUnderParent; } } if (minimumResource == 0) { - updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( context.getChildQueue().getQueuePath()).withInfo("Resource name: " + context.getResourceName())); } @@ -386,10 +308,73 @@ private Pair validateCalculatedResources(CalculationContext contex } private void validateRemainingResource() { - for (String label : parent.getConfiguredNodeLabels()) { - if (!batchRemainingResource.get(label).equals(ResourceVector.newInstance())) { - updateContext.addUpdateWarning(QueueUpdateWarning.QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( - parent.getQueuePath()).withInfo("Label: " + label)); + for (String label : queue.getConfiguredNodeLabels()) { + if (!batchRemainingResourcePerLabel.get(label).equals(ResourceVector.newInstance())) { + updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_UNDERUTILIZED.ofQueue( + queue.getQueuePath()).withInfo("Label: " + label)); + } + } + } + + /** + * Calculates the normalized resource ratio of a parent queue, under which children are defined + * with absolute capacity type. If the effective resource of the parent is less, than the + * aggregated configured absolute resource of its children, the resource ratio will be less, + * than 1. + * + */ + private void setNormalizedResourceRatio() { + for (String label : getQueue().getConfiguredNodeLabels()) { + // ManagedParents assign zero capacity to queues in case of overutilization, downscaling is + // turned off for their children + if (queue instanceof ManagedParentQueue) { + return; + } + + for (QueueCapacityVectorEntry capacityVectorEntry : queue.getConfiguredCapacityVector( + label)) { + String resourceName = capacityVectorEntry.getResourceName(); + long childrenConfiguredResource = 0; + long effectiveMinResource = queue.getQueueResourceQuotas().getEffectiveMinResource( + label).getResourceValue(resourceName); + + // Total configured min resources of direct children of this given parent + // queue + for (CSQueue childQueue : queue.getChildQueues()) { + if (!childQueue.getConfiguredNodeLabels().contains(label)) { + continue; + } + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + if (capacityVector.isResourceOfType(resourceName, ResourceUnitCapacityType.ABSOLUTE)) { + childrenConfiguredResource += capacityVector.getResource(resourceName).getResourceValue(); + } + } + // If no children is using ABSOLUTE capacity type, normalization is not needed + if (childrenConfiguredResource == 0) { + continue; + } + // Factor to scale down effective resource: When cluster has sufficient + // resources, effective_min_resources will be same as configured + // min_resources. + float numeratorForMinRatio = childrenConfiguredResource; + if (effectiveMinResource < childrenConfiguredResource) { + numeratorForMinRatio = queue.getQueueResourceQuotas().getEffectiveMinResource(label) + .getResourceValue(resourceName); + updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_DOWNSCALED.ofQueue( + queue.getQueuePath())); + } + + String unit = resourceName.equals(MEMORY_URI) ? MB_UNIT : ""; + long convertedValue = UnitsConversionUtil.convert(unit, + updateContext.getUpdatedClusterResource(label).getResourceInformation(resourceName) + .getUnits(), childrenConfiguredResource); + + if (convertedValue != 0) { + Map normalizedResourceRatios = getNormalizedResourceRatios(); + normalizedResourceRatios.putIfAbsent(label, ResourceVector.newInstance()); + normalizedResourceRatios.get(label).setValue(resourceName, numeratorForMinRatio / + convertedValue); + } } } } 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/ResourceVector.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/ResourceVector.java index df1eac8aa3c4f..8a417b0e66b01 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/ResourceVector.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/ResourceVector.java @@ -28,11 +28,10 @@ import java.util.Set; /** - * Represents a simple resource floating point value storage - * grouped by resource names. + * Represents a simple resource floating point value grouped by resource names. */ -public class ResourceVector implements Iterable> { - private final Map resourcesByName = new HashMap<>(); +public class ResourceVector implements Iterable> { + private final Map resourcesByName = new HashMap<>(); /** * Creates a new {@code ResourceVector} with all pre-defined resources set to @@ -54,7 +53,7 @@ public static ResourceVector newInstance() { * @param value the value to set all resources to * @return uniform resource vector */ - public static ResourceVector of(float value) { + public static ResourceVector of(double value) { ResourceVector emptyResourceVector = new ResourceVector(); for (ResourceInformation resource : ResourceUtils.getResourceTypesArray()) { emptyResourceVector.setValue(resource.getName(), value); @@ -84,7 +83,7 @@ public static ResourceVector of(Resource resource) { * @param otherResourceVector rhs resource vector of the subtraction */ public void decrement(ResourceVector otherResourceVector) { - for (Map.Entry resource : otherResourceVector) { + for (Map.Entry resource : otherResourceVector) { setValue(resource.getKey(), getValue(resource.getKey()) - resource.getValue()); } } @@ -94,7 +93,7 @@ public void decrement(ResourceVector otherResourceVector) { * @param resourceName name of the resource * @param value value to be subtracted from the resource's current value */ - public void decrement(String resourceName, float value) { + public void decrement(String resourceName, double value) { setValue(resourceName, getValue(resourceName) - value); } @@ -103,15 +102,15 @@ public void decrement(String resourceName, float value) { * @param resourceName name of the resource * @param value value to be added to the resource's current value */ - public void increment(String resourceName, float value) { + public void increment(String resourceName, double value) { setValue(resourceName, getValue(resourceName) + value); } - public float getValue(String resourceName) { + public double getValue(String resourceName) { return resourcesByName.get(resourceName); } - public void setValue(String resourceName, float value) { + public void setValue(String resourceName, double value) { resourcesByName.put(resourceName, value); } @@ -124,7 +123,7 @@ public Set getResourceNames() { } @Override - public Iterator> iterator() { + public Iterator> iterator() { return resourcesByName.entrySet().iterator(); } 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/RootCalculationDriver.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/RootCalculationDriver.java index 9a7b03e385d51..1bed62657e05e 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/RootCalculationDriver.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/RootCalculationDriver.java @@ -38,23 +38,21 @@ public RootCalculationDriver(CSQueue rootQueue, QueueCapacityUpdateContext updat @Override public void calculateResources() { - for (String label : parent.getConfiguredNodeLabels()) { - for (QueueCapacityVector.QueueCapacityVectorEntry capacityVectorEntry : parent.getConfiguredCapacityVector(label)) { + for (String label : queue.getConfiguredNodeLabels()) { + for (QueueCapacityVector.QueueCapacityVectorEntry capacityVectorEntry : queue.getConfiguredCapacityVector(label)) { String resourceName = capacityVectorEntry.getResourceName(); - parent.getOrCreateAbsoluteMinCapacityVector(label).setValue(resourceName, 1); - parent.getOrCreateAbsoluteMaxCapacityVector(label).setValue(resourceName, 1); - CalculationContext context = new CalculationContext(resourceName, PERCENTAGE, parent); - float minimumResource = rootCalculator.calculateMinimumResource(this, context, label); - float maximumResource = rootCalculator.calculateMaximumResource(this, context, label); + CalculationContext context = new CalculationContext(resourceName, PERCENTAGE, queue); + double minimumResource = rootCalculator.calculateMinimumResource(this, context, label); + double maximumResource = rootCalculator.calculateMaximumResource(this, context, label); long roundedMinResource = (long) roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); - long roundedMaxResource = (long) roundingStrategy.getRoundedResource(maximumResource, parent.getConfiguredMaxCapacityVector(label).getResource(resourceName)); - parent.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + long roundedMaxResource = (long) roundingStrategy.getRoundedResource(maximumResource, queue.getConfiguredMaxCapacityVector(label).getResource(resourceName)); + queue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( resourceName, roundedMinResource); - parent.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + queue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( resourceName, roundedMaxResource); } - rootCalculator.updateCapacitiesAfterCalculation(this, parent, label); + rootCalculator.updateCapacitiesAfterCalculation(this, queue, label); } rootCalculator.calculateResourcePrerequisites(this); 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index 0c1547e0cef4c..e4a0c4b019ce8 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -18,20 +18,18 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; - import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; public class RootQueueCapacityCalculator extends AbstractQueueCapacityCalculator { @Override - public float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + public double calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(context.getResourceName()); } @Override - public float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + public double calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(context.getResourceName()); } 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 1c03f73f5f7e9..23a51167f76e5 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.CalculationContext; import java.util.Collection; @@ -31,7 +30,8 @@ public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculat public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { super.calculateResourcePrerequisites(resourceCalculationDriver); - for (CSQueue childQueue : resourceCalculationDriver.getParent().getChildQueues()) { + // Precalculate the summary of children's weight + for (CSQueue childQueue : resourceCalculationDriver.getQueue().getChildQueues()) { for (String label : childQueue.getConfiguredNodeLabels()) { for (String resourceName : childQueue.getConfiguredCapacityVector(label) .getResourceNamesByCapacityType(getCapacityType())) { @@ -43,15 +43,14 @@ public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCal } @Override - public float calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + public double calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { - CSQueue parentQueue = resourceCalculationDriver.getParent(); String resourceName = context.getResourceName(); - float normalizedWeight = context.getCurrentMinimumCapacityEntry(label) + double normalizedWeight = context.getCurrentMinimumCapacityEntry(label) .getResourceValue() / resourceCalculationDriver.getSumWeightsByResource(label, resourceName); - float remainingResource = resourceCalculationDriver.getBatchRemainingResource(label).getValue( + double remainingResource = resourceCalculationDriver.getBatchRemainingResource(label).getValue( resourceName); // Due to rounding loss it is better to use all remaining resources if no other resource uses @@ -60,24 +59,19 @@ public float calculateMinimumResource(ResourceCalculationDriver resourceCalculat return remainingResource; } - float remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( + double remainingResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( label, resourceName); - - float parentAbsoluteCapacity = parentQueue.getOrCreateAbsoluteMinCapacityVector(label) - .getValue(resourceName); - float queueAbsoluteCapacity = parentAbsoluteCapacity * remainingResourceRatio + double parentAbsoluteCapacity = resourceCalculationDriver.getParentAbsoluteMinCapacity( + label, resourceName); + double queueAbsoluteCapacity = parentAbsoluteCapacity * remainingResourceRatio * normalizedWeight; - // Weight capacity types are the last to consider, therefore it is safe to assign all remaining - // effective resources between queues. The strategy is to round values to the closest whole - // number. - return resourceCalculationDriver.getUpdateContext() .getUpdatedClusterResource(label).getResourceValue(resourceName) * queueAbsoluteCapacity; } @Override - public float calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + public double calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { throw new IllegalStateException("Resource " + context.getCurrentMinimumCapacityEntry( @@ -92,17 +86,17 @@ public ResourceUnitCapacityType getCapacityType() { @Override public void updateCapacitiesAfterCalculation( ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { - float sumCapacityPerResource = 0f; + double sumCapacityPerResource = 0f; Collection resourceNames = getResourceNames(queue, label); for (String resourceName : resourceNames) { - float sumBranchWeight = resourceCalculationDriver.getSumWeightsByResource(label, resourceName); - float capacity = queue.getConfiguredCapacityVector( + double sumBranchWeight = resourceCalculationDriver.getSumWeightsByResource(label, resourceName); + double capacity = queue.getConfiguredCapacityVector( label).getResource(resourceName).getResourceValue() / sumBranchWeight; sumCapacityPerResource += capacity; } - queue.getQueueCapacities().setNormalizedWeight(label, sumCapacityPerResource / resourceNames.size()); + queue.getQueueCapacities().setNormalizedWeight(label, (float) (sumCapacityPerResource / resourceNames.size())); ((AbstractCSQueue) queue).updateAbsoluteCapacities(); } } 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 e760b0586539e..f62945c7a5a11 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 @@ -96,8 +96,9 @@ protected QueueCapacityUpdateContext update( new CapacitySchedulerQueueCapacityHandler(mgr); mgr.setResourceForLabel(CommonNodeLabelsManager.NO_LABEL, emptyLabelResource); + queueController.updateRoot(cs.getQueue("root"), clusterResource); QueueCapacityUpdateContext updateContext = - queueController.update(clusterResource, cs.getQueue("root")); + queueController.updateChildren(clusterResource, cs.getQueue("root")); assertions.finishAssertion(); 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 1a25d4bc8d680..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 @@ -101,7 +101,7 @@ public QueueAssertion assertEffectiveMinResource(Resource expected) { return assertEffectiveMinResource(expected, NO_LABEL); } - public QueueAssertion assertCapacity(float expected) { + public QueueAssertion assertCapacity(double expected) { ValueAssertion valueAssertion = new ValueAssertion(expected); valueAssertion.withCapacitySupplier(CAPACITY, CAPACITY_INFO); assertions.add(valueAssertion); @@ -109,7 +109,7 @@ public QueueAssertion assertCapacity(float expected) { return this; } - public QueueAssertion assertAbsoluteCapacity(float expected) { + public QueueAssertion assertAbsoluteCapacity(double expected) { ValueAssertion valueAssertion = new ValueAssertion(expected); valueAssertion.withCapacitySupplier(ABS_CAPACITY, ABS_CAPACITY_INFO); assertions.add(valueAssertion); @@ -118,14 +118,14 @@ public QueueAssertion assertAbsoluteCapacity(float expected) { } private class ValueAssertion { - private float expectedValue = 0; + private double expectedValue = 0; private Resource expectedResource = null; private String assertionType; private Supplier valueSupplier; private Supplier resourceSupplier; private String label = ""; - ValueAssertion(float expectedValue) { + ValueAssertion(double expectedValue) { this.expectedValue = expectedValue; } 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 54210aae98270..e5b7cc964e3ab 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 @@ -359,10 +359,10 @@ public void testDifferentMinimumAndMaximumCapacityTypes() throws IOException { QueueAssertionBuilder assertionBuilder = createAssertionBuilder() .withQueue(A) - .assertEffectiveMinResource(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(UPDATE_RESOURCE, 0.5d)) .assertEffectiveMaxResource(A_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX) .withQueue(B) - .assertEffectiveMinResource(ResourceUtils.multiply(UPDATE_RESOURCE, 0.5f)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(UPDATE_RESOURCE, 0.5d)) .assertEffectiveMaxResource(B_EXPECTED_MAX_RESOURCE_DIFFERENT_MIN_MAX) .build(); 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 64c0cdd340ebd..17c4aa763aad0 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 @@ -43,12 +43,12 @@ public class TestUniformQueueResourceCalculation extends CapacitySchedulerQueueC private static final Resource UPDATE_RES = Resource.newInstance(250 * GB, 40); private static final Resource PERCENTAGE_ALL_RES = Resource.newInstance(10 * GB, 20); - public static final float A_CAPACITY = 0.3f; - public static final float B_CAPACITY = 0.7f; - public static final float A1_CAPACITY = 0.17f; - public static final float A11_CAPACITY = 0.25f; - public static final float A12_CAPACITY = 0.75f; - public static final float A2_CAPACITY = 0.83f; + public static final double A_CAPACITY = 0.3; + public static final double B_CAPACITY = 0.7; + public static final double A1_CAPACITY = 0.17; + public static final double A11_CAPACITY = 0.25; + public static final double A12_CAPACITY = 0.75; + public static final double A2_CAPACITY = 0.83; public static final float A_WEIGHT = 3; public static final float B_WEIGHT = 6; @@ -57,12 +57,12 @@ public class TestUniformQueueResourceCalculation extends CapacitySchedulerQueueC public static final float A12_WEIGHT = 8; public static final float A2_WEIGHT = 3; - public static final float A_NORMALIZED_WEIGHT = A_WEIGHT / (A_WEIGHT + B_WEIGHT); - public static final float B_NORMALIZED_WEIGHT = B_WEIGHT / (A_WEIGHT + B_WEIGHT); - public static final float A1_NORMALIZED_WEIGHT = A1_WEIGHT / (A1_WEIGHT + A2_WEIGHT); - public static final float A2_NORMALIZED_WEIGHT = A2_WEIGHT / (A1_WEIGHT + A2_WEIGHT); - public static final float A11_NORMALIZED_WEIGHT = A11_WEIGHT / (A11_WEIGHT + A12_WEIGHT); - public static final float A12_NORMALIZED_WEIGHT = A12_WEIGHT / (A11_WEIGHT + A12_WEIGHT); + public static final double A_NORMALIZED_WEIGHT = A_WEIGHT / (A_WEIGHT + B_WEIGHT); + public static final double B_NORMALIZED_WEIGHT = B_WEIGHT / (A_WEIGHT + B_WEIGHT); + public static final double A1_NORMALIZED_WEIGHT = A1_WEIGHT / (A1_WEIGHT + A2_WEIGHT); + public static final double A2_NORMALIZED_WEIGHT = A2_WEIGHT / (A1_WEIGHT + A2_WEIGHT); + public static final double A11_NORMALIZED_WEIGHT = A11_WEIGHT / (A11_WEIGHT + A12_WEIGHT); + public static final double A12_NORMALIZED_WEIGHT = A12_WEIGHT / (A11_WEIGHT + A12_WEIGHT); @Test public void testWeightResourceCalculation() throws IOException { @@ -103,39 +103,39 @@ public void testWeightResourceCalculation() throws IOException { @Test public void testPercentageResourceCalculation() throws IOException { - csConf.setCapacity(A, A_CAPACITY * 100); - csConf.setCapacity(B, B_CAPACITY * 100); - csConf.setCapacity(A1, A1_CAPACITY * 100); - csConf.setCapacity(A11, A11_CAPACITY * 100); - csConf.setCapacity(A12, A12_CAPACITY * 100); - csConf.setCapacity(A2, A2_CAPACITY * 100); + csConf.setCapacity(A, (float) (A_CAPACITY * 100)); + csConf.setCapacity(B, (float) (B_CAPACITY * 100)); + csConf.setCapacity(A1, (float) (A1_CAPACITY * 100)); + csConf.setCapacity(A11, (float) (A11_CAPACITY * 100)); + csConf.setCapacity(A12, (float) (A12_CAPACITY * 100)); + csConf.setCapacity(A2, (float) (A2_CAPACITY * 100)); QueueAssertionBuilder queueAssertionBuilder = createAssertionBuilder() .withQueue(A) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, A_CAPACITY)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, A_CAPACITY)) .assertCapacity(A_CAPACITY) .assertAbsoluteCapacity(A_CAPACITY) .withQueue(B) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, B_CAPACITY)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, B_CAPACITY)) .assertCapacity(B_CAPACITY) .assertAbsoluteCapacity(B_CAPACITY) .withQueue(A1) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, A_CAPACITY * A1_CAPACITY)) .assertCapacity(A1_CAPACITY) .assertAbsoluteCapacity(A_CAPACITY * A1_CAPACITY) .withQueue(A2) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, A_CAPACITY * A2_CAPACITY)) .assertCapacity(A2_CAPACITY) .assertAbsoluteCapacity(A_CAPACITY * A2_CAPACITY) .withQueue(A11) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, A11_CAPACITY * A_CAPACITY * A1_CAPACITY)) .assertCapacity(A11_CAPACITY) .assertAbsoluteCapacity(A11_CAPACITY * A_CAPACITY * A1_CAPACITY) .withQueue(A12) - .assertEffectiveMinResource(ResourceUtils.multiply(PERCENTAGE_ALL_RES, + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(PERCENTAGE_ALL_RES, A12_CAPACITY * A_CAPACITY * A1_CAPACITY)) .assertCapacity(A12_CAPACITY) .assertAbsoluteCapacity(A12_CAPACITY * A_CAPACITY * A1_CAPACITY) @@ -172,20 +172,20 @@ public void testAbsoluteResourceCalculation() throws IOException { QueueAssertionBuilder queueAssertionHalfClusterResource = createAssertionBuilder() .withQueue(A) - .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A_RES, 0.5f)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A_RES, 0.5f)) .withQueue(B) - .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_B_RES, 0.5f)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_B_RES, 0.5f)) .withQueue(A1) - .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A1_RES, 0.5f)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A1_RES, 0.5f)) .withQueue(A2) - .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A2_RES, 0.5f)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A2_RES, 0.5f)) .withQueue(A11) - .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A11_RES, 0.5f)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A11_RES, 0.5f)) .withQueue(A12) - .assertEffectiveMinResource(ResourceUtils.multiply(QUEUE_A12_RES, 0.5f)) + .assertEffectiveMinResource(ResourceUtils.multiplyFloor(QUEUE_A12_RES, 0.5f)) .build(); - update(queueAssertionHalfClusterResource, ResourceUtils.multiply(UPDATE_RES, 0.5f)); + update(queueAssertionHalfClusterResource, ResourceUtils.multiplyFloor(UPDATE_RES, 0.5f)); } } \ No newline at end of file From faa9a9a9ee6fba0f98096b471a1efdf0569a962f Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Wed, 2 Feb 2022 09:11:43 +0100 Subject: [PATCH 23/27] YARN-10965. Remove unnecessary change in LeafQueue --- .../server/resourcemanager/scheduler/capacity/LeafQueue.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index 48d97b97db7b3..f33de96e2c881 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -25,7 +25,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - @Private @Unstable public class LeafQueue extends AbstractLeafQueue { From c16c335be8c803b6b87ac67d764c11582dc94670 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 17 Feb 2022 16:05:20 +0100 Subject: [PATCH 24/27] YARN-10965. Fix review feedback --- .../AbsoluteResourceCapacityCalculator.java | 75 ++++++++++++++++++ .../AbstractQueueCapacityCalculator.java | 4 +- .../PercentageQueueCapacityCalculator.java | 5 ++ .../capacity/ResourceCalculationDriver.java | 76 +++---------------- .../capacity/RootQueueCapacityCalculator.java | 24 ++++-- .../WeightQueueCapacityCalculator.java | 4 +- 6 files changed, 110 insertions(+), 78 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index 6d1bd5cffe796..2bc798c599ac3 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -19,9 +19,21 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; +import org.apache.hadoop.yarn.util.UnitsConversionUtil; + +import java.util.Map; + +import static org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueUpdateWarning.QueueUpdateWarningType.BRANCH_DOWNSCALED; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ResourceCalculationDriver.MB_UNIT; public class AbsoluteResourceCapacityCalculator extends AbstractQueueCapacityCalculator { + @Override + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + setNormalizedResourceRatio(resourceCalculationDriver); + } + @Override public double calculateMinimumResource( ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { @@ -53,4 +65,67 @@ public ResourceUnitCapacityType getCapacityType() { return ResourceUnitCapacityType.ABSOLUTE; } + /** + * Calculates the normalized resource ratio of a parent queue, under which children are defined + * with absolute capacity type. If the effective resource of the parent is less, than the + * aggregated configured absolute resource of its children, the resource ratio will be less, + * than 1. + * + */ + public static void setNormalizedResourceRatio(ResourceCalculationDriver calculationDriver) { + CSQueue queue = calculationDriver.getQueue(); + + for (String label : queue.getConfiguredNodeLabels()) { + // ManagedParents assign zero capacity to queues in case of overutilization, downscaling is + // turned off for their children + if (queue instanceof ManagedParentQueue) { + return; + } + + for (String resourceName : queue.getConfiguredCapacityVector(label).getResourceNames()) { + long childrenConfiguredResource = 0; + long effectiveMinResource = queue.getQueueResourceQuotas().getEffectiveMinResource( + label).getResourceValue(resourceName); + + // Total configured min resources of direct children of the queue + for (CSQueue childQueue : queue.getChildQueues()) { + if (!childQueue.getConfiguredNodeLabels().contains(label)) { + continue; + } + QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + if (capacityVector.isResourceOfType(resourceName, ResourceUnitCapacityType.ABSOLUTE)) { + childrenConfiguredResource += capacityVector.getResource(resourceName) + .getResourceValue(); + } + } + // If no children is using ABSOLUTE capacity type, normalization is not needed + if (childrenConfiguredResource == 0) { + continue; + } + // Factor to scale down effective resource: When cluster has sufficient + // resources, effective_min_resources will be same as configured + // min_resources. + float numeratorForMinRatio = childrenConfiguredResource; + if (effectiveMinResource < childrenConfiguredResource) { + numeratorForMinRatio = queue.getQueueResourceQuotas().getEffectiveMinResource(label) + .getResourceValue(resourceName); + calculationDriver.getUpdateContext().addUpdateWarning(BRANCH_DOWNSCALED.ofQueue( + queue.getQueuePath())); + } + + String unit = resourceName.equals(MEMORY_URI) ? MB_UNIT : ""; + long convertedValue = UnitsConversionUtil.convert(unit, calculationDriver.getUpdateContext() + .getUpdatedClusterResource(label).getResourceInformation(resourceName).getUnits(), + childrenConfiguredResource); + + if (convertedValue != 0) { + Map normalizedResourceRatios = + calculationDriver.getNormalizedResourceRatios(); + normalizedResourceRatios.putIfAbsent(label, ResourceVector.newInstance()); + normalizedResourceRatios.get(label).setValue(resourceName, numeratorForMinRatio / + convertedValue); + } + } + } + } } 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 3c1e9c0e69ceb..369492b39f9bd 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -75,8 +75,8 @@ public abstract double calculateMaximumResource(ResourceCalculationDriver resour * @param resourceCalculationDriver driver that contains the parent queue on which the prerequisite * calculation should be made */ - public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { - } + public abstract void calculateResourcePrerequisites( + ResourceCalculationDriver resourceCalculationDriver); /** * Returns all resource names that are defined for the capacity type that is 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index b9619aa59aabd..6cf1259c7a788 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -50,6 +50,11 @@ public double calculateMaximumResource( .getResourceValue(resourceName) * absoluteMaxCapacity; } + @Override + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + + } + @Override public void updateCapacitiesAfterCalculation(ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { ((AbstractCSQueue) queue).updateAbsoluteCapacities(); 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index 176539969e0b3..ae37dddb2546a 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -23,7 +23,6 @@ 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.QueueUpdateWarning.QueueUpdateWarningType; -import org.apache.hadoop.yarn.util.UnitsConversionUtil; import java.util.Collection; import java.util.HashMap; @@ -66,7 +65,6 @@ public ResourceCalculationDriver( this.updateContext = updateContext; this.calculators = calculators; this.definedResources = definedResources; - setNormalizedResourceRatio(); } @@ -79,6 +77,15 @@ public CSQueue getQueue() { return queue; } + /** + * Returns all the children defined under the driver parent queue. + * + * @return child queues + */ + public Collection getChildQueues() { + return queue.getChildQueues(); + } + /** * Returns the context that is used throughout the whole update phase. * @@ -191,7 +198,7 @@ public void calculateResources() { for (String resourceName : definedResources) { for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { - for (CSQueue childQueue : queue.getChildQueues()) { + for (CSQueue childQueue : getChildQueues()) { CalculationContext context = new CalculationContext(resourceName, capacityType, childQueue); calculateResourceOnChild(context); } @@ -315,67 +322,4 @@ private void validateRemainingResource() { } } } - - /** - * Calculates the normalized resource ratio of a parent queue, under which children are defined - * with absolute capacity type. If the effective resource of the parent is less, than the - * aggregated configured absolute resource of its children, the resource ratio will be less, - * than 1. - * - */ - private void setNormalizedResourceRatio() { - for (String label : getQueue().getConfiguredNodeLabels()) { - // ManagedParents assign zero capacity to queues in case of overutilization, downscaling is - // turned off for their children - if (queue instanceof ManagedParentQueue) { - return; - } - - for (QueueCapacityVectorEntry capacityVectorEntry : queue.getConfiguredCapacityVector( - label)) { - String resourceName = capacityVectorEntry.getResourceName(); - long childrenConfiguredResource = 0; - long effectiveMinResource = queue.getQueueResourceQuotas().getEffectiveMinResource( - label).getResourceValue(resourceName); - - // Total configured min resources of direct children of this given parent - // queue - for (CSQueue childQueue : queue.getChildQueues()) { - if (!childQueue.getConfiguredNodeLabels().contains(label)) { - continue; - } - QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); - if (capacityVector.isResourceOfType(resourceName, ResourceUnitCapacityType.ABSOLUTE)) { - childrenConfiguredResource += capacityVector.getResource(resourceName).getResourceValue(); - } - } - // If no children is using ABSOLUTE capacity type, normalization is not needed - if (childrenConfiguredResource == 0) { - continue; - } - // Factor to scale down effective resource: When cluster has sufficient - // resources, effective_min_resources will be same as configured - // min_resources. - float numeratorForMinRatio = childrenConfiguredResource; - if (effectiveMinResource < childrenConfiguredResource) { - numeratorForMinRatio = queue.getQueueResourceQuotas().getEffectiveMinResource(label) - .getResourceValue(resourceName); - updateContext.addUpdateWarning(QueueUpdateWarningType.BRANCH_DOWNSCALED.ofQueue( - queue.getQueuePath())); - } - - String unit = resourceName.equals(MEMORY_URI) ? MB_UNIT : ""; - long convertedValue = UnitsConversionUtil.convert(unit, - updateContext.getUpdatedClusterResource(label).getResourceInformation(resourceName) - .getUnits(), childrenConfiguredResource); - - if (convertedValue != 0) { - Map normalizedResourceRatios = getNormalizedResourceRatios(); - normalizedResourceRatios.putIfAbsent(label, ResourceVector.newInstance()); - normalizedResourceRatios.get(label).setValue(resourceName, numeratorForMinRatio / - convertedValue); - } - } - } - } } 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java index e4a0c4b019ce8..8da1aeab282b5 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/RootQueueCapacityCalculator.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/RootQueueCapacityCalculator.java @@ -18,19 +18,29 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; + import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.PERCENTAGE; -public class RootQueueCapacityCalculator extends - AbstractQueueCapacityCalculator { +public class RootQueueCapacityCalculator extends AbstractQueueCapacityCalculator { + + @Override + public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { + AbsoluteResourceCapacityCalculator.setNormalizedResourceRatio(resourceCalculationDriver); + } @Override - public double calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { - return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(context.getResourceName()); + public double calculateMinimumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, String label) { + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) + .getResourceValue(context.getResourceName()); } @Override - public double calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { - return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label).getResourceValue(context.getResourceName()); + public double calculateMaximumResource(ResourceCalculationDriver resourceCalculationDriver, + CalculationContext context, String label) { + return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) + .getResourceValue(context.getResourceName()); } @Override @@ -43,7 +53,7 @@ public void updateCapacitiesAfterCalculation( } @Override - public QueueCapacityVector.ResourceUnitCapacityType getCapacityType() { + public ResourceUnitCapacityType getCapacityType() { return 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/capacity/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 23a51167f76e5..3ace63afa9cf0 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -28,10 +28,8 @@ public class WeightQueueCapacityCalculator extends AbstractQueueCapacityCalculat @Override public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCalculationDriver) { - super.calculateResourcePrerequisites(resourceCalculationDriver); - // Precalculate the summary of children's weight - for (CSQueue childQueue : resourceCalculationDriver.getQueue().getChildQueues()) { + for (CSQueue childQueue : resourceCalculationDriver.getChildQueues()) { for (String label : childQueue.getConfiguredNodeLabels()) { for (String resourceName : childQueue.getConfiguredCapacityVector(label) .getResourceNamesByCapacityType(getCapacityType())) { From 785f2fa35f3699b903226023bd1342e720a79d60 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 22 Mar 2022 15:55:16 +0100 Subject: [PATCH 25/27] YARN-10965. Fix checkstyle issues and nomenclature nits --- .../AbsoluteResourceCapacityCalculator.java | 2 ++ .../AbstractQueueCapacityCalculator.java | 15 ++++++----- .../capacity/CalculationContext.java | 20 +++++++------- ...CapacitySchedulerQueueCapacityHandler.java | 23 +++++++++------- .../capacity/ResourceCalculationDriver.java | 26 +++++++++---------- 5 files changed, 48 insertions(+), 38 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index 2bc798c599ac3..c9dc47b3b853d 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -71,6 +71,8 @@ public ResourceUnitCapacityType getCapacityType() { * aggregated configured absolute resource of its children, the resource ratio will be less, * than 1. * + * @param calculationDriver the driver, which contains the parent queue that will form the base + * of the normalization calculation */ public static void setNormalizedResourceRatio(ResourceCalculationDriver calculationDriver) { CSQueue queue = calculationDriver.getQueue(); 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 369492b39f9bd..11e179ff34096 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -30,8 +30,9 @@ public abstract class AbstractQueueCapacityCalculator { /** * Sets the metrics and statistics after effective resource values calculation. * - * @param resourceCalculationDriver driver that contains the current resource unit and child to - * process + * @param queue the queue on which the calculations are based + * @param resourceCalculationDriver driver that contains the intermediate calculation results for + * a queue branch * @param label node label */ public abstract void updateCapacitiesAfterCalculation( @@ -48,8 +49,9 @@ public abstract void updateCapacitiesAfterCalculation( /** * Calculates the minimum effective resource. * - * @param resourceCalculationDriver driver that contains the current resource unit and child to - * process + * @param resourceCalculationDriver driver that contains the intermediate calculation results for + * a queue branch + * @param context the units evaluated in the current iteration phase * @param label node label * @return minimum effective resource */ @@ -60,8 +62,9 @@ public abstract double calculateMinimumResource(ResourceCalculationDriver resour /** * Calculates the maximum effective resource. * - * @param resourceCalculationDriver driver that contains the current resource unit and child to - * process + * @param resourceCalculationDriver driver that contains the intermediate calculation results for + * a queue branch + * @param context the units evaluated in the current iteration phase * @param label node label * @return minimum effective resource */ 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/CalculationContext.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/CalculationContext.java index 71c4bd4641556..7ec85e19b1f0c 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/CalculationContext.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/CalculationContext.java @@ -19,31 +19,33 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; /** * A storage class that wraps arguments used in a resource calculation iteration. */ public class CalculationContext { private final String resourceName; - private final QueueCapacityVector.ResourceUnitCapacityType capacityType; - private final CSQueue childQueue; + private final ResourceUnitCapacityType capacityType; + private final CSQueue queue; - public CalculationContext(String resourceName, QueueCapacityVector.ResourceUnitCapacityType capacityType, CSQueue childQueue) { + public CalculationContext(String resourceName, ResourceUnitCapacityType capacityType, + CSQueue queue) { this.resourceName = resourceName; this.capacityType = capacityType; - this.childQueue = childQueue; + this.queue = queue; } public String getResourceName() { return resourceName; } - public QueueCapacityVector.ResourceUnitCapacityType getCapacityType() { + public ResourceUnitCapacityType getCapacityType() { return capacityType; } - public CSQueue getChildQueue() { - return childQueue; + public CSQueue getQueue() { + return queue; } /** @@ -54,7 +56,7 @@ public CSQueue getChildQueue() { * @return capacity vector entry */ public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { - return childQueue.getConfiguredCapacityVector(label).getResource(resourceName); + return queue.getConfiguredCapacityVector(label).getResource(resourceName); } /** @@ -65,6 +67,6 @@ public QueueCapacityVectorEntry getCurrentMinimumCapacityEntry(String label) { * @return capacity vector entry */ public QueueCapacityVectorEntry getCurrentMaximumCapacityEntry(String label) { - return childQueue.getConfiguredMaxCapacityVector(label).getResource(resourceName); + return queue.getConfiguredMaxCapacityVector(label).getResource(resourceName); } } 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index d9844b724aad7..fd7e2845b3c88 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -121,7 +121,7 @@ private void updateChildrenAfterCalculation( ResourceCalculationDriver resourceCalculationDriver, ResourceLimits resourceLimits) { ParentQueue parentQueue = (ParentQueue) resourceCalculationDriver.getQueue(); for (CSQueue childQueue : parentQueue.getChildQueues()) { - updateChildCapacities(resourceCalculationDriver, childQueue); + updateQueueCapacities(resourceCalculationDriver, childQueue); ResourceLimits childLimit = parentQueue.getResourceLimitsOfChild(childQueue, resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(), @@ -135,27 +135,29 @@ private void updateChildrenAfterCalculation( /** * Updates the capacity values of the currently evaluated child. - * @param childQueue child queue on which the capacities are set + * @param queue queue on which the capacities are set */ - private void updateChildCapacities(ResourceCalculationDriver resourceCalculationDriver, CSQueue childQueue) { - childQueue.getWriteLock().lock(); + private void updateQueueCapacities( + ResourceCalculationDriver resourceCalculationDriver, CSQueue queue) { + queue.getWriteLock().lock(); try { - for (String label : childQueue.getConfiguredNodeLabels()) { - QueueCapacityVector capacityVector = childQueue.getConfiguredCapacityVector(label); + for (String label : queue.getConfiguredNodeLabels()) { + QueueCapacityVector capacityVector = queue.getConfiguredCapacityVector(label); if (capacityVector.isMixedCapacityVector()) { // Post update capacities based on the calculated effective resource values - setQueueCapacities(resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label), childQueue, label); + setQueueCapacities(resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource( + label), queue, label); } else { // Update capacities according to the legacy logic for (ResourceUnitCapacityType capacityType : - childQueue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { + queue.getConfiguredCapacityVector(label).getDefinedCapacityTypes()) { AbstractQueueCapacityCalculator calculator = calculators.get(capacityType); - calculator.updateCapacitiesAfterCalculation(resourceCalculationDriver, childQueue, label); + calculator.updateCapacitiesAfterCalculation(resourceCalculationDriver, queue, label); } } } } finally { - childQueue.getWriteLock().unlock(); + queue.getWriteLock().unlock(); } } @@ -163,6 +165,7 @@ private void updateChildCapacities(ResourceCalculationDriver resourceCalculation * Sets capacity and absolute capacity values of a queue based on minimum and * maximum effective resources. * + * @param clusterResource overall cluster resource * @param queue child queue for which the capacities are set * @param label node 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index ae37dddb2546a..23847aa020835 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -216,10 +216,10 @@ public void calculateResources() { } private void calculateResourceOnChild(CalculationContext context) { - context.getChildQueue().getWriteLock().lock(); + context.getQueue().getWriteLock().lock(); try { - for (String label : context.getChildQueue().getConfiguredNodeLabels()) { - if (!context.getChildQueue().getConfiguredCapacityVector(label).isResourceOfType(context.getResourceName(), + for (String label : context.getQueue().getConfiguredNodeLabels()) { + if (!context.getQueue().getConfiguredCapacityVector(label).isResourceOfType(context.getResourceName(), context.getCapacityType())) { continue; } @@ -232,14 +232,14 @@ private void calculateResourceOnChild(CalculationContext context) { usedResourceByCurrentCalculatorPerLabel.put(label, resourceUsedByLabel); } } finally { - context.getChildQueue().getWriteLock().unlock(); + context.getQueue().getWriteLock().unlock(); } } private double setChildResources(CalculationContext context, String label) { - QueueCapacityVectorEntry capacityVectorEntry = context.getChildQueue().getConfiguredCapacityVector( + QueueCapacityVectorEntry capacityVectorEntry = context.getQueue().getConfiguredCapacityVector( label).getResource(context.getResourceName()); - QueueCapacityVectorEntry maximumCapacityVectorEntry = context.getChildQueue() + QueueCapacityVectorEntry maximumCapacityVectorEntry = context.getQueue() .getConfiguredMaxCapacityVector(label).getResource(context.getResourceName()); AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( maximumCapacityVectorEntry.getVectorResourceType()); @@ -255,9 +255,9 @@ private double setChildResources(CalculationContext context, String label) { minimumResource = resources.getLeft(); maximumResource = resources.getRight(); - context.getChildQueue().getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( + context.getQueue().getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( context.getResourceName(), (long) minimumResource); - context.getChildQueue().getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( + context.getQueue().getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( context.getResourceName(), (long) maximumResource); return minimumResource; @@ -266,7 +266,7 @@ private double setChildResources(CalculationContext context, String label) { private Pair validateCalculatedResources(CalculationContext context, String label, Pair calculatedResources) { double minimumResource = calculatedResources.getLeft(); - long minimumMemoryResource = context.getChildQueue().getQueueResourceQuotas().getEffectiveMinResource(label) + long minimumMemoryResource = context.getQueue().getQueueResourceQuotas().getEffectiveMinResource(label) .getMemorySize(); double remainingResourceUnderParent = overallRemainingResourcePerLabel.get(label).getValue( @@ -283,14 +283,14 @@ private Pair validateCalculatedResources(CalculationContext cont if (maximumResource != 0 && maximumResource > parentMaximumResource) { updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( - context.getChildQueue().getQueuePath())); + context.getQueue().getQueuePath())); } maximumResource = maximumResource == 0 ? parentMaximumResource : Math.min(maximumResource, parentMaximumResource); if (maximumResource < minimumResource) { updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_EXCEEDS_MAX_RESOURCE.ofQueue( - context.getChildQueue().getQueuePath())); + context.getQueue().getQueuePath())); minimumResource = maximumResource; } @@ -300,7 +300,7 @@ private Pair validateCalculatedResources(CalculationContext cont minimumResource = 0; } else { updateContext.addUpdateWarning( - QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(context.getChildQueue().getQueuePath()).withInfo( + QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(context.getQueue().getQueuePath()).withInfo( "Resource name: " + context.getResourceName() + " resource value: " + minimumResource)); minimumResource = remainingResourceUnderParent; } @@ -308,7 +308,7 @@ private Pair validateCalculatedResources(CalculationContext cont if (minimumResource == 0) { updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( - context.getChildQueue().getQueuePath()).withInfo("Resource name: " + context.getResourceName())); + context.getQueue().getQueuePath()).withInfo("Resource name: " + context.getResourceName())); } return new ImmutablePair<>(minimumResource, maximumResource); From 673a8c0eadad70f0a77eb25e82d0464c267d3c64 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 30 Aug 2022 15:46:21 +0200 Subject: [PATCH 26/27] YARN-10965. Fix javac and spotbugs errors --- .../resourcemanager/scheduler/capacity/AbstractLeafQueue.java | 2 +- .../scheduler/capacity/ResourceCalculationDriver.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 d716b858481b3..669b1370af80a 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 @@ -167,7 +167,7 @@ public AbstractLeafQueue(CapacitySchedulerQueueContext queueContext, resourceCalculator); // One time initialization is enough since it is static ordering policy - this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps(); + this.pendingOrderingPolicy = new FifoOrderingPolicyForPendingApps<>(); } @SuppressWarnings("checkstyle:nowhitespaceafter") 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index 23847aa020835..47056a4dac16e 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -35,7 +35,7 @@ * bookkeeper of disposable update information that is used by all children under the common parent. */ public class ResourceCalculationDriver { - protected static final ResourceUnitCapacityType[] CALCULATOR_PRECEDENCE = + private static final ResourceUnitCapacityType[] CALCULATOR_PRECEDENCE = new ResourceUnitCapacityType[] { ResourceUnitCapacityType.ABSOLUTE, ResourceUnitCapacityType.PERCENTAGE, From ce2618d57d01b87942aa4414762405166504e3d5 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Thu, 26 Jan 2023 14:54:48 +0100 Subject: [PATCH 27/27] Fix checkstyle --- .../AbsoluteResourceCapacityCalculator.java | 6 ++- .../scheduler/capacity/AbstractCSQueue.java | 4 +- .../scheduler/capacity/AbstractLeafQueue.java | 3 +- .../AbstractQueueCapacityCalculator.java | 4 +- .../scheduler/capacity/CSQueue.java | 4 +- .../capacity/CSQueueUsageTracker.java | 3 -- .../CapacitySchedulerConfiguration.java | 3 +- ...CapacitySchedulerQueueCapacityHandler.java | 6 ++- .../DefaultQueueResourceRoundingStrategy.java | 10 +---- .../scheduler/capacity/ParentQueue.java | 3 +- .../PercentageQueueCapacityCalculator.java | 21 +++++---- .../capacity/QueueCapacityUpdateContext.java | 3 -- .../capacity/QueueUpdateWarning.java | 9 ++-- .../capacity/ResourceCalculationDriver.java | 45 ++++++++++++------- .../capacity/RootCalculationDriver.java | 10 +++-- .../WeightQueueCapacityCalculator.java | 21 +++++---- .../nodelabels/NullRMNodeLabelsManager.java | 2 +- .../capacity/TestQueueCapacityVector.java | 6 ++- .../TestUniformQueueResourceCalculation.java | 2 +- .../conf/TestQueueCapacityConfigParser.java | 27 ++++++----- 20 files changed, 110 insertions(+), 82 deletions(-) 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java index c9dc47b3b853d..33b457410799b 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/AbsoluteResourceCapacityCalculator.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/AbsoluteResourceCapacityCalculator.java @@ -36,7 +36,8 @@ public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCal @Override public double calculateMinimumResource( - ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, + String label) { String resourceName = context.getResourceName(); double normalizedRatio = resourceCalculationDriver.getNormalizedResourceRatios().getOrDefault( label, ResourceVector.of(1)).getValue(resourceName); @@ -49,7 +50,8 @@ public double calculateMinimumResource( @Override public double calculateMaximumResource( - ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, + String label) { return context.getCurrentMaximumCapacityEntry(label).getResourceValue(); } 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 739f22c983dd1..f9304cc960417 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 @@ -381,7 +381,9 @@ protected void setupQueueConfigs(Resource clusterResource) throws .parseConfiguredResourceVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels()); this.configuredMaxCapacityVectors = configuration - .parseConfiguredMaximumCapacityVector(queuePath.getFullPath(), this.queueNodeLabelsSettings.getConfiguredNodeLabels(), QueueCapacityVector.newInstance()); + .parseConfiguredMaximumCapacityVector(queuePath.getFullPath(), + this.queueNodeLabelsSettings.getConfiguredNodeLabels(), + QueueCapacityVector.newInstance()); // Update metrics CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, this, labelManager, null); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/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 669b1370af80a..72ea63a2fc50d 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 @@ -1940,7 +1940,8 @@ private void updateCurrentResourceLimits( } @Override - public void refreshAfterResourceCalculation(Resource clusterResource, ResourceLimits resourceLimits) { + public void refreshAfterResourceCalculation(Resource clusterResource, + ResourceLimits resourceLimits) { lastClusterResource = clusterResource; // Update maximum applications for the queue and for users updateMaximumApplications(); 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java index 11e179ff34096..8b48da88ff835 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/AbstractQueueCapacityCalculator.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/AbstractQueueCapacityCalculator.java @@ -75,8 +75,8 @@ public abstract double calculateMaximumResource(ResourceCalculationDriver resour /** * Executes all logic that must be called prior to the effective resource value calculations. * - * @param resourceCalculationDriver driver that contains the parent queue on which the prerequisite - * calculation should be made + * @param resourceCalculationDriver driver that contains the parent queue on which the + * prerequisite calculation should be made */ public abstract void calculateResourcePrerequisites( ResourceCalculationDriver resourceCalculationDriver); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index 90abe0214954c..91dab98ce76cf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -458,14 +458,14 @@ public void validateSubmitApplication(ApplicationId applicationId, QueueCapacityVector getConfiguredMaxCapacityVector(String label); /** - * Sets the configured minimum capacity vector to a specific value + * Sets the configured minimum capacity vector to a specific value. * @param label node label (partition) * @param minCapacityVector capacity vector */ void setConfiguredMinCapacityVector(String label, QueueCapacityVector minCapacityVector); /** - * Sets the configured maximum capacity vector to a specific value + * Sets the configured maximum capacity vector to a specific value. * @param label node label (partition) * @param maxCapacityVector capacity vector */ 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/CSQueueUsageTracker.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/CSQueueUsageTracker.java index a983540f12600..dd6b9b17ac0c4 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/CSQueueUsageTracker.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/CSQueueUsageTracker.java @@ -21,9 +21,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; -import java.util.HashMap; -import java.util.Map; - public class CSQueueUsageTracker { private final CSQueueMetrics metrics; private int numContainers; 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 b47ac02fa3888..757120e1621db 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 @@ -2732,7 +2732,8 @@ public Map parseConfiguredMaximumCapacityVector( String propertyName = CapacitySchedulerConfiguration.getNodeLabelPrefix( queuePath, label) + CapacitySchedulerConfiguration.MAXIMUM_CAPACITY; String capacityString = get(propertyName); - QueueCapacityVector capacityVector = queueCapacityConfigParser.parse(capacityString, queuePath); + QueueCapacityVector capacityVector = queueCapacityConfigParser.parse(capacityString, + queuePath); if (capacityVector.isEmpty()) { capacityVector = defaultVector; } 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java index fd7e2845b3c88..f197ccf6be28f 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/CapacitySchedulerQueueCapacityHandler.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/CapacitySchedulerQueueCapacityHandler.java @@ -98,10 +98,12 @@ public void updateRoot(CSQueue rootQueue, Resource clusterResource) { QueueCapacityUpdateContext updateContext = new QueueCapacityUpdateContext(clusterResource, labelsManager); - RootCalculationDriver rootCalculationDriver = new RootCalculationDriver(rootQueue, updateContext, + RootCalculationDriver rootCalculationDriver = new RootCalculationDriver(rootQueue, + updateContext, rootCalculator, definedResources); rootCalculationDriver.calculateResources(); - rootQueue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), resourceLimits); + rootQueue.refreshAfterResourceCalculation(updateContext.getUpdatedClusterResource(), + resourceLimits); } private void update( 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java index a1ffcd5e225a4..3a0254cdc531a 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/DefaultQueueResourceRoundingStrategy.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/DefaultQueueResourceRoundingStrategy.java @@ -21,14 +21,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.QueueCapacityVectorEntry; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType; -import java.util.ArrayList; -import java.util.Collection; -import java.util.LinkedHashSet; -import java.util.Set; -import java.util.SortedSet; - -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector.ResourceUnitCapacityType.WEIGHT; - /** * The default rounding strategy for resource calculation. Uses floor for all types except WEIGHT, * which is always the last type to consider, therefore it is safe to round up. @@ -39,7 +31,7 @@ public class DefaultQueueResourceRoundingStrategy implements QueueResourceRoundi public DefaultQueueResourceRoundingStrategy( ResourceUnitCapacityType[] capacityTypePrecedence) { if (capacityTypePrecedence.length == 0) { - throw new IllegalArgumentException("Capacity type precedence collection is empty"); + throw new IllegalArgumentException("Capacity type precedence collection is empty"); } lastCapacityType = capacityTypePrecedence[capacityTypePrecedence.length - 1]; 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 9ce719ffb7a4b..a816b91034caa 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 @@ -1212,7 +1212,8 @@ public void completedContainer(Resource clusterResource, } @Override - public void refreshAfterResourceCalculation(Resource clusterResource, ResourceLimits resourceLimits) { + public void refreshAfterResourceCalculation(Resource clusterResource, + ResourceLimits resourceLimits) { CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, this, labelManager, null); // Update configured capacity/max-capacity for default partition only 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java index 6cf1259c7a788..6a73459aaf418 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/PercentageQueueCapacityCalculator.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/PercentageQueueCapacityCalculator.java @@ -24,25 +24,29 @@ public class PercentageQueueCapacityCalculator extends AbstractQueueCapacityCalc @Override public double calculateMinimumResource( - ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, + String label) { String resourceName = context.getResourceName(); - double parentAbsoluteCapacity = resourceCalculationDriver.getParentAbsoluteMinCapacity(label, resourceName); - double remainingPerEffectiveResourceRatio = resourceCalculationDriver.getRemainingRatioOfResource( - label, resourceName); + double parentAbsoluteCapacity = resourceCalculationDriver.getParentAbsoluteMinCapacity(label, + resourceName); + double remainingPerEffectiveResourceRatio = + resourceCalculationDriver.getRemainingRatioOfResource(label, resourceName); double absoluteCapacity = parentAbsoluteCapacity * remainingPerEffectiveResourceRatio * context.getCurrentMinimumCapacityEntry(label).getResourceValue() / 100; return resourceCalculationDriver.getUpdateContext().getUpdatedClusterResource(label) .getResourceValue(resourceName) * absoluteCapacity; -} + } @Override public double calculateMaximumResource( - ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, String label) { + ResourceCalculationDriver resourceCalculationDriver, CalculationContext context, + String label) { String resourceName = context.getResourceName(); - double parentAbsoluteMaxCapacity = resourceCalculationDriver.getParentAbsoluteMaxCapacity(label, resourceName); + double parentAbsoluteMaxCapacity = + resourceCalculationDriver.getParentAbsoluteMaxCapacity(label, resourceName); double absoluteMaxCapacity = parentAbsoluteMaxCapacity * context.getCurrentMaximumCapacityEntry(label).getResourceValue() / 100; @@ -56,7 +60,8 @@ public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCal } @Override - public void updateCapacitiesAfterCalculation(ResourceCalculationDriver resourceCalculationDriver, CSQueue queue, String label) { + public void updateCapacitiesAfterCalculation(ResourceCalculationDriver resourceCalculationDriver, + CSQueue queue, String label) { ((AbstractCSQueue) queue).updateAbsoluteCapacities(); } 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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java index 6bd8d5cb361f6..4eb270be515c5 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/QueueCapacityUpdateContext.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/QueueCapacityUpdateContext.java @@ -18,14 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.commons.collections.map.LazyMap; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; /** * A storage that encapsulates intermediate calculation values throughout a 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/QueueUpdateWarning.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/QueueUpdateWarning.java index 826de78c2822b..43c345b1bc3ef 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/QueueUpdateWarning.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/QueueUpdateWarning.java @@ -33,11 +33,14 @@ public QueueUpdateWarning(QueueUpdateWarningType queueUpdateWarningType, String public enum QueueUpdateWarningType { BRANCH_UNDERUTILIZED("Remaining resource found in branch under parent queue '%s'. %s"), - QUEUE_OVERUTILIZED("Queue '%s' is configured to use more resources than what is available under its parent. %s"), + QUEUE_OVERUTILIZED("Queue '%s' is configured to use more resources than what is available " + + "under its parent. %s"), QUEUE_ZERO_RESOURCE("Queue '%s' is assigned zero resource. %s"), - BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are downscaled due to insufficient cluster resource. %s"), + BRANCH_DOWNSCALED("Child queues with absolute configured capacity under parent queue '%s' are" + + " downscaled due to insufficient cluster resource. %s"), QUEUE_EXCEEDS_MAX_RESOURCE("Queue '%s' exceeds its maximum available resources. %s"), - QUEUE_MAX_RESOURCE_EXCEEDS_PARENT("Maximum resources of queue '%s' are greater than its parent's. %s"); + QUEUE_MAX_RESOURCE_EXCEEDS_PARENT("Maximum resources of queue '%s' are greater than its " + + "parent's. %s"); private final String template; 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java index 47056a4dac16e..5993042c0e5da 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/ResourceCalculationDriver.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/ResourceCalculationDriver.java @@ -188,8 +188,10 @@ public ResourceVector getBatchRemainingResource(String label) { public void calculateResources() { // Reset both remaining resource storage to the parent's available resource for (String label : queue.getConfiguredNodeLabels()) { - overallRemainingResourcePerLabel.put(label, ResourceVector.of(queue.getEffectiveCapacity(label))); - batchRemainingResourcePerLabel.put(label, ResourceVector.of(queue.getEffectiveCapacity(label))); + overallRemainingResourcePerLabel.put(label, + ResourceVector.of(queue.getEffectiveCapacity(label))); + batchRemainingResourcePerLabel.put(label, + ResourceVector.of(queue.getEffectiveCapacity(label))); } for (AbstractQueueCapacityCalculator capacityCalculator : calculators.values()) { @@ -199,13 +201,15 @@ public void calculateResources() { for (String resourceName : definedResources) { for (ResourceUnitCapacityType capacityType : CALCULATOR_PRECEDENCE) { for (CSQueue childQueue : getChildQueues()) { - CalculationContext context = new CalculationContext(resourceName, capacityType, childQueue); + CalculationContext context = new CalculationContext(resourceName, capacityType, + childQueue); calculateResourceOnChild(context); } // Flush aggregated used resource by labels at the end of a calculator phase for (Map.Entry entry : usedResourceByCurrentCalculatorPerLabel.entrySet()) { - batchRemainingResourcePerLabel.get(entry.getKey()).decrement(resourceName, entry.getValue()); + batchRemainingResourcePerLabel.get(entry.getKey()).decrement(resourceName, + entry.getValue()); } usedResourceByCurrentCalculatorPerLabel = new HashMap<>(); @@ -219,8 +223,8 @@ private void calculateResourceOnChild(CalculationContext context) { context.getQueue().getWriteLock().lock(); try { for (String label : context.getQueue().getConfiguredNodeLabels()) { - if (!context.getQueue().getConfiguredCapacityVector(label).isResourceOfType(context.getResourceName(), - context.getCapacityType())) { + if (!context.getQueue().getConfiguredCapacityVector(label).isResourceOfType( + context.getResourceName(), context.getCapacityType())) { continue; } double usedResourceByChild = setChildResources(context, label); @@ -228,7 +232,8 @@ private void calculateResourceOnChild(CalculationContext context) { 0d); double resourceUsedByLabel = aggregatedUsedResource + usedResourceByChild; - overallRemainingResourcePerLabel.get(label).decrement(context.getResourceName(), usedResourceByChild); + overallRemainingResourcePerLabel.get(label).decrement(context.getResourceName(), + usedResourceByChild); usedResourceByCurrentCalculatorPerLabel.put(label, resourceUsedByLabel); } } finally { @@ -244,13 +249,16 @@ private double setChildResources(CalculationContext context, String label) { AbstractQueueCapacityCalculator maximumCapacityCalculator = calculators.get( maximumCapacityVectorEntry.getVectorResourceType()); - double minimumResource = calculators.get(context.getCapacityType()).calculateMinimumResource(this, context, label); - double maximumResource = maximumCapacityCalculator.calculateMaximumResource(this, context, label); + double minimumResource = + calculators.get(context.getCapacityType()).calculateMinimumResource(this, context, label); + double maximumResource = maximumCapacityCalculator.calculateMaximumResource(this, context, + label); minimumResource = roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); maximumResource = roundingStrategy.getRoundedResource(maximumResource, maximumCapacityVectorEntry); - Pair resources = validateCalculatedResources(context, label, new ImmutablePair<>( + Pair resources = validateCalculatedResources(context, label, + new ImmutablePair<>( minimumResource, maximumResource)); minimumResource = resources.getLeft(); maximumResource = resources.getRight(); @@ -266,8 +274,8 @@ private double setChildResources(CalculationContext context, String label) { private Pair validateCalculatedResources(CalculationContext context, String label, Pair calculatedResources) { double minimumResource = calculatedResources.getLeft(); - long minimumMemoryResource = context.getQueue().getQueueResourceQuotas().getEffectiveMinResource(label) - .getMemorySize(); + long minimumMemoryResource = + context.getQueue().getQueueResourceQuotas().getEffectiveMinResource(label).getMemorySize(); double remainingResourceUnderParent = overallRemainingResourcePerLabel.get(label).getValue( context.getResourceName()); @@ -282,8 +290,8 @@ private Pair validateCalculatedResources(CalculationContext cont } if (maximumResource != 0 && maximumResource > parentMaximumResource) { - updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT.ofQueue( - context.getQueue().getQueuePath())); + updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_MAX_RESOURCE_EXCEEDS_PARENT + .ofQueue(context.getQueue().getQueuePath())); } maximumResource = maximumResource == 0 ? parentMaximumResource : Math.min(maximumResource, parentMaximumResource); @@ -300,15 +308,18 @@ private Pair validateCalculatedResources(CalculationContext cont minimumResource = 0; } else { updateContext.addUpdateWarning( - QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue(context.getQueue().getQueuePath()).withInfo( - "Resource name: " + context.getResourceName() + " resource value: " + minimumResource)); + QueueUpdateWarningType.QUEUE_OVERUTILIZED.ofQueue( + context.getQueue().getQueuePath()).withInfo( + "Resource name: " + context.getResourceName() + + " resource value: " + minimumResource)); minimumResource = remainingResourceUnderParent; } } if (minimumResource == 0) { updateContext.addUpdateWarning(QueueUpdateWarningType.QUEUE_ZERO_RESOURCE.ofQueue( - context.getQueue().getQueuePath()).withInfo("Resource name: " + context.getResourceName())); + context.getQueue().getQueuePath()) + .withInfo("Resource name: " + context.getResourceName())); } return new ImmutablePair<>(minimumResource, maximumResource); 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/RootCalculationDriver.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/RootCalculationDriver.java index 1bed62657e05e..530c5c1086ff6 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/RootCalculationDriver.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/RootCalculationDriver.java @@ -39,14 +39,18 @@ public RootCalculationDriver(CSQueue rootQueue, QueueCapacityUpdateContext updat @Override public void calculateResources() { for (String label : queue.getConfiguredNodeLabels()) { - for (QueueCapacityVector.QueueCapacityVectorEntry capacityVectorEntry : queue.getConfiguredCapacityVector(label)) { + for (QueueCapacityVector.QueueCapacityVectorEntry capacityVectorEntry : + queue.getConfiguredCapacityVector(label)) { String resourceName = capacityVectorEntry.getResourceName(); CalculationContext context = new CalculationContext(resourceName, PERCENTAGE, queue); double minimumResource = rootCalculator.calculateMinimumResource(this, context, label); double maximumResource = rootCalculator.calculateMaximumResource(this, context, label); - long roundedMinResource = (long) roundingStrategy.getRoundedResource(minimumResource, capacityVectorEntry); - long roundedMaxResource = (long) roundingStrategy.getRoundedResource(maximumResource, queue.getConfiguredMaxCapacityVector(label).getResource(resourceName)); + long roundedMinResource = (long) roundingStrategy + .getRoundedResource(minimumResource, capacityVectorEntry); + long roundedMaxResource = (long) roundingStrategy + .getRoundedResource(maximumResource, + queue.getConfiguredMaxCapacityVector(label).getResource(resourceName)); queue.getQueueResourceQuotas().getEffectiveMinResource(label).setResourceValue( resourceName, roundedMinResource); queue.getQueueResourceQuotas().getEffectiveMaxResource(label).setResourceValue( 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java index 3ace63afa9cf0..4121a6bf056e2 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/WeightQueueCapacityCalculator.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/WeightQueueCapacityCalculator.java @@ -33,8 +33,8 @@ public void calculateResourcePrerequisites(ResourceCalculationDriver resourceCal for (String label : childQueue.getConfiguredNodeLabels()) { for (String resourceName : childQueue.getConfiguredCapacityVector(label) .getResourceNamesByCapacityType(getCapacityType())) { - resourceCalculationDriver.incrementWeight(label, resourceName, childQueue - .getConfiguredCapacityVector(label).getResource(resourceName).getResourceValue()); + resourceCalculationDriver.incrementWeight(label, resourceName, childQueue + .getConfiguredCapacityVector(label).getResource(resourceName).getResourceValue()); } } } @@ -45,11 +45,11 @@ public double calculateMinimumResource(ResourceCalculationDriver resourceCalcula CalculationContext context, String label) { String resourceName = context.getResourceName(); - double normalizedWeight = context.getCurrentMinimumCapacityEntry(label) - .getResourceValue() / resourceCalculationDriver.getSumWeightsByResource(label, resourceName); + double normalizedWeight = context.getCurrentMinimumCapacityEntry(label).getResourceValue() / + resourceCalculationDriver.getSumWeightsByResource(label, resourceName); - double remainingResource = resourceCalculationDriver.getBatchRemainingResource(label).getValue( - resourceName); + double remainingResource = resourceCalculationDriver.getBatchRemainingResource(label) + .getValue(resourceName); // Due to rounding loss it is better to use all remaining resources if no other resource uses // weight @@ -73,7 +73,8 @@ public double calculateMaximumResource(ResourceCalculationDriver resourceCalcula CalculationContext context, String label) { throw new IllegalStateException("Resource " + context.getCurrentMinimumCapacityEntry( - label).getResourceName() + " has " + "WEIGHT maximum capacity type, which is not supported"); + label).getResourceName() + + " has " + "WEIGHT maximum capacity type, which is not supported"); } @Override @@ -88,13 +89,15 @@ public void updateCapacitiesAfterCalculation( Collection resourceNames = getResourceNames(queue, label); for (String resourceName : resourceNames) { - double sumBranchWeight = resourceCalculationDriver.getSumWeightsByResource(label, resourceName); + double sumBranchWeight = resourceCalculationDriver.getSumWeightsByResource(label, + resourceName); double capacity = queue.getConfiguredCapacityVector( label).getResource(resourceName).getResourceValue() / sumBranchWeight; sumCapacityPerResource += capacity; } - queue.getQueueCapacities().setNormalizedWeight(label, (float) (sumCapacityPerResource / resourceNames.size())); + queue.getQueueCapacities().setNormalizedWeight(label, + (float) (sumCapacityPerResource / resourceNames.size())); ((AbstractCSQueue) queue).updateAbsoluteCapacities(); } } 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/NullRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java index 180c2d6442c29..10d98455851f4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java @@ -112,7 +112,7 @@ public void setResourceForLabel(String label, Resource resource) { private static class FakeLabel extends RMNodeLabel { - FakeLabel(String label, Resource resource) { + FakeLabel(String label, Resource resource) { super(label, resource, 1, false); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacityVector.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/TestQueueCapacityVector.java index 7b5ab6e225cda..18eead5d8ece9 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/TestQueueCapacityVector.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/TestQueueCapacityVector.java @@ -78,8 +78,10 @@ public void isResourceOfType() { capacityVector.setResource(CUSTOM_RESOURCE, 3, ResourceUnitCapacityType.ABSOLUTE); Assert.assertTrue(capacityVector.isResourceOfType(MEMORY_URI, ResourceUnitCapacityType.WEIGHT)); - Assert.assertTrue(capacityVector.isResourceOfType(VCORES_URI, ResourceUnitCapacityType.PERCENTAGE)); - Assert.assertTrue(capacityVector.isResourceOfType(CUSTOM_RESOURCE, ResourceUnitCapacityType.ABSOLUTE)); + Assert.assertTrue(capacityVector.isResourceOfType(VCORES_URI, + ResourceUnitCapacityType.PERCENTAGE)); + Assert.assertTrue(capacityVector.isResourceOfType(CUSTOM_RESOURCE, + ResourceUnitCapacityType.ABSOLUTE)); } @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/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 17c4aa763aad0..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 @@ -30,7 +30,7 @@ public class TestUniformQueueResourceCalculation extends CapacitySchedulerQueueC private static final Resource QUEUE_A_RES = Resource.newInstance(80 * GB, 10); - private static final Resource QUEUE_B_RES = Resource.newInstance( 170 * GB, + private static final Resource QUEUE_B_RES = Resource.newInstance(170 * GB, 30); private static final Resource QUEUE_A1_RES = Resource.newInstance(50 * GB, 4); 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 c126da407ed7f..4e8f31e1a8515 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 @@ -69,7 +69,8 @@ public class TestQueueCapacityConfigParser { @Test public void testPercentageCapacityConfig() { - QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), QUEUE); + QueueCapacityVector percentageCapacityVector = + capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), QUEUE); QueueCapacityVectorEntry memory = percentageCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcore = percentageCapacityVector.getResource(VCORES_URI); @@ -79,7 +80,8 @@ public void testPercentageCapacityConfig() { Assert.assertEquals(ResourceUnitCapacityType.PERCENTAGE, vcore.getVectorResourceType()); Assert.assertEquals(PERCENTAGE_VALUE, vcore.getResourceValue(), EPSILON); - QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), + QueueCapacityVector rootCapacityVector = + capacityConfigParser.parse(Float.toString(PERCENTAGE_VALUE), CapacitySchedulerConfiguration.ROOT); QueueCapacityVectorEntry memoryRoot = rootCapacityVector.getResource(MEMORY_URI); @@ -94,7 +96,8 @@ public void testPercentageCapacityConfig() { @Test public void testWeightCapacityConfig() { - QueueCapacityVector weightCapacityVector = capacityConfigParser.parse(WEIGHT_VALUE + "w", QUEUE); + QueueCapacityVector weightCapacityVector = capacityConfigParser.parse(WEIGHT_VALUE + "w", + QUEUE); QueueCapacityVectorEntry memory = weightCapacityVector.getResource(MEMORY_URI); QueueCapacityVectorEntry vcore = weightCapacityVector.getResource(VCORES_URI); @@ -114,24 +117,26 @@ public void testAbsoluteCapacityVectorConfig() { conf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_TYPES); ResourceUtils.resetResourceTypes(conf); - QueueCapacityVector absoluteCapacityVector = capacityConfigParser.parse(ABSOLUTE_RESOURCE, QUEUE); + QueueCapacityVector absoluteCapacityVector = capacityConfigParser.parse(ABSOLUTE_RESOURCE, + QUEUE); - Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(MEMORY_URI) - .getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, + absoluteCapacityVector.getResource(MEMORY_URI).getVectorResourceType()); Assert.assertEquals(12 * GB, absoluteCapacityVector.getResource(MEMORY_URI) .getResourceValue(), EPSILON); - Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) - .getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, + absoluteCapacityVector.getResource(VCORES_URI).getVectorResourceType()); Assert.assertEquals(VCORE_ABSOLUTE, absoluteCapacityVector.getResource(VCORES_URI) .getResourceValue(), EPSILON); - Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) - .getVectorResourceType()); + Assert.assertEquals(ResourceUnitCapacityType.ABSOLUTE, + absoluteCapacityVector.getResource(GPU_URI).getVectorResourceType()); Assert.assertEquals(GPU_ABSOLUTE, absoluteCapacityVector.getResource(GPU_URI) .getResourceValue(), EPSILON); - QueueCapacityVector withoutGpuVector = capacityConfigParser.parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); + QueueCapacityVector withoutGpuVector = capacityConfigParser + .parse(ABSOLUTE_RESOURCE_MEMORY_VCORE, QUEUE); Assert.assertEquals(3, withoutGpuVector.getResourceCount()); Assert.assertEquals(0f, withoutGpuVector.getResource(GPU_URI).getResourceValue(), EPSILON);