Ver código fonte

YARN-11033. isAbsoluteResource is not correct for dynamically created queues. Contributed by Tamas Domok

Szilard Nemeth 3 anos atrás
pai
commit
f544ef4a51
16 arquivos alterados com 1645 adições e 24 exclusões
  1. 5 0
      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
  2. 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
  3. 32 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java
  4. 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
  5. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteMode.json
  6. 1582 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json
  7. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-NodeLabelDefaultAPI.xml
  8. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.json
  9. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.xml
  10. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PercentageMode.json
  11. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PercentageModeLegacyAutoCreation.json
  12. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json
  13. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json
  14. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json
  15. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.json
  16. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.xml

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

@@ -24,6 +24,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.XmlType;
 
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@@ -49,6 +50,7 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
   protected String queueName;
   private String queuePath;
   protected int maxParallelApps;
+  private boolean isAbsoluteResource;
   protected CapacitySchedulerQueueInfoList queues;
   protected QueueCapacitiesInfo capacities;
   protected CapacitySchedulerHealthInfo health;
@@ -90,6 +92,9 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
     health = new CapacitySchedulerHealthInfo(cs);
     maximumAllocation = new ResourceInfo(parent.getMaximumAllocation());
 
+    isAbsoluteResource = parent.getCapacityConfigType() ==
+        AbstractCSQueue.CapacityConfigType.ABSOLUTE_RESOURCE;
+
     CapacitySchedulerConfiguration conf = cs.getConfiguration();
     queueAcls = new QueueAclsInfo();
     queueAcls.addAll(getSortedQueueAclInfoList(queueName, conf));

+ 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

@@ -35,6 +35,7 @@ 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.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
@@ -43,11 +44,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueu
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.helper.CapacitySchedulerInfoHelper;
 
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.
-    CapacitySchedulerConfiguration.RESOURCE_PATTERN;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.
-    CapacitySchedulerConfiguration.CAPACITY;
-
 @XmlRootElement
 @XmlAccessorType(XmlAccessType.FIELD)
 @XmlSeeAlso({CapacitySchedulerLeafQueueInfo.class})
@@ -179,10 +175,8 @@ public class CapacitySchedulerQueueInfo {
               .getLeafOnlyProperties());
     }
 
-    String configuredCapacity = conf.get(
-        CapacitySchedulerConfiguration.getQueuePrefix(queuePath) + CAPACITY);
-    isAbsoluteResource = (configuredCapacity != null)
-        && RESOURCE_PATTERN.matcher(configuredCapacity).find();
+    isAbsoluteResource = q.getCapacityConfigType() ==
+        AbstractCSQueue.CapacityConfigType.ABSOLUTE_RESOURCE;
 
     autoCreateChildQueueEnabled = conf.
         isAutoCreateChildQueueEnabled(queuePath);

+ 32 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java

@@ -136,6 +136,22 @@ public class TestRMWebServicesCapacitySchedDynamicConfig extends
         "webapp/scheduler-response-PercentageModeLegacyAutoCreation.json");
   }
 
+  @Test
+  public void testSchedulerResponseAbsoluteModeLegacyAutoCreation()
+      throws Exception {
+    Configuration config = CSConfigGenerator
+        .createAbsoluteConfigLegacyAutoCreation();
+    config.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.MEMORY_CONFIGURATION_STORE);
+
+    initResourceManager(config);
+    initAutoQueueHandler(8192 * GB);
+    createQueue("root.managed.queue1");
+
+    assertJsonResponse(sendRequest(),
+        "webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json");
+  }
+
   @Test
   public void testSchedulerResponseAbsoluteMode()
       throws Exception {
@@ -189,7 +205,7 @@ public class TestRMWebServicesCapacitySchedDynamicConfig extends
         "maximum-applications", 300);
 
     initResourceManager(config);
-    initAutoQueueHandler();
+    initAutoQueueHandler(1200 * GB);
 
     // same as webapp/scheduler-response-WeightMode.json, but with effective resources filled in
     assertJsonResponse(sendRequest(),
@@ -212,10 +228,10 @@ public class TestRMWebServicesCapacitySchedDynamicConfig extends
         "webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json");
   }
 
-  private void initAutoQueueHandler() throws Exception {
+  private void initAutoQueueHandler(int nodeMemory) throws Exception {
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
     autoQueueHandler = cs.getCapacitySchedulerQueueManager();
-    rm.registerNode("h1:1234", 1200 * GB); // label = x
+    rm.registerNode("h1:1234", nodeMemory); // label = x
   }
 
   private void createQueue(String queuePath) throws YarnException,
@@ -255,6 +271,19 @@ public class TestRMWebServicesCapacitySchedDynamicConfig extends
       return createConfiguration(conf);
     }
 
+    public static Configuration createAbsoluteConfigLegacyAutoCreation() {
+      Map<String, String> conf = new HashMap<>();
+      conf.put("yarn.scheduler.capacity.root.queues", "default, managed");
+      conf.put("yarn.scheduler.capacity.root.default.state", "STOPPED");
+      conf.put("yarn.scheduler.capacity.root.managed.capacity", "[memory=4096,vcores=4]");
+      conf.put("yarn.scheduler.capacity.root.managed.leaf-queue-template.capacity",
+          "[memory=2048,vcores=2]");
+      conf.put("yarn.scheduler.capacity.root.managed.state", "RUNNING");
+      conf.put("yarn.scheduler.capacity.root.managed." +
+          "auto-create-child-queue.enabled", "true");
+      return createConfiguration(conf);
+    }
+
     public static Configuration createAbsoluteConfig() {
       Map<String, String> conf = new HashMap<>();
       conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2");

+ 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

@@ -574,7 +574,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", 23, info.length());
+    assertEquals("incorrect number of elements", 24, info.length());
     JSONObject capacitiesJsonObject = info.getJSONObject(CAPACITIES);
     JSONArray partitionsCapsArray =
         capacitiesJsonObject.getJSONArray(QUEUE_CAPACITIES_BY_PARTITION);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteMode.json

@@ -8,6 +8,7 @@
   "queueName": "root",
   "queuePath": "root",
   "maxParallelApps": 2147483647,
+  "isAbsoluteResource": true,
   "queues": {"queue": [
     {
       "type": "capacitySchedulerLeafQueueInfo",
@@ -1735,4 +1736,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

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

@@ -0,0 +1,1582 @@
+{"scheduler": {"schedulerInfo": {
+  "type": "capacityScheduler",
+  "capacity": 100,
+  "usedCapacity": 0,
+  "maxCapacity": 100,
+  "weight": -1,
+  "normalizedWeight": 0,
+  "queueName": "root",
+  "queuePath": "root",
+  "maxParallelApps": 2147483647,
+  "isAbsoluteResource": false,
+  "queues": {"queue": [
+    {
+      "type": "capacitySchedulerLeafQueueInfo",
+      "queuePath": "root.default",
+      "capacity": 0,
+      "usedCapacity": 0,
+      "maxCapacity": 100,
+      "absoluteCapacity": 0,
+      "absoluteMaxCapacity": 100,
+      "absoluteUsedCapacity": 0,
+      "weight": -1,
+      "normalizedWeight": 0,
+      "numApplications": 0,
+      "maxParallelApps": 2147483647,
+      "queueName": "default",
+      "isAbsoluteResource": false,
+      "state": "STOPPED",
+      "resourcesUsed": {
+        "memory": 0,
+        "vCores": 0,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 0
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 0
+          }
+        ]}
+      },
+      "hideReservationQueues": false,
+      "nodeLabels": ["*"],
+      "allocatedContainers": 0,
+      "reservedContainers": 0,
+      "pendingContainers": 0,
+      "capacities": {"queueCapacitiesByPartition": [{
+        "partitionName": "",
+        "capacity": 0,
+        "usedCapacity": 0,
+        "maxCapacity": 100,
+        "absoluteCapacity": 0,
+        "absoluteUsedCapacity": 0,
+        "absoluteMaxCapacity": 100,
+        "maxAMLimitPercentage": 10,
+        "weight": -1,
+        "normalizedWeight": 0,
+        "configuredMinResource": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 8192,
+              "minimumAllocation": 1024,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 4,
+              "minimumAllocation": 1,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "configuredMaxResource": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 8192,
+              "minimumAllocation": 1024,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 4,
+              "minimumAllocation": 1,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "effectiveMinResource": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "effectiveMaxResource": {
+          "memory": 8388608,
+          "vCores": 8192,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 8388608
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 8192
+            }
+          ]}
+        }
+      }]},
+      "resources": {"resourceUsagesByPartition": [{
+        "partitionName": "",
+        "used": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "reserved": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "pending": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "amUsed": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "amLimit": {
+          "memory": 839680,
+          "vCores": 1,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 839680
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 1
+            }
+          ]}
+        },
+        "userAmLimit": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        }
+      }]},
+      "minEffectiveCapacity": {
+        "memory": 0,
+        "vCores": 0,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 0
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 0
+          }
+        ]}
+      },
+      "maxEffectiveCapacity": {
+        "memory": 8388608,
+        "vCores": 8192,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 8388608
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 8192
+          }
+        ]}
+      },
+      "maximumAllocation": {
+        "memory": 8192,
+        "vCores": 4,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 8192
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 4
+          }
+        ]}
+      },
+      "queueAcls": {"queueAcl": [
+        {
+          "accessType": "ADMINISTER_QUEUE",
+          "accessControlList": " "
+        },
+        {
+          "accessType": "APPLICATION_MAX_PRIORITY",
+          "accessControlList": "*"
+        },
+        {
+          "accessType": "SUBMIT_APP",
+          "accessControlList": " "
+        }
+      ]},
+      "queuePriority": 0,
+      "orderingPolicyInfo": "fifo",
+      "autoCreateChildQueueEnabled": false,
+      "leafQueueTemplate": {},
+      "mode": "percentage",
+      "queueType": "leaf",
+      "creationMethod": "static",
+      "autoCreationEligibility": "off",
+      "autoQueueTemplateProperties": {},
+      "autoQueueParentTemplateProperties": {},
+      "autoQueueLeafTemplateProperties": {},
+      "numActiveApplications": 0,
+      "numPendingApplications": 0,
+      "numContainers": 0,
+      "maxApplications": 0,
+      "maxApplicationsPerUser": 0,
+      "userLimit": 100,
+      "users": {},
+      "userLimitFactor": 1,
+      "configuredMaxAMResourceLimit": 0.1,
+      "AMResourceLimit": {
+        "memory": 839680,
+        "vCores": 1,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 839680
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 1
+          }
+        ]}
+      },
+      "usedAMResource": {
+        "memory": 0,
+        "vCores": 0,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 0
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 0
+          }
+        ]}
+      },
+      "userAMResourceLimit": {
+        "memory": 839680,
+        "vCores": 1,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 839680
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 1
+          }
+        ]}
+      },
+      "preemptionDisabled": true,
+      "intraQueuePreemptionDisabled": true,
+      "defaultPriority": 0,
+      "isAutoCreatedLeafQueue": false,
+      "maxApplicationLifetime": -1,
+      "defaultApplicationLifetime": -1
+    },
+    {
+      "queuePath": "root.managed",
+      "capacity": 0.048828125,
+      "usedCapacity": 0,
+      "maxCapacity": 100,
+      "absoluteCapacity": 0.048828125,
+      "absoluteMaxCapacity": 100,
+      "absoluteUsedCapacity": 0,
+      "weight": -1,
+      "normalizedWeight": 0,
+      "numApplications": 0,
+      "maxParallelApps": 2147483647,
+      "queueName": "managed",
+      "isAbsoluteResource": true,
+      "state": "RUNNING",
+      "queues": {"queue": [{
+        "type": "capacitySchedulerLeafQueueInfo",
+        "queuePath": "root.managed.queue1",
+        "capacity": 50,
+        "usedCapacity": 0,
+        "maxCapacity": 100,
+        "absoluteCapacity": 0.024414062,
+        "absoluteMaxCapacity": 100,
+        "absoluteUsedCapacity": 0,
+        "weight": -1,
+        "normalizedWeight": 0,
+        "numApplications": 0,
+        "maxParallelApps": 2147483647,
+        "queueName": "queue1",
+        "isAbsoluteResource": true,
+        "state": "RUNNING",
+        "resourcesUsed": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "hideReservationQueues": false,
+        "nodeLabels": ["*"],
+        "allocatedContainers": 0,
+        "reservedContainers": 0,
+        "pendingContainers": 0,
+        "capacities": {"queueCapacitiesByPartition": [{
+          "partitionName": "",
+          "capacity": 50,
+          "usedCapacity": 0,
+          "maxCapacity": 100,
+          "absoluteCapacity": 0.024414062,
+          "absoluteUsedCapacity": 0,
+          "absoluteMaxCapacity": 100,
+          "maxAMLimitPercentage": 10,
+          "weight": -1,
+          "normalizedWeight": 0,
+          "configuredMinResource": {
+            "memory": 2048,
+            "vCores": 2,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 2048
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 2
+              }
+            ]}
+          },
+          "configuredMaxResource": {
+            "memory": 0,
+            "vCores": 0,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 8192,
+                "minimumAllocation": 1024,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 0
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 4,
+                "minimumAllocation": 1,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 0
+              }
+            ]}
+          },
+          "effectiveMinResource": {
+            "memory": 2048,
+            "vCores": 2,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 2048
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 2
+              }
+            ]}
+          },
+          "effectiveMaxResource": {
+            "memory": 8388608,
+            "vCores": 8192,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 8388608
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 8192
+              }
+            ]}
+          }
+        }]},
+        "resources": {"resourceUsagesByPartition": [{
+          "partitionName": "",
+          "used": {
+            "memory": 0,
+            "vCores": 0,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 0
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 0
+              }
+            ]}
+          },
+          "reserved": {
+            "memory": 0,
+            "vCores": 0,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 0
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 0
+              }
+            ]}
+          },
+          "pending": {
+            "memory": 0,
+            "vCores": 0,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 0
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 0
+              }
+            ]}
+          },
+          "amUsed": {
+            "memory": 0,
+            "vCores": 0,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 0
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 0
+              }
+            ]}
+          },
+          "amLimit": {
+            "memory": 839680,
+            "vCores": 1,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 839680
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 1
+              }
+            ]}
+          },
+          "userAmLimit": {
+            "memory": 0,
+            "vCores": 0,
+            "resourceInformations": {"resourceInformation": [
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "memory-mb",
+                "resourceType": "COUNTABLE",
+                "units": "Mi",
+                "value": 0
+              },
+              {
+                "attributes": {},
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
+                "name": "vcores",
+                "resourceType": "COUNTABLE",
+                "units": "",
+                "value": 0
+              }
+            ]}
+          }
+        }]},
+        "minEffectiveCapacity": {
+          "memory": 2048,
+          "vCores": 2,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 2048
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 2
+            }
+          ]}
+        },
+        "maxEffectiveCapacity": {
+          "memory": 8388608,
+          "vCores": 8192,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 8388608
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 8192
+            }
+          ]}
+        },
+        "maximumAllocation": {
+          "memory": 8192,
+          "vCores": 4,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 8192
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 4
+            }
+          ]}
+        },
+        "queueAcls": {"queueAcl": [
+          {
+            "accessType": "ADMINISTER_QUEUE",
+            "accessControlList": " "
+          },
+          {
+            "accessType": "APPLICATION_MAX_PRIORITY",
+            "accessControlList": "*"
+          },
+          {
+            "accessType": "SUBMIT_APP",
+            "accessControlList": " "
+          }
+        ]},
+        "queuePriority": 0,
+        "orderingPolicyInfo": "fifo",
+        "autoCreateChildQueueEnabled": false,
+        "leafQueueTemplate": {},
+        "mode": "absolute",
+        "queueType": "leaf",
+        "creationMethod": "dynamicLegacy",
+        "autoCreationEligibility": "off",
+        "autoQueueTemplateProperties": {},
+        "autoQueueParentTemplateProperties": {},
+        "autoQueueLeafTemplateProperties": {},
+        "numActiveApplications": 0,
+        "numPendingApplications": 0,
+        "numContainers": 0,
+        "maxApplications": 2,
+        "maxApplicationsPerUser": 2,
+        "userLimit": 100,
+        "users": {},
+        "userLimitFactor": 1,
+        "configuredMaxAMResourceLimit": 0.1,
+        "AMResourceLimit": {
+          "memory": 839680,
+          "vCores": 1,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 839680
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 1
+            }
+          ]}
+        },
+        "usedAMResource": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "userAMResourceLimit": {
+          "memory": 839680,
+          "vCores": 1,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 839680
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 1
+            }
+          ]}
+        },
+        "preemptionDisabled": true,
+        "intraQueuePreemptionDisabled": true,
+        "defaultPriority": 0,
+        "isAutoCreatedLeafQueue": true,
+        "maxApplicationLifetime": -1,
+        "defaultApplicationLifetime": -1
+      }]},
+      "resourcesUsed": {
+        "memory": 0,
+        "vCores": 0,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 0
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 0
+          }
+        ]}
+      },
+      "hideReservationQueues": false,
+      "nodeLabels": ["*"],
+      "allocatedContainers": 0,
+      "reservedContainers": 0,
+      "pendingContainers": 0,
+      "capacities": {"queueCapacitiesByPartition": [{
+        "partitionName": "",
+        "capacity": 0.048828125,
+        "usedCapacity": 0,
+        "maxCapacity": 100,
+        "absoluteCapacity": 0.048828125,
+        "absoluteUsedCapacity": 0,
+        "absoluteMaxCapacity": 100,
+        "maxAMLimitPercentage": 0,
+        "weight": -1,
+        "normalizedWeight": 0,
+        "configuredMinResource": {
+          "memory": 4096,
+          "vCores": 4,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 4096
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 4
+            }
+          ]}
+        },
+        "configuredMaxResource": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 8192,
+              "minimumAllocation": 1024,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 4,
+              "minimumAllocation": 1,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "effectiveMinResource": {
+          "memory": 4096,
+          "vCores": 4,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 4096
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 4
+            }
+          ]}
+        },
+        "effectiveMaxResource": {
+          "memory": 8388608,
+          "vCores": 8192,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 8388608
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 8192
+            }
+          ]}
+        }
+      }]},
+      "resources": {"resourceUsagesByPartition": [{
+        "partitionName": "",
+        "used": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "reserved": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        },
+        "pending": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        }
+      }]},
+      "minEffectiveCapacity": {
+        "memory": 4096,
+        "vCores": 4,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 4096
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 4
+          }
+        ]}
+      },
+      "maxEffectiveCapacity": {
+        "memory": 8388608,
+        "vCores": 8192,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 8388608
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 8192
+          }
+        ]}
+      },
+      "maximumAllocation": {
+        "memory": 8192,
+        "vCores": 4,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 8192
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 4
+          }
+        ]}
+      },
+      "queueAcls": {"queueAcl": [
+        {
+          "accessType": "ADMINISTER_QUEUE",
+          "accessControlList": " "
+        },
+        {
+          "accessType": "APPLICATION_MAX_PRIORITY",
+          "accessControlList": "*"
+        },
+        {
+          "accessType": "SUBMIT_APP",
+          "accessControlList": " "
+        }
+      ]},
+      "queuePriority": 0,
+      "orderingPolicyInfo": "utilization",
+      "autoCreateChildQueueEnabled": true,
+      "leafQueueTemplate": {"property": [{
+        "name": "leaf-queue-template.capacity",
+        "value": "[memory=2048,vcores=2]"
+      }]},
+      "mode": "absolute",
+      "queueType": "parent",
+      "creationMethod": "static",
+      "autoCreationEligibility": "legacy",
+      "autoQueueTemplateProperties": {},
+      "autoQueueParentTemplateProperties": {},
+      "autoQueueLeafTemplateProperties": {}
+    }
+  ]},
+  "capacities": {"queueCapacitiesByPartition": [{
+    "partitionName": "",
+    "capacity": 100,
+    "usedCapacity": 0,
+    "maxCapacity": 100,
+    "absoluteCapacity": 100,
+    "absoluteUsedCapacity": 0,
+    "absoluteMaxCapacity": 100,
+    "maxAMLimitPercentage": 0,
+    "weight": -1,
+    "normalizedWeight": 0,
+    "configuredMinResource": {
+      "memory": 0,
+      "vCores": 0,
+      "resourceInformations": {"resourceInformation": [
+        {
+          "attributes": {},
+          "maximumAllocation": 8192,
+          "minimumAllocation": 1024,
+          "name": "memory-mb",
+          "resourceType": "COUNTABLE",
+          "units": "Mi",
+          "value": 0
+        },
+        {
+          "attributes": {},
+          "maximumAllocation": 4,
+          "minimumAllocation": 1,
+          "name": "vcores",
+          "resourceType": "COUNTABLE",
+          "units": "",
+          "value": 0
+        }
+      ]}
+    },
+    "configuredMaxResource": {
+      "memory": 0,
+      "vCores": 0,
+      "resourceInformations": {"resourceInformation": [
+        {
+          "attributes": {},
+          "maximumAllocation": 8192,
+          "minimumAllocation": 1024,
+          "name": "memory-mb",
+          "resourceType": "COUNTABLE",
+          "units": "Mi",
+          "value": 0
+        },
+        {
+          "attributes": {},
+          "maximumAllocation": 4,
+          "minimumAllocation": 1,
+          "name": "vcores",
+          "resourceType": "COUNTABLE",
+          "units": "",
+          "value": 0
+        }
+      ]}
+    },
+    "effectiveMinResource": {
+      "memory": 8388608,
+      "vCores": 8192,
+      "resourceInformations": {"resourceInformation": [
+        {
+          "attributes": {},
+          "maximumAllocation": 9223372036854775807,
+          "minimumAllocation": 0,
+          "name": "memory-mb",
+          "resourceType": "COUNTABLE",
+          "units": "Mi",
+          "value": 8388608
+        },
+        {
+          "attributes": {},
+          "maximumAllocation": 9223372036854775807,
+          "minimumAllocation": 0,
+          "name": "vcores",
+          "resourceType": "COUNTABLE",
+          "units": "",
+          "value": 8192
+        }
+      ]}
+    },
+    "effectiveMaxResource": {
+      "memory": 8388608,
+      "vCores": 8192,
+      "resourceInformations": {"resourceInformation": [
+        {
+          "attributes": {},
+          "maximumAllocation": 9223372036854775807,
+          "minimumAllocation": 0,
+          "name": "memory-mb",
+          "resourceType": "COUNTABLE",
+          "units": "Mi",
+          "value": 8388608
+        },
+        {
+          "attributes": {},
+          "maximumAllocation": 9223372036854775807,
+          "minimumAllocation": 0,
+          "name": "vcores",
+          "resourceType": "COUNTABLE",
+          "units": "",
+          "value": 8192
+        }
+      ]}
+    }
+  }]},
+  "health": {
+    "lastrun": 0,
+    "operationsInfo": [
+      {
+        "operation": "last-allocation",
+        "nodeId": "N\/A",
+        "containerId": "N\/A",
+        "queue": "N\/A"
+      },
+      {
+        "operation": "last-release",
+        "nodeId": "N\/A",
+        "containerId": "N\/A",
+        "queue": "N\/A"
+      },
+      {
+        "operation": "last-preemption",
+        "nodeId": "N\/A",
+        "containerId": "N\/A",
+        "queue": "N\/A"
+      },
+      {
+        "operation": "last-reservation",
+        "nodeId": "N\/A",
+        "containerId": "N\/A",
+        "queue": "N\/A"
+      }
+    ],
+    "lastRunDetails": [
+      {
+        "operation": "releases",
+        "count": 0,
+        "resources": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        }
+      },
+      {
+        "operation": "allocations",
+        "count": 0,
+        "resources": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        }
+      },
+      {
+        "operation": "reservations",
+        "count": 0,
+        "resources": {
+          "memory": 0,
+          "vCores": 0,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 0
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 0
+            }
+          ]}
+        }
+      }
+    ]
+  },
+  "maximumAllocation": {
+    "memory": 8192,
+    "vCores": 4,
+    "resourceInformations": {"resourceInformation": [
+      {
+        "attributes": {},
+        "maximumAllocation": 9223372036854775807,
+        "minimumAllocation": 0,
+        "name": "memory-mb",
+        "resourceType": "COUNTABLE",
+        "units": "Mi",
+        "value": 8192
+      },
+      {
+        "attributes": {},
+        "maximumAllocation": 9223372036854775807,
+        "minimumAllocation": 0,
+        "name": "vcores",
+        "resourceType": "COUNTABLE",
+        "units": "",
+        "value": 4
+      }
+    ]}
+  },
+  "queueAcls": {"queueAcl": [
+    {
+      "accessType": "ADMINISTER_QUEUE",
+      "accessControlList": "*"
+    },
+    {
+      "accessType": "APPLICATION_MAX_PRIORITY",
+      "accessControlList": "*"
+    },
+    {
+      "accessType": "SUBMIT_APP",
+      "accessControlList": "*"
+    }
+  ]},
+  "queuePriority": 0,
+  "orderingPolicyInfo": "utilization",
+  "mode": "percentage",
+  "queueType": "parent",
+  "creationMethod": "static",
+  "autoCreationEligibility": "off",
+  "autoQueueTemplateProperties": {},
+  "autoQueueParentTemplateProperties": {},
+  "autoQueueLeafTemplateProperties": {}
+}}}

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

@@ -8,6 +8,7 @@
     <queueName>root</queueName>
     <queuePath>root</queuePath>
     <maxParallelApps>2147483647</maxParallelApps>
+    <isAbsoluteResource>false</isAbsoluteResource>
     <queues>
       <queue>
         <queuePath>root.a</queuePath>
@@ -4546,4 +4547,4 @@
     <autoQueueParentTemplateProperties/>
     <autoQueueLeafTemplateProperties/>
   </schedulerInfo>
-</scheduler>
+</scheduler>

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

@@ -8,6 +8,7 @@
   "queueName": "root",
   "queuePath": "root",
   "maxParallelApps": 2147483647,
+  "isAbsoluteResource": false,
   "queues": {"queue": [
     {
       "queuePath": "root.a",
@@ -4953,4 +4954,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

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

@@ -8,6 +8,7 @@
     <queueName>root</queueName>
     <queuePath>root</queuePath>
     <maxParallelApps>2147483647</maxParallelApps>
+    <isAbsoluteResource>false</isAbsoluteResource>
     <queues>
       <queue>
         <queuePath>root.a</queuePath>
@@ -4992,4 +4993,4 @@
     <autoQueueParentTemplateProperties/>
     <autoQueueLeafTemplateProperties/>
   </schedulerInfo>
-</scheduler>
+</scheduler>

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PercentageMode.json

@@ -8,6 +8,7 @@
   "queueName": "root",
   "queuePath": "root",
   "maxParallelApps": 2147483647,
+  "isAbsoluteResource": false,
   "queues": {"queue": [
     {
       "type": "capacitySchedulerLeafQueueInfo",
@@ -1735,4 +1736,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

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

@@ -8,6 +8,7 @@
   "queueName": "root",
   "queuePath": "root",
   "maxParallelApps": 2147483647,
+  "isAbsoluteResource": false,
   "queues": {"queue": [
     {
       "type": "capacitySchedulerLeafQueueInfo",
@@ -1576,4 +1577,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json

@@ -8,6 +8,7 @@
   "queueName": "root",
   "queuePath": "root",
   "maxParallelApps": 2147483647,
+  "isAbsoluteResource": false,
   "queues": {"queue": [
     {
       "type": "capacitySchedulerLeafQueueInfo",
@@ -1735,4 +1736,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json

@@ -8,6 +8,7 @@
   "queueName": "root",
   "queuePath": "root",
   "maxParallelApps": 2147483647,
+  "isAbsoluteResource": false,
   "queues": {"queue": [
     {
       "type": "capacitySchedulerLeafQueueInfo",
@@ -4003,4 +4004,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json

@@ -8,6 +8,7 @@
   "queueName": "root",
   "queuePath": "root",
   "maxParallelApps": 2147483647,
+  "isAbsoluteResource": false,
   "queues": {"queue": [
     {
       "type": "capacitySchedulerLeafQueueInfo",
@@ -1735,4 +1736,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

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

@@ -8,6 +8,7 @@
   "queueName": "root",
   "queuePath": "root",
   "maxParallelApps": 2147483647,
+  "isAbsoluteResource": false,
   "queues": {"queue": [
     {
       "queuePath": "root.a",
@@ -4500,4 +4501,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

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

@@ -8,6 +8,7 @@
     <queueName>root</queueName>
     <queuePath>root</queuePath>
     <maxParallelApps>2147483647</maxParallelApps>
+    <isAbsoluteResource>false</isAbsoluteResource>
     <queues>
       <queue>
         <queuePath>root.a</queuePath>
@@ -4535,4 +4536,4 @@
     <autoQueueParentTemplateProperties/>
     <autoQueueLeafTemplateProperties/>
   </schedulerInfo>
-</scheduler>
+</scheduler>