You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2021/08/31 10:41:29 UTC

[GitHub] [hadoop] 9uapaw opened a new pull request #3358: YARN-10930. Introduce universal capacity resource vector

9uapaw opened a new pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358


   <!--
     Thanks for sending a pull request!
       1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
       2. Make sure your PR title starts with JIRA issue id, e.g., 'HADOOP-17799. Your PR title ...'.
   -->
   
   ### Description of PR
   
   
   ### How was this patch tested?
   
   
   ### For code changes:
   
   - [ ] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
   - [ ] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, `NOTICE-binary` files?
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r733555091



##########
File path: 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() {

Review comment:
       QueueCapacityVector is Iterable itself. This test basically checks the edge cases on how the class behaves when its empty and when it is defined.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r731581051



##########
File path: 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,210 @@
+/**
+ * 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.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.util.UnitsConversionUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * 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.]+)(.*)";
+  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);
+
+  private final List<Parser> parsers = new ArrayList<>();
+
+  public QueueCapacityConfigParser() {
+    parsers.add(new Parser(RESOURCE_PATTERN, this::heterogeneousParser));
+    parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser));
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  public QueueCapacityVector parse(CapacitySchedulerConfiguration conf,
+                                   String queuePath, String label) {
+
+    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();
+    }
+
+    for (Parser parser : parsers) {
+      Matcher matcher = parser.regex.matcher(capacityString);
+      if (matcher.find()) {
+        return parser.parser.apply(matcher);
+      }
+    }
+
+    return new QueueCapacityVector();
+  }
+
+  /**
+   * 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) {
+    QueueCapacityType capacityType = QueueCapacityType.PERCENTAGE;
+    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;
+          }
+        }
+      }
+    }

Review comment:
       Should we handle errors here? I mean if the suffix is not one of the strings defined in QueueCapacityType, what will happen? 

##########
File path: 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,210 @@
+/**
+ * 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.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.util.UnitsConversionUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * 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.]+)(.*)";
+  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);
+
+  private final List<Parser> parsers = new ArrayList<>();
+
+  public QueueCapacityConfigParser() {
+    parsers.add(new Parser(RESOURCE_PATTERN, this::heterogeneousParser));
+    parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser));
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  public QueueCapacityVector parse(CapacitySchedulerConfiguration conf,
+                                   String queuePath, String label) {
+
+    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();
+    }
+
+    for (Parser parser : parsers) {
+      Matcher matcher = parser.regex.matcher(capacityString);
+      if (matcher.find()) {
+        return parser.parser.apply(matcher);
+      }
+    }
+
+    return new QueueCapacityVector();
+  }
+
+  /**
+   * A parser method that is usable on uniform capacity values eg. percentage or

Review comment:
       Nit: e.g.

##########
File path: 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,210 @@
+/**
+ * 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.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.util.UnitsConversionUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * 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.]+)(.*)";
+  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);
+
+  private final List<Parser> parsers = new ArrayList<>();
+
+  public QueueCapacityConfigParser() {
+    parsers.add(new Parser(RESOURCE_PATTERN, this::heterogeneousParser));
+    parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser));
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  public QueueCapacityVector parse(CapacitySchedulerConfiguration conf,
+                                   String queuePath, String label) {
+
+    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();
+    }
+
+    for (Parser parser : parsers) {
+      Matcher matcher = parser.regex.matcher(capacityString);
+      if (matcher.find()) {
+        return parser.parser.apply(matcher);
+      }
+    }
+
+    return new QueueCapacityVector();
+  }
+
+  /**
+   * 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) {
+    QueueCapacityType capacityType = QueueCapacityType.PERCENTAGE;
+    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;
+          }
+        }
+      }
+    }
+
+    return QueueCapacityVector.of(Float.parseFloat(value), capacityType);
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  private QueueCapacityVector heterogeneousParser(Matcher matcher) {
+    // Define resource here.
+    QueueCapacityVector capacityVector = QueueCapacityVector.newInstance();
+
+    /*
+     * 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 capacityVector;
+    }
+    bracketedGroup = bracketedGroup.substring(1, bracketedGroup.length() - 1);
+    // Split by comma and equals delimiter eg. memory=1024, vcores=6 to
+    // [[memory, 1024], [vcores, 6]]

Review comment:
       Nit: This seems like an example config but as per our discussion you represent the array with it. Please refine the comment.

##########
File path: 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,210 @@
+/**
+ * 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.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.util.UnitsConversionUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * 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.]+)(.*)";
+  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);
+
+  private final List<Parser> parsers = new ArrayList<>();
+
+  public QueueCapacityConfigParser() {
+    parsers.add(new Parser(RESOURCE_PATTERN, this::heterogeneousParser));
+    parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser));
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  public QueueCapacityVector parse(CapacitySchedulerConfiguration conf,
+                                   String queuePath, String label) {
+
+    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();
+    }
+
+    for (Parser parser : parsers) {
+      Matcher matcher = parser.regex.matcher(capacityString);
+      if (matcher.find()) {
+        return parser.parser.apply(matcher);
+      }
+    }
+
+    return new QueueCapacityVector();
+  }
+
+  /**
+   * 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) {
+    QueueCapacityType capacityType = QueueCapacityType.PERCENTAGE;
+    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;
+          }
+        }
+      }
+    }
+
+    return QueueCapacityVector.of(Float.parseFloat(value), capacityType);
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  private QueueCapacityVector heterogeneousParser(Matcher matcher) {
+    // Define resource here.
+    QueueCapacityVector capacityVector = QueueCapacityVector.newInstance();
+
+    /*
+     * 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 capacityVector;

Review comment:
       Should this be an invalid scenario? 

##########
File path: 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<String, Long> 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() {

Review comment:
       Could you add a test that tests the float overflow scenario?

##########
File path: 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,210 @@
+/**
+ * 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.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.util.UnitsConversionUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * 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.]+)(.*)";
+  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);
+
+  private final List<Parser> parsers = new ArrayList<>();
+
+  public QueueCapacityConfigParser() {
+    parsers.add(new Parser(RESOURCE_PATTERN, this::heterogeneousParser));
+    parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser));
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  public QueueCapacityVector parse(CapacitySchedulerConfiguration conf,
+                                   String queuePath, String label) {
+
+    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();
+    }
+
+    for (Parser parser : parsers) {
+      Matcher matcher = parser.regex.matcher(capacityString);
+      if (matcher.find()) {
+        return parser.parser.apply(matcher);
+      }
+    }
+
+    return new QueueCapacityVector();
+  }
+
+  /**
+   * 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) {
+    QueueCapacityType capacityType = QueueCapacityType.PERCENTAGE;
+    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;
+          }
+        }
+      }
+    }
+
+    return QueueCapacityVector.of(Float.parseFloat(value), capacityType);
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  private QueueCapacityVector heterogeneousParser(Matcher matcher) {
+    // Define resource here.

Review comment:
       Nit: Comment is a bit misleading, resource -> capacityVector.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r732534390



##########
File path: 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,210 @@
+/**
+ * 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.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.util.UnitsConversionUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * 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.]+)(.*)";
+  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);
+
+  private final List<Parser> parsers = new ArrayList<>();
+
+  public QueueCapacityConfigParser() {
+    parsers.add(new Parser(RESOURCE_PATTERN, this::heterogeneousParser));
+    parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser));
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  public QueueCapacityVector parse(CapacitySchedulerConfiguration conf,
+                                   String queuePath, String label) {
+
+    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();
+    }
+
+    for (Parser parser : parsers) {
+      Matcher matcher = parser.regex.matcher(capacityString);
+      if (matcher.find()) {
+        return parser.parser.apply(matcher);
+      }
+    }
+
+    return new QueueCapacityVector();
+  }
+
+  /**
+   * 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) {
+    QueueCapacityType capacityType = QueueCapacityType.PERCENTAGE;
+    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;
+          }
+        }
+      }
+    }

Review comment:
       Its an invalid config, added a test case to cover this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth edited a comment on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth edited a comment on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-949739306


   Thanks @9uapaw for working on this.
   Very good patch in overall.
   The latest state LGTM, commited to trunk.
   The build failures are not related.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r733560867



##########
File path: 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() {

Review comment:
       The QueueCapacityVector implements Iterable itself. It tests the two edge cases here:
   1. defined vector
   2. empty vector




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth closed pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth closed pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-909173124


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 22s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  40m  0s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m 23s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   1m  4s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 55s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 19s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 59s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m 36s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 44s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 15s | [/patch-mvninstall-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-mvninstall-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  compile  |   0m 16s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  javac  |   0m 16s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  compile  |   0m 17s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | -1 :x: |  javac  |   0m 17s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 49s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 18 new + 67 unchanged - 0 fixed = 85 total (was 67)  |
   | -1 :x: |  mvnsite  |   0m 18s | [/patch-mvnsite-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-mvnsite-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  javadoc  |   0m 19s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  javadoc  |   0m 18s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | -1 :x: |  spotbugs  |   0m 20s | [/patch-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  shadedclient  |   8m 21s |  |  patch has errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 19s | [/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 37s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  81m 36s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3358 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux fb45154c4496 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / af15fd7874cda8f6f8a48452e0e85ed8b90e8432 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/testReport/ |
   | Max. process+thread count | 578 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-947781823


   Thanks @9uapaw ,
   Prod code looks fine for a second loook as well, I only left comments for the test code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r734604592



##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.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 weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(weightCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6);
+  }
+
+  @Test
+  public void testAbsoluteCapacityVectorConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    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, "");
+
+    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("vcores").getVectorResourceType());
+    Assert.assertEquals(6f, absoluteCapacityVector.getResource("vcores").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(3, withoutGpuVector.getResourceCount());
+    Assert.assertEquals(0f, withoutGpuVector.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 mixedCapacityVector =
+        capacityConfigParser.parse(conf, QUEUE, "");
+
+    Assert.assertEquals(QueueCapacityType.ABSOLUTE,
+        mixedCapacityVector.getResource("memory-mb").getVectorResourceType());
+    Assert.assertEquals(1024, mixedCapacityVector.getResource("memory-mb").getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        mixedCapacityVector.getResource("vcores").getVectorResourceType());
+    Assert.assertEquals(50f,
+        mixedCapacityVector.getResource("vcores").getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT,
+        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
+  public void testInvalidCapacityConfigs() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+
+    conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE)
+        + CapacitySchedulerConfiguration.CAPACITY, "50nonexistingsuffix");
+    QueueCapacityVector capacityVectorWithInvalidSuffix =
+        capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> entriesWithInvalidSuffix =
+        Lists.newArrayList(capacityVectorWithInvalidSuffix.iterator());
+    Assert.assertEquals(0, entriesWithInvalidSuffix.size());

Review comment:
       We discussed we need to keep this behavior as the old code also did not fail on these invalid resoure formats.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] brumi1024 commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
brumi1024 commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-942363742


   @9uapaw thanks for the updates. Apart from the additional "missing resource" test case, the change looks good to me.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r712991616



##########
File path: 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.
+ */

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] brumi1024 commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
brumi1024 commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-942363742


   @9uapaw thanks for the updates. Apart from the additional "missing resource" test case, the change looks good to me.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] brumi1024 removed a comment on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
brumi1024 removed a comment on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-942363742


   @9uapaw thanks for the updates. Apart from the additional "missing resource" test case, the change looks good to me.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r734547927



##########
File path: 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() {

Review comment:
       I see, but you are not calling hasNext or next, they are the methods to check the behaviour of your iterator implementation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r712991505



##########
File path: 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]";

Review comment:
       A good question, we should get back to it. What happens, when a resource is not defined in configuration, but is present as a resource type? Should it be zero by default?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r733559765



##########
File path: 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<QueueCapacityVectorEntry> entries = Lists.newArrayList(capacityVector);
+
+    Assert.assertEquals(3, entries.size());
+
+    QueueCapacityVector emptyCapacityVector = new QueueCapacityVector();
+    List<QueueCapacityVectorEntry> 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());

Review comment:
       The guarantee of the resource order is the same as of a HashMap. I do not think that we need to guarantee any specific order as it is intended to be used for two purposes:
   1. human readable format
   2. parsing as a key-value pair
   For both cases the order does not matter.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r734629036



##########
File path: 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() {

Review comment:
       Thanks for explaning. You are right, makes sense.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r734598293



##########
File path: 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() {

Review comment:
       They are called under the hood. Basically you are creating an ArrayList from an Iterator here, so it is a shorthand for calling if (hasNext()) list.append(next())).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r734549822



##########
File path: 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<QueueCapacityVectorEntry> entries = Lists.newArrayList(capacityVector);
+
+    Assert.assertEquals(3, entries.size());
+
+    QueueCapacityVector emptyCapacityVector = new QueueCapacityVector();
+    List<QueueCapacityVectorEntry> 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());

Review comment:
       We don't need to guarantee any order in the production code, I just don't want to have fragile test code that sometimes fails because the order of the elements are not the same all the time, this is what I referred to. Sorry for the confusion.
   Anyway, my problem is that HashMap is not guaranteeing any ordering so the test could break between two runs. https://docs.oracle.com/javase/8/docs/api/java/util/HashMap.html




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-909173124


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 22s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  40m  0s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m 23s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   1m  4s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 55s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 19s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 59s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m 36s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 44s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 15s | [/patch-mvninstall-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-mvninstall-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  compile  |   0m 16s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  javac  |   0m 16s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  compile  |   0m 17s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | -1 :x: |  javac  |   0m 17s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 49s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 18 new + 67 unchanged - 0 fixed = 85 total (was 67)  |
   | -1 :x: |  mvnsite  |   0m 18s | [/patch-mvnsite-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-mvnsite-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  javadoc  |   0m 19s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  javadoc  |   0m 18s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | -1 :x: |  spotbugs  |   0m 20s | [/patch-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  shadedclient  |   8m 21s |  |  patch has errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 19s | [/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 37s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  81m 36s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3358 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux fb45154c4496 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / af15fd7874cda8f6f8a48452e0e85ed8b90e8432 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/testReport/ |
   | Max. process+thread count | 578 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r712991998



##########
File path: 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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<QueueCapacityVector.QueueResourceVectorEntry> {
+  private final ResourceVector resource;
+  private final Map<String, QueueVectorResourceType> resourceTypes
+      = new HashMap<>();
+  private final Set<QueueVectorResourceType>
+      definedResourceTypes = new HashSet<>();
+
+  public QueueCapacityVector(ResourceVector resource) {
+    this.resource = resource;
+  }
+
+  public static QueueCapacityVector empty() {

Review comment:
       I agree with these. Fixed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-929511731


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 16s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  2s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 14s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 53s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 46s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 58s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 44s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 55s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m  5s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 50s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 57s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 47s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 47s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 39s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 14 new + 119 unchanged - 0 fixed = 133 total (was 119)  |
   | +1 :green_heart: |  mvnsite  |   0m 51s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javadoc  |   0m 33s | [/results-javadoc-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/artifact/out/results-javadoc-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 2 new + 344 unchanged - 0 fixed = 346 total (was 344)  |
   | +1 :green_heart: |  spotbugs  |   1m 58s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  22m 53s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  99m 45s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 196m 42s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3358 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux cc3f0e335772 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f3fe410e221383d514d3bf5a075c86a0d7515bb6 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/testReport/ |
   | Max. process+thread count | 912 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-929511731


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 16s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  2s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 14s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 53s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 46s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 58s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 44s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 55s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m  5s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 50s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 57s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 47s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 47s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 39s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 14 new + 119 unchanged - 0 fixed = 133 total (was 119)  |
   | +1 :green_heart: |  mvnsite  |   0m 51s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javadoc  |   0m 33s | [/results-javadoc-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/artifact/out/results-javadoc-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 2 new + 344 unchanged - 0 fixed = 346 total (was 344)  |
   | +1 :green_heart: |  spotbugs  |   1m 58s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  22m 53s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  99m 45s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 196m 42s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3358 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux cc3f0e335772 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f3fe410e221383d514d3bf5a075c86a0d7515bb6 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/testReport/ |
   | Max. process+thread count | 912 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/7/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] brumi1024 commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
brumi1024 commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r707501296



##########
File path: 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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<QueueCapacityVector.QueueResourceVectorEntry> {
+  private final ResourceVector resource;
+  private final Map<String, QueueVectorResourceType> resourceTypes
+      = new HashMap<>();
+  private final Set<QueueVectorResourceType>
+      definedResourceTypes = new HashSet<>();
+
+  public QueueCapacityVector(ResourceVector resource) {
+    this.resource = resource;
+  }
+
+  public static QueueCapacityVector empty() {

Review comment:
       Empty is a bit misleading here (at least for me), it can mean that the values are emptied for the object in question. newInstance for example is a bit more straightforward: https://www.informit.com/articles/article.aspx?p=1216151 
   
   Also can you please add javadoc to these factory methods?

##########
File path: 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]";

Review comment:
       Can you please add a testcase where the gpu resource is added to the resource_types, but it isn't configured in absolute_resources? Just for future development, to ensure that this will be supported in the future.

##########
File path: 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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<Map.Entry<String, Float>> {
+  private final Map<String, Float> resourcesByName = new HashMap<>();
+
+  public static ResourceVector empty() {

Review comment:
       Same here, newInstance would be more straightforward.

##########
File path: 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
##########
@@ -399,6 +401,9 @@ protected void setupQueueConfigs(Resource clusterResource,
             .getConfiguredNodeLabels(queuePath);
       }
 
+      configuredCapacityVectors = csContext.getConfiguration()

Review comment:
       No need to use csContext here, configuration is a method parameter.

##########
File path: 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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<Map.Entry<String, Float>> {
+  private final Map<String, Float> 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) {

Review comment:
       Instead of uniform maybe .of(float value) or valueOf()?

##########
File path: 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.
+ */

Review comment:
       If we plan to make this as flexible as possible can you please write a short summary on how to add a new parser for a possible new resource description method?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r732879744



##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(1).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(1).getResourceValue(), 1e-6);

Review comment:
       can you also add the resource types (memory, vcores) assertions here as well?

##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.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 weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(weightCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6);
+  }
+
+  @Test
+  public void testAbsoluteCapacityVectorConfig() {

Review comment:
       Please use constants for "memory-mb" & "yarn.io/gpu" from class: ResourceInformation.

##########
File path: 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,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.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<String, Long> 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() {

Review comment:
       all testcase names should start with "test" by JUnit convention

##########
File path: 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,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.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);

Review comment:
       Pls extract the precision into a constant.

##########
File path: 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,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.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<String, Long> 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() {

Review comment:
       Can you also add a subtract case that goes to negative, e.g. 10-25?

##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(1).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(1).getResourceValue(), 1e-6);

Review comment:
       Note: Applies to all other testcases.

##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.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 weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(weightCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6);
+  }
+
+  @Test
+  public void testAbsoluteCapacityVectorConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    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, "");
+
+    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("vcores").getVectorResourceType());
+    Assert.assertEquals(6f, absoluteCapacityVector.getResource("vcores").getResourceValue(), 1e-6);

Review comment:
       Can you extract the values of vcores, gpu and memory (6, 10 and 12, respectively) to some constants? 
   I can see they are coming from here: 
   ```
     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]";
   ```
   I think it would greatly improve readability to extract the numbers to constants and use them in these 3 constants + the expectations as well.

##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.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 weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(weightCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6);
+  }
+
+  @Test
+  public void testAbsoluteCapacityVectorConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    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, "");
+
+    Assert.assertEquals(QueueCapacityType.ABSOLUTE, absoluteCapacityVector.getResource("memory-mb").getVectorResourceType());

Review comment:
       Could you extract the precision to a constant? (1e-6)

##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);

Review comment:
       Can you add testing the resource names (memory, vcores I assume) as well?
   I suppose both the memory and vcores should be set to 50, but it's not too straightforward from the test code.

##########
File path: 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,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.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<String, Long> 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);
+
+    // 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
+  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));

Review comment:
       IntelliJ complains with: "'equals' between objects of inconvertible types 'ResourceVector' and 'Resource' "

##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.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 weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(weightCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6);
+  }
+
+  @Test
+  public void testAbsoluteCapacityVectorConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    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, "");
+
+    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("vcores").getVectorResourceType());
+    Assert.assertEquals(6f, absoluteCapacityVector.getResource("vcores").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(3, withoutGpuVector.getResourceCount());
+    Assert.assertEquals(0f, withoutGpuVector.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 mixedCapacityVector =
+        capacityConfigParser.parse(conf, QUEUE, "");
+
+    Assert.assertEquals(QueueCapacityType.ABSOLUTE,
+        mixedCapacityVector.getResource("memory-mb").getVectorResourceType());
+    Assert.assertEquals(1024, mixedCapacityVector.getResource("memory-mb").getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        mixedCapacityVector.getResource("vcores").getVectorResourceType());
+    Assert.assertEquals(50f,
+        mixedCapacityVector.getResource("vcores").getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT,
+        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
+  public void testInvalidCapacityConfigs() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+
+    conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE)
+        + CapacitySchedulerConfiguration.CAPACITY, "50nonexistingsuffix");
+    QueueCapacityVector capacityVectorWithInvalidSuffix =
+        capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> entriesWithInvalidSuffix =
+        Lists.newArrayList(capacityVectorWithInvalidSuffix.iterator());
+    Assert.assertEquals(0, entriesWithInvalidSuffix.size());

Review comment:
       Shouldn't this be an erroneous case instead?

##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.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 weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(weightCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6);
+  }
+
+  @Test
+  public void testAbsoluteCapacityVectorConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    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, "");
+
+    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("vcores").getVectorResourceType());
+    Assert.assertEquals(6f, absoluteCapacityVector.getResource("vcores").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(3, withoutGpuVector.getResourceCount());
+    Assert.assertEquals(0f, withoutGpuVector.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 mixedCapacityVector =
+        capacityConfigParser.parse(conf, QUEUE, "");
+
+    Assert.assertEquals(QueueCapacityType.ABSOLUTE,
+        mixedCapacityVector.getResource("memory-mb").getVectorResourceType());
+    Assert.assertEquals(1024, mixedCapacityVector.getResource("memory-mb").getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        mixedCapacityVector.getResource("vcores").getVectorResourceType());
+    Assert.assertEquals(50f,
+        mixedCapacityVector.getResource("vcores").getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT,
+        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
+  public void testInvalidCapacityConfigs() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+
+    conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE)
+        + CapacitySchedulerConfiguration.CAPACITY, "50nonexistingsuffix");
+    QueueCapacityVector capacityVectorWithInvalidSuffix =
+        capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> 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, "");
+    List<QueueCapacityVectorEntry> 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<QueueCapacityVectorEntry> resources =
+        Lists.newArrayList(invalidCapacityVector.iterator());
+    Assert.assertEquals(0, resources.size());
+
+    conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE)
+        + CapacitySchedulerConfiguration.CAPACITY, "[]");
+    QueueCapacityVector emptyBracketCapacityVector =
+        capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> emptyEntries =
+        Lists.newArrayList(emptyBracketCapacityVector.iterator());
+    Assert.assertEquals(0, resources.size());
+
+    conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE)
+        + CapacitySchedulerConfiguration.CAPACITY, "");
+    QueueCapacityVector emptyCapacity =
+        capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> emptyResources =
+        Lists.newArrayList(emptyCapacity.iterator());
+    Assert.assertEquals(emptyResources.size(), 0);
+
+    conf.unset(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE)
+        + CapacitySchedulerConfiguration.CAPACITY);
+    QueueCapacityVector nonSetCapacity =
+        capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> nonSetResources =
+        Lists.newArrayList(nonSetCapacity.iterator());
+    Assert.assertEquals(nonSetResources.size(), 0);
+  }
+
+  @Test
+  public void testResourceVector() {

Review comment:
       why this is empty?

##########
File path: 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,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.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);

Review comment:
       Pls use resource type name constants from ResourceInformation.

##########
File path: 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,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.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<String, Long> 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() {

Review comment:
       Can you also add an increment case that goes from negative to positive, e.g. -10+25?

##########
File path: 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);

Review comment:
       Pls use resource type name constants from ResourceInformation.
   

##########
File path: 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,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.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<String, Long> 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);
+
+    // 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
+  public void testEquals() {
+    ResourceVector resourceVector = ResourceVector.of(13);
+    ResourceVector resourceVectorOther = ResourceVector.of(14);
+    Resource resource = Resource.newInstance(13, 13);
+
+    Assert.assertFalse(resourceVector.equals(null));

Review comment:
       Use this: 
   `Assert.assertNotEquals(null, resourceVector);`

##########
File path: 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,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.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<String, Long> 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);
+
+    // 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
+  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));

Review comment:
       Use this: 
   `Assert.assertEquals(resourceVector, resourceVectorOther)`

##########
File path: 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() {

Review comment:
       All testcase names should start with "test"

##########
File path: 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(

Review comment:
       You are not testing here that memory=10 and vcores=6

##########
File path: 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() {

Review comment:
       The name of the testcase is misleading: I can't see any real testing for the iterator logic here.

##########
File path: 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,208 @@
+/**
+ * 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.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;
+
+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 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 final QueueCapacityConfigParser capacityConfigParser
+      = new QueueCapacityConfigParser();
+
+  @Test
+  public void testPercentageCapacityConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setCapacity(QUEUE, 50);
+
+    QueueCapacityVector percentageCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(percentageCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(50f, resources.get(1).getResourceValue(), 1e-6);
+
+    QueueCapacityVector rootCapacityVector = capacityConfigParser.parse(conf,
+        CapacitySchedulerConfiguration.ROOT, "");
+    List<QueueCapacityVectorEntry> rootResources =
+        Lists.newArrayList(rootCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        rootResources.get(0).getVectorResourceType());
+    Assert.assertEquals(100f,
+        rootResources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.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 weightCapacityVector = capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> resources = Lists.newArrayList(weightCapacityVector.iterator());
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(0).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(0).getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT, resources.get(1).getVectorResourceType());
+    Assert.assertEquals(6f, resources.get(1).getResourceValue(), 1e-6);
+  }
+
+  @Test
+  public void testAbsoluteCapacityVectorConfig() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    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, "");
+
+    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("vcores").getVectorResourceType());
+    Assert.assertEquals(6f, absoluteCapacityVector.getResource("vcores").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(3, withoutGpuVector.getResourceCount());
+    Assert.assertEquals(0f, withoutGpuVector.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 mixedCapacityVector =
+        capacityConfigParser.parse(conf, QUEUE, "");
+
+    Assert.assertEquals(QueueCapacityType.ABSOLUTE,
+        mixedCapacityVector.getResource("memory-mb").getVectorResourceType());
+    Assert.assertEquals(1024, mixedCapacityVector.getResource("memory-mb").getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.PERCENTAGE,
+        mixedCapacityVector.getResource("vcores").getVectorResourceType());
+    Assert.assertEquals(50f,
+        mixedCapacityVector.getResource("vcores").getResourceValue(), 1e-6);
+
+    Assert.assertEquals(QueueCapacityType.WEIGHT,
+        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
+  public void testInvalidCapacityConfigs() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+
+    conf.set(CapacitySchedulerConfiguration.getQueuePrefix(QUEUE)
+        + CapacitySchedulerConfiguration.CAPACITY, "50nonexistingsuffix");
+    QueueCapacityVector capacityVectorWithInvalidSuffix =
+        capacityConfigParser.parse(conf, QUEUE, "");
+    List<QueueCapacityVectorEntry> entriesWithInvalidSuffix =
+        Lists.newArrayList(capacityVectorWithInvalidSuffix.iterator());
+    Assert.assertEquals(0, entriesWithInvalidSuffix.size());

Review comment:
       TBH, most of these cases should be an error case.
   I would also prefer if you could add all of these cases to a separate testcase if possible.

##########
File path: 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<QueueCapacityVectorEntry> entries = Lists.newArrayList(capacityVector);
+
+    Assert.assertEquals(3, entries.size());
+
+    QueueCapacityVector emptyCapacityVector = new QueueCapacityVector();
+    List<QueueCapacityVectorEntry> 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());

Review comment:
       Is it guaranteed that the custom resource always goes first? 
   Is this intentionally passing so that the resource names sorted by their names or something else is happening? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-949728230


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 58s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 17s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m 11s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 54s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 45s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m  3s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 51s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m  6s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  24m 11s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |   1m  0s | [/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/11/artifact/out/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 14 new + 42 unchanged - 14 fixed = 56 total (was 56)  |
   | +1 :green_heart: |  compile  |   0m 49s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |   0m 49s | [/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/11/artifact/out/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 8 new + 37 unchanged - 8 fixed = 45 total (was 45)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 39s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/11/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 2 new + 116 unchanged - 0 fixed = 118 total (was 116)  |
   | +1 :green_heart: |  mvnsite  |   0m 52s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m  1s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  23m 57s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 100m 23s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 31s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 199m 24s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/11/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3358 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 69871bfecd68 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f4d253faebebbff7ae4580b2c56fdd42bde965e4 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/11/testReport/ |
   | Max. process+thread count | 956 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/11/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] brumi1024 commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
brumi1024 commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r728123268



##########
File path: 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]";

Review comment:
       After an offline discussion the missing resources should be zero by default.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r732535035



##########
File path: 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,210 @@
+/**
+ * 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.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.util.UnitsConversionUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * 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.]+)(.*)";
+  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);
+
+  private final List<Parser> parsers = new ArrayList<>();
+
+  public QueueCapacityConfigParser() {
+    parsers.add(new Parser(RESOURCE_PATTERN, this::heterogeneousParser));
+    parsers.add(new Parser(UNIFORM_PATTERN, this::uniformParser));
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  public QueueCapacityVector parse(CapacitySchedulerConfiguration conf,
+                                   String queuePath, String label) {
+
+    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();
+    }
+
+    for (Parser parser : parsers) {
+      Matcher matcher = parser.regex.matcher(capacityString);
+      if (matcher.find()) {
+        return parser.parser.apply(matcher);
+      }
+    }
+
+    return new QueueCapacityVector();
+  }
+
+  /**
+   * 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) {
+    QueueCapacityType capacityType = QueueCapacityType.PERCENTAGE;
+    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;
+          }
+        }
+      }
+    }
+
+    return QueueCapacityVector.of(Float.parseFloat(value), capacityType);
+  }
+
+  /**
+   * 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 capacity vector
+   */
+  private QueueCapacityVector heterogeneousParser(Matcher matcher) {
+    // Define resource here.
+    QueueCapacityVector capacityVector = QueueCapacityVector.newInstance();
+
+    /*
+     * 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 capacityVector;

Review comment:
       Its an impossible scenario, because we skip this parser for empty brackets, therefore I have removed this redundant check.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] brumi1024 commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
brumi1024 commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r728123268



##########
File path: 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]";

Review comment:
       After an offline discussion the missing resources should be zero by default.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-949739306


   Thanks @9uapaw for working on this.
   Very good patch in overall.
   The latest state LGTM, commited to trunk.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-909173124


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 22s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  40m  0s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m 23s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   1m  4s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 55s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 19s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 59s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m 36s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 44s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 15s | [/patch-mvninstall-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-mvninstall-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  compile  |   0m 16s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  javac  |   0m 16s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  compile  |   0m 17s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | -1 :x: |  javac  |   0m 17s | [/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-compile-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 49s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 18 new + 67 unchanged - 0 fixed = 85 total (was 67)  |
   | -1 :x: |  mvnsite  |   0m 18s | [/patch-mvnsite-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-mvnsite-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  javadoc  |   0m 19s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  javadoc  |   0m 18s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-server-resourcemanager in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | -1 :x: |  spotbugs  |   0m 20s | [/patch-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | -1 :x: |  shadedclient  |   8m 21s |  |  patch has errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 19s | [/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-server-resourcemanager in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 37s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  81m 36s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3358 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux fb45154c4496 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / af15fd7874cda8f6f8a48452e0e85ed8b90e8432 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/testReport/ |
   | Max. process+thread count | 578 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] szilard-nemeth commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
szilard-nemeth commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-947758872


   Hi @9uapaw,
   I can't see that the patch would be formatted for 100 chars line length. 
   Please reformat your new files + modified lines accordingly. 
   I will leave some comments as well.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-947604815


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 53s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m  2s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 52s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 46s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 57s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 45s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 55s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  24m  2s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 56s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |   1m  2s | [/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/10/artifact/out/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 22 new + 33 unchanged - 22 fixed = 55 total (was 55)  |
   | +1 :green_heart: |  compile  |   0m 49s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |   0m 49s | [/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/10/artifact/out/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 16 new + 28 unchanged - 16 fixed = 44 total (was 44)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 40s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/10/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 12 new + 116 unchanged - 0 fixed = 128 total (was 116)  |
   | +1 :green_heart: |  mvnsite  |   0m 53s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m 10s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  23m 40s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  99m 35s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 198m 34s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3358 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux e03724905626 4.15.0-147-generic #151-Ubuntu SMP Fri Jun 18 19:21:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / efc535e90e7d9fe83622410944c0dcea4b5928f0 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/10/testReport/ |
   | Max. process+thread count | 935 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/10/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#discussion_r733562371



##########
File path: 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<QueueCapacityVectorEntry> entries = Lists.newArrayList(capacityVector);
+
+    Assert.assertEquals(3, entries.size());
+
+    QueueCapacityVector emptyCapacityVector = new QueueCapacityVector();
+    List<QueueCapacityVectorEntry> 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());

Review comment:
       The guarantee of the order is the same as of a HashMap. I do not think we need any kind of guarantee in ordering as this string is intended to be used for two reasons:
   1. human readable format
   2. parsing it as key-value pairs
   For both cases the ordering should not matter.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-949745182


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  3s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m  2s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 54s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 46s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m  0s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m  1s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 32s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 53s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |   1m  0s | [/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/12/artifact/out/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 14 new + 41 unchanged - 15 fixed = 55 total (was 56)  |
   | +1 :green_heart: |  compile  |   0m 48s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |   0m 48s | [/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/12/artifact/out/results-compile-javac-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 11 new + 33 unchanged - 11 fixed = 44 total (was 44)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 39s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/12/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 2 new + 116 unchanged - 0 fixed = 118 total (was 116)  |
   | +1 :green_heart: |  mvnsite  |   0m 52s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m  7s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  24m  8s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 101m 36s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 30s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 200m 45s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/12/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3358 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 6ac6132efe23 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 057ecd823a7001ad46f0f4bfc4197ba1f8ba90ae |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/12/testReport/ |
   | Max. process+thread count | 940 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3358/12/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] brumi1024 removed a comment on pull request #3358: YARN-10930. Introduce universal capacity resource vector

Posted by GitBox <gi...@apache.org>.
brumi1024 removed a comment on pull request #3358:
URL: https://github.com/apache/hadoop/pull/3358#issuecomment-942363742


   @9uapaw thanks for the updates. Apart from the additional "missing resource" test case, the change looks good to me.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org