Quellcode durchsuchen

YARN-11514. Extend SchedulerResponse with capacityVector (#5989)

Co-authored-by: Benjamin Teke <bteke@cloudera.com>
Benjamin Teke vor 1 Jahr
Ursprung
Commit
f51162d70b
49 geänderte Dateien mit 3389 neuen und 28 gelöschten Zeilen
  1. 4 0
      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
  2. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
  3. 3 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
  4. 3 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
  5. 13 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/PartitionQueueCapacitiesInfo.java
  6. 8 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacitiesInfo.java
  7. 53 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacityVectorEntryInfo.java
  8. 64 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacityVectorInfo.java
  9. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java
  10. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-0.json
  11. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-16.json
  12. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-32.json
  13. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-legacy-0.json
  14. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-0.json
  15. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-16.json
  16. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-32.json
  17. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-legacy-0.json
  18. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-0.json
  19. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-16.json
  20. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-32.json
  21. 200 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-after-aqc.json
  22. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-before-aqc.json
  23. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-0.json
  24. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-16.json
  25. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-32.json
  26. 200 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-after-aqc.json
  27. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-before-aqc.json
  28. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentage-0.json
  29. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentage-16.json
  30. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentage-32.json
  31. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeight-0.json
  32. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeight-16.json
  33. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeight-32.json
  34. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeightMixed-0.json
  35. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeightMixed-16.json
  36. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeightMixed-32.json
  37. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndWeight-0.json
  38. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndWeight-16.json
  39. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndWeight-32.json
  40. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerPercentageAndWeight-0.json
  41. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerPercentageAndWeight-16.json
  42. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerPercentageAndWeight-32.json
  43. 50 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json
  44. 55 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-NodeLabelDefaultAPI.xml
  45. 50 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.json
  46. 55 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.xml
  47. 40 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PercentageModeLegacyAutoCreation.json
  48. 50 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.json
  49. 55 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.xml

+ 4 - 0
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

@@ -281,5 +281,9 @@ public class QueueCapacityVector implements
     public String getResourceName() {
       return resourceName;
     }
+
+    public String getResourceWithPostfix() {
+      return resourceValue + vectorResourceType.getPostfix();
+    }
   }
 }

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper.CapacityS
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL;
 import static org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo.getSortedQueueAclInfoList;
 
 @XmlRootElement(name = "capacityScheduler")
@@ -47,6 +48,7 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
   protected float maxCapacity;
   protected float weight;
   protected float normalizedWeight;
+  protected QueueCapacityVectorInfo queueCapacityVectorInfo;
   protected String queueName;
   private String queuePath;
   protected int maxParallelApps;
@@ -78,6 +80,8 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
     this.queuePath = parent.getQueuePath();
     this.usedCapacity = parent.getUsedCapacity() * 100;
     this.capacity = parent.getCapacity() * 100;
+    this.queueCapacityVectorInfo = new QueueCapacityVectorInfo(
+            parent.getConfiguredCapacityVector(NO_LABEL));
     float max = parent.getMaximumCapacity();
     if (max < EPSILON || max > 1f)
       max = 1f;
@@ -86,8 +90,7 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
     this.normalizedWeight = parent.getQueueCapacities().getNormalizedWeight();
     this.maxParallelApps = parent.getMaxParallelApps();
 
-    capacities = new QueueCapacitiesInfo(parent.getQueueCapacities(),
-        parent.getQueueResourceQuotas(), false);
+    capacities = new QueueCapacitiesInfo(parent, false);
     queues = getQueues(cs, parent);
     health = new CapacitySchedulerHealthInfo(cs);
     maximumAllocation = new ResourceInfo(parent.getMaximumAllocation());

+ 3 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java

@@ -25,13 +25,12 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractLeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
     .AutoCreatedLeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo;
 
 @XmlRootElement
@@ -105,9 +104,8 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   }
 
   @Override
-  protected void populateQueueCapacities(QueueCapacities qCapacities,
-      QueueResourceQuotas qResQuotas) {
-    capacities = new QueueCapacitiesInfo(qCapacities, qResQuotas);
+  protected void populateQueueCapacities(CSQueue queue) {
+    capacities = new QueueCapacitiesInfo(queue, true);
   }
 
   public int getNumActiveApplications() {

+ 3 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.security.AccessType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractParentQueue;
@@ -41,7 +40,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper.CapacitySchedulerInfoHelper;
 
 @XmlRootElement
@@ -136,9 +134,7 @@ public class CapacitySchedulerQueueInfo {
       nodeLabels.addAll(labelSet);
       Collections.sort(nodeLabels);
     }
-    QueueCapacities qCapacities = q.getQueueCapacities();
-    QueueResourceQuotas qResQuotas = q.getQueueResourceQuotas();
-    populateQueueCapacities(qCapacities, qResQuotas);
+    populateQueueCapacities(q);
 
     mode = CapacitySchedulerInfoHelper.getMode(q);
     queueType = CapacitySchedulerInfoHelper.getQueueType(q);
@@ -210,10 +206,8 @@ public class CapacitySchedulerQueueInfo {
     resources = new ResourcesInfo(queueResourceUsage, false);
   }
 
-  protected void populateQueueCapacities(QueueCapacities qCapacities,
-      QueueResourceQuotas qResQuotas) {
-    capacities = new QueueCapacitiesInfo(qCapacities, qResQuotas,
-        false);
+  protected void populateQueueCapacities(CSQueue queue) {
+    capacities = new QueueCapacitiesInfo(queue, false);
   }
 
   public float getCapacity() {

+ 13 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/PartitionQueueCapacitiesInfo.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 public class PartitionQueueCapacitiesInfo {
   private String partitionName;
 
+  private QueueCapacityVectorInfo queueCapacityVectorInfo;
   private float capacity;
   private float usedCapacity;
   private float maxCapacity = 100;
@@ -49,13 +50,15 @@ public class PartitionQueueCapacitiesInfo {
   public PartitionQueueCapacitiesInfo() {
   }
 
-  public PartitionQueueCapacitiesInfo(String partitionName, float capacity,
-      float usedCapacity, float maxCapacity, float absCapacity,
+  public PartitionQueueCapacitiesInfo(String partitionName,
+      QueueCapacityVectorInfo queueCapacityVectorInfo,
+      float capacity, float usedCapacity, float maxCapacity, float absCapacity,
       float absUsedCapacity, float absMaxCapacity, float maxAMLimitPercentage,
       float weight, float normalizedWeight,
       Resource confMinRes, Resource confMaxRes, Resource effMinRes,
       Resource effMaxRes) {
     super();
+    this.queueCapacityVectorInfo = queueCapacityVectorInfo;
     this.partitionName = partitionName;
     this.capacity = capacity;
     this.usedCapacity = usedCapacity;
@@ -72,6 +75,14 @@ public class PartitionQueueCapacitiesInfo {
     this.effectiveMaxResource = new ResourceInfo(effMaxRes);
   }
 
+  public QueueCapacityVectorInfo getQueueCapacityVectorInfo() {
+    return queueCapacityVectorInfo;
+  }
+
+  public void setQueueCapacityVectorInfo(QueueCapacityVectorInfo queueCapacityVectorInfo) {
+    this.queueCapacityVectorInfo = queueCapacityVectorInfo;
+  }
+
   public float getCapacity() {
     return capacity;
   }

+ 8 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacitiesInfo.java

@@ -25,6 +25,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 
 /**
@@ -39,12 +40,13 @@ public class QueueCapacitiesInfo {
   public QueueCapacitiesInfo() {
   }
 
-  public QueueCapacitiesInfo(QueueCapacities capacities,
-      QueueResourceQuotas resourceQuotas,
-      boolean considerAMUsage) {
+  public QueueCapacitiesInfo(CSQueue queue, boolean considerAMUsage) {
+    QueueCapacities capacities = queue.getQueueCapacities();
+    QueueResourceQuotas resourceQuotas = queue.getQueueResourceQuotas();
     if (capacities == null) {
       return;
     }
+    QueueCapacityVectorInfo queueCapacityVectorInfo;
     float capacity;
     float usedCapacity;
     float maxCapacity;
@@ -55,6 +57,8 @@ public class QueueCapacitiesInfo {
     float weight;
     float normalizedWeight;
     for (String partitionName : capacities.getExistingNodeLabels()) {
+      queueCapacityVectorInfo = new QueueCapacityVectorInfo(
+           queue.getConfiguredCapacityVector(partitionName));
       usedCapacity = capacities.getUsedCapacity(partitionName) * 100;
       capacity = capacities.getCapacity(partitionName) * 100;
       maxCapacity = capacities.getMaximumCapacity(partitionName);
@@ -72,7 +76,7 @@ public class QueueCapacitiesInfo {
       weight = capacities.getWeight(partitionName);
       normalizedWeight = capacities.getNormalizedWeight(partitionName);
       queueCapacitiesByPartition.add(new PartitionQueueCapacitiesInfo(
-          partitionName, capacity, usedCapacity, maxCapacity, absCapacity,
+          partitionName, queueCapacityVectorInfo, capacity, usedCapacity, maxCapacity, absCapacity,
           absUsedCapacity, absMaxCapacity,
           considerAMUsage ? maxAMLimitPercentage : 0f,
           weight, normalizedWeight,
@@ -83,11 +87,6 @@ public class QueueCapacitiesInfo {
     }
   }
 
-  public QueueCapacitiesInfo(QueueCapacities capacities,
-      QueueResourceQuotas resourceQuotas) {
-    this(capacities, resourceQuotas, true);
-  }
-
   public void add(PartitionQueueCapacitiesInfo partitionQueueCapacitiesInfo) {
     queueCapacitiesByPartition.add(partitionQueueCapacitiesInfo);
   }

+ 53 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacityVectorEntryInfo.java

@@ -0,0 +1,53 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class QueueCapacityVectorEntryInfo {
+  private String resourceName;
+  private String resourceValue;
+
+  public QueueCapacityVectorEntryInfo() {
+  }
+
+  public QueueCapacityVectorEntryInfo(String resourceName, String resourceValue) {
+    this.resourceName = resourceName;
+    this.resourceValue = resourceValue;
+  }
+
+  public String getResourceName() {
+    return this.resourceName;
+  }
+
+  public void setResourceName(String resourceName) {
+    this.resourceName = resourceName;
+  }
+
+  public String getResourceValue() {
+    return this.resourceValue;
+  }
+
+  public void setResourceValue(String resourceValue) {
+    this.resourceValue = resourceValue;
+  }
+}

+ 64 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueCapacityVectorInfo.java

@@ -0,0 +1,64 @@
+/**
+ * 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.webapp.dao;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacityVector;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import java.util.ArrayList;
+import java.util.List;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class QueueCapacityVectorInfo {
+  private String configuredCapacityVector;
+  private List<QueueCapacityVectorEntryInfo> capacityVectorEntries;
+
+  public QueueCapacityVectorInfo() {
+  }
+
+  public QueueCapacityVectorInfo(QueueCapacityVector queueCapacityVector) {
+    this.configuredCapacityVector = queueCapacityVector.toString();
+    this.capacityVectorEntries = new ArrayList<>();
+    for (QueueCapacityVector.QueueCapacityVectorEntry
+            queueCapacityVectorEntry : queueCapacityVector) {
+      this.capacityVectorEntries.add(
+              new QueueCapacityVectorEntryInfo(queueCapacityVectorEntry.getResourceName(),
+                    queueCapacityVectorEntry.getResourceWithPostfix()));
+    }
+  }
+
+  public String getConfiguredCapacityVector() {
+    return configuredCapacityVector;
+  }
+
+  public void setConfiguredCapacityVector(String configuredCapacityVector) {
+    this.configuredCapacityVector = configuredCapacityVector;
+  }
+
+  public List<QueueCapacityVectorEntryInfo> getCapacityVectorEntries() {
+    return capacityVectorEntries;
+  }
+
+  public void setCapacityVectorEntries(List<QueueCapacityVectorEntryInfo> capacityVectorEntries) {
+    this.capacityVectorEntries = capacityVectorEntries;
+  }
+}

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java

@@ -580,7 +580,7 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase {
     JSONObject info = json.getJSONObject("scheduler");
     assertEquals("incorrect number of elements", 1, info.length());
     info = info.getJSONObject("schedulerInfo");
-    assertEquals("incorrect number of elements", 24, info.length());
+    assertEquals("incorrect number of elements", 25, info.length());
     JSONObject capacitiesJsonObject = info.getJSONObject(CAPACITIES);
     JSONArray partitionsCapsArray =
         capacitiesJsonObject.getJSONArray(QUEUE_CAPACITIES_BY_PARTITION);

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12288.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4096.0,vcores=4.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4096.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12288.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4096.0,vcores=4.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4096.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12288.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4096.0,vcores=4.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4096.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-legacy-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12288.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4096.0,vcores=4.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4096.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=37.5%,vcores=37.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "37.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "37.5%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.5%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "75.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "75.0%"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=37.5%,vcores=37.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "37.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "37.5%"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.5%"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "75.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "75.0%"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=37.5%,vcores=37.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "37.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "37.5%"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.5%"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "75.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "75.0%"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-legacy-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=37.5%,vcores=37.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "37.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "37.5%"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.5%"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "75.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "75.0%"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 200 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-after-aqc.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -544,6 +564,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=10.0w,vcores=10.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "10.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "10.0w"
+                    } ]
+                  },
                   "capacity" : 41.666664,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=10.0w,vcores=10.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "10.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "10.0w"
+                    } ]
+                  },
                   "capacity" : 41.666664,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1498,6 +1538,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 50,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -1967,6 +2017,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 50,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -2422,6 +2482,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 4.166667,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2767,6 +2837,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 100,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -3222,6 +3302,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 4.166667,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -3562,6 +3652,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 100,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -4017,6 +4117,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 4.166667,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -4373,6 +4483,16 @@
                       "capacities" : {
                         "queueCapacitiesByPartition" : [ {
                           "partitionName" : "",
+                          "queueCapacityVectorInfo" : {
+                            "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                            "capacityVectorEntries" : [ {
+                              "resourceName" : "memory-mb",
+                              "resourceValue" : "1.0w"
+                            }, {
+                              "resourceName" : "vcores",
+                              "resourceValue" : "1.0w"
+                            } ]
+                          },
                           "capacity" : 100,
                           "usedCapacity" : 0,
                           "maxCapacity" : 100,
@@ -4828,6 +4948,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 100,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -5138,6 +5268,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 4.166667,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -5448,6 +5588,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -5793,6 +5943,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -6262,6 +6422,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -6731,6 +6901,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -7186,6 +7366,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -7460,6 +7650,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-before-aqc.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -873,6 +903,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1342,6 +1382,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1811,6 +1861,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2266,6 +2326,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2540,6 +2610,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 200 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-after-aqc.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -544,6 +564,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=10.0w,vcores=10.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "10.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "10.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=10.0w,vcores=10.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "10.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "10.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1498,6 +1538,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 0,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -1967,6 +2017,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 0,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -2422,6 +2482,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2767,6 +2837,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 0,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -3222,6 +3302,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -3562,6 +3652,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 0,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -4017,6 +4117,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -4373,6 +4483,16 @@
                       "capacities" : {
                         "queueCapacitiesByPartition" : [ {
                           "partitionName" : "",
+                          "queueCapacityVectorInfo" : {
+                            "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                            "capacityVectorEntries" : [ {
+                              "resourceName" : "memory-mb",
+                              "resourceValue" : "1.0w"
+                            }, {
+                              "resourceName" : "vcores",
+                              "resourceValue" : "1.0w"
+                            } ]
+                          },
                           "capacity" : 0,
                           "usedCapacity" : 0,
                           "maxCapacity" : 100,
@@ -4828,6 +4948,16 @@
                   "capacities" : {
                     "queueCapacitiesByPartition" : [ {
                       "partitionName" : "",
+                      "queueCapacityVectorInfo" : {
+                        "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                        "capacityVectorEntries" : [ {
+                          "resourceName" : "memory-mb",
+                          "resourceValue" : "1.0w"
+                        }, {
+                          "resourceName" : "vcores",
+                          "resourceValue" : "1.0w"
+                        } ]
+                      },
                       "capacity" : 0,
                       "usedCapacity" : 0,
                       "maxCapacity" : 100,
@@ -5138,6 +5268,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -5448,6 +5588,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -5793,6 +5943,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -6262,6 +6422,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -6731,6 +6901,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -7186,6 +7366,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -7460,6 +7650,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-legacy-before-aqc.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4.0w,vcores=4.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -873,6 +903,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1342,6 +1382,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2.0w,vcores=2.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1811,6 +1861,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.0w,vcores=12.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2266,6 +2326,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16.0w,vcores=16.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2540,6 +2610,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentage-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=25.0%,vcores=25.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "25.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "25.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "75.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "75.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "100.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "100.0%"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentage-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=25.0%,vcores=25.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "25.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "25.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "75.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "75.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "100.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "100.0%"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 100,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentage-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=25.0%,vcores=25.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "25.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "25.0%"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "75.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "75.0%"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "100.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "100.0%"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeight-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "75.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "75.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "50.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "50.0%"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeight-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "75.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "75.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "50.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "50.0%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 100,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeight-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "75.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "75.0%"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "50.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "50.0%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=12.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.0"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeightMixed-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=4.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=12.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=50.0%,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "50.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=86.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "86.0%"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=100.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "100.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeightMixed-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=4.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=12.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=50.0%,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "50.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=86.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "86.0%"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=100.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "100.0%"
+                } ]
+              },
               "capacity" : 100,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndPercentageAndWeightMixed-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=4.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=12.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.0"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=50.0%,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "50.0%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12288.0,vcores=86.0%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12288.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "86.0%"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=100.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "100.0%"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndWeight-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=3.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "3.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndWeight-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=3.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "3.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 100,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerAbsoluteAndWeight-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=3.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "3.0w"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=16384.0,vcores=16.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "16384.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "16.0"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerPercentageAndWeight-0.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=3.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "3.0w"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 0,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 0,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerPercentageAndWeight-16.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=3.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "3.0w"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/mixed-testSchedulerPercentageAndWeight-32.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "1.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "1.0w"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=3.0w,vcores=3.0w]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "3.0w"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "3.0w"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1013,6 +1043,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1482,6 +1522,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "12.5%"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "12.5%"
+                    } ]
+                  },
                   "capacity" : 12.5,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -1951,6 +2001,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=1.0w,vcores=1.0w]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "1.0w"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "1.0w"
+                    } ]
+                  },
                   "capacity" : 75,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -2406,6 +2466,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -2680,6 +2750,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 50 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=28672.0,vcores=28.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "28672.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "28.0"
+                } ]
+              },
               "capacity" : 87.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -544,6 +564,16 @@
               "capacities" : {
                 "queueCapacitiesByPartition" : [ {
                   "partitionName" : "",
+                  "queueCapacityVectorInfo" : {
+                    "configuredCapacityVector" : "[memory-mb=2048.0,vcores=2.0]",
+                    "capacityVectorEntries" : [ {
+                      "resourceName" : "memory-mb",
+                      "resourceValue" : "2048.0"
+                    }, {
+                      "resourceName" : "vcores",
+                      "resourceValue" : "2.0"
+                    } ]
+                  },
                   "capacity" : 50,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
@@ -999,6 +1029,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=4096.0,vcores=4.0]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "4096.0"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "4.0"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1284,6 +1324,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 55 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-NodeLabelDefaultAPI.xml

@@ -5,6 +5,17 @@
     <maxCapacity>100.0</maxCapacity>
     <weight>-1.0</weight>
     <normalizedWeight>0.0</normalizedWeight>
+    <queueCapacityVectorInfo>
+      <configuredCapacityVector>[memory-mb=100.0%,vcores=100.0%]</configuredCapacityVector>
+      <capacityVectorEntries>
+        <resourceName>memory-mb</resourceName>
+        <resourceValue>100.0%</resourceValue>
+      </capacityVectorEntries>
+      <capacityVectorEntries>
+        <resourceName>vcores</resourceName>
+        <resourceValue>100.0%</resourceValue>
+      </capacityVectorEntries>
+    </queueCapacityVectorInfo>
     <queueName>root</queueName>
     <queuePath>root</queuePath>
     <maxParallelApps>2147483647</maxParallelApps>
@@ -57,6 +68,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=12.5%,vcores=12.5%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>12.5%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>12.5%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>12.5</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>50.0</maxCapacity>
@@ -546,6 +568,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=50.0%,vcores=50.0%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>50.0%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>50.0%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>50.0</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>100.0</maxCapacity>
@@ -1035,6 +1068,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=37.5%,vcores=37.5%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>37.5%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>37.5%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>37.5</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>100.0</maxCapacity>
@@ -1481,6 +1525,17 @@
     <capacities>
       <queueCapacitiesByPartition>
         <partitionName/>
+        <queueCapacityVectorInfo>
+          <configuredCapacityVector>[memory-mb=100.0%,vcores=100.0%]</configuredCapacityVector>
+          <capacityVectorEntries>
+            <resourceName>memory-mb</resourceName>
+            <resourceValue>100.0%</resourceValue>
+          </capacityVectorEntries>
+          <capacityVectorEntries>
+            <resourceName>vcores</resourceName>
+            <resourceValue>100.0%</resourceValue>
+          </capacityVectorEntries>
+        </queueCapacityVectorInfo>
         <capacity>100.0</capacity>
         <usedCapacity>0.0</usedCapacity>
         <maxCapacity>100.0</maxCapacity>

+ 50 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.5%"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 50,
@@ -748,6 +768,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1437,6 +1467,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=37.5%,vcores=37.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "37.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "37.5%"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1864,6 +1904,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 55 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.xml

@@ -5,6 +5,17 @@
     <maxCapacity>100.0</maxCapacity>
     <weight>-1.0</weight>
     <normalizedWeight>0.0</normalizedWeight>
+    <queueCapacityVectorInfo>
+      <configuredCapacityVector>[memory-mb=100.0%,vcores=100.0%]</configuredCapacityVector>
+      <capacityVectorEntries>
+        <resourceName>memory-mb</resourceName>
+        <resourceValue>100.0%</resourceValue>
+      </capacityVectorEntries>
+      <capacityVectorEntries>
+        <resourceName>vcores</resourceName>
+        <resourceValue>100.0%</resourceValue>
+      </capacityVectorEntries>
+    </queueCapacityVectorInfo>
     <queueName>root</queueName>
     <queuePath>root</queuePath>
     <maxParallelApps>2147483647</maxParallelApps>
@@ -57,6 +68,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=12.5%,vcores=12.5%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>12.5%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>12.5%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>12.5</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>50.0</maxCapacity>
@@ -774,6 +796,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=50.0%,vcores=50.0%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>50.0%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>50.0%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>50.0</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>100.0</maxCapacity>
@@ -1491,6 +1524,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=37.5%,vcores=37.5%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>37.5%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>37.5%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>37.5</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>100.0</maxCapacity>
@@ -1936,6 +1980,17 @@
     <capacities>
       <queueCapacitiesByPartition>
         <partitionName/>
+        <queueCapacityVectorInfo>
+          <configuredCapacityVector>[memory-mb=100.0%,vcores=100.0%]</configuredCapacityVector>
+          <capacityVectorEntries>
+            <resourceName>memory-mb</resourceName>
+            <resourceValue>100.0%</resourceValue>
+          </capacityVectorEntries>
+          <capacityVectorEntries>
+            <resourceName>vcores</resourceName>
+            <resourceValue>100.0%</resourceValue>
+          </capacityVectorEntries>
+        </queueCapacityVectorInfo>
         <capacity>100.0</capacity>
         <usedCapacity>0.0</usedCapacity>
         <maxCapacity>100.0</maxCapacity>

+ 40 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PercentageModeLegacyAutoCreation.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=25.0%,vcores=25.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "25.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "25.0%"
+                } ]
+              },
               "capacity" : 25,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=75.0%,vcores=75.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "75.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "75.0%"
+                } ]
+              },
               "capacity" : 75,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -802,6 +832,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 50 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.json

@@ -7,6 +7,16 @@
       "maxCapacity" : 100,
       "weight" : -1,
       "normalizedWeight" : 0,
+      "queueCapacityVectorInfo" : {
+        "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+        "capacityVectorEntries" : [ {
+          "resourceName" : "memory-mb",
+          "resourceValue" : "100.0%"
+        }, {
+          "resourceName" : "vcores",
+          "resourceValue" : "100.0%"
+        } ]
+      },
       "queueName" : "root",
       "queuePath" : "root",
       "maxParallelApps" : 2147483647,
@@ -59,6 +69,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=12.5%,vcores=12.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "12.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "12.5%"
+                } ]
+              },
               "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 50,
@@ -528,6 +548,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=50.0%,vcores=50.0%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "50.0%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "50.0%"
+                } ]
+              },
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -997,6 +1027,16 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
+              "queueCapacityVectorInfo" : {
+                "configuredCapacityVector" : "[memory-mb=37.5%,vcores=37.5%]",
+                "capacityVectorEntries" : [ {
+                  "resourceName" : "memory-mb",
+                  "resourceValue" : "37.5%"
+                }, {
+                  "resourceName" : "vcores",
+                  "resourceValue" : "37.5%"
+                } ]
+              },
               "capacity" : 37.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
@@ -1424,6 +1464,16 @@
       "capacities" : {
         "queueCapacitiesByPartition" : [ {
           "partitionName" : "",
+          "queueCapacityVectorInfo" : {
+            "configuredCapacityVector" : "[memory-mb=100.0%,vcores=100.0%]",
+            "capacityVectorEntries" : [ {
+              "resourceName" : "memory-mb",
+              "resourceValue" : "100.0%"
+            }, {
+              "resourceName" : "vcores",
+              "resourceValue" : "100.0%"
+            } ]
+          },
           "capacity" : 100,
           "usedCapacity" : 0,
           "maxCapacity" : 100,

+ 55 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.xml

@@ -5,6 +5,17 @@
     <maxCapacity>100.0</maxCapacity>
     <weight>-1.0</weight>
     <normalizedWeight>0.0</normalizedWeight>
+    <queueCapacityVectorInfo>
+      <configuredCapacityVector>[memory-mb=100.0%,vcores=100.0%]</configuredCapacityVector>
+      <capacityVectorEntries>
+        <resourceName>memory-mb</resourceName>
+        <resourceValue>100.0%</resourceValue>
+      </capacityVectorEntries>
+      <capacityVectorEntries>
+        <resourceName>vcores</resourceName>
+        <resourceValue>100.0%</resourceValue>
+      </capacityVectorEntries>
+    </queueCapacityVectorInfo>
     <queueName>root</queueName>
     <queuePath>root</queuePath>
     <maxParallelApps>2147483647</maxParallelApps>
@@ -57,6 +68,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=12.5%,vcores=12.5%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>12.5%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>12.5%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>12.5</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>50.0</maxCapacity>
@@ -545,6 +567,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=50.0%,vcores=50.0%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>50.0%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>50.0%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>50.0</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>100.0</maxCapacity>
@@ -1033,6 +1066,17 @@
         <capacities>
           <queueCapacitiesByPartition>
             <partitionName/>
+            <queueCapacityVectorInfo>
+              <configuredCapacityVector>[memory-mb=37.5%,vcores=37.5%]</configuredCapacityVector>
+              <capacityVectorEntries>
+                <resourceName>memory-mb</resourceName>
+                <resourceValue>37.5%</resourceValue>
+              </capacityVectorEntries>
+              <capacityVectorEntries>
+                <resourceName>vcores</resourceName>
+                <resourceValue>37.5%</resourceValue>
+              </capacityVectorEntries>
+            </queueCapacityVectorInfo>
             <capacity>37.5</capacity>
             <usedCapacity>0.0</usedCapacity>
             <maxCapacity>100.0</maxCapacity>
@@ -1478,6 +1522,17 @@
     <capacities>
       <queueCapacitiesByPartition>
         <partitionName/>
+        <queueCapacityVectorInfo>
+          <configuredCapacityVector>[memory-mb=100.0%,vcores=100.0%]</configuredCapacityVector>
+          <capacityVectorEntries>
+            <resourceName>memory-mb</resourceName>
+            <resourceValue>100.0%</resourceValue>
+          </capacityVectorEntries>
+          <capacityVectorEntries>
+            <resourceName>vcores</resourceName>
+            <resourceValue>100.0%</resourceValue>
+          </capacityVectorEntries>
+        </queueCapacityVectorInfo>
         <capacity>100.0</capacity>
         <usedCapacity>0.0</usedCapacity>
         <maxCapacity>100.0</maxCapacity>