瀏覽代碼

YARN-11511. Improve TestRMWebServices test config and data. (#5745)

Co-authored-by: Tamas Domok <tdomok@cloudera.com>
K0K0V0K 2 年之前
父節點
當前提交
3f4de919bd
共有 27 個文件被更改,包括 38718 次插入21720 次删除
  1. 70 360
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
  2. 117 273
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java
  3. 93 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedLegacyQueueCreation.java
  4. 80 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java
  5. 265 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestWebServiceUtil.java
  6. 2935 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-0.json
  7. 2935 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-16.json
  8. 2935 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-32.json
  9. 3292 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testAbsoluteMode-laqc.json
  10. 1211 180
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-0.json
  11. 2935 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-16.json
  12. 2935 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-32.json
  13. 3292 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-laqc.json
  14. 766 321
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-0.json
  15. 465 140
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-16.json
  16. 2935 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-32.json
  17. 7710 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-after-aqc.json
  18. 2790 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-before-aqc.json
  19. 0 1679
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteMode.json
  20. 77 77
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json
  21. 106 1737
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-NodeLabelDefaultAPI.xml
  22. 264 3568
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.json
  23. 294 3686
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.xml
  24. 12 481
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PercentageModeLegacyAutoCreation.json
  25. 0 5818
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json
  26. 99 1662
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.json
  27. 105 1732
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.xml

+ 70 - 360
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java

@@ -18,273 +18,103 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.inject.Guice;
-import com.google.inject.servlet.ServletModule;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
-import com.sun.jersey.test.framework.WebAppDescriptor;
-
-import java.io.BufferedReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.net.URISyntaxException;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
 import javax.ws.rs.core.MediaType;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.transform.OutputKeys;
-import javax.xml.transform.Transformer;
-import javax.xml.transform.TransformerException;
-import javax.xml.transform.TransformerFactory;
-import javax.xml.transform.dom.DOMSource;
-import javax.xml.transform.stream.StreamResult;
+
+import com.sun.jersey.api.client.ClientResponse;
+import org.codehaus.jettison.json.JSONObject;
+import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.JettyUtils;
-import org.apache.hadoop.util.XMLUtils;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmissionData;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
-import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONObject;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.w3c.dom.Document;
-import org.xml.sax.InputSource;
 
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.CAPACITY;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.MAX_PARALLEL_APPLICATIONS;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.GB;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.assertJsonResponse;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.assertJsonType;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.assertXmlResponse;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.createRM;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.createWebAppDescriptor;
 import static org.junit.Assert.assertEquals;
 
 public class TestRMWebServicesCapacitySched extends JerseyTestBase {
 
-  private MockRM rm;
-
-  public static class WebServletModule extends ServletModule {
-    private final MockRM rm;
-
-    WebServletModule(MockRM rm) {
-      this.rm = rm;
-    }
-
-    @Override
-    protected void configureServlets() {
-      bind(JAXBContextResolver.class);
-      bind(RMWebServices.class);
-      bind(GenericExceptionHandler.class);
-      bind(ResourceManager.class).toInstance(rm);
-      serve("/*").with(GuiceContainer.class);
-    }
-  }
-
   public TestRMWebServicesCapacitySched() {
     super(createWebAppDescriptor());
   }
 
-  @Before
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    rm = createMockRM(new CapacitySchedulerConfiguration(
-        new Configuration(false)));
-    GuiceServletConfig.setInjector(
-        Guice.createInjector(new WebServletModule(rm)));
-  }
-
-  public static void setupQueueConfiguration(
-      CapacitySchedulerConfiguration config) {
-
-    // Define top-level queues
-    config.setQueues(CapacitySchedulerConfiguration.ROOT,
-        new String[] {"a", "b", "c"});
-
-    final String a = CapacitySchedulerConfiguration.ROOT + ".a";
-    config.setCapacity(a, 10.5f);
-    config.setMaximumCapacity(a, 50);
-    config.setInt(CapacitySchedulerConfiguration.getQueuePrefix(a) + MAX_PARALLEL_APPLICATIONS, 42);
-
-    final String b = CapacitySchedulerConfiguration.ROOT + ".b";
-    config.setCapacity(b, 89.5f);
-
-    final String c = CapacitySchedulerConfiguration.ROOT + ".c";
-    config.setCapacity(c, "[memory=1024]");
-
-    // Define 2nd-level queues
-    final String a1 = a + ".a1";
-    final String a2 = a + ".a2";
-    config.setQueues(a, new String[] {"a1", "a2"});
-    config.setCapacity(a1, 30);
-    config.setMaximumCapacity(a1, 50);
-    config.setMaximumLifetimePerQueue(a2, 100);
-    config.setDefaultLifetimePerQueue(a2, 50);
-
-    config.setUserLimitFactor(a1, 100.0f);
-    config.setCapacity(a2, 70);
-    config.setUserLimitFactor(a2, 100.0f);
-
-    final String b1 = b + ".b1";
-    final String b2 = b + ".b2";
-    final String b3 = b + ".b3";
-    config.setQueues(b, new String[] {"b1", "b2", "b3"});
-    config.setCapacity(b1, 60);
-    config.setUserLimitFactor(b1, 100.0f);
-    config.setCapacity(b2, 39.5f);
-    config.setUserLimitFactor(b2, 100.0f);
-    config.setCapacity(b3, 0.5f);
-    config.setUserLimitFactor(b3, 100.0f);
-
-    config.setQueues(a1, new String[] {"a1a", "a1b", "a1c"});
-    final String a1A = a1 + ".a1a";
-    config.setCapacity(a1A, 65);
-    final String a1B = a1 + ".a1b";
-    config.setCapacity(a1B, 15);
-    final String a1C = a1 + ".a1c";
-    config.setCapacity(a1C, 20);
-
-    config.setAutoCreateChildQueueEnabled(a1C, true);
-    config.setInt(PREFIX + a1C + DOT + AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX
-        + DOT + CAPACITY, 50);
-  }
-
   @Test
   public void testClusterScheduler() throws Exception {
-    ClientResponse response = resource().path("ws").path("v1").path("cluster")
-        .path("scheduler").accept(MediaType.APPLICATION_JSON)
-        .get(ClientResponse.class);
-    assertJsonResponse(response, "webapp/scheduler-response.json");
-  }
-
-  @Test
-  public void testClusterSchedulerSlash() throws Exception {
-    ClientResponse response = resource().path("ws").path("v1").path("cluster")
-        .path("scheduler/").accept(MediaType.APPLICATION_JSON)
-        .get(ClientResponse.class);
-    assertJsonResponse(response, "webapp/scheduler-response.json");
-  }
-
-  @Test
-  public void testClusterSchedulerDefault() throws Exception {
-    ClientResponse response = resource().path("ws").path("v1").path("cluster")
-        .path("scheduler").get(ClientResponse.class);
-    assertJsonResponse(response, "webapp/scheduler-response.json");
-  }
-
-  @Test
-  public void testClusterSchedulerXML() throws Exception {
-    ClientResponse response = resource().path("ws").path("v1").path("cluster")
-        .path("scheduler/").accept(MediaType.APPLICATION_XML)
-        .get(ClientResponse.class);
-    assertXmlResponse(response, "webapp/scheduler-response.xml");
+    try (MockRM rm = createRM(createConfig())){
+      assertJsonResponse(resource().path("ws/v1/cluster/scheduler")
+              .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class),
+          "webapp/scheduler-response.json");
+      assertJsonResponse(resource().path("ws/v1/cluster/scheduler/")
+              .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class),
+          "webapp/scheduler-response.json");
+      assertJsonResponse(resource().path("ws/v1/cluster/scheduler").get(ClientResponse.class),
+          "webapp/scheduler-response.json");
+      assertXmlResponse(resource().path("ws/v1/cluster/scheduler/")
+              .accept(MediaType.APPLICATION_XML).get(ClientResponse.class),
+          "webapp/scheduler-response.xml");
+    }
   }
 
   @Test
-  public void testPerUserResourcesXML() throws Exception {
-    // Start RM so that it accepts app submissions
-    rm.start();
-    try {
-      MockRMAppSubmissionData data1 =
-          MockRMAppSubmissionData.Builder.createWithMemory(10, rm)
-              .withAppName("app1")
-              .withUser("user1")
-              .withAcls(null)
-              .withQueue("b1")
-              .withUnmanagedAM(false)
-              .build();
-      MockRMAppSubmitter.submit(rm, data1);
-      MockRMAppSubmissionData data =
-          MockRMAppSubmissionData.Builder.createWithMemory(20, rm)
-              .withAppName("app2")
-              .withUser("user2")
-              .withAcls(null)
-              .withQueue("b1")
-              .withUnmanagedAM(false)
-              .build();
-      MockRMAppSubmitter.submit(rm, data);
+  public void testPerUserResources() throws Exception {
+    try (MockRM rm = createRM(createConfig())){
+      rm.registerNode("h1:1234", 10 * GB, 10);
+      MockRMAppSubmitter.submit(rm, MockRMAppSubmissionData.Builder
+          .createWithMemory(10, rm)
+          .withAppName("app1")
+          .withUser("user1")
+          .withAcls(null)
+          .withQueue("a")
+          .withUnmanagedAM(false)
+          .build()
+      );
+      MockRMAppSubmitter.submit(rm, MockRMAppSubmissionData.Builder
+          .createWithMemory(20, rm)
+          .withAppName("app2")
+          .withUser("user2")
+          .withAcls(null)
+          .withQueue("b")
+          .withUnmanagedAM(false)
+          .build()
+      );
+      assertXmlResponse(resource().path("ws/v1/cluster/scheduler")
+              .accept(MediaType.APPLICATION_XML).get(ClientResponse.class),
+          "webapp/scheduler-response-PerUserResources.xml");
+      assertJsonResponse(resource().path("ws/v1/cluster/scheduler")
+          .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class),
+          "webapp/scheduler-response-PerUserResources.json");
 
-      //Get the XML from ws/v1/cluster/scheduler
-      ClientResponse response = resource().path("ws/v1/cluster/scheduler")
-          .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
-      assertXmlResponse(response, "webapp/scheduler-response-PerUserResources.xml");
-    } finally {
-      rm.stop();
     }
   }
 
   @Test
   public void testNodeLabelDefaultAPI() throws Exception {
-    CapacitySchedulerConfiguration config =
-        ((CapacityScheduler)rm.getResourceScheduler()).getConfiguration();
-
-    config.setDefaultNodeLabelExpression("root", "ROOT-INHERITED");
-    config.setDefaultNodeLabelExpression("root.a", "root-a-default-label");
-    rm.getResourceScheduler().reinitialize(config, rm.getRMContext());
-
-    //Start RM so that it accepts app submissions
-    rm.start();
-    try {
-      //Get the XML from ws/v1/cluster/scheduler
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(createConfig());
+    conf.setDefaultNodeLabelExpression("root", "ROOT-INHERITED");
+    conf.setDefaultNodeLabelExpression("root.a", "root-a-default-label");
+    try (MockRM rm = createRM(conf)) {
       ClientResponse response = resource().path("ws/v1/cluster/scheduler")
           .accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
       assertXmlResponse(response, "webapp/scheduler-response-NodeLabelDefaultAPI.xml");
-    } finally {
-      rm.stop();
     }
   }
-
   @Test
-  public void testPerUserResourcesJSON() throws Exception {
-    //Start RM so that it accepts app submissions
-    rm.start();
-    try {
-      MockRMAppSubmissionData data1 =
-          MockRMAppSubmissionData.Builder.createWithMemory(10, rm)
-              .withAppName("app1")
-              .withUser("user1")
-              .withAcls(null)
-              .withQueue("b1")
-              .withUnmanagedAM(false)
-              .build();
-      MockRMAppSubmitter.submit(rm, data1);
-      MockRMAppSubmissionData data =
-          MockRMAppSubmissionData.Builder.createWithMemory(20, rm)
-              .withAppName("app2")
-              .withUser("user2")
-              .withAcls(null)
-              .withQueue("b1")
-              .withUnmanagedAM(false)
-              .build();
-      MockRMAppSubmitter.submit(rm, data);
-
-      //Get JSON
-      ClientResponse response = resource().path("ws").path("v1").path("cluster")
-          .path("scheduler/").accept(MediaType.APPLICATION_JSON)
-          .get(ClientResponse.class);
-      assertJsonResponse(response, "webapp/scheduler-response-PerUserResources.json");
-    } finally {
-      rm.stop();
+  public void testClusterSchedulerOverviewCapacity() throws Exception {
+    try (MockRM rm = createRM(createConfig())) {
+      ClientResponse response = resource().path("ws/v1/cluster/scheduler-overview")
+          .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+      assertJsonType(response);
+      JSONObject json = response.getEntity(JSONObject.class);
+      TestRMWebServices.verifyClusterSchedulerOverView(json, "Capacity Scheduler");
     }
   }
 
@@ -297,134 +127,14 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     assertEquals("<memory:10, vCores:1>", res.toString());
   }
 
-  public static void assertXmlType(ClientResponse response) {
-    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
-        response.getType().toString());
-  }
-
-  public static void assertXmlResponse(ClientResponse response,
-                                       String expectedResourceFilename) throws
-      Exception {
-    assertXmlType(response);
-    Document document = loadDocument(response.getEntity(String.class));
-    String actual = serializeDocument(document).trim();
-    updateTestDataAutomatically(expectedResourceFilename, actual);
-    assertEquals(getResourceAsString(expectedResourceFilename), actual);
-  }
-
-  public static String serializeDocument(Document document) throws TransformerException {
-    DOMSource domSource = new DOMSource(document);
-    StringWriter writer = new StringWriter();
-    StreamResult result = new StreamResult(writer);
-    TransformerFactory tf = XMLUtils.newSecureTransformerFactory();
-    Transformer transformer = tf.newTransformer();
-    transformer.setOutputProperty(OutputKeys.INDENT, "yes");
-    transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
-    transformer.transform(domSource, result);
-    return writer.toString();
-  }
-
-  public static Document loadDocument(String xml) throws Exception {
-    DocumentBuilderFactory factory = XMLUtils.newSecureDocumentBuilderFactory();
-    DocumentBuilder builder = factory.newDocumentBuilder();
-    InputSource is = new InputSource(new StringReader(xml));
-    return builder.parse(is);
-  }
-
-  public static void assertJsonResponse(ClientResponse response,
-                                        String expectedResourceFilename) throws
-      JSONException, IOException {
-    assertJsonType(response);
-    JSONObject json = response.getEntity(JSONObject.class);
-    String actual = prettyPrintJson(json.toString(2));
-    updateTestDataAutomatically(expectedResourceFilename, actual);
-    assertEquals(
-        prettyPrintJson(getResourceAsString(expectedResourceFilename)),
-        prettyPrintJson(actual));
-  }
-
-  private static String prettyPrintJson(String in) throws JsonProcessingException {
-    ObjectMapper objectMapper = new ObjectMapper();
-    return objectMapper
-        .writerWithDefaultPrettyPrinter()
-        .writeValueAsString(objectMapper.readTree(in));
-  }
-
-  public static void assertJsonType(ClientResponse response) {
-    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
-        response.getType().toString());
-  }
-
-  public static InputStream getResourceAsStream(String configFilename) {
-    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
-    return classLoader.getResourceAsStream(configFilename);
-  }
-
-  public static String getResourceAsString(String configFilename) throws IOException {
-    try (InputStream is = getResourceAsStream(configFilename)) {
-      if (is == null) {
-        return null;
-      }
-      try (InputStreamReader isr = new InputStreamReader(is);
-           BufferedReader reader = new BufferedReader(isr)) {
-        return reader.lines().collect(Collectors.joining(System.lineSeparator()));
-      }
-    }
-  }
-
-  public static void updateTestDataAutomatically(String configFilename, String actualContent) {
-    /*
-     Set UPDATE_TESTDATA=1 environment variable for auto update the expected data
-     or uncomment this return statement.
-
-     It's safe in a way that, this updates the source directory so the test will still fail,
-     because the target directory is untouched.
-     */
-    if (System.getenv("UPDATE_TESTDATA") == null) {
-      return;
-    }
-
-    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
-    try {
-      String resource = Objects.requireNonNull(
-          Objects.requireNonNull(classLoader.getResource(configFilename)).toURI().getPath())
-          .replaceAll("/target/test-classes/", "/src/test/resources/");
-      try (FileWriter writer = new FileWriter(resource, false)) {
-        writer.write(actualContent);
-      }
-    } catch (URISyntaxException | IOException e) {
-      e.printStackTrace();
-      Assert.fail("overwrite should not fail " + e.getMessage());
-    }
-  }
-
-  public static WebAppDescriptor createWebAppDescriptor() {
-    return new WebAppDescriptor.Builder(
-        TestRMWebServicesCapacitySched.class.getPackage().getName())
-        .contextListenerClass(GuiceServletConfig.class)
-        .filterClass(com.google.inject.servlet.GuiceFilter.class)
-        .contextPath("jersey-guice-filter").servletPath("/").build();
-  }
-
-  public static MockRM createMockRM(CapacitySchedulerConfiguration csConf) {
-    setupQueueConfiguration(csConf);
-    YarnConfiguration conf = new YarnConfiguration(csConf);
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-    conf.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
-        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
-    return new MockRM(conf);
-  }
-
-  @Test
-  public void testClusterSchedulerOverviewCapacity() throws Exception {
-    WebResource r = resource();
-    ClientResponse response = r.path("ws").path("v1").path("cluster")
-        .path("scheduler-overview").accept(MediaType.APPLICATION_JSON)
-        .get(ClientResponse.class);
-    assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-        response.getType().toString());
-    JSONObject json = response.getEntity(JSONObject.class);
-    TestRMWebServices.verifyClusterSchedulerOverView(json, "Capacity Scheduler");
+  private Configuration createConfig() {
+    Configuration conf = new Configuration();
+    conf.set("yarn.scheduler.capacity.root.queues", "a, b, c");
+    conf.set("yarn.scheduler.capacity.root.a.capacity", "20");
+    conf.set("yarn.scheduler.capacity.root.a.maximum-capacity", "50");
+    conf.set("yarn.scheduler.capacity.root.a.max-parallel-app", "42");
+    conf.set("yarn.scheduler.capacity.root.b.capacity", "70");
+    conf.set("yarn.scheduler.capacity.root.c.capacity", "10");
+    return conf;
   }
 }

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

@@ -18,311 +18,155 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
-import com.google.inject.Guice;
-import com.sun.jersey.api.client.ClientResponse;
-
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import javax.ws.rs.core.MediaType;
+import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate;
 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.CapacitySchedulerQueueManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
-import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
-import org.junit.Test;
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfigGeneratorForTest.createConfiguration;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.GB;
-import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.assertJsonResponse;
-import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.createMockRM;
-import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.createWebAppDescriptor;
-
-public class TestRMWebServicesCapacitySchedDynamicConfig extends
-    JerseyTestBase {
-  private MockRM rm;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.assertJsonResponse;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.createMutableRM;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.createWebAppDescriptor;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.reinitialize;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.runTest;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.sendRequest;
+import static org.assertj.core.api.Assertions.fail;
+
+
+/*
+ *                                         EffectiveMin (32GB 32VCores)     AbsoluteCapacity
+ *     root.default              4/32      [memory=4096,  vcores=4]       12.5%
+ *     root.test_1              16/32      [memory=16384, vcores=16]
+ *     root.test_2              12/32      [memory=12288, vcores=12]      37.5%
+ *     root.test_1.test_1_1      2/16      [memory=2048,  vcores=2]       6.25%
+ *     root.test_1.test_1_2      2/16      [memory=2048,  vcores=2]       6.25%
+ *     root.test_1.test_1_3     12/16      [memory=12288, vcores=12]      37.5%
+ */
+public class TestRMWebServicesCapacitySchedDynamicConfig extends JerseyTestBase {
 
-  private CapacitySchedulerQueueManager autoQueueHandler;
+  private static final String EXPECTED_FILE_TMPL = "webapp/dynamic-%s-%s.json";
 
   public TestRMWebServicesCapacitySchedDynamicConfig() {
     super(createWebAppDescriptor());
   }
 
   @Test
-  public void testSchedulerResponsePercentageMode()
-      throws Exception {
-    Configuration config = CSConfigGenerator
-        .createPercentageConfig();
-
-    initResourceManager(config);
-
-    /*
-     * mode: percentage
-     * autoCreationEligibility: off
-     * weight: -1, normalizedWeight: 0
-     * root.queueType: parent, others.queueType: leaf
-     */
-    assertJsonResponse(sendRequest(), "webapp/scheduler-response-PercentageMode.json");
-  }
-
-  @Test
-  public void testSchedulerResponsePercentageModeLegacyAutoCreation()
-      throws Exception {
-    Configuration config = CSConfigGenerator
-        .createPercentageConfigLegacyAutoCreation();
-
-    initResourceManager(config);
-
-    /*
-     * mode: percentage
-     * managedtest2.autoCreationEligibility: legacy, others.autoCreationEligibility: off
-     * weight: -1, normalizedWeight: 0
-     * root.queueType: parent, others.queueType: leaf
-     */
-    assertJsonResponse(sendRequest(),
-        "webapp/scheduler-response-PercentageModeLegacyAutoCreation.json");
-  }
-
-  @Test
-  public void testSchedulerResponseAbsoluteModeLegacyAutoCreation()
-      throws Exception {
-    Configuration config = CSConfigGenerator
-        .createAbsoluteConfigLegacyAutoCreation();
-
-    initResourceManager(config);
-    initAutoQueueHandler(8192 * GB);
-    createQueue("root.managed.queue1");
-
-    assertJsonResponse(sendRequest(),
-        "webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json");
+  public void testPercentageMode() throws Exception {
+    Map<String, String> conf = new HashMap<>();
+    conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2");
+    conf.put("yarn.scheduler.capacity.root.test1.queues", "test1_1, test1_2, test1_3");
+    conf.put("yarn.scheduler.capacity.root.default.capacity", "12.5");
+    conf.put("yarn.scheduler.capacity.root.test1.capacity", "50");
+    conf.put("yarn.scheduler.capacity.root.test2.capacity", "37.5");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_1.capacity", "12.5");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_2.capacity", "12.5");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_3.capacity", "75");
+    try (MockRM rm = createMutableRM(createConfiguration(conf))) {
+      runTest(EXPECTED_FILE_TMPL, "testPercentageMode", rm, resource());
+    }
   }
-
   @Test
-  public void testSchedulerResponseAbsoluteMode()
-      throws Exception {
-    Configuration config = CSConfigGenerator
-        .createAbsoluteConfig();
-
-    initResourceManager(config);
-
-    /*
-     * mode: absolute
-     * autoCreationEligibility: off
-     * weight: -1, normalizedWeight: 0
-     * root.queueType: parent, others.queueType: leaf
-     */
-    assertJsonResponse(sendRequest(), "webapp/scheduler-response-AbsoluteMode.json");
+  public void testAbsoluteMode() throws Exception {
+    Map<String, String> conf = new HashMap<>();
+    conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2");
+    conf.put("yarn.scheduler.capacity.root.test1.queues", "test1_1, test1_2, test1_3");
+    conf.put("yarn.scheduler.capacity.root.default.capacity", "[memory=4096,vcores=4]");
+    conf.put("yarn.scheduler.capacity.root.test1.capacity", "[memory=16384,vcores=16]");
+    conf.put("yarn.scheduler.capacity.root.test2.capacity", "[memory=12288,vcores=12]");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_1.capacity", "[memory=2048,vcores=2]");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_2.capacity", "[memory=2048,vcores=2]");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_3.capacity", "[memory=12288,vcores=12]");
+    try (MockRM rm = createMutableRM(createConfiguration(conf))) {
+      runTest(EXPECTED_FILE_TMPL, "testAbsoluteMode", rm, resource());
+    }
   }
 
   @Test
-  public void testSchedulerResponseWeightMode()
-      throws Exception {
-    Configuration config = CSConfigGenerator
-        .createWeightConfig();
-
-    initResourceManager(config);
-
-    /*
-     * mode: weight
-     * autoCreationEligibility: off
-     *                   root   default  test1  test2
-     * weight:            1       10       4     6
-     * normalizedWeight:  1       0.5      0.2   0.3
-     * root.queueType: parent, others.queueType: leaf
-     */
-    assertJsonResponse(sendRequest(), "webapp/scheduler-response-WeightMode.json");
+  public void testWeightMode() throws Exception {
+    Map<String, String> conf = new HashMap<>();
+    conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2");
+    conf.put("yarn.scheduler.capacity.root.test1.queues", "test1_1, test1_2, test1_3");
+    conf.put("yarn.scheduler.capacity.root.default.capacity", "4w");
+    conf.put("yarn.scheduler.capacity.root.test1.capacity", "16w");
+    conf.put("yarn.scheduler.capacity.root.test2.capacity", "12w");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_1.capacity", "2w");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_2.capacity", "2w");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_3.capacity", "12w");
+    try (MockRM rm = createMutableRM(createConfiguration(conf))) {
+      runTest(EXPECTED_FILE_TMPL, "testWeightMode", rm, resource());
+    }
   }
 
   @Test
-  public void testSchedulerResponseWeightModeWithAutoCreatedQueues()
-      throws Exception {
-    Configuration config = CSConfigGenerator
-        .createWeightConfigWithAutoQueueCreationEnabled();
-    config.setInt(CapacitySchedulerConfiguration
-        .getQueuePrefix("root.autoParent1") +
-        AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX +
-        "maximum-applications", 300);
-
-    initResourceManager(config);
-    initAutoQueueHandler(1200 * GB);
-
-    // same as webapp/scheduler-response-WeightMode.json, but with effective resources filled in
-    assertJsonResponse(sendRequest(),
-        "webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json");
-
-    //Now create some auto created queues
-    createQueue("root.auto1");
-    createQueue("root.auto2");
-    createQueue("root.auto3");
-    createQueue("root.autoParent1.auto4");
-    createQueue("root.autoParent2.auto5");
-    createQueue("root.parent.autoParent3.auto6");
-
-    assertJsonResponse(sendRequest(),
-        "webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json");
-  }
-
-  private void initAutoQueueHandler(int nodeMemory) throws Exception {
-    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
-    autoQueueHandler = cs.getCapacitySchedulerQueueManager();
-    rm.registerNode("h1:1234", nodeMemory); // label = x
+  public void testWeightModeFlexibleAQC() throws Exception {
+    Map<String, String> conf = new HashMap<>();
+    conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2");
+    conf.put("yarn.scheduler.capacity.root.test1.queues", "test1_1, test1_2, test1_3");
+    conf.put("yarn.scheduler.capacity.root.default.capacity", "4w");
+    conf.put("yarn.scheduler.capacity.root.test1.capacity", "16w");
+    conf.put("yarn.scheduler.capacity.root.test2.capacity", "12w");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_1.capacity", "2w");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_2.capacity", "2w");
+    conf.put("yarn.scheduler.capacity.root.test1.test1_3.capacity", "12w");
+
+    Configuration config = createConfiguration(conf);
+    setupAQC(config, "yarn.scheduler.capacity.root.test2.");
+    try (MockRM rm = createMutableRM(config)) {
+      rm.registerNode("h1:1234", 32 * GB, 32);
+      assertJsonResponse(sendRequest(resource()),
+          String.format(EXPECTED_FILE_TMPL, "testWeightMode", "before-aqc"));
+      createAQC(rm, "test2");
+      reinitialize(rm, config);
+      assertJsonResponse(sendRequest(resource()),
+          String.format(EXPECTED_FILE_TMPL, "testWeightMode", "after-aqc"));
+    }
   }
 
-  private void createQueue(String queuePath) throws YarnException,
-      IOException {
-    autoQueueHandler.createQueue(new QueuePath(queuePath));
-  }
 
-  private ClientResponse sendRequest() {
-    return resource().path("ws").path("v1").path("cluster")
-        .path("scheduler").accept(MediaType.APPLICATION_JSON)
-        .get(ClientResponse.class);
+  private void setupAQC(Configuration config, String queue) {
+    config.set(queue + "auto-queue-creation-v2.enabled", "true");
+    config.set(queue + "auto-queue-creation-v2.maximum-queue-depth", "10");
+    config.set(queue + "auto-queue-creation-v2.parent-template.acl_submit_applications",
+        "parentUser");
+    config.set(queue + "auto-queue-creation-v2.parent-template.acl_administer_queue",
+        "parentAdmin");
+    config.set(queue + "autoParent1.auto-queue-creation-v2.leaf-template.acl_submit_applications",
+        "ap1User");
+    config.set(queue + "autoParent1.auto-queue-creation-v2.leaf-template.acl_administer_queue",
+        "ap1Admin");
+    config.set(queue + "*.auto-queue-creation-v2.leaf-template.acl_submit_applications",
+        "leafUser");
+    config.set(queue + "*.auto-queue-creation-v2.leaf-template.acl_administer_queue",
+        "leafAdmin");
+    config.set(queue + "parent.*.auto-queue-creation-v2.leaf-template.acl_submit_applications",
+        "pLeafUser");
+    config.set(queue + "parent.*.auto-queue-creation-v2.leaf-template.acl_administer_queue",
+        "pLeafAdmin");
+    config.set(queue + "autoParent1.auto-queue-creation-v2.template.maximum-applications", "300");
   }
-
-  private static class CSConfigGenerator {
-    public static Configuration createPercentageConfig() {
-      Map<String, String> conf = new HashMap<>();
-      conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2");
-      conf.put("yarn.scheduler.capacity.root.test1.capacity", "50");
-      conf.put("yarn.scheduler.capacity.root.test2.capacity", "50");
-      conf.put("yarn.scheduler.capacity.root.test1.maximum-capacity", "100");
-      conf.put("yarn.scheduler.capacity.root.test1.state", "RUNNING");
-      conf.put("yarn.scheduler.capacity.root.test2.state", "RUNNING");
-      return createConfiguration(conf);
-    }
-
-    public static Configuration createPercentageConfigLegacyAutoCreation() {
-      Map<String, String> conf = new HashMap<>();
-      conf.put("yarn.scheduler.capacity.root.queues", "default, test1, " +
-          "managedtest2");
-      conf.put("yarn.scheduler.capacity.root.test1.capacity", "50");
-      conf.put("yarn.scheduler.capacity.root.managedtest2.capacity", "50");
-      conf.put("yarn.scheduler.capacity.root.test1.maximum-capacity", "100");
-      conf.put("yarn.scheduler.capacity.root.test1.state", "RUNNING");
-      conf.put("yarn.scheduler.capacity.root.managedtest2.state", "RUNNING");
-      conf.put("yarn.scheduler.capacity.root.managedtest2." +
-          "auto-create-child-queue.enabled", "true");
-      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");
-      conf.put("yarn.scheduler.capacity.root.managed.leaf-queue-template.acl_submit_applications",
-          "user");
-      conf.put("yarn.scheduler.capacity.root.managed.leaf-queue-template.acl_administer_queue",
-          "admin");
-      return createConfiguration(conf);
-    }
-
-    public static Configuration createAbsoluteConfig() {
-      Map<String, String> conf = new HashMap<>();
-      conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2");
-      conf.put("yarn.scheduler.capacity.root.capacity",
-          "[memory=6136,vcores=30]");
-      conf.put("yarn.scheduler.capacity.root.default.capacity",
-          "[memory=3064,vcores=15]");
-      conf.put("yarn.scheduler.capacity.root.test1.capacity",
-          "[memory=2048,vcores=10]");
-      conf.put("yarn.scheduler.capacity.root.test2.capacity",
-          "[memory=1024,vcores=5]");
-      conf.put("yarn.scheduler.capacity.root.test1.state", "RUNNING");
-      conf.put("yarn.scheduler.capacity.root.test2.state", "RUNNING");
-      return createConfiguration(conf);
-    }
-
-    public static Configuration createWeightConfig() {
-      return createWeightConfigInternal(false);
-    }
-
-    public static Configuration createWeightConfigWithAutoQueueCreationEnabled() {
-      return createWeightConfigInternal(true);
-    }
-
-    private static Configuration createWeightConfigInternal(boolean enableAqc) {
-      Map<String, String> conf = new HashMap<>();
-      conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2, parent");
-      conf.put("yarn.scheduler.capacity.root.capacity", "1w");
-      conf.put("yarn.scheduler.capacity.root.default.capacity", "10w");
-      conf.put("yarn.scheduler.capacity.root.test1.capacity", "5w");
-      conf.put("yarn.scheduler.capacity.root.test2.capacity", "10w");
-      conf.put("yarn.scheduler.capacity.root.parent.capacity", "20w");
-      conf.put("yarn.scheduler.capacity.root.test1.state", "RUNNING");
-      conf.put("yarn.scheduler.capacity.root.test2.state", "RUNNING");
-
-      if (enableAqc) {
-        final String root = "yarn.scheduler.capacity.root.";
-        conf.put(root +  "auto-queue-creation-v2.enabled", "true");
-
-        conf.put(root + "auto-queue-creation-v2.parent-template.acl_submit_applications",
-            "parentUser1");
-        conf.put(root + "auto-queue-creation-v2.parent-template.acl_administer_queue",
-            "parentAdmin1");
-
-        conf.put(root + "autoParent1.auto-queue-creation-v2.leaf-template.acl_submit_applications",
-            "user1");
-        conf.put(root + "autoParent1.auto-queue-creation-v2.leaf-template.acl_administer_queue",
-            "admin1");
-
-        conf.put(root + "*.auto-queue-creation-v2.leaf-template.acl_submit_applications",
-            "wildUser1");
-        conf.put(root + "*.auto-queue-creation-v2.leaf-template.acl_administer_queue",
-            "wildAdmin1");
-
-
-        conf.put(root + "parent.auto-queue-creation-v2.enabled", "true");
-        conf.put(root + "parent.auto-queue-creation-v2.parent-template.acl_submit_applications",
-            "parentUser2");
-        conf.put(root + "parent.auto-queue-creation-v2.parent-template.acl_administer_queue",
-            "parentAdmin2");
-
-        conf.put(root + "parent.*.auto-queue-creation-v2.leaf-template.acl_submit_applications",
-            "wildUser2");
-        conf.put(root + "parent.*.auto-queue-creation-v2.leaf-template.acl_administer_queue",
-            "wildAdmin2");
-      }
-      return createConfiguration(conf);
+  private void createAQC(MockRM rm, String queue) {
+    try {
+      CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+      CapacitySchedulerQueueManager autoQueueHandler = cs.getCapacitySchedulerQueueManager();
+      autoQueueHandler.createQueue(new QueuePath("root." + queue + ".auto1"));
+      autoQueueHandler.createQueue(new QueuePath("root." + queue + ".auto2"));
+      autoQueueHandler.createQueue(new QueuePath("root." + queue + ".autoParent1.auto3"));
+      autoQueueHandler.createQueue(new QueuePath("root." + queue + ".autoParent1.auto4"));
+      autoQueueHandler.createQueue(new QueuePath("root." + queue + ".autoParent2.auto5"));
+      autoQueueHandler.createQueue(new QueuePath("root." + queue + ".parent.autoParent2.auto6"));
+      autoQueueHandler.createQueue(new QueuePath("root." + queue + ".parent2.auto7"));
+    } catch (YarnException | IOException e) {
+      fail("Can not auto create queues under " + queue, e);
     }
-
-    public static Configuration createConfiguration(
-        Map<String, String> configs) {
-      Configuration config = new Configuration();
-
-      for (Map.Entry<String, String> entry : configs.entrySet()) {
-        config.set(entry.getKey(), entry.getValue());
-      }
-
-      config.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
-          YarnConfiguration.MEMORY_CONFIGURATION_STORE);
-
-      return config;
-    }
-  }
-
-  private void initResourceManager(Configuration conf) throws IOException {
-    rm = createMockRM(new CapacitySchedulerConfiguration(conf));
-    GuiceServletConfig.setInjector(
-        Guice.createInjector(new TestRMWebServicesCapacitySched.WebServletModule(rm)));
-    rm.start();
-    //Need to call reinitialize as
-    //MutableCSConfigurationProvider with InMemoryConfigurationStore
-    //somehow does not load the queues properly and falls back to default config.
-    //Therefore CS will think there's only the default queue there.
-    ((CapacityScheduler) rm.getResourceScheduler()).reinitialize(conf,
-        rm.getRMContext(), true);
   }
-}
+}

+ 93 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedLegacyQueueCreation.java

@@ -0,0 +1,93 @@
+/**
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import javax.ws.rs.core.MediaType;
+
+import com.sun.jersey.api.client.ClientResponse;
+import org.junit.Test;
+
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
+import org.apache.hadoop.yarn.webapp.JerseyTestBase;
+
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfigGeneratorForTest.createConfiguration;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.GB;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.assertJsonResponse;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.createMutableRM;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.createWebAppDescriptor;
+
+public class TestRMWebServicesCapacitySchedLegacyQueueCreation extends
+    JerseyTestBase {
+
+  public TestRMWebServicesCapacitySchedLegacyQueueCreation() {
+    super(createWebAppDescriptor());
+  }
+
+  @Test
+  public void testSchedulerResponsePercentageModeLegacyAutoCreation()
+      throws Exception {
+    Map<String, String> conf = new HashMap<>();
+    conf.put("yarn.scheduler.capacity.root.queues", "default, managed");
+    conf.put("yarn.scheduler.capacity.root.default.capacity", "20");
+    conf.put("yarn.scheduler.capacity.root.managed.capacity", "80");
+    conf.put("yarn.scheduler.capacity.root.managed." +
+        "auto-create-child-queue.enabled", "true");
+    try (MockRM rm = createMutableRM(createConfiguration(conf))) {
+      assertJsonResponse(sendRequest(),
+          "webapp/scheduler-response-PercentageModeLegacyAutoCreation.json");
+    }
+  }
+
+  @Test
+  public void testSchedulerResponseAbsoluteModeLegacyAutoCreation()
+      throws Exception {
+    Map<String, String> conf = new HashMap<>();
+    conf.put("yarn.scheduler.capacity.root.queues", "default, managed");
+    conf.put("yarn.scheduler.capacity.root.default.capacity", "[memory=28672,vcores=28]");
+    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." +
+        "auto-create-child-queue.enabled", "true");
+    conf.put("yarn.scheduler.capacity.root.managed.leaf-queue-template.acl_submit_applications",
+        "user");
+    conf.put("yarn.scheduler.capacity.root.managed.leaf-queue-template.acl_administer_queue",
+        "admin");
+    try (MockRM rm = createMutableRM(createConfiguration(conf))) {
+      rm.registerNode("h1:1234", 32 * GB, 32);
+      CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+      CapacitySchedulerQueueManager autoQueueHandler = cs.getCapacitySchedulerQueueManager();
+      autoQueueHandler.createQueue(new QueuePath("root.managed.queue1"));
+      assertJsonResponse(sendRequest(),
+          "webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json");
+    }
+  }
+
+  private ClientResponse sendRequest() {
+    return resource().path("ws").path("v1").path("cluster")
+        .path("scheduler").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+  }
+
+}

+ 80 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java

@@ -26,6 +26,8 @@ import com.sun.jersey.core.util.MultivaluedMapImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@@ -88,8 +90,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.webapp.ActivitiesTes
 import static org.apache.hadoop.yarn.server.resourcemanager.webapp.ActivitiesTestUtils.verifyNumberOfNodes;
 import static org.apache.hadoop.yarn.server.resourcemanager.webapp.ActivitiesTestUtils.verifyQueueOrder;
 import static org.apache.hadoop.yarn.server.resourcemanager.webapp.ActivitiesTestUtils.verifyStateOfAllocations;
-import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.createMockRM;
-import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.createWebAppDescriptor;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestWebServiceUtil.createWebAppDescriptor;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -110,10 +111,52 @@ public class TestRMWebServicesSchedulerActivities extends JerseyTestBase {
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    rm = createMockRM(new CapacitySchedulerConfiguration(
-        new Configuration(false)));
+    CapacitySchedulerConfiguration config =
+        createConfig(new CapacitySchedulerConfiguration(new Configuration(false)));
+    rm = createMockRM(config);
     GuiceServletConfig.setInjector(
-        Guice.createInjector(new TestRMWebServicesCapacitySched.WebServletModule(rm)));
+        Guice.createInjector(new TestWebServiceUtil.WebServletModule(rm)));
+  }
+
+  public static MockRM createMockRM(CapacitySchedulerConfiguration csConf) {
+    setupQueueConfiguration(csConf);
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
+        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
+    return new MockRM(conf);
+  }
+
+  public static void setupQueueConfiguration(
+      CapacitySchedulerConfiguration config) {
+    config.set("yarn.scheduler.capacity.root.queues", "a, b, c");
+    config.set("yarn.scheduler.capacity.root.a.queues", "a1, a2");
+    config.set("yarn.scheduler.capacity.root.b.queues", "b1, b2, b3");
+    config.set("yarn.scheduler.capacity.root.a.a1.queues", "a1a, a1b, a1c");
+    config.set("yarn.scheduler.capacity.root.a.capacity", "10.5");
+    config.set("yarn.scheduler.capacity.root.a.maximum-capacity", "50");
+    config.set("yarn.scheduler.capacity.root.a.max-parallel-app", "42");
+    config.set("yarn.scheduler.capacity.root.b.capacity", "79.5");
+    config.set("yarn.scheduler.capacity.root.c.capacity", "10");
+    config.set("yarn.scheduler.capacity.root.a.a1.capacity", "30");
+    config.set("yarn.scheduler.capacity.root.a.a1.maximum-capacity", "50");
+    config.set("yarn.scheduler.capacity.root.a.a1.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.a.a2.capacity", "70");
+    config.set("yarn.scheduler.capacity.root.a.a2.maximum-application-lifetime", "100");
+    config.set("yarn.scheduler.capacity.root.a.a2.default-application-lifetime", "50");
+    config.set("yarn.scheduler.capacity.root.a.a2.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.b.b1.capacity", "60");
+    config.set("yarn.scheduler.capacity.root.b.b2.capacity", "39.5");
+    config.set("yarn.scheduler.capacity.root.b.b3.capacity", "0.5");
+    config.set("yarn.scheduler.capacity.root.b.b1.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.b.b2.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.b.b3.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1a.capacity", "65");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1b.capacity", "15");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1c.capacity", "20");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1c.auto-create-child-queue.enabled", "true");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1c.leaf-queue-template.capacity", "50");
   }
 
   @Test
@@ -399,7 +442,7 @@ public class TestRMWebServicesSchedulerActivities extends JerseyTestBase {
       JSONObject allocations = getFirstSubNodeFromJson(json,
           FN_SCHEDULER_ACT_ROOT, FN_ACT_ALLOCATIONS);
       verifyQueueOrder(allocations,
-          "root-root.c-root.a-root.b-root.b.b3-root.b.b1");
+          "root-root.a-root.c-root.b-root.b.b3-root.b.b1");
       verifyStateOfAllocations(allocations, FN_ACT_FINAL_ALLOCATION_STATE,
           "RESERVED");
 
@@ -1705,4 +1748,35 @@ public class TestRMWebServicesSchedulerActivities extends JerseyTestBase {
       return restClient.isDone();
     }, 10, 20000);
   }
+
+  private CapacitySchedulerConfiguration createConfig(CapacitySchedulerConfiguration config) {
+    config.set("yarn.scheduler.capacity.root.queues", "a, b, c");
+    config.set("yarn.scheduler.capacity.root.a.queues", "a1, a2");
+    config.set("yarn.scheduler.capacity.root.b.queues", "b1, b2, b3");
+    config.set("yarn.scheduler.capacity.root.a.a1.queues", "a1a, a1b, a1c");
+    config.set("yarn.scheduler.capacity.root.a.capacity", "10.5");
+    config.set("yarn.scheduler.capacity.root.a.maximum-capacity", "50");
+    config.set("yarn.scheduler.capacity.root.a.max-parallel-app", "42");
+    config.set("yarn.scheduler.capacity.root.b.capacity", "79.5");
+    config.set("yarn.scheduler.capacity.root.c.capacity", "10");
+    config.set("yarn.scheduler.capacity.root.a.a1.capacity", "30");
+    config.set("yarn.scheduler.capacity.root.a.a1.maximum-capacity", "50");
+    config.set("yarn.scheduler.capacity.root.a.a1.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.a.a2.capacity", "70");
+    config.set("yarn.scheduler.capacity.root.a.a2.maximum-application-lifetime", "100");
+    config.set("yarn.scheduler.capacity.root.a.a2.default-application-lifetime", "50");
+    config.set("yarn.scheduler.capacity.root.a.a2.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.b.b1.capacity", "60");
+    config.set("yarn.scheduler.capacity.root.b.b2.capacity", "39.5");
+    config.set("yarn.scheduler.capacity.root.b.b3.capacity", "0.5");
+    config.set("yarn.scheduler.capacity.root.b.b1.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.b.b2.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.b.b3.user-limit-factor", "100");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1a.capacity", "65");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1b.capacity", "15");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1c.capacity", "20");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1c.auto-create-child-queue.enabled", "true");
+    config.set("yarn.scheduler.capacity.root.a.a1.a1c.leaf-queue-template.capacity", "50");
+    return config;
+  }
 }

+ 265 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestWebServiceUtil.java

@@ -0,0 +1,265 @@
+/**
+ * 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;
+
+import java.io.BufferedReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.net.URISyntaxException;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import javax.ws.rs.core.MediaType;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Guice;
+import com.google.inject.servlet.ServletModule;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
+import com.sun.jersey.test.framework.WebAppDescriptor;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.junit.Assert;
+import org.w3c.dom.Document;
+import org.xml.sax.InputSource;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
+import org.apache.hadoop.util.XMLUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
+
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.MEMORY_CONFIGURATION_STORE;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.GB;
+import static org.junit.Assert.assertEquals;
+
+public final class TestWebServiceUtil {
+  private TestWebServiceUtil(){
+  }
+
+  public static class WebServletModule extends ServletModule {
+    private final MockRM rm;
+
+    WebServletModule(MockRM rm) {
+      this.rm = rm;
+    }
+
+    @Override
+    protected void configureServlets() {
+      bind(JAXBContextResolver.class);
+      bind(RMWebServices.class);
+      bind(GenericExceptionHandler.class);
+      bind(ResourceManager.class).toInstance(rm);
+      serve("/*").with(GuiceContainer.class);
+    }
+  }
+
+  public static void runTest(String template, String name,
+      MockRM rm,
+      WebResource resource) throws Exception {
+    final boolean reinitAfterNodeChane = isMutableConfig(rm.getConfig());
+    try {
+      assertJsonResponse(sendRequest(resource), String.format(template, name, 0));
+      MockNM nm1 = rm.registerNode("h1:1234", 8 * GB, 8);
+      rm.registerNode("h2:1234", 8 * GB, 8);
+      if (reinitAfterNodeChane) {
+        reinitialize(rm, rm.getConfig());
+      }
+      assertJsonResponse(sendRequest(resource), String.format(template, name, 16));
+      rm.registerNode("h3:1234", 8 * GB, 8);
+      MockNM nm4 = rm.registerNode("h4:1234", 8 * GB, 8);
+      if (reinitAfterNodeChane) {
+        reinitialize(rm, rm.getConfig());
+      }
+      assertJsonResponse(sendRequest(resource), String.format(template, name, 32));
+      rm.unRegisterNode(nm1);
+      rm.unRegisterNode(nm4);
+      assertJsonResponse(sendRequest(resource), String.format(template, name, 16));
+    } finally {
+      rm.close();
+    }
+  }
+  public static boolean isMutableConfig(Configuration config) {
+    return Objects.equals(config.get(SCHEDULER_CONFIGURATION_STORE_CLASS),
+        MEMORY_CONFIGURATION_STORE);
+  }
+
+  public static ClientResponse sendRequest(WebResource resource) {
+    return resource.path("ws").path("v1").path("cluster")
+        .path("scheduler").accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+  }
+
+  public static void assertXmlType(ClientResponse response) {
+    assertEquals(MediaType.APPLICATION_XML_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+  }
+
+  public static void assertXmlResponse(ClientResponse response,
+      String expectedResourceFilename) throws
+      Exception {
+    assertXmlType(response);
+    Document document = loadDocument(response.getEntity(String.class));
+    String actual = serializeDocument(document).trim();
+    updateTestDataAutomatically(expectedResourceFilename, actual);
+    assertEquals(getResourceAsString(expectedResourceFilename), actual);
+  }
+
+  public static String serializeDocument(Document document) throws TransformerException {
+    DOMSource domSource = new DOMSource(document);
+    StringWriter writer = new StringWriter();
+    StreamResult result = new StreamResult(writer);
+    TransformerFactory tf = XMLUtils.newSecureTransformerFactory();
+    Transformer transformer = tf.newTransformer();
+    transformer.setOutputProperty(OutputKeys.INDENT, "yes");
+    transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
+    transformer.transform(domSource, result);
+    return writer.toString();
+  }
+
+  public static Document loadDocument(String xml) throws Exception {
+    DocumentBuilderFactory factory = XMLUtils.newSecureDocumentBuilderFactory();
+    DocumentBuilder builder = factory.newDocumentBuilder();
+    InputSource is = new InputSource(new StringReader(xml));
+    return builder.parse(is);
+  }
+
+  public static void assertJsonResponse(ClientResponse response,
+      String expectedResourceFilename) throws JSONException,
+      IOException {
+    assertJsonType(response);
+    JSONObject json = response.getEntity(JSONObject.class);
+    String actual = prettyPrintJson(json.toString(2));
+    updateTestDataAutomatically(expectedResourceFilename, actual);
+    assertEquals(
+        prettyPrintJson(getResourceAsString(expectedResourceFilename)),
+        actual);
+  }
+
+  private static String prettyPrintJson(String in) throws JsonProcessingException {
+    ObjectMapper objectMapper = new ObjectMapper();
+    return objectMapper
+        .writerWithDefaultPrettyPrinter()
+        .writeValueAsString(objectMapper.readTree(in));
+  }
+
+  public static void assertJsonType(ClientResponse response) {
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+  }
+
+  public static InputStream getResourceAsStream(String configFilename) {
+    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+    return classLoader.getResourceAsStream(configFilename);
+  }
+
+  public static String getResourceAsString(String configFilename) throws IOException {
+    try (InputStream is = getResourceAsStream(configFilename)) {
+      if (is == null) {
+        return null;
+      }
+      try (InputStreamReader isr = new InputStreamReader(is);
+          BufferedReader reader = new BufferedReader(isr)) {
+        return reader.lines().collect(Collectors.joining(System.lineSeparator()));
+      }
+    }
+  }
+
+  public static void updateTestDataAutomatically(String configFilename, String actualContent) {
+    /*
+     Set UPDATE_TESTDATA=1 environment variable for auto update the expected data
+     or uncomment this return statement.
+
+     It's safe in a way that, this updates the source directory so the test will still fail,
+     because the target directory is untouched.
+     */
+    if (System.getenv("UPDATE_TESTDATA") == null) {
+      return;
+    }
+
+    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+    try {
+      String resource = Objects.requireNonNull(
+              Objects.requireNonNull(classLoader.getResource(configFilename)).toURI().getPath())
+          .replaceAll("/target/test-classes/", "/src/test/resources/");
+      try (FileWriter writer = new FileWriter(resource, false)) {
+        writer.write(actualContent);
+      }
+    } catch (URISyntaxException | IOException e) {
+      e.printStackTrace();
+      Assert.fail("overwrite should not fail " + e.getMessage());
+    }
+  }
+  public static WebAppDescriptor createWebAppDescriptor() {
+    return new WebAppDescriptor.Builder(
+        TestRMWebServicesCapacitySched.class.getPackage().getName())
+        .contextListenerClass(GuiceServletConfig.class)
+        .filterClass(com.google.inject.servlet.GuiceFilter.class)
+        .contextPath("jersey-guice-filter").servletPath("/").build();
+  }
+
+  public static MockRM createRM(Configuration config) {
+    config.setClass(YarnConfiguration.RM_SCHEDULER,
+        CapacityScheduler.class, ResourceScheduler.class);
+    config.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
+        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
+    MockRM rm = new MockRM(config);
+    GuiceServletConfig.setInjector(Guice.createInjector(new WebServletModule(rm)));
+    rm.start();
+    return rm;
+  }
+
+  public static MockRM createMutableRM(Configuration conf) throws IOException {
+    conf.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.MEMORY_CONFIGURATION_STORE);
+    MockRM rm = createRM(new CapacitySchedulerConfiguration(conf));
+    reinitialize(rm, conf);
+    return rm;
+  }
+
+  public static void reinitialize(MockRM rm, Configuration conf) throws IOException {
+    // Need to call reinitialize as
+    // MutableCSConfigurationProvider with InMemoryConfigurationStore
+    // somehow does not load the queues properly and falls back to default config.
+    // Therefore CS will think there's only the default queue there.
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    cs.reinitialize(conf, rm.getRMContext(), true);
+  }
+}

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

@@ -0,0 +1,2935 @@
+{
+  "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.test2",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 0,
+          "absoluteMaxCapacity" : 0,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test2",
+          "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" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 0,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 0,
+              "maxAMLimitPercentage" : 10,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "configuredMinResource" : {
+                "memory" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "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
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 12288,
+            "vCores" : 12,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 12288
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 12
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "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" : "fifo",
+          "autoCreateChildQueueEnabled" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "absolute",
+          "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" : 2048,
+            "vCores" : 1,
+            "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" : 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" : 2048,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "type" : "capacitySchedulerLeafQueueInfo",
+          "queuePath" : "root.default",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 0,
+          "absoluteMaxCapacity" : 0,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "default",
+          "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" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 0,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 0,
+              "maxAMLimitPercentage" : 10,
+              "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" : 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
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "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" : 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" : 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" : "fifo",
+          "autoCreateChildQueueEnabled" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "absolute",
+          "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" : 1024,
+            "vCores" : 1,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 1024
+              }, {
+                "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" : 1024,
+            "vCores" : 1,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 1024
+              }, {
+                "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.test1",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 0,
+          "absoluteMaxCapacity" : 0,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test1",
+          "isAbsoluteResource" : true,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_1",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 0,
+              "absoluteMaxCapacity" : 0,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_1",
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 0,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 0,
+                  "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" : 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
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 1024,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 1024
+                      }, {
+                        "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" : 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" : "fifo",
+              "autoCreateChildQueueEnabled" : false,
+              "leafQueueTemplate" : { },
+              "mode" : "absolute",
+              "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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_2",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 0,
+              "absoluteMaxCapacity" : 0,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_2",
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 0,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 0,
+                  "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" : 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
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 1024,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 1024
+                      }, {
+                        "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" : 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" : "fifo",
+              "autoCreateChildQueueEnabled" : false,
+              "leafQueueTemplate" : { },
+              "mode" : "absolute",
+              "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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_3",
+              "capacity" : 75,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 0,
+              "absoluteMaxCapacity" : 0,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_3",
+              "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" : 75,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 0,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 0,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : -1,
+                  "normalizedWeight" : 0,
+                  "configuredMinResource" : {
+                    "memory" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "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" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "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
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 2048,
+                    "vCores" : 1,
+                    "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" : 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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "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" : "fifo",
+              "autoCreateChildQueueEnabled" : false,
+              "leafQueueTemplate" : { },
+              "mode" : "absolute",
+              "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" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 2048,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "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,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 0,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 0,
+              "maxAMLimitPercentage" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "configuredMinResource" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "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
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "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",
+          "autoCreateChildQueueEnabled" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "absolute",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "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" : 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" : 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
+              } ]
+            }
+          }
+        } ]
+      },
+      "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" : { }
+    }
+  }
+}

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

@@ -0,0 +1,2935 @@
+{
+  "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.test2",
+          "capacity" : 37.5,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 37.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test2",
+          "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" : 37.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 10,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "configuredMinResource" : {
+                "memory" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "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" : 6144,
+                "vCores" : 6,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 6144
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 6
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 6144,
+            "vCores" : 6,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 6144
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 6
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "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" : "static",
+          "autoCreationEligibility" : "off",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : { },
+          "autoQueueLeafTemplateProperties" : { },
+          "numActiveApplications" : 0,
+          "numPendingApplications" : 0,
+          "numContainers" : 0,
+          "maxApplications" : 3750,
+          "maxApplicationsPerUser" : 3750,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 2048,
+            "vCores" : 1,
+            "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" : 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" : 2048,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "type" : "capacitySchedulerLeafQueueInfo",
+          "queuePath" : "root.default",
+          "capacity" : 12.5,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 12.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "default",
+          "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" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 12.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 10,
+              "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" : 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" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "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" : "static",
+          "autoCreationEligibility" : "off",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : { },
+          "autoQueueLeafTemplateProperties" : { },
+          "numActiveApplications" : 0,
+          "numPendingApplications" : 0,
+          "numContainers" : 0,
+          "maxApplications" : 1250,
+          "maxApplicationsPerUser" : 1250,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 2048,
+            "vCores" : 1,
+            "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" : 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" : 2048,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "queuePath" : "root.test1",
+          "capacity" : 50,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 50,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test1",
+          "isAbsoluteResource" : true,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_2",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_2",
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "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" : 1024,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 1024
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 16384,
+                    "vCores" : 16,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 16384
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 16
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 2048,
+                    "vCores" : 1,
+                    "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" : 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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 1
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 2048,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_1",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_1",
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "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" : 1024,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 1024
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 16384,
+                    "vCores" : 16,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 16384
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 16
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 2048,
+                    "vCores" : 1,
+                    "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" : 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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 1
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 2048,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_3",
+              "capacity" : 75,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_3",
+              "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" : 75,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 37.5,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : -1,
+                  "normalizedWeight" : 0,
+                  "configuredMinResource" : {
+                    "memory" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "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" : 6144,
+                    "vCores" : 6,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 6144
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 6
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 16384,
+                    "vCores" : 16,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 16384
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 16
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 2048,
+                    "vCores" : 1,
+                    "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" : 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" : 6144,
+                "vCores" : 6,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 6144
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 6
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 3750,
+              "maxApplicationsPerUser" : 3750,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 2048,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "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" : 50,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 50,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "configuredMinResource" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : 8192,
+                "vCores" : 8,
+                "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" : 8
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 8192,
+            "vCores" : 8,
+            "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" : 8
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "absolute",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "effectiveMaxResource" : {
+            "memory" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          }
+        } ]
+      },
+      "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" : { }
+    }
+  }
+}

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

@@ -0,0 +1,2935 @@
+{
+  "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.test2",
+          "capacity" : 37.5,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 37.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test2",
+          "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" : 37.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 10,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "configuredMinResource" : {
+                "memory" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 12288,
+            "vCores" : 12,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 12288
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 12
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : "static",
+          "autoCreationEligibility" : "off",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : { },
+          "autoQueueLeafTemplateProperties" : { },
+          "numActiveApplications" : 0,
+          "numPendingApplications" : 0,
+          "numContainers" : 0,
+          "maxApplications" : 3750,
+          "maxApplicationsPerUser" : 3750,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 4096,
+            "vCores" : 1,
+            "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" : 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" : 4096,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "type" : "capacitySchedulerLeafQueueInfo",
+          "queuePath" : "root.default",
+          "capacity" : 12.5,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 12.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "default",
+          "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" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 12.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 10,
+              "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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : "static",
+          "autoCreationEligibility" : "off",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : { },
+          "autoQueueLeafTemplateProperties" : { },
+          "numActiveApplications" : 0,
+          "numPendingApplications" : 0,
+          "numContainers" : 0,
+          "maxApplications" : 1250,
+          "maxApplicationsPerUser" : 1250,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 4096,
+            "vCores" : 1,
+            "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" : 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" : 4096,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "queuePath" : "root.test1",
+          "capacity" : 50,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 50,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test1",
+          "isAbsoluteResource" : true,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_2",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_2",
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_1",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_1",
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_3",
+              "capacity" : 75,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_3",
+              "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" : 75,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 37.5,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : -1,
+                  "normalizedWeight" : 0,
+                  "configuredMinResource" : {
+                    "memory" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "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" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 3750,
+              "maxApplicationsPerUser" : 3750,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "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" : 50,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 50,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "configuredMinResource" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "absolute",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "effectiveMaxResource" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          }
+        } ]
+      },
+      "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" : { }
+    }
+  }
+}

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

@@ -0,0 +1,3292 @@
+{
+  "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.test2",
+            "capacity": 37.5,
+            "usedCapacity": 0,
+            "maxCapacity": 100,
+            "absoluteCapacity": 37.5,
+            "absoluteMaxCapacity": 100,
+            "absoluteUsedCapacity": 0,
+            "weight": -1,
+            "normalizedWeight": 0,
+            "numApplications": 0,
+            "maxParallelApps": 2147483647,
+            "queueName": "test2",
+            "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": 37.5,
+                "usedCapacity": 0,
+                "maxCapacity": 100,
+                "absoluteCapacity": 37.5,
+                "absoluteUsedCapacity": 0,
+                "absoluteMaxCapacity": 100,
+                "maxAMLimitPercentage": 10,
+                "weight": -1,
+                "normalizedWeight": 0,
+                "configuredMinResource": {
+                  "memory": 12288,
+                  "vCores": 12,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 12288
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 12
+                      }
+                    ]
+                  }
+                },
+                "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": 12288,
+                  "vCores": 12,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 12288
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 12
+                      }
+                    ]
+                  }
+                },
+                "effectiveMaxResource": {
+                  "memory": 32768,
+                  "vCores": 32,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 32768
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 32
+                      }
+                    ]
+                  }
+                }
+              }]
+            },
+            "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": 4096,
+                  "vCores": 1,
+                  "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": 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": 12288,
+              "vCores": 12,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 12288
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 12
+                  }
+                ]
+              }
+            },
+            "maxEffectiveCapacity": {
+              "memory": 32768,
+              "vCores": 32,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 32768
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 32
+                  }
+                ]
+              }
+            },
+            "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": "static",
+            "autoCreationEligibility": "off",
+            "autoQueueTemplateProperties": {},
+            "autoQueueParentTemplateProperties": {},
+            "autoQueueLeafTemplateProperties": {},
+            "numActiveApplications": 0,
+            "numPendingApplications": 0,
+            "numContainers": 0,
+            "maxApplications": 3750,
+            "maxApplicationsPerUser": 3750,
+            "userLimit": 100,
+            "users": {},
+            "userLimitFactor": 1,
+            "configuredMaxAMResourceLimit": 0.1,
+            "AMResourceLimit": {
+              "memory": 4096,
+              "vCores": 1,
+              "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": 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": 4096,
+              "vCores": 1,
+              "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": 1
+                  }
+                ]
+              }
+            },
+            "preemptionDisabled": true,
+            "intraQueuePreemptionDisabled": true,
+            "defaultPriority": 0,
+            "isAutoCreatedLeafQueue": false,
+            "maxApplicationLifetime": -1,
+            "defaultApplicationLifetime": -1
+          },
+          {
+            "type": "capacitySchedulerLeafQueueInfo",
+            "queuePath": "root.default",
+            "capacity": 12.5,
+            "usedCapacity": 0,
+            "maxCapacity": 100,
+            "absoluteCapacity": 12.5,
+            "absoluteMaxCapacity": 100,
+            "absoluteUsedCapacity": 0,
+            "weight": -1,
+            "normalizedWeight": 0,
+            "numApplications": 0,
+            "maxParallelApps": 2147483647,
+            "queueName": "default",
+            "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": 12.5,
+                "usedCapacity": 0,
+                "maxCapacity": 100,
+                "absoluteCapacity": 12.5,
+                "absoluteUsedCapacity": 0,
+                "absoluteMaxCapacity": 100,
+                "maxAMLimitPercentage": 10,
+                "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": 32768,
+                  "vCores": 32,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 32768
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 32
+                      }
+                    ]
+                  }
+                }
+              }]
+            },
+            "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": 4096,
+                  "vCores": 1,
+                  "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": 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": 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": 32768,
+              "vCores": 32,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 32768
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 32
+                  }
+                ]
+              }
+            },
+            "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": "static",
+            "autoCreationEligibility": "off",
+            "autoQueueTemplateProperties": {},
+            "autoQueueParentTemplateProperties": {},
+            "autoQueueLeafTemplateProperties": {},
+            "numActiveApplications": 0,
+            "numPendingApplications": 0,
+            "numContainers": 0,
+            "maxApplications": 1250,
+            "maxApplicationsPerUser": 1250,
+            "userLimit": 100,
+            "users": {},
+            "userLimitFactor": 1,
+            "configuredMaxAMResourceLimit": 0.1,
+            "AMResourceLimit": {
+              "memory": 4096,
+              "vCores": 1,
+              "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": 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": 4096,
+              "vCores": 1,
+              "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": 1
+                  }
+                ]
+              }
+            },
+            "preemptionDisabled": true,
+            "intraQueuePreemptionDisabled": true,
+            "defaultPriority": 0,
+            "isAutoCreatedLeafQueue": false,
+            "maxApplicationLifetime": -1,
+            "defaultApplicationLifetime": -1
+          },
+          {
+            "queuePath": "root.test1",
+            "capacity": 50,
+            "usedCapacity": 0,
+            "maxCapacity": 100,
+            "absoluteCapacity": 50,
+            "absoluteMaxCapacity": 100,
+            "absoluteUsedCapacity": 0,
+            "weight": -1,
+            "normalizedWeight": 0,
+            "numApplications": 0,
+            "maxParallelApps": 2147483647,
+            "queueName": "test1",
+            "isAbsoluteResource": true,
+            "state": "RUNNING",
+            "queues": {
+              "queue": [
+                {
+                  "type": "capacitySchedulerLeafQueueInfo",
+                  "queuePath": "root.test1.test1_2",
+                  "capacity": 12.5,
+                  "usedCapacity": 0,
+                  "maxCapacity": 100,
+                  "absoluteCapacity": 6.25,
+                  "absoluteMaxCapacity": 100,
+                  "absoluteUsedCapacity": 0,
+                  "weight": -1,
+                  "normalizedWeight": 0,
+                  "numApplications": 0,
+                  "maxParallelApps": 2147483647,
+                  "queueName": "test1_2",
+                  "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": 12.5,
+                      "usedCapacity": 0,
+                      "maxCapacity": 100,
+                      "absoluteCapacity": 6.25,
+                      "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": 32768,
+                        "vCores": 32,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 32768
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 32
+                            }
+                          ]
+                        }
+                      }
+                    }]
+                  },
+                  "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": 4096,
+                        "vCores": 1,
+                        "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": 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": 32768,
+                    "vCores": 32,
+                    "resourceInformations": {
+                      "resourceInformation": [
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "memory-mb",
+                          "resourceType": "COUNTABLE",
+                          "units": "Mi",
+                          "value": 32768
+                        },
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "vcores",
+                          "resourceType": "COUNTABLE",
+                          "units": "",
+                          "value": 32
+                        }
+                      ]
+                    }
+                  },
+                  "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": "static",
+                  "autoCreationEligibility": "off",
+                  "autoQueueTemplateProperties": {},
+                  "autoQueueParentTemplateProperties": {},
+                  "autoQueueLeafTemplateProperties": {},
+                  "numActiveApplications": 0,
+                  "numPendingApplications": 0,
+                  "numContainers": 0,
+                  "maxApplications": 625,
+                  "maxApplicationsPerUser": 625,
+                  "userLimit": 100,
+                  "users": {},
+                  "userLimitFactor": 1,
+                  "configuredMaxAMResourceLimit": 0.1,
+                  "AMResourceLimit": {
+                    "memory": 4096,
+                    "vCores": 1,
+                    "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": 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": 4096,
+                    "vCores": 1,
+                    "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": 1
+                        }
+                      ]
+                    }
+                  },
+                  "preemptionDisabled": true,
+                  "intraQueuePreemptionDisabled": true,
+                  "defaultPriority": 0,
+                  "isAutoCreatedLeafQueue": false,
+                  "maxApplicationLifetime": -1,
+                  "defaultApplicationLifetime": -1
+                },
+                {
+                  "type": "capacitySchedulerLeafQueueInfo",
+                  "queuePath": "root.test1.test1_1",
+                  "capacity": 12.5,
+                  "usedCapacity": 0,
+                  "maxCapacity": 100,
+                  "absoluteCapacity": 6.25,
+                  "absoluteMaxCapacity": 100,
+                  "absoluteUsedCapacity": 0,
+                  "weight": -1,
+                  "normalizedWeight": 0,
+                  "numApplications": 0,
+                  "maxParallelApps": 2147483647,
+                  "queueName": "test1_1",
+                  "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": 12.5,
+                      "usedCapacity": 0,
+                      "maxCapacity": 100,
+                      "absoluteCapacity": 6.25,
+                      "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": 32768,
+                        "vCores": 32,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 32768
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 32
+                            }
+                          ]
+                        }
+                      }
+                    }]
+                  },
+                  "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": 4096,
+                        "vCores": 1,
+                        "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": 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": 32768,
+                    "vCores": 32,
+                    "resourceInformations": {
+                      "resourceInformation": [
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "memory-mb",
+                          "resourceType": "COUNTABLE",
+                          "units": "Mi",
+                          "value": 32768
+                        },
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "vcores",
+                          "resourceType": "COUNTABLE",
+                          "units": "",
+                          "value": 32
+                        }
+                      ]
+                    }
+                  },
+                  "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": "static",
+                  "autoCreationEligibility": "off",
+                  "autoQueueTemplateProperties": {},
+                  "autoQueueParentTemplateProperties": {},
+                  "autoQueueLeafTemplateProperties": {},
+                  "numActiveApplications": 0,
+                  "numPendingApplications": 0,
+                  "numContainers": 0,
+                  "maxApplications": 625,
+                  "maxApplicationsPerUser": 625,
+                  "userLimit": 100,
+                  "users": {},
+                  "userLimitFactor": 1,
+                  "configuredMaxAMResourceLimit": 0.1,
+                  "AMResourceLimit": {
+                    "memory": 4096,
+                    "vCores": 1,
+                    "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": 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": 4096,
+                    "vCores": 1,
+                    "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": 1
+                        }
+                      ]
+                    }
+                  },
+                  "preemptionDisabled": true,
+                  "intraQueuePreemptionDisabled": true,
+                  "defaultPriority": 0,
+                  "isAutoCreatedLeafQueue": false,
+                  "maxApplicationLifetime": -1,
+                  "defaultApplicationLifetime": -1
+                },
+                {
+                  "type": "capacitySchedulerLeafQueueInfo",
+                  "queuePath": "root.test1.test1_3",
+                  "capacity": 75,
+                  "usedCapacity": 0,
+                  "maxCapacity": 100,
+                  "absoluteCapacity": 37.5,
+                  "absoluteMaxCapacity": 100,
+                  "absoluteUsedCapacity": 0,
+                  "weight": -1,
+                  "normalizedWeight": 0,
+                  "numApplications": 0,
+                  "maxParallelApps": 2147483647,
+                  "queueName": "test1_3",
+                  "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": 75,
+                      "usedCapacity": 0,
+                      "maxCapacity": 100,
+                      "absoluteCapacity": 37.5,
+                      "absoluteUsedCapacity": 0,
+                      "absoluteMaxCapacity": 100,
+                      "maxAMLimitPercentage": 10,
+                      "weight": -1,
+                      "normalizedWeight": 0,
+                      "configuredMinResource": {
+                        "memory": 12288,
+                        "vCores": 12,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 12288
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 12
+                            }
+                          ]
+                        }
+                      },
+                      "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": 12288,
+                        "vCores": 12,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 12288
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 12
+                            }
+                          ]
+                        }
+                      },
+                      "effectiveMaxResource": {
+                        "memory": 32768,
+                        "vCores": 32,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 32768
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 32
+                            }
+                          ]
+                        }
+                      }
+                    }]
+                  },
+                  "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": 4096,
+                        "vCores": 1,
+                        "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": 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": 12288,
+                    "vCores": 12,
+                    "resourceInformations": {
+                      "resourceInformation": [
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "memory-mb",
+                          "resourceType": "COUNTABLE",
+                          "units": "Mi",
+                          "value": 12288
+                        },
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "vcores",
+                          "resourceType": "COUNTABLE",
+                          "units": "",
+                          "value": 12
+                        }
+                      ]
+                    }
+                  },
+                  "maxEffectiveCapacity": {
+                    "memory": 32768,
+                    "vCores": 32,
+                    "resourceInformations": {
+                      "resourceInformation": [
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "memory-mb",
+                          "resourceType": "COUNTABLE",
+                          "units": "Mi",
+                          "value": 32768
+                        },
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "vcores",
+                          "resourceType": "COUNTABLE",
+                          "units": "",
+                          "value": 32
+                        }
+                      ]
+                    }
+                  },
+                  "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": "static",
+                  "autoCreationEligibility": "off",
+                  "autoQueueTemplateProperties": {},
+                  "autoQueueParentTemplateProperties": {},
+                  "autoQueueLeafTemplateProperties": {},
+                  "numActiveApplications": 0,
+                  "numPendingApplications": 0,
+                  "numContainers": 0,
+                  "maxApplications": 3750,
+                  "maxApplicationsPerUser": 3750,
+                  "userLimit": 100,
+                  "users": {},
+                  "userLimitFactor": 1,
+                  "configuredMaxAMResourceLimit": 0.1,
+                  "AMResourceLimit": {
+                    "memory": 4096,
+                    "vCores": 1,
+                    "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": 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": 4096,
+                    "vCores": 1,
+                    "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": 1
+                        }
+                      ]
+                    }
+                  },
+                  "preemptionDisabled": true,
+                  "intraQueuePreemptionDisabled": true,
+                  "defaultPriority": 0,
+                  "isAutoCreatedLeafQueue": false,
+                  "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": 50,
+                "usedCapacity": 0,
+                "maxCapacity": 100,
+                "absoluteCapacity": 50,
+                "absoluteUsedCapacity": 0,
+                "absoluteMaxCapacity": 100,
+                "maxAMLimitPercentage": 0,
+                "weight": -1,
+                "normalizedWeight": 0,
+                "configuredMinResource": {
+                  "memory": 16384,
+                  "vCores": 16,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 16384
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 16
+                      }
+                    ]
+                  }
+                },
+                "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": 16384,
+                  "vCores": 16,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 16384
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 16
+                      }
+                    ]
+                  }
+                },
+                "effectiveMaxResource": {
+                  "memory": 32768,
+                  "vCores": 32,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 32768
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 32
+                      }
+                    ]
+                  }
+                }
+              }]
+            },
+            "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": 16384,
+              "vCores": 16,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 16384
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 16
+                  }
+                ]
+              }
+            },
+            "maxEffectiveCapacity": {
+              "memory": 32768,
+              "vCores": 32,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 32768
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 32
+                  }
+                ]
+              }
+            },
+            "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": false,
+            "leafQueueTemplate": {},
+            "mode": "absolute",
+            "queueType": "parent",
+            "creationMethod": "static",
+            "autoCreationEligibility": "off",
+            "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": 32768,
+            "vCores": 32,
+            "resourceInformations": {
+              "resourceInformation": [
+                {
+                  "attributes": {},
+                  "maximumAllocation": 9223372036854775807,
+                  "minimumAllocation": 0,
+                  "name": "memory-mb",
+                  "resourceType": "COUNTABLE",
+                  "units": "Mi",
+                  "value": 32768
+                },
+                {
+                  "attributes": {},
+                  "maximumAllocation": 9223372036854775807,
+                  "minimumAllocation": 0,
+                  "name": "vcores",
+                  "resourceType": "COUNTABLE",
+                  "units": "",
+                  "value": 32
+                }
+              ]
+            }
+          },
+          "effectiveMaxResource": {
+            "memory": 32768,
+            "vCores": 32,
+            "resourceInformations": {
+              "resourceInformation": [
+                {
+                  "attributes": {},
+                  "maximumAllocation": 9223372036854775807,
+                  "minimumAllocation": 0,
+                  "name": "memory-mb",
+                  "resourceType": "COUNTABLE",
+                  "units": "Mi",
+                  "value": 32768
+                },
+                {
+                  "attributes": {},
+                  "maximumAllocation": 9223372036854775807,
+                  "minimumAllocation": 0,
+                  "name": "vcores",
+                  "resourceType": "COUNTABLE",
+                  "units": "",
+                  "value": 32
+                }
+              ]
+            }
+          }
+        }]
+      },
+      "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": {}
+    }
+  }
+}

File diff suppressed because it is too large
+ 1211 - 180
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testPercentageMode-0.json


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

@@ -0,0 +1,2935 @@
+{
+  "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.test2",
+          "capacity" : 37.5,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 37.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test2",
+          "isAbsoluteResource" : false,
+          "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" : 37.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "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" : 6144,
+                "vCores" : 6,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 6144
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 6
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 6144,
+            "vCores" : 6,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 6144
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 6
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "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" : 3750,
+          "maxApplicationsPerUser" : 3750,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 2048,
+            "vCores" : 1,
+            "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" : 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" : 2048,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "type" : "capacitySchedulerLeafQueueInfo",
+          "queuePath" : "root.default",
+          "capacity" : 12.5,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 12.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "default",
+          "isAbsoluteResource" : false,
+          "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" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 12.5,
+              "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" : 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" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "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" : 1250,
+          "maxApplicationsPerUser" : 1250,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 2048,
+            "vCores" : 1,
+            "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" : 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" : 2048,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "queuePath" : "root.test1",
+          "capacity" : 50,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 50,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test1",
+          "isAbsoluteResource" : false,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_2",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_2",
+              "isAbsoluteResource" : false,
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "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" : 1024,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 1024
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 16384,
+                    "vCores" : 16,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 16384
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 16
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 2048,
+                    "vCores" : 1,
+                    "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" : 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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 1
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 2048,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_1",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_1",
+              "isAbsoluteResource" : false,
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "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" : 1024,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 1024
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 16384,
+                    "vCores" : 16,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 16384
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 16
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 2048,
+                    "vCores" : 1,
+                    "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" : 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" : 1024,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 1024
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 1
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 2048,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_3",
+              "capacity" : 75,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_3",
+              "isAbsoluteResource" : false,
+              "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" : 75,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 37.5,
+                  "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" : 6144,
+                    "vCores" : 6,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 6144
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 6
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 16384,
+                    "vCores" : 16,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 16384
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 16
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 2048,
+                    "vCores" : 1,
+                    "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" : 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" : 6144,
+                "vCores" : 6,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 6144
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 6
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "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" : 3750,
+              "maxApplicationsPerUser" : 3750,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 2048,
+                "vCores" : 1,
+                "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" : 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" : 2048,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "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" : 50,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 50,
+              "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" : 8192,
+                "vCores" : 8,
+                "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" : 8
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 8192,
+            "vCores" : 8,
+            "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" : 8
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "percentage",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "effectiveMaxResource" : {
+            "memory" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          }
+        } ]
+      },
+      "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" : { }
+    }
+  }
+}

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

@@ -0,0 +1,2935 @@
+{
+  "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.test2",
+          "capacity" : 37.5,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 37.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test2",
+          "isAbsoluteResource" : false,
+          "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" : 37.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 12288,
+            "vCores" : 12,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 12288
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 12
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : 3750,
+          "maxApplicationsPerUser" : 3750,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 4096,
+            "vCores" : 1,
+            "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" : 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" : 4096,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "type" : "capacitySchedulerLeafQueueInfo",
+          "queuePath" : "root.default",
+          "capacity" : 12.5,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 12.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "default",
+          "isAbsoluteResource" : false,
+          "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" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 12.5,
+              "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : 1250,
+          "maxApplicationsPerUser" : 1250,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 4096,
+            "vCores" : 1,
+            "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" : 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" : 4096,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "queuePath" : "root.test1",
+          "capacity" : 50,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 50,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : -1,
+          "normalizedWeight" : 0,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test1",
+          "isAbsoluteResource" : false,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_2",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_2",
+              "isAbsoluteResource" : false,
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_1",
+              "capacity" : 12.5,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_1",
+              "isAbsoluteResource" : false,
+              "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" : 12.5,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_3",
+              "capacity" : 75,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : -1,
+              "normalizedWeight" : 0,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_3",
+              "isAbsoluteResource" : false,
+              "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" : 75,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 37.5,
+                  "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" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : 3750,
+              "maxApplicationsPerUser" : 3750,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "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" : 50,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 50,
+              "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" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "percentage",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "effectiveMaxResource" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          }
+        } ]
+      },
+      "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" : { }
+    }
+  }
+}

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

@@ -0,0 +1,3292 @@
+{
+  "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.test2",
+            "capacity": 37.5,
+            "usedCapacity": 0,
+            "maxCapacity": 100,
+            "absoluteCapacity": 37.5,
+            "absoluteMaxCapacity": 100,
+            "absoluteUsedCapacity": 0,
+            "weight": -1,
+            "normalizedWeight": 0,
+            "numApplications": 0,
+            "maxParallelApps": 2147483647,
+            "queueName": "test2",
+            "isAbsoluteResource": false,
+            "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": 37.5,
+                "usedCapacity": 0,
+                "maxCapacity": 100,
+                "absoluteCapacity": 37.5,
+                "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": 12288,
+                  "vCores": 12,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 12288
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 12
+                      }
+                    ]
+                  }
+                },
+                "effectiveMaxResource": {
+                  "memory": 32768,
+                  "vCores": 32,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 32768
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 32
+                      }
+                    ]
+                  }
+                }
+              }]
+            },
+            "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": 4096,
+                  "vCores": 1,
+                  "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": 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": 12288,
+              "vCores": 12,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 12288
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 12
+                  }
+                ]
+              }
+            },
+            "maxEffectiveCapacity": {
+              "memory": 32768,
+              "vCores": 32,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 32768
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 32
+                  }
+                ]
+              }
+            },
+            "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": 3750,
+            "maxApplicationsPerUser": 3750,
+            "userLimit": 100,
+            "users": {},
+            "userLimitFactor": 1,
+            "configuredMaxAMResourceLimit": 0.1,
+            "AMResourceLimit": {
+              "memory": 4096,
+              "vCores": 1,
+              "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": 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": 4096,
+              "vCores": 1,
+              "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": 1
+                  }
+                ]
+              }
+            },
+            "preemptionDisabled": true,
+            "intraQueuePreemptionDisabled": true,
+            "defaultPriority": 0,
+            "isAutoCreatedLeafQueue": false,
+            "maxApplicationLifetime": -1,
+            "defaultApplicationLifetime": -1
+          },
+          {
+            "type": "capacitySchedulerLeafQueueInfo",
+            "queuePath": "root.default",
+            "capacity": 12.5,
+            "usedCapacity": 0,
+            "maxCapacity": 100,
+            "absoluteCapacity": 12.5,
+            "absoluteMaxCapacity": 100,
+            "absoluteUsedCapacity": 0,
+            "weight": -1,
+            "normalizedWeight": 0,
+            "numApplications": 0,
+            "maxParallelApps": 2147483647,
+            "queueName": "default",
+            "isAbsoluteResource": false,
+            "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": 12.5,
+                "usedCapacity": 0,
+                "maxCapacity": 100,
+                "absoluteCapacity": 12.5,
+                "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": 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": 32768,
+                  "vCores": 32,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 32768
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 32
+                      }
+                    ]
+                  }
+                }
+              }]
+            },
+            "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": 4096,
+                  "vCores": 1,
+                  "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": 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": 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": 32768,
+              "vCores": 32,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 32768
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 32
+                  }
+                ]
+              }
+            },
+            "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": 1250,
+            "maxApplicationsPerUser": 1250,
+            "userLimit": 100,
+            "users": {},
+            "userLimitFactor": 1,
+            "configuredMaxAMResourceLimit": 0.1,
+            "AMResourceLimit": {
+              "memory": 4096,
+              "vCores": 1,
+              "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": 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": 4096,
+              "vCores": 1,
+              "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": 1
+                  }
+                ]
+              }
+            },
+            "preemptionDisabled": true,
+            "intraQueuePreemptionDisabled": true,
+            "defaultPriority": 0,
+            "isAutoCreatedLeafQueue": false,
+            "maxApplicationLifetime": -1,
+            "defaultApplicationLifetime": -1
+          },
+          {
+            "queuePath": "root.test1",
+            "capacity": 50,
+            "usedCapacity": 0,
+            "maxCapacity": 100,
+            "absoluteCapacity": 50,
+            "absoluteMaxCapacity": 100,
+            "absoluteUsedCapacity": 0,
+            "weight": -1,
+            "normalizedWeight": 0,
+            "numApplications": 0,
+            "maxParallelApps": 2147483647,
+            "queueName": "test1",
+            "isAbsoluteResource": false,
+            "state": "RUNNING",
+            "queues": {
+              "queue": [
+                {
+                  "type": "capacitySchedulerLeafQueueInfo",
+                  "queuePath": "root.test1.test1_2",
+                  "capacity": 12.5,
+                  "usedCapacity": 0,
+                  "maxCapacity": 100,
+                  "absoluteCapacity": 6.25,
+                  "absoluteMaxCapacity": 100,
+                  "absoluteUsedCapacity": 0,
+                  "weight": -1,
+                  "normalizedWeight": 0,
+                  "numApplications": 0,
+                  "maxParallelApps": 2147483647,
+                  "queueName": "test1_2",
+                  "isAbsoluteResource": false,
+                  "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": 12.5,
+                      "usedCapacity": 0,
+                      "maxCapacity": 100,
+                      "absoluteCapacity": 6.25,
+                      "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": 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": 32768,
+                        "vCores": 32,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 32768
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 32
+                            }
+                          ]
+                        }
+                      }
+                    }]
+                  },
+                  "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": 4096,
+                        "vCores": 1,
+                        "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": 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": 32768,
+                    "vCores": 32,
+                    "resourceInformations": {
+                      "resourceInformation": [
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "memory-mb",
+                          "resourceType": "COUNTABLE",
+                          "units": "Mi",
+                          "value": 32768
+                        },
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "vcores",
+                          "resourceType": "COUNTABLE",
+                          "units": "",
+                          "value": 32
+                        }
+                      ]
+                    }
+                  },
+                  "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": 625,
+                  "maxApplicationsPerUser": 625,
+                  "userLimit": 100,
+                  "users": {},
+                  "userLimitFactor": 1,
+                  "configuredMaxAMResourceLimit": 0.1,
+                  "AMResourceLimit": {
+                    "memory": 4096,
+                    "vCores": 1,
+                    "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": 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": 4096,
+                    "vCores": 1,
+                    "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": 1
+                        }
+                      ]
+                    }
+                  },
+                  "preemptionDisabled": true,
+                  "intraQueuePreemptionDisabled": true,
+                  "defaultPriority": 0,
+                  "isAutoCreatedLeafQueue": false,
+                  "maxApplicationLifetime": -1,
+                  "defaultApplicationLifetime": -1
+                },
+                {
+                  "type": "capacitySchedulerLeafQueueInfo",
+                  "queuePath": "root.test1.test1_1",
+                  "capacity": 12.5,
+                  "usedCapacity": 0,
+                  "maxCapacity": 100,
+                  "absoluteCapacity": 6.25,
+                  "absoluteMaxCapacity": 100,
+                  "absoluteUsedCapacity": 0,
+                  "weight": -1,
+                  "normalizedWeight": 0,
+                  "numApplications": 0,
+                  "maxParallelApps": 2147483647,
+                  "queueName": "test1_1",
+                  "isAbsoluteResource": false,
+                  "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": 12.5,
+                      "usedCapacity": 0,
+                      "maxCapacity": 100,
+                      "absoluteCapacity": 6.25,
+                      "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": 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": 32768,
+                        "vCores": 32,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 32768
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 32
+                            }
+                          ]
+                        }
+                      }
+                    }]
+                  },
+                  "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": 4096,
+                        "vCores": 1,
+                        "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": 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": 32768,
+                    "vCores": 32,
+                    "resourceInformations": {
+                      "resourceInformation": [
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "memory-mb",
+                          "resourceType": "COUNTABLE",
+                          "units": "Mi",
+                          "value": 32768
+                        },
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "vcores",
+                          "resourceType": "COUNTABLE",
+                          "units": "",
+                          "value": 32
+                        }
+                      ]
+                    }
+                  },
+                  "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": 625,
+                  "maxApplicationsPerUser": 625,
+                  "userLimit": 100,
+                  "users": {},
+                  "userLimitFactor": 1,
+                  "configuredMaxAMResourceLimit": 0.1,
+                  "AMResourceLimit": {
+                    "memory": 4096,
+                    "vCores": 1,
+                    "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": 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": 4096,
+                    "vCores": 1,
+                    "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": 1
+                        }
+                      ]
+                    }
+                  },
+                  "preemptionDisabled": true,
+                  "intraQueuePreemptionDisabled": true,
+                  "defaultPriority": 0,
+                  "isAutoCreatedLeafQueue": false,
+                  "maxApplicationLifetime": -1,
+                  "defaultApplicationLifetime": -1
+                },
+                {
+                  "type": "capacitySchedulerLeafQueueInfo",
+                  "queuePath": "root.test1.test1_3",
+                  "capacity": 75,
+                  "usedCapacity": 0,
+                  "maxCapacity": 100,
+                  "absoluteCapacity": 37.5,
+                  "absoluteMaxCapacity": 100,
+                  "absoluteUsedCapacity": 0,
+                  "weight": -1,
+                  "normalizedWeight": 0,
+                  "numApplications": 0,
+                  "maxParallelApps": 2147483647,
+                  "queueName": "test1_3",
+                  "isAbsoluteResource": false,
+                  "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": 75,
+                      "usedCapacity": 0,
+                      "maxCapacity": 100,
+                      "absoluteCapacity": 37.5,
+                      "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": 12288,
+                        "vCores": 12,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 12288
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 12
+                            }
+                          ]
+                        }
+                      },
+                      "effectiveMaxResource": {
+                        "memory": 32768,
+                        "vCores": 32,
+                        "resourceInformations": {
+                          "resourceInformation": [
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "memory-mb",
+                              "resourceType": "COUNTABLE",
+                              "units": "Mi",
+                              "value": 32768
+                            },
+                            {
+                              "attributes": {},
+                              "maximumAllocation": 9223372036854775807,
+                              "minimumAllocation": 0,
+                              "name": "vcores",
+                              "resourceType": "COUNTABLE",
+                              "units": "",
+                              "value": 32
+                            }
+                          ]
+                        }
+                      }
+                    }]
+                  },
+                  "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": 4096,
+                        "vCores": 1,
+                        "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": 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": 12288,
+                    "vCores": 12,
+                    "resourceInformations": {
+                      "resourceInformation": [
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "memory-mb",
+                          "resourceType": "COUNTABLE",
+                          "units": "Mi",
+                          "value": 12288
+                        },
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "vcores",
+                          "resourceType": "COUNTABLE",
+                          "units": "",
+                          "value": 12
+                        }
+                      ]
+                    }
+                  },
+                  "maxEffectiveCapacity": {
+                    "memory": 32768,
+                    "vCores": 32,
+                    "resourceInformations": {
+                      "resourceInformation": [
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "memory-mb",
+                          "resourceType": "COUNTABLE",
+                          "units": "Mi",
+                          "value": 32768
+                        },
+                        {
+                          "attributes": {},
+                          "maximumAllocation": 9223372036854775807,
+                          "minimumAllocation": 0,
+                          "name": "vcores",
+                          "resourceType": "COUNTABLE",
+                          "units": "",
+                          "value": 32
+                        }
+                      ]
+                    }
+                  },
+                  "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": 3750,
+                  "maxApplicationsPerUser": 3750,
+                  "userLimit": 100,
+                  "users": {},
+                  "userLimitFactor": 1,
+                  "configuredMaxAMResourceLimit": 0.1,
+                  "AMResourceLimit": {
+                    "memory": 4096,
+                    "vCores": 1,
+                    "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": 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": 4096,
+                    "vCores": 1,
+                    "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": 1
+                        }
+                      ]
+                    }
+                  },
+                  "preemptionDisabled": true,
+                  "intraQueuePreemptionDisabled": true,
+                  "defaultPriority": 0,
+                  "isAutoCreatedLeafQueue": false,
+                  "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": 50,
+                "usedCapacity": 0,
+                "maxCapacity": 100,
+                "absoluteCapacity": 50,
+                "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": 16384,
+                  "vCores": 16,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 16384
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 16
+                      }
+                    ]
+                  }
+                },
+                "effectiveMaxResource": {
+                  "memory": 32768,
+                  "vCores": 32,
+                  "resourceInformations": {
+                    "resourceInformation": [
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "memory-mb",
+                        "resourceType": "COUNTABLE",
+                        "units": "Mi",
+                        "value": 32768
+                      },
+                      {
+                        "attributes": {},
+                        "maximumAllocation": 9223372036854775807,
+                        "minimumAllocation": 0,
+                        "name": "vcores",
+                        "resourceType": "COUNTABLE",
+                        "units": "",
+                        "value": 32
+                      }
+                    ]
+                  }
+                }
+              }]
+            },
+            "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": 16384,
+              "vCores": 16,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 16384
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 16
+                  }
+                ]
+              }
+            },
+            "maxEffectiveCapacity": {
+              "memory": 32768,
+              "vCores": 32,
+              "resourceInformations": {
+                "resourceInformation": [
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "memory-mb",
+                    "resourceType": "COUNTABLE",
+                    "units": "Mi",
+                    "value": 32768
+                  },
+                  {
+                    "attributes": {},
+                    "maximumAllocation": 9223372036854775807,
+                    "minimumAllocation": 0,
+                    "name": "vcores",
+                    "resourceType": "COUNTABLE",
+                    "units": "",
+                    "value": 32
+                  }
+                ]
+              }
+            },
+            "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": false,
+            "leafQueueTemplate": {},
+            "mode": "percentage",
+            "queueType": "parent",
+            "creationMethod": "static",
+            "autoCreationEligibility": "off",
+            "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": 32768,
+            "vCores": 32,
+            "resourceInformations": {
+              "resourceInformation": [
+                {
+                  "attributes": {},
+                  "maximumAllocation": 9223372036854775807,
+                  "minimumAllocation": 0,
+                  "name": "memory-mb",
+                  "resourceType": "COUNTABLE",
+                  "units": "Mi",
+                  "value": 32768
+                },
+                {
+                  "attributes": {},
+                  "maximumAllocation": 9223372036854775807,
+                  "minimumAllocation": 0,
+                  "name": "vcores",
+                  "resourceType": "COUNTABLE",
+                  "units": "",
+                  "value": 32
+                }
+              ]
+            }
+          },
+          "effectiveMaxResource": {
+            "memory": 32768,
+            "vCores": 32,
+            "resourceInformations": {
+              "resourceInformation": [
+                {
+                  "attributes": {},
+                  "maximumAllocation": 9223372036854775807,
+                  "minimumAllocation": 0,
+                  "name": "memory-mb",
+                  "resourceType": "COUNTABLE",
+                  "units": "Mi",
+                  "value": 32768
+                },
+                {
+                  "attributes": {},
+                  "maximumAllocation": 9223372036854775807,
+                  "minimumAllocation": 0,
+                  "name": "vcores",
+                  "resourceType": "COUNTABLE",
+                  "units": "",
+                  "value": 32
+                }
+              ]
+            }
+          }
+        }]
+      },
+      "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": {}
+    }
+  }
+}

File diff suppressed because it is too large
+ 766 - 321
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-0.json


File diff suppressed because it is too large
+ 465 - 140
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/dynamic-testWeightMode-16.json


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

@@ -0,0 +1,2935 @@
+{
+  "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.test2",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 37.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 12,
+          "normalizedWeight" : 0.375,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test2",
+          "isAbsoluteResource" : false,
+          "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" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 10,
+              "weight" : 12,
+              "normalizedWeight" : 0.375,
+              "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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 12288,
+            "vCores" : 12,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 12288
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 12
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : "weight",
+          "queueType" : "leaf",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : { },
+          "autoQueueLeafTemplateProperties" : { },
+          "numActiveApplications" : 0,
+          "numPendingApplications" : 0,
+          "numContainers" : 0,
+          "maxApplications" : 3750,
+          "maxApplicationsPerUser" : 3750,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 4096,
+            "vCores" : 1,
+            "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" : 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" : 4096,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "type" : "capacitySchedulerLeafQueueInfo",
+          "queuePath" : "root.default",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 12.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 4,
+          "normalizedWeight" : 0.125,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "default",
+          "isAbsoluteResource" : false,
+          "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" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 12.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 10,
+              "weight" : 4,
+              "normalizedWeight" : 0.125,
+              "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : "weight",
+          "queueType" : "leaf",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : { },
+          "autoQueueLeafTemplateProperties" : { },
+          "numActiveApplications" : 0,
+          "numPendingApplications" : 0,
+          "numContainers" : 0,
+          "maxApplications" : 1250,
+          "maxApplicationsPerUser" : 1250,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 4096,
+            "vCores" : 1,
+            "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" : 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" : 4096,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "queuePath" : "root.test1",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 50,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 16,
+          "normalizedWeight" : 0.5,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test1",
+          "isAbsoluteResource" : false,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_2",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 2,
+              "normalizedWeight" : 0.125,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_2",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 2,
+                  "normalizedWeight" : 0.125,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_1",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 2,
+              "normalizedWeight" : 0.125,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_1",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 2,
+                  "normalizedWeight" : 0.125,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_3",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 12,
+              "normalizedWeight" : 0.75,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_3",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 37.5,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 12,
+                  "normalizedWeight" : 0.75,
+                  "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" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 3750,
+              "maxApplicationsPerUser" : 3750,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "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,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 50,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 0,
+              "weight" : 16,
+              "normalizedWeight" : 0.5,
+              "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" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "weight",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "effectiveMaxResource" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          }
+        } ]
+      },
+      "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" : { }
+    }
+  }
+}

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

@@ -0,0 +1,7710 @@
+{
+  "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" : 12.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 4,
+          "normalizedWeight" : 0.125,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "default",
+          "isAbsoluteResource" : false,
+          "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" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 12.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 10,
+              "weight" : 4,
+              "normalizedWeight" : 0.125,
+              "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : "weight",
+          "queueType" : "leaf",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : { },
+          "autoQueueLeafTemplateProperties" : { },
+          "numActiveApplications" : 0,
+          "numPendingApplications" : 0,
+          "numContainers" : 0,
+          "maxApplications" : 1250,
+          "maxApplicationsPerUser" : 1250,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 4096,
+            "vCores" : 1,
+            "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" : 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" : 4096,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "queuePath" : "root.test2",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 37.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 12,
+          "normalizedWeight" : 0.375,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test2",
+          "isAbsoluteResource" : false,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test2.auto1",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 1,
+              "normalizedWeight" : 0.16666667,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "auto1",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 100,
+                  "weight" : 1,
+                  "normalizedWeight" : 0.16666667,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "dynamicFlexible",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : -1,
+              "configuredMaxAMResourceLimit" : 1,
+              "AMResourceLimit" : {
+                "memory" : 32768,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "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" : 32768,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test2.auto2",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 1,
+              "normalizedWeight" : 0.16666667,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "auto2",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 100,
+                  "weight" : 1,
+                  "normalizedWeight" : 0.16666667,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "dynamicFlexible",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : -1,
+              "configuredMaxAMResourceLimit" : 1,
+              "AMResourceLimit" : {
+                "memory" : 32768,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "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" : 32768,
+                "vCores" : 1,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "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.test2.autoParent1",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 1,
+              "normalizedWeight" : 0.16666667,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "autoParent1",
+              "isAbsoluteResource" : false,
+              "state" : "RUNNING",
+              "queues" : {
+                "queue" : [ {
+                  "type" : "capacitySchedulerLeafQueueInfo",
+                  "queuePath" : "root.test2.autoParent1.auto4",
+                  "capacity" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 3.125,
+                  "absoluteMaxCapacity" : 100,
+                  "absoluteUsedCapacity" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 0.5,
+                  "numApplications" : 0,
+                  "maxParallelApps" : 2147483647,
+                  "queueName" : "auto4",
+                  "isAbsoluteResource" : false,
+                  "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" : 0,
+                      "usedCapacity" : 0,
+                      "maxCapacity" : 100,
+                      "absoluteCapacity" : 3.125,
+                      "absoluteUsedCapacity" : 0,
+                      "absoluteMaxCapacity" : 100,
+                      "maxAMLimitPercentage" : 100,
+                      "weight" : 1,
+                      "normalizedWeight" : 0.5,
+                      "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" : 1024,
+                        "vCores" : 1,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 1024
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 1
+                          } ]
+                        }
+                      },
+                      "effectiveMaxResource" : {
+                        "memory" : 32768,
+                        "vCores" : 32,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 32
+                          } ]
+                        }
+                      }
+                    } ]
+                  },
+                  "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" : 32768,
+                        "vCores" : 1,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "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" : 1024,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 1024
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "maxEffectiveCapacity" : {
+                    "memory" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  },
+                  "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" : "ap1Admin "
+                    }, {
+                      "accessType" : "APPLICATION_MAX_PRIORITY",
+                      "accessControlList" : "*"
+                    }, {
+                      "accessType" : "SUBMIT_APP",
+                      "accessControlList" : "ap1User "
+                    } ]
+                  },
+                  "queuePriority" : 0,
+                  "orderingPolicyInfo" : "fifo",
+                  "autoCreateChildQueueEnabled" : false,
+                  "leafQueueTemplate" : { },
+                  "mode" : "weight",
+                  "queueType" : "leaf",
+                  "creationMethod" : "dynamicFlexible",
+                  "autoCreationEligibility" : "off",
+                  "autoQueueTemplateProperties" : { },
+                  "autoQueueParentTemplateProperties" : { },
+                  "autoQueueLeafTemplateProperties" : { },
+                  "numActiveApplications" : 0,
+                  "numPendingApplications" : 0,
+                  "numContainers" : 0,
+                  "maxApplications" : 300,
+                  "maxApplicationsPerUser" : 300,
+                  "userLimit" : 100,
+                  "users" : { },
+                  "userLimitFactor" : -1,
+                  "configuredMaxAMResourceLimit" : 1,
+                  "AMResourceLimit" : {
+                    "memory" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "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" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "preemptionDisabled" : true,
+                  "intraQueuePreemptionDisabled" : true,
+                  "defaultPriority" : 0,
+                  "isAutoCreatedLeafQueue" : false,
+                  "maxApplicationLifetime" : -1,
+                  "defaultApplicationLifetime" : -1
+                }, {
+                  "type" : "capacitySchedulerLeafQueueInfo",
+                  "queuePath" : "root.test2.autoParent1.auto3",
+                  "capacity" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 3.125,
+                  "absoluteMaxCapacity" : 100,
+                  "absoluteUsedCapacity" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 0.5,
+                  "numApplications" : 0,
+                  "maxParallelApps" : 2147483647,
+                  "queueName" : "auto3",
+                  "isAbsoluteResource" : false,
+                  "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" : 0,
+                      "usedCapacity" : 0,
+                      "maxCapacity" : 100,
+                      "absoluteCapacity" : 3.125,
+                      "absoluteUsedCapacity" : 0,
+                      "absoluteMaxCapacity" : 100,
+                      "maxAMLimitPercentage" : 100,
+                      "weight" : 1,
+                      "normalizedWeight" : 0.5,
+                      "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" : 1024,
+                        "vCores" : 1,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 1024
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 1
+                          } ]
+                        }
+                      },
+                      "effectiveMaxResource" : {
+                        "memory" : 32768,
+                        "vCores" : 32,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 32
+                          } ]
+                        }
+                      }
+                    } ]
+                  },
+                  "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" : 32768,
+                        "vCores" : 1,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "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" : 1024,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 1024
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "maxEffectiveCapacity" : {
+                    "memory" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  },
+                  "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" : "ap1Admin "
+                    }, {
+                      "accessType" : "APPLICATION_MAX_PRIORITY",
+                      "accessControlList" : "*"
+                    }, {
+                      "accessType" : "SUBMIT_APP",
+                      "accessControlList" : "ap1User "
+                    } ]
+                  },
+                  "queuePriority" : 0,
+                  "orderingPolicyInfo" : "fifo",
+                  "autoCreateChildQueueEnabled" : false,
+                  "leafQueueTemplate" : { },
+                  "mode" : "weight",
+                  "queueType" : "leaf",
+                  "creationMethod" : "dynamicFlexible",
+                  "autoCreationEligibility" : "off",
+                  "autoQueueTemplateProperties" : { },
+                  "autoQueueParentTemplateProperties" : { },
+                  "autoQueueLeafTemplateProperties" : { },
+                  "numActiveApplications" : 0,
+                  "numPendingApplications" : 0,
+                  "numContainers" : 0,
+                  "maxApplications" : 300,
+                  "maxApplicationsPerUser" : 300,
+                  "userLimit" : 100,
+                  "users" : { },
+                  "userLimitFactor" : -1,
+                  "configuredMaxAMResourceLimit" : 1,
+                  "AMResourceLimit" : {
+                    "memory" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "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" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "preemptionDisabled" : true,
+                  "intraQueuePreemptionDisabled" : true,
+                  "defaultPriority" : 0,
+                  "isAutoCreatedLeafQueue" : false,
+                  "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,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 0.16666667,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "parentAdmin "
+                }, {
+                  "accessType" : "APPLICATION_MAX_PRIORITY",
+                  "accessControlList" : "*"
+                }, {
+                  "accessType" : "SUBMIT_APP",
+                  "accessControlList" : "parentUser "
+                } ]
+              },
+              "queuePriority" : 0,
+              "orderingPolicyInfo" : "utilization",
+              "autoCreateChildQueueEnabled" : false,
+              "leafQueueTemplate" : { },
+              "mode" : "weight",
+              "queueType" : "parent",
+              "creationMethod" : "dynamicFlexible",
+              "autoCreationEligibility" : "flexible",
+              "autoQueueTemplateProperties" : {
+                "property" : [ {
+                  "name" : "maximum-applications",
+                  "value" : "300"
+                } ]
+              },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : {
+                "property" : [ {
+                  "name" : "acl_administer_queue",
+                  "value" : "ap1Admin"
+                }, {
+                  "name" : "acl_submit_applications",
+                  "value" : "ap1User"
+                } ]
+              }
+            }, {
+              "queuePath" : "root.test2.autoParent2",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 1,
+              "normalizedWeight" : 0.16666667,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "autoParent2",
+              "isAbsoluteResource" : false,
+              "state" : "RUNNING",
+              "queues" : {
+                "queue" : [ {
+                  "type" : "capacitySchedulerLeafQueueInfo",
+                  "queuePath" : "root.test2.autoParent2.auto5",
+                  "capacity" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteMaxCapacity" : 100,
+                  "absoluteUsedCapacity" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 1,
+                  "numApplications" : 0,
+                  "maxParallelApps" : 2147483647,
+                  "queueName" : "auto5",
+                  "isAbsoluteResource" : false,
+                  "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" : 0,
+                      "usedCapacity" : 0,
+                      "maxCapacity" : 100,
+                      "absoluteCapacity" : 6.25,
+                      "absoluteUsedCapacity" : 0,
+                      "absoluteMaxCapacity" : 100,
+                      "maxAMLimitPercentage" : 100,
+                      "weight" : 1,
+                      "normalizedWeight" : 1,
+                      "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" : 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" : 32768,
+                        "vCores" : 32,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 32
+                          } ]
+                        }
+                      }
+                    } ]
+                  },
+                  "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" : 32768,
+                        "vCores" : 1,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  },
+                  "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" : "leafAdmin "
+                    }, {
+                      "accessType" : "APPLICATION_MAX_PRIORITY",
+                      "accessControlList" : "*"
+                    }, {
+                      "accessType" : "SUBMIT_APP",
+                      "accessControlList" : "leafUser "
+                    } ]
+                  },
+                  "queuePriority" : 0,
+                  "orderingPolicyInfo" : "fifo",
+                  "autoCreateChildQueueEnabled" : false,
+                  "leafQueueTemplate" : { },
+                  "mode" : "weight",
+                  "queueType" : "leaf",
+                  "creationMethod" : "dynamicFlexible",
+                  "autoCreationEligibility" : "off",
+                  "autoQueueTemplateProperties" : { },
+                  "autoQueueParentTemplateProperties" : { },
+                  "autoQueueLeafTemplateProperties" : { },
+                  "numActiveApplications" : 0,
+                  "numPendingApplications" : 0,
+                  "numContainers" : 0,
+                  "maxApplications" : 625,
+                  "maxApplicationsPerUser" : 625,
+                  "userLimit" : 100,
+                  "users" : { },
+                  "userLimitFactor" : -1,
+                  "configuredMaxAMResourceLimit" : 1,
+                  "AMResourceLimit" : {
+                    "memory" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "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" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "preemptionDisabled" : true,
+                  "intraQueuePreemptionDisabled" : true,
+                  "defaultPriority" : 0,
+                  "isAutoCreatedLeafQueue" : false,
+                  "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,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 0.16666667,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "parentAdmin "
+                }, {
+                  "accessType" : "APPLICATION_MAX_PRIORITY",
+                  "accessControlList" : "*"
+                }, {
+                  "accessType" : "SUBMIT_APP",
+                  "accessControlList" : "parentUser "
+                } ]
+              },
+              "queuePriority" : 0,
+              "orderingPolicyInfo" : "utilization",
+              "autoCreateChildQueueEnabled" : false,
+              "leafQueueTemplate" : { },
+              "mode" : "weight",
+              "queueType" : "parent",
+              "creationMethod" : "dynamicFlexible",
+              "autoCreationEligibility" : "flexible",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : {
+                "property" : [ {
+                  "name" : "acl_administer_queue",
+                  "value" : "leafAdmin"
+                }, {
+                  "name" : "acl_submit_applications",
+                  "value" : "leafUser"
+                } ]
+              }
+            }, {
+              "queuePath" : "root.test2.parent2",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 1,
+              "normalizedWeight" : 0.16666667,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "parent2",
+              "isAbsoluteResource" : false,
+              "state" : "RUNNING",
+              "queues" : {
+                "queue" : [ {
+                  "type" : "capacitySchedulerLeafQueueInfo",
+                  "queuePath" : "root.test2.parent2.auto7",
+                  "capacity" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteMaxCapacity" : 100,
+                  "absoluteUsedCapacity" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 1,
+                  "numApplications" : 0,
+                  "maxParallelApps" : 2147483647,
+                  "queueName" : "auto7",
+                  "isAbsoluteResource" : false,
+                  "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" : 0,
+                      "usedCapacity" : 0,
+                      "maxCapacity" : 100,
+                      "absoluteCapacity" : 6.25,
+                      "absoluteUsedCapacity" : 0,
+                      "absoluteMaxCapacity" : 100,
+                      "maxAMLimitPercentage" : 100,
+                      "weight" : 1,
+                      "normalizedWeight" : 1,
+                      "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" : 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" : 32768,
+                        "vCores" : 32,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 32
+                          } ]
+                        }
+                      }
+                    } ]
+                  },
+                  "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" : 32768,
+                        "vCores" : 1,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  },
+                  "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" : "leafAdmin "
+                    }, {
+                      "accessType" : "APPLICATION_MAX_PRIORITY",
+                      "accessControlList" : "*"
+                    }, {
+                      "accessType" : "SUBMIT_APP",
+                      "accessControlList" : "leafUser "
+                    } ]
+                  },
+                  "queuePriority" : 0,
+                  "orderingPolicyInfo" : "fifo",
+                  "autoCreateChildQueueEnabled" : false,
+                  "leafQueueTemplate" : { },
+                  "mode" : "weight",
+                  "queueType" : "leaf",
+                  "creationMethod" : "dynamicFlexible",
+                  "autoCreationEligibility" : "off",
+                  "autoQueueTemplateProperties" : { },
+                  "autoQueueParentTemplateProperties" : { },
+                  "autoQueueLeafTemplateProperties" : { },
+                  "numActiveApplications" : 0,
+                  "numPendingApplications" : 0,
+                  "numContainers" : 0,
+                  "maxApplications" : 625,
+                  "maxApplicationsPerUser" : 625,
+                  "userLimit" : 100,
+                  "users" : { },
+                  "userLimitFactor" : -1,
+                  "configuredMaxAMResourceLimit" : 1,
+                  "AMResourceLimit" : {
+                    "memory" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "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" : 32768,
+                    "vCores" : 1,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 1
+                      } ]
+                    }
+                  },
+                  "preemptionDisabled" : true,
+                  "intraQueuePreemptionDisabled" : true,
+                  "defaultPriority" : 0,
+                  "isAutoCreatedLeafQueue" : false,
+                  "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,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 0.16666667,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "parentAdmin "
+                }, {
+                  "accessType" : "APPLICATION_MAX_PRIORITY",
+                  "accessControlList" : "*"
+                }, {
+                  "accessType" : "SUBMIT_APP",
+                  "accessControlList" : "parentUser "
+                } ]
+              },
+              "queuePriority" : 0,
+              "orderingPolicyInfo" : "utilization",
+              "autoCreateChildQueueEnabled" : false,
+              "leafQueueTemplate" : { },
+              "mode" : "weight",
+              "queueType" : "parent",
+              "creationMethod" : "dynamicFlexible",
+              "autoCreationEligibility" : "flexible",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : {
+                "property" : [ {
+                  "name" : "acl_administer_queue",
+                  "value" : "leafAdmin"
+                }, {
+                  "name" : "acl_submit_applications",
+                  "value" : "leafUser"
+                } ]
+              }
+            }, {
+              "queuePath" : "root.test2.parent",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 1,
+              "normalizedWeight" : 0.16666667,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "parent",
+              "isAbsoluteResource" : false,
+              "state" : "RUNNING",
+              "queues" : {
+                "queue" : [ {
+                  "queuePath" : "root.test2.parent.autoParent2",
+                  "capacity" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteMaxCapacity" : 100,
+                  "absoluteUsedCapacity" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 1,
+                  "numApplications" : 0,
+                  "maxParallelApps" : 2147483647,
+                  "queueName" : "autoParent2",
+                  "isAbsoluteResource" : false,
+                  "state" : "RUNNING",
+                  "queues" : {
+                    "queue" : [ {
+                      "type" : "capacitySchedulerLeafQueueInfo",
+                      "queuePath" : "root.test2.parent.autoParent2.auto6",
+                      "capacity" : 0,
+                      "usedCapacity" : 0,
+                      "maxCapacity" : 100,
+                      "absoluteCapacity" : 6.25,
+                      "absoluteMaxCapacity" : 100,
+                      "absoluteUsedCapacity" : 0,
+                      "weight" : 1,
+                      "normalizedWeight" : 1,
+                      "numApplications" : 0,
+                      "maxParallelApps" : 2147483647,
+                      "queueName" : "auto6",
+                      "isAbsoluteResource" : false,
+                      "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" : 0,
+                          "usedCapacity" : 0,
+                          "maxCapacity" : 100,
+                          "absoluteCapacity" : 6.25,
+                          "absoluteUsedCapacity" : 0,
+                          "absoluteMaxCapacity" : 100,
+                          "maxAMLimitPercentage" : 100,
+                          "weight" : 1,
+                          "normalizedWeight" : 1,
+                          "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" : 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" : 32768,
+                            "vCores" : 32,
+                            "resourceInformations" : {
+                              "resourceInformation" : [ {
+                                "attributes" : { },
+                                "maximumAllocation" : 9223372036854775807,
+                                "minimumAllocation" : 0,
+                                "name" : "memory-mb",
+                                "resourceType" : "COUNTABLE",
+                                "units" : "Mi",
+                                "value" : 32768
+                              }, {
+                                "attributes" : { },
+                                "maximumAllocation" : 9223372036854775807,
+                                "minimumAllocation" : 0,
+                                "name" : "vcores",
+                                "resourceType" : "COUNTABLE",
+                                "units" : "",
+                                "value" : 32
+                              } ]
+                            }
+                          }
+                        } ]
+                      },
+                      "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" : 32768,
+                            "vCores" : 1,
+                            "resourceInformations" : {
+                              "resourceInformation" : [ {
+                                "attributes" : { },
+                                "maximumAllocation" : 9223372036854775807,
+                                "minimumAllocation" : 0,
+                                "name" : "memory-mb",
+                                "resourceType" : "COUNTABLE",
+                                "units" : "Mi",
+                                "value" : 32768
+                              }, {
+                                "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" : 32768,
+                        "vCores" : 32,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 32
+                          } ]
+                        }
+                      },
+                      "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" : "pLeafAdmin "
+                        }, {
+                          "accessType" : "APPLICATION_MAX_PRIORITY",
+                          "accessControlList" : "*"
+                        }, {
+                          "accessType" : "SUBMIT_APP",
+                          "accessControlList" : "pLeafUser "
+                        } ]
+                      },
+                      "queuePriority" : 0,
+                      "orderingPolicyInfo" : "fifo",
+                      "autoCreateChildQueueEnabled" : false,
+                      "leafQueueTemplate" : { },
+                      "mode" : "weight",
+                      "queueType" : "leaf",
+                      "creationMethod" : "dynamicFlexible",
+                      "autoCreationEligibility" : "off",
+                      "autoQueueTemplateProperties" : { },
+                      "autoQueueParentTemplateProperties" : { },
+                      "autoQueueLeafTemplateProperties" : { },
+                      "numActiveApplications" : 0,
+                      "numPendingApplications" : 0,
+                      "numContainers" : 0,
+                      "maxApplications" : 625,
+                      "maxApplicationsPerUser" : 625,
+                      "userLimit" : 100,
+                      "users" : { },
+                      "userLimitFactor" : -1,
+                      "configuredMaxAMResourceLimit" : 1,
+                      "AMResourceLimit" : {
+                        "memory" : 32768,
+                        "vCores" : 1,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "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" : 32768,
+                        "vCores" : 1,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 1
+                          } ]
+                        }
+                      },
+                      "preemptionDisabled" : true,
+                      "intraQueuePreemptionDisabled" : true,
+                      "defaultPriority" : 0,
+                      "isAutoCreatedLeafQueue" : false,
+                      "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,
+                      "usedCapacity" : 0,
+                      "maxCapacity" : 100,
+                      "absoluteCapacity" : 6.25,
+                      "absoluteUsedCapacity" : 0,
+                      "absoluteMaxCapacity" : 100,
+                      "maxAMLimitPercentage" : 0,
+                      "weight" : 1,
+                      "normalizedWeight" : 1,
+                      "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" : 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" : 32768,
+                        "vCores" : 32,
+                        "resourceInformations" : {
+                          "resourceInformation" : [ {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "memory-mb",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "Mi",
+                            "value" : 32768
+                          }, {
+                            "attributes" : { },
+                            "maximumAllocation" : 9223372036854775807,
+                            "minimumAllocation" : 0,
+                            "name" : "vcores",
+                            "resourceType" : "COUNTABLE",
+                            "units" : "",
+                            "value" : 32
+                          } ]
+                        }
+                      }
+                    } ]
+                  },
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  },
+                  "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" : false,
+                  "leafQueueTemplate" : { },
+                  "mode" : "weight",
+                  "queueType" : "parent",
+                  "creationMethod" : "dynamicFlexible",
+                  "autoCreationEligibility" : "flexible",
+                  "autoQueueTemplateProperties" : { },
+                  "autoQueueParentTemplateProperties" : { },
+                  "autoQueueLeafTemplateProperties" : {
+                    "property" : [ {
+                      "name" : "acl_administer_queue",
+                      "value" : "pLeafAdmin"
+                    }, {
+                      "name" : "acl_submit_applications",
+                      "value" : "pLeafUser"
+                    } ]
+                  }
+                } ]
+              },
+              "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" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 0,
+                  "weight" : 1,
+                  "normalizedWeight" : 0.16666667,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "parentAdmin "
+                }, {
+                  "accessType" : "APPLICATION_MAX_PRIORITY",
+                  "accessControlList" : "*"
+                }, {
+                  "accessType" : "SUBMIT_APP",
+                  "accessControlList" : "parentUser "
+                } ]
+              },
+              "queuePriority" : 0,
+              "orderingPolicyInfo" : "utilization",
+              "autoCreateChildQueueEnabled" : false,
+              "leafQueueTemplate" : { },
+              "mode" : "weight",
+              "queueType" : "parent",
+              "creationMethod" : "dynamicFlexible",
+              "autoCreationEligibility" : "flexible",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : {
+                "property" : [ {
+                  "name" : "acl_administer_queue",
+                  "value" : "leafAdmin"
+                }, {
+                  "name" : "acl_submit_applications",
+                  "value" : "leafUser"
+                } ]
+              }
+            } ]
+          },
+          "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" : 37.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 0,
+              "weight" : 12,
+              "normalizedWeight" : 0.375,
+              "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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 12288,
+            "vCores" : 12,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 12288
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 12
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "weight",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "flexible",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : {
+            "property" : [ {
+              "name" : "acl_administer_queue",
+              "value" : "parentAdmin"
+            }, {
+              "name" : "acl_submit_applications",
+              "value" : "parentUser"
+            } ]
+          },
+          "autoQueueLeafTemplateProperties" : { }
+        }, {
+          "queuePath" : "root.test1",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 50,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 16,
+          "normalizedWeight" : 0.5,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test1",
+          "isAbsoluteResource" : false,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_2",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 2,
+              "normalizedWeight" : 0.125,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_2",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 2,
+                  "normalizedWeight" : 0.125,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_1",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 2,
+              "normalizedWeight" : 0.125,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_1",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 2,
+                  "normalizedWeight" : 0.125,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_3",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 12,
+              "normalizedWeight" : 0.75,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_3",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 37.5,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 12,
+                  "normalizedWeight" : 0.75,
+                  "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" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 3750,
+              "maxApplicationsPerUser" : 3750,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "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,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 50,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 0,
+              "weight" : 16,
+              "normalizedWeight" : 0.5,
+              "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" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "weight",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "effectiveMaxResource" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          }
+        } ]
+      },
+      "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" : { }
+    }
+  }
+}

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

@@ -0,0 +1,2790 @@
+{
+  "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" : 12.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 4,
+          "normalizedWeight" : 0.125,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "default",
+          "isAbsoluteResource" : false,
+          "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" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 12.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 10,
+              "weight" : 4,
+              "normalizedWeight" : 0.125,
+              "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : "weight",
+          "queueType" : "leaf",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : { },
+          "autoQueueLeafTemplateProperties" : { },
+          "numActiveApplications" : 0,
+          "numPendingApplications" : 0,
+          "numContainers" : 0,
+          "maxApplications" : 1250,
+          "maxApplicationsPerUser" : 1250,
+          "userLimit" : 100,
+          "users" : { },
+          "userLimitFactor" : 1,
+          "configuredMaxAMResourceLimit" : 0.1,
+          "AMResourceLimit" : {
+            "memory" : 4096,
+            "vCores" : 1,
+            "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" : 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" : 4096,
+            "vCores" : 1,
+            "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" : 1
+              } ]
+            }
+          },
+          "preemptionDisabled" : true,
+          "intraQueuePreemptionDisabled" : true,
+          "defaultPriority" : 0,
+          "isAutoCreatedLeafQueue" : false,
+          "maxApplicationLifetime" : -1,
+          "defaultApplicationLifetime" : -1
+        }, {
+          "queuePath" : "root.test2",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 37.5,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 12,
+          "normalizedWeight" : 0.375,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test2",
+          "isAbsoluteResource" : false,
+          "state" : "RUNNING",
+          "queues" : { },
+          "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" : 37.5,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 0,
+              "weight" : 12,
+              "normalizedWeight" : 0.375,
+              "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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 12288,
+            "vCores" : 12,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 12288
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 12
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "weight",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "flexible",
+          "autoQueueTemplateProperties" : { },
+          "autoQueueParentTemplateProperties" : {
+            "property" : [ {
+              "name" : "acl_administer_queue",
+              "value" : "parentAdmin"
+            }, {
+              "name" : "acl_submit_applications",
+              "value" : "parentUser"
+            } ]
+          },
+          "autoQueueLeafTemplateProperties" : { }
+        }, {
+          "queuePath" : "root.test1",
+          "capacity" : 0,
+          "usedCapacity" : 0,
+          "maxCapacity" : 100,
+          "absoluteCapacity" : 50,
+          "absoluteMaxCapacity" : 100,
+          "absoluteUsedCapacity" : 0,
+          "weight" : 16,
+          "normalizedWeight" : 0.5,
+          "numApplications" : 0,
+          "maxParallelApps" : 2147483647,
+          "queueName" : "test1",
+          "isAbsoluteResource" : false,
+          "state" : "RUNNING",
+          "queues" : {
+            "queue" : [ {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_1",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 2,
+              "normalizedWeight" : 0.125,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_1",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 2,
+                  "normalizedWeight" : 0.125,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_2",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 6.25,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 2,
+              "normalizedWeight" : 0.125,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_2",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 6.25,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 2,
+                  "normalizedWeight" : 0.125,
+                  "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" : 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" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "maxApplicationLifetime" : -1,
+              "defaultApplicationLifetime" : -1
+            }, {
+              "type" : "capacitySchedulerLeafQueueInfo",
+              "queuePath" : "root.test1.test1_3",
+              "capacity" : 0,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 37.5,
+              "absoluteMaxCapacity" : 100,
+              "absoluteUsedCapacity" : 0,
+              "weight" : 12,
+              "normalizedWeight" : 0.75,
+              "numApplications" : 0,
+              "maxParallelApps" : 2147483647,
+              "queueName" : "test1_3",
+              "isAbsoluteResource" : false,
+              "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" : 0,
+                  "usedCapacity" : 0,
+                  "maxCapacity" : 100,
+                  "absoluteCapacity" : 37.5,
+                  "absoluteUsedCapacity" : 0,
+                  "absoluteMaxCapacity" : 100,
+                  "maxAMLimitPercentage" : 10,
+                  "weight" : 12,
+                  "normalizedWeight" : 0.75,
+                  "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" : 12288,
+                    "vCores" : 12,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 12288
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 12
+                      } ]
+                    }
+                  },
+                  "effectiveMaxResource" : {
+                    "memory" : 32768,
+                    "vCores" : 32,
+                    "resourceInformations" : {
+                      "resourceInformation" : [ {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "memory-mb",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "Mi",
+                        "value" : 32768
+                      }, {
+                        "attributes" : { },
+                        "maximumAllocation" : 9223372036854775807,
+                        "minimumAllocation" : 0,
+                        "name" : "vcores",
+                        "resourceType" : "COUNTABLE",
+                        "units" : "",
+                        "value" : 32
+                      } ]
+                    }
+                  }
+                } ]
+              },
+              "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" : 4096,
+                    "vCores" : 1,
+                    "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" : 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" : 12288,
+                "vCores" : 12,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 12288
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 12
+                  } ]
+                }
+              },
+              "maxEffectiveCapacity" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              },
+              "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" : "weight",
+              "queueType" : "leaf",
+              "creationMethod" : "static",
+              "autoCreationEligibility" : "off",
+              "autoQueueTemplateProperties" : { },
+              "autoQueueParentTemplateProperties" : { },
+              "autoQueueLeafTemplateProperties" : { },
+              "numActiveApplications" : 0,
+              "numPendingApplications" : 0,
+              "numContainers" : 0,
+              "maxApplications" : 3750,
+              "maxApplicationsPerUser" : 3750,
+              "userLimit" : 100,
+              "users" : { },
+              "userLimitFactor" : 1,
+              "configuredMaxAMResourceLimit" : 0.1,
+              "AMResourceLimit" : {
+                "memory" : 4096,
+                "vCores" : 1,
+                "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" : 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" : 4096,
+                "vCores" : 1,
+                "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" : 1
+                  } ]
+                }
+              },
+              "preemptionDisabled" : true,
+              "intraQueuePreemptionDisabled" : true,
+              "defaultPriority" : 0,
+              "isAutoCreatedLeafQueue" : false,
+              "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,
+              "usedCapacity" : 0,
+              "maxCapacity" : 100,
+              "absoluteCapacity" : 50,
+              "absoluteUsedCapacity" : 0,
+              "absoluteMaxCapacity" : 100,
+              "maxAMLimitPercentage" : 0,
+              "weight" : 16,
+              "normalizedWeight" : 0.5,
+              "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" : 16384,
+                "vCores" : 16,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 16384
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 16
+                  } ]
+                }
+              },
+              "effectiveMaxResource" : {
+                "memory" : 32768,
+                "vCores" : 32,
+                "resourceInformations" : {
+                  "resourceInformation" : [ {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "memory-mb",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "Mi",
+                    "value" : 32768
+                  }, {
+                    "attributes" : { },
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
+                    "name" : "vcores",
+                    "resourceType" : "COUNTABLE",
+                    "units" : "",
+                    "value" : 32
+                  } ]
+                }
+              }
+            } ]
+          },
+          "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" : 16384,
+            "vCores" : 16,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 16384
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 16
+              } ]
+            }
+          },
+          "maxEffectiveCapacity" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "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" : false,
+          "leafQueueTemplate" : { },
+          "mode" : "weight",
+          "queueType" : "parent",
+          "creationMethod" : "static",
+          "autoCreationEligibility" : "off",
+          "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" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          },
+          "effectiveMaxResource" : {
+            "memory" : 32768,
+            "vCores" : 32,
+            "resourceInformations" : {
+              "resourceInformation" : [ {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "memory-mb",
+                "resourceType" : "COUNTABLE",
+                "units" : "Mi",
+                "value" : 32768
+              }, {
+                "attributes" : { },
+                "maximumAllocation" : 9223372036854775807,
+                "minimumAllocation" : 0,
+                "name" : "vcores",
+                "resourceType" : "COUNTABLE",
+                "units" : "",
+                "value" : 32
+              } ]
+            }
+          }
+        } ]
+      },
+      "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" : { }
+    }
+  }
+}

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

@@ -1,1679 +0,0 @@
-{
-  "scheduler" : {
-    "schedulerInfo" : {
-      "type" : "capacityScheduler",
-      "capacity" : 100,
-      "usedCapacity" : 0,
-      "maxCapacity" : 100,
-      "weight" : -1,
-      "normalizedWeight" : 0,
-      "queueName" : "root",
-      "queuePath" : "root",
-      "maxParallelApps" : 2147483647,
-      "isAbsoluteResource" : true,
-      "queues" : {
-        "queue" : [ {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.test2",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 0,
-          "absoluteMaxCapacity" : 0,
-          "absoluteUsedCapacity" : 0,
-          "weight" : -1,
-          "normalizedWeight" : 0,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "test2",
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 0,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 0,
-              "maxAMLimitPercentage" : 10,
-              "weight" : -1,
-              "normalizedWeight" : 0,
-              "configuredMinResource" : {
-                "memory" : 1024,
-                "vCores" : 5,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1024
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 5
-                  } ]
-                }
-              },
-              "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" : 1024,
-                "vCores" : 5,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1024
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 5
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "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
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 1024,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1024
-                  }, {
-                    "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" : 1024,
-            "vCores" : 5,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1024
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 5
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "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" : "fifo",
-          "autoCreateChildQueueEnabled" : false,
-          "leafQueueTemplate" : { },
-          "mode" : "absolute",
-          "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" : 1024,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1024
-              }, {
-                "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" : 1024,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1024
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        }, {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.test1",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 0,
-          "absoluteMaxCapacity" : 0,
-          "absoluteUsedCapacity" : 0,
-          "weight" : -1,
-          "normalizedWeight" : 0,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "test1",
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 0,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 0,
-              "maxAMLimitPercentage" : 10,
-              "weight" : -1,
-              "normalizedWeight" : 0,
-              "configuredMinResource" : {
-                "memory" : 2048,
-                "vCores" : 10,
-                "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" : 10
-                  } ]
-                }
-              },
-              "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" : 10,
-                "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" : 10
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "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
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 1024,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1024
-                  }, {
-                    "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" : 10,
-            "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" : 10
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "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" : "fifo",
-          "autoCreateChildQueueEnabled" : false,
-          "leafQueueTemplate" : { },
-          "mode" : "absolute",
-          "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" : 1024,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1024
-              }, {
-                "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" : 1024,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1024
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        }, {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.default",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 0,
-          "absoluteMaxCapacity" : 0,
-          "absoluteUsedCapacity" : 0,
-          "weight" : -1,
-          "normalizedWeight" : 0,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "default",
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 0,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 0,
-              "maxAMLimitPercentage" : 10,
-              "weight" : -1,
-              "normalizedWeight" : 0,
-              "configuredMinResource" : {
-                "memory" : 3064,
-                "vCores" : 15,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 3064
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 15
-                  } ]
-                }
-              },
-              "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" : 3064,
-                "vCores" : 15,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 3064
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 15
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "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
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 1024,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1024
-                  }, {
-                    "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" : 3064,
-            "vCores" : 15,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 3064
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 15
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "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" : "fifo",
-          "autoCreateChildQueueEnabled" : false,
-          "leafQueueTemplate" : { },
-          "mode" : "absolute",
-          "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" : 1024,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1024
-              }, {
-                "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" : 1024,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1024
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        } ]
-      },
-      "capacities" : {
-        "queueCapacitiesByPartition" : [ {
-          "partitionName" : "",
-          "capacity" : 100,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "absoluteMaxCapacity" : 100,
-          "maxAMLimitPercentage" : 0,
-          "weight" : -1,
-          "normalizedWeight" : 0,
-          "configuredMinResource" : {
-            "memory" : 6136,
-            "vCores" : 30,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 6136
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 30
-              } ]
-            }
-          },
-          "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" : 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
-              } ]
-            }
-          }
-        } ]
-      },
-      "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" : "absolute",
-      "queueType" : "parent",
-      "creationMethod" : "static",
-      "autoCreationEligibility" : "off",
-      "autoQueueTemplateProperties" : { },
-      "autoQueueParentTemplateProperties" : { },
-      "autoQueueLeafTemplateProperties" : { }
-    }
-  }
-}

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

@@ -15,10 +15,10 @@
         "queue" : [ {
           "type" : "capacitySchedulerLeafQueueInfo",
           "queuePath" : "root.default",
-          "capacity" : 0,
+          "capacity" : 87.5,
           "usedCapacity" : 0,
           "maxCapacity" : 100,
-          "absoluteCapacity" : 0,
+          "absoluteCapacity" : 87.5,
           "absoluteMaxCapacity" : 100,
           "absoluteUsedCapacity" : 0,
           "weight" : -1,
@@ -26,8 +26,8 @@
           "numApplications" : 0,
           "maxParallelApps" : 2147483647,
           "queueName" : "default",
-          "isAbsoluteResource" : false,
-          "state" : "STOPPED",
+          "isAbsoluteResource" : true,
+          "state" : "RUNNING",
           "resourcesUsed" : {
             "memory" : 0,
             "vCores" : 0,
@@ -59,35 +59,35 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
-              "capacity" : 0,
+              "capacity" : 87.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
-              "absoluteCapacity" : 0,
+              "absoluteCapacity" : 87.5,
               "absoluteUsedCapacity" : 0,
               "absoluteMaxCapacity" : 100,
               "maxAMLimitPercentage" : 10,
               "weight" : -1,
               "normalizedWeight" : 0,
               "configuredMinResource" : {
-                "memory" : 0,
-                "vCores" : 0,
+                "memory" : 28672,
+                "vCores" : 28,
                 "resourceInformations" : {
                   "resourceInformation" : [ {
                     "attributes" : { },
-                    "maximumAllocation" : 8192,
-                    "minimumAllocation" : 1024,
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
                     "name" : "memory-mb",
                     "resourceType" : "COUNTABLE",
                     "units" : "Mi",
-                    "value" : 0
+                    "value" : 28672
                   }, {
                     "attributes" : { },
-                    "maximumAllocation" : 4,
-                    "minimumAllocation" : 1,
+                    "maximumAllocation" : 9223372036854775807,
+                    "minimumAllocation" : 0,
                     "name" : "vcores",
                     "resourceType" : "COUNTABLE",
                     "units" : "",
-                    "value" : 0
+                    "value" : 28
                   } ]
                 }
               },
@@ -115,8 +115,8 @@
                 }
               },
               "effectiveMinResource" : {
-                "memory" : 0,
-                "vCores" : 0,
+                "memory" : 28672,
+                "vCores" : 28,
                 "resourceInformations" : {
                   "resourceInformation" : [ {
                     "attributes" : { },
@@ -125,7 +125,7 @@
                     "name" : "memory-mb",
                     "resourceType" : "COUNTABLE",
                     "units" : "Mi",
-                    "value" : 0
+                    "value" : 28672
                   }, {
                     "attributes" : { },
                     "maximumAllocation" : 9223372036854775807,
@@ -133,13 +133,13 @@
                     "name" : "vcores",
                     "resourceType" : "COUNTABLE",
                     "units" : "",
-                    "value" : 0
+                    "value" : 28
                   } ]
                 }
               },
               "effectiveMaxResource" : {
-                "memory" : 8388608,
-                "vCores" : 8192,
+                "memory" : 32768,
+                "vCores" : 32,
                 "resourceInformations" : {
                   "resourceInformation" : [ {
                     "attributes" : { },
@@ -148,7 +148,7 @@
                     "name" : "memory-mb",
                     "resourceType" : "COUNTABLE",
                     "units" : "Mi",
-                    "value" : 8388608
+                    "value" : 32768
                   }, {
                     "attributes" : { },
                     "maximumAllocation" : 9223372036854775807,
@@ -156,7 +156,7 @@
                     "name" : "vcores",
                     "resourceType" : "COUNTABLE",
                     "units" : "",
-                    "value" : 8192
+                    "value" : 32
                   } ]
                 }
               }
@@ -258,7 +258,7 @@
                 }
               },
               "amLimit" : {
-                "memory" : 839680,
+                "memory" : 4096,
                 "vCores" : 1,
                 "resourceInformations" : {
                   "resourceInformation" : [ {
@@ -268,7 +268,7 @@
                     "name" : "memory-mb",
                     "resourceType" : "COUNTABLE",
                     "units" : "Mi",
-                    "value" : 839680
+                    "value" : 4096
                   }, {
                     "attributes" : { },
                     "maximumAllocation" : 9223372036854775807,
@@ -306,8 +306,8 @@
             } ]
           },
           "minEffectiveCapacity" : {
-            "memory" : 0,
-            "vCores" : 0,
+            "memory" : 28672,
+            "vCores" : 28,
             "resourceInformations" : {
               "resourceInformation" : [ {
                 "attributes" : { },
@@ -316,7 +316,7 @@
                 "name" : "memory-mb",
                 "resourceType" : "COUNTABLE",
                 "units" : "Mi",
-                "value" : 0
+                "value" : 28672
               }, {
                 "attributes" : { },
                 "maximumAllocation" : 9223372036854775807,
@@ -324,13 +324,13 @@
                 "name" : "vcores",
                 "resourceType" : "COUNTABLE",
                 "units" : "",
-                "value" : 0
+                "value" : 28
               } ]
             }
           },
           "maxEffectiveCapacity" : {
-            "memory" : 8388608,
-            "vCores" : 8192,
+            "memory" : 32768,
+            "vCores" : 32,
             "resourceInformations" : {
               "resourceInformation" : [ {
                 "attributes" : { },
@@ -339,7 +339,7 @@
                 "name" : "memory-mb",
                 "resourceType" : "COUNTABLE",
                 "units" : "Mi",
-                "value" : 8388608
+                "value" : 32768
               }, {
                 "attributes" : { },
                 "maximumAllocation" : 9223372036854775807,
@@ -347,7 +347,7 @@
                 "name" : "vcores",
                 "resourceType" : "COUNTABLE",
                 "units" : "",
-                "value" : 8192
+                "value" : 32
               } ]
             }
           },
@@ -390,7 +390,7 @@
           "orderingPolicyInfo" : "fifo",
           "autoCreateChildQueueEnabled" : false,
           "leafQueueTemplate" : { },
-          "mode" : "percentage",
+          "mode" : "absolute",
           "queueType" : "leaf",
           "creationMethod" : "static",
           "autoCreationEligibility" : "off",
@@ -400,14 +400,14 @@
           "numActiveApplications" : 0,
           "numPendingApplications" : 0,
           "numContainers" : 0,
-          "maxApplications" : 0,
-          "maxApplicationsPerUser" : 0,
+          "maxApplications" : 8750,
+          "maxApplicationsPerUser" : 8750,
           "userLimit" : 100,
           "users" : { },
           "userLimitFactor" : 1,
           "configuredMaxAMResourceLimit" : 0.1,
           "AMResourceLimit" : {
-            "memory" : 839680,
+            "memory" : 4096,
             "vCores" : 1,
             "resourceInformations" : {
               "resourceInformation" : [ {
@@ -417,7 +417,7 @@
                 "name" : "memory-mb",
                 "resourceType" : "COUNTABLE",
                 "units" : "Mi",
-                "value" : 839680
+                "value" : 4096
               }, {
                 "attributes" : { },
                 "maximumAllocation" : 9223372036854775807,
@@ -453,7 +453,7 @@
             }
           },
           "userAMResourceLimit" : {
-            "memory" : 839680,
+            "memory" : 4096,
             "vCores" : 1,
             "resourceInformations" : {
               "resourceInformation" : [ {
@@ -463,7 +463,7 @@
                 "name" : "memory-mb",
                 "resourceType" : "COUNTABLE",
                 "units" : "Mi",
-                "value" : 839680
+                "value" : 4096
               }, {
                 "attributes" : { },
                 "maximumAllocation" : 9223372036854775807,
@@ -483,10 +483,10 @@
           "defaultApplicationLifetime" : -1
         }, {
           "queuePath" : "root.managed",
-          "capacity" : 0.048828125,
+          "capacity" : 12.5,
           "usedCapacity" : 0,
           "maxCapacity" : 100,
-          "absoluteCapacity" : 0.048828125,
+          "absoluteCapacity" : 12.5,
           "absoluteMaxCapacity" : 100,
           "absoluteUsedCapacity" : 0,
           "weight" : -1,
@@ -503,7 +503,7 @@
               "capacity" : 50,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
-              "absoluteCapacity" : 0.024414062,
+              "absoluteCapacity" : 6.25,
               "absoluteMaxCapacity" : 100,
               "absoluteUsedCapacity" : 0,
               "weight" : -1,
@@ -547,7 +547,7 @@
                   "capacity" : 50,
                   "usedCapacity" : 0,
                   "maxCapacity" : 100,
-                  "absoluteCapacity" : 0.024414062,
+                  "absoluteCapacity" : 6.25,
                   "absoluteUsedCapacity" : 0,
                   "absoluteMaxCapacity" : 100,
                   "maxAMLimitPercentage" : 10,
@@ -623,8 +623,8 @@
                     }
                   },
                   "effectiveMaxResource" : {
-                    "memory" : 8388608,
-                    "vCores" : 8192,
+                    "memory" : 32768,
+                    "vCores" : 32,
                     "resourceInformations" : {
                       "resourceInformation" : [ {
                         "attributes" : { },
@@ -633,7 +633,7 @@
                         "name" : "memory-mb",
                         "resourceType" : "COUNTABLE",
                         "units" : "Mi",
-                        "value" : 8388608
+                        "value" : 32768
                       }, {
                         "attributes" : { },
                         "maximumAllocation" : 9223372036854775807,
@@ -641,7 +641,7 @@
                         "name" : "vcores",
                         "resourceType" : "COUNTABLE",
                         "units" : "",
-                        "value" : 8192
+                        "value" : 32
                       } ]
                     }
                   }
@@ -743,7 +743,7 @@
                     }
                   },
                   "amLimit" : {
-                    "memory" : 839680,
+                    "memory" : 4096,
                     "vCores" : 1,
                     "resourceInformations" : {
                       "resourceInformation" : [ {
@@ -753,7 +753,7 @@
                         "name" : "memory-mb",
                         "resourceType" : "COUNTABLE",
                         "units" : "Mi",
-                        "value" : 839680
+                        "value" : 4096
                       }, {
                         "attributes" : { },
                         "maximumAllocation" : 9223372036854775807,
@@ -814,8 +814,8 @@
                 }
               },
               "maxEffectiveCapacity" : {
-                "memory" : 8388608,
-                "vCores" : 8192,
+                "memory" : 32768,
+                "vCores" : 32,
                 "resourceInformations" : {
                   "resourceInformation" : [ {
                     "attributes" : { },
@@ -824,7 +824,7 @@
                     "name" : "memory-mb",
                     "resourceType" : "COUNTABLE",
                     "units" : "Mi",
-                    "value" : 8388608
+                    "value" : 32768
                   }, {
                     "attributes" : { },
                     "maximumAllocation" : 9223372036854775807,
@@ -832,7 +832,7 @@
                     "name" : "vcores",
                     "resourceType" : "COUNTABLE",
                     "units" : "",
-                    "value" : 8192
+                    "value" : 32
                   } ]
                 }
               },
@@ -885,14 +885,14 @@
               "numActiveApplications" : 0,
               "numPendingApplications" : 0,
               "numContainers" : 0,
-              "maxApplications" : 2,
-              "maxApplicationsPerUser" : 2,
+              "maxApplications" : 625,
+              "maxApplicationsPerUser" : 625,
               "userLimit" : 100,
               "users" : { },
               "userLimitFactor" : 1,
               "configuredMaxAMResourceLimit" : 0.1,
               "AMResourceLimit" : {
-                "memory" : 839680,
+                "memory" : 4096,
                 "vCores" : 1,
                 "resourceInformations" : {
                   "resourceInformation" : [ {
@@ -902,7 +902,7 @@
                     "name" : "memory-mb",
                     "resourceType" : "COUNTABLE",
                     "units" : "Mi",
-                    "value" : 839680
+                    "value" : 4096
                   }, {
                     "attributes" : { },
                     "maximumAllocation" : 9223372036854775807,
@@ -938,7 +938,7 @@
                 }
               },
               "userAMResourceLimit" : {
-                "memory" : 839680,
+                "memory" : 4096,
                 "vCores" : 1,
                 "resourceInformations" : {
                   "resourceInformation" : [ {
@@ -948,7 +948,7 @@
                     "name" : "memory-mb",
                     "resourceType" : "COUNTABLE",
                     "units" : "Mi",
-                    "value" : 839680
+                    "value" : 4096
                   }, {
                     "attributes" : { },
                     "maximumAllocation" : 9223372036854775807,
@@ -999,10 +999,10 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
-              "capacity" : 0.048828125,
+              "capacity" : 12.5,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
-              "absoluteCapacity" : 0.048828125,
+              "absoluteCapacity" : 12.5,
               "absoluteUsedCapacity" : 0,
               "absoluteMaxCapacity" : 100,
               "maxAMLimitPercentage" : 0,
@@ -1078,8 +1078,8 @@
                 }
               },
               "effectiveMaxResource" : {
-                "memory" : 8388608,
-                "vCores" : 8192,
+                "memory" : 32768,
+                "vCores" : 32,
                 "resourceInformations" : {
                   "resourceInformation" : [ {
                     "attributes" : { },
@@ -1088,7 +1088,7 @@
                     "name" : "memory-mb",
                     "resourceType" : "COUNTABLE",
                     "units" : "Mi",
-                    "value" : 8388608
+                    "value" : 32768
                   }, {
                     "attributes" : { },
                     "maximumAllocation" : 9223372036854775807,
@@ -1096,7 +1096,7 @@
                     "name" : "vcores",
                     "resourceType" : "COUNTABLE",
                     "units" : "",
-                    "value" : 8192
+                    "value" : 32
                   } ]
                 }
               }
@@ -1200,8 +1200,8 @@
             }
           },
           "maxEffectiveCapacity" : {
-            "memory" : 8388608,
-            "vCores" : 8192,
+            "memory" : 32768,
+            "vCores" : 32,
             "resourceInformations" : {
               "resourceInformation" : [ {
                 "attributes" : { },
@@ -1210,7 +1210,7 @@
                 "name" : "memory-mb",
                 "resourceType" : "COUNTABLE",
                 "units" : "Mi",
-                "value" : 8388608
+                "value" : 32768
               }, {
                 "attributes" : { },
                 "maximumAllocation" : 9223372036854775807,
@@ -1218,7 +1218,7 @@
                 "name" : "vcores",
                 "resourceType" : "COUNTABLE",
                 "units" : "",
-                "value" : 8192
+                "value" : 32
               } ]
             }
           },
@@ -1340,8 +1340,8 @@
             }
           },
           "effectiveMinResource" : {
-            "memory" : 8388608,
-            "vCores" : 8192,
+            "memory" : 32768,
+            "vCores" : 32,
             "resourceInformations" : {
               "resourceInformation" : [ {
                 "attributes" : { },
@@ -1350,7 +1350,7 @@
                 "name" : "memory-mb",
                 "resourceType" : "COUNTABLE",
                 "units" : "Mi",
-                "value" : 8388608
+                "value" : 32768
               }, {
                 "attributes" : { },
                 "maximumAllocation" : 9223372036854775807,
@@ -1358,13 +1358,13 @@
                 "name" : "vcores",
                 "resourceType" : "COUNTABLE",
                 "units" : "",
-                "value" : 8192
+                "value" : 32
               } ]
             }
           },
           "effectiveMaxResource" : {
-            "memory" : 8388608,
-            "vCores" : 8192,
+            "memory" : 32768,
+            "vCores" : 32,
             "resourceInformations" : {
               "resourceInformation" : [ {
                 "attributes" : { },
@@ -1373,7 +1373,7 @@
                 "name" : "memory-mb",
                 "resourceType" : "COUNTABLE",
                 "units" : "Mi",
-                "value" : 8388608
+                "value" : 32768
               }, {
                 "attributes" : { },
                 "maximumAllocation" : 9223372036854775807,
@@ -1381,7 +1381,7 @@
                 "name" : "vcores",
                 "resourceType" : "COUNTABLE",
                 "units" : "",
-                "value" : 8192
+                "value" : 32
               } ]
             }
           }

File diff suppressed because it is too large
+ 106 - 1737
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-NodeLabelDefaultAPI.xml


File diff suppressed because it is too large
+ 264 - 3568
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.json


File diff suppressed because it is too large
+ 294 - 3686
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-PerUserResources.xml


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

@@ -13,481 +13,12 @@
       "isAbsoluteResource" : false,
       "queues" : {
         "queue" : [ {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.test1",
-          "capacity" : 50,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 50,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : -1,
-          "normalizedWeight" : 0,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "test1",
-          "isAbsoluteResource" : false,
-          "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" : 50,
-              "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" : 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
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 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
-                  } ]
-                }
-              },
-              "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" : 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" : "fifo",
-          "autoCreateChildQueueEnabled" : false,
-          "leafQueueTemplate" : { },
-          "mode" : "percentage",
-          "queueType" : "leaf",
-          "creationMethod" : "static",
-          "autoCreationEligibility" : "off",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : { },
-          "numActiveApplications" : 0,
-          "numPendingApplications" : 0,
-          "numContainers" : 0,
-          "maxApplications" : 5000,
-          "maxApplicationsPerUser" : 5000,
-          "userLimit" : 100,
-          "users" : { },
-          "userLimitFactor" : 1,
-          "configuredMaxAMResourceLimit" : 0.1,
-          "AMResourceLimit" : {
-            "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
-              } ]
-            }
-          },
-          "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" : 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
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        }, {
           "type" : "capacitySchedulerLeafQueueInfo",
           "queuePath" : "root.default",
-          "capacity" : 0,
+          "capacity" : 20,
           "usedCapacity" : 0,
           "maxCapacity" : 100,
-          "absoluteCapacity" : 0,
+          "absoluteCapacity" : 20,
           "absoluteMaxCapacity" : 100,
           "absoluteUsedCapacity" : 0,
           "weight" : -1,
@@ -528,10 +59,10 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
-              "capacity" : 0,
+              "capacity" : 20,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
-              "absoluteCapacity" : 0,
+              "absoluteCapacity" : 20,
               "absoluteUsedCapacity" : 0,
               "absoluteMaxCapacity" : 100,
               "maxAMLimitPercentage" : 10,
@@ -869,8 +400,8 @@
           "numActiveApplications" : 0,
           "numPendingApplications" : 0,
           "numContainers" : 0,
-          "maxApplications" : 0,
-          "maxApplicationsPerUser" : 0,
+          "maxApplications" : 2000,
+          "maxApplicationsPerUser" : 2000,
           "userLimit" : 100,
           "users" : { },
           "userLimitFactor" : 1,
@@ -951,18 +482,18 @@
           "maxApplicationLifetime" : -1,
           "defaultApplicationLifetime" : -1
         }, {
-          "queuePath" : "root.managedtest2",
-          "capacity" : 50,
+          "queuePath" : "root.managed",
+          "capacity" : 80,
           "usedCapacity" : 0,
           "maxCapacity" : 100,
-          "absoluteCapacity" : 50,
+          "absoluteCapacity" : 80,
           "absoluteMaxCapacity" : 100,
           "absoluteUsedCapacity" : 0,
           "weight" : -1,
           "normalizedWeight" : 0,
           "numApplications" : 0,
           "maxParallelApps" : 2147483647,
-          "queueName" : "managedtest2",
+          "queueName" : "managed",
           "isAbsoluteResource" : false,
           "state" : "RUNNING",
           "queues" : { },
@@ -997,10 +528,10 @@
           "capacities" : {
             "queueCapacitiesByPartition" : [ {
               "partitionName" : "",
-              "capacity" : 50,
+              "capacity" : 80,
               "usedCapacity" : 0,
               "maxCapacity" : 100,
-              "absoluteCapacity" : 50,
+              "absoluteCapacity" : 80,
               "absoluteUsedCapacity" : 0,
               "absoluteMaxCapacity" : 100,
               "maxAMLimitPercentage" : 0,

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

@@ -1,5818 +0,0 @@
-{
-  "scheduler" : {
-    "schedulerInfo" : {
-      "type" : "capacityScheduler",
-      "capacity" : 100,
-      "usedCapacity" : 0,
-      "maxCapacity" : 100,
-      "weight" : 1,
-      "normalizedWeight" : 1,
-      "queueName" : "root",
-      "queuePath" : "root",
-      "maxParallelApps" : 2147483647,
-      "isAbsoluteResource" : false,
-      "queues" : {
-        "queue" : [ {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.test2",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 20,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 10,
-          "normalizedWeight" : 0.2,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "test2",
-          "isAbsoluteResource" : false,
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 20,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 10,
-              "weight" : 10,
-              "normalizedWeight" : 0.2,
-              "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" : 245760,
-                "vCores" : 240,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 245760
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 240
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 122880,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 122880
-                  }, {
-                    "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" : 245760,
-            "vCores" : 240,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 245760
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 240
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : "weight",
-          "queueType" : "leaf",
-          "creationMethod" : "static",
-          "autoCreationEligibility" : "off",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : { },
-          "numActiveApplications" : 0,
-          "numPendingApplications" : 0,
-          "numContainers" : 0,
-          "maxApplications" : 2000,
-          "maxApplicationsPerUser" : 2000,
-          "userLimit" : 100,
-          "users" : { },
-          "userLimitFactor" : 1,
-          "configuredMaxAMResourceLimit" : 0.1,
-          "AMResourceLimit" : {
-            "memory" : 122880,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 122880
-              }, {
-                "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" : 122880,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 122880
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        }, {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.test1",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 10,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 5,
-          "normalizedWeight" : 0.1,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "test1",
-          "isAbsoluteResource" : false,
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 10,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 10,
-              "weight" : 5,
-              "normalizedWeight" : 0.1,
-              "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" : 122880,
-                "vCores" : 120,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 122880
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 120
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 122880,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 122880
-                  }, {
-                    "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" : 122880,
-            "vCores" : 120,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 122880
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 120
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : "weight",
-          "queueType" : "leaf",
-          "creationMethod" : "static",
-          "autoCreationEligibility" : "off",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : { },
-          "numActiveApplications" : 0,
-          "numPendingApplications" : 0,
-          "numContainers" : 0,
-          "maxApplications" : 1000,
-          "maxApplicationsPerUser" : 1000,
-          "userLimit" : 100,
-          "users" : { },
-          "userLimitFactor" : 1,
-          "configuredMaxAMResourceLimit" : 0.1,
-          "AMResourceLimit" : {
-            "memory" : 122880,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 122880
-              }, {
-                "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" : 122880,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 122880
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        }, {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.default",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 20,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 10,
-          "normalizedWeight" : 0.2,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "default",
-          "isAbsoluteResource" : false,
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 20,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 10,
-              "weight" : 10,
-              "normalizedWeight" : 0.2,
-              "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" : 245760,
-                "vCores" : 240,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 245760
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 240
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 122880,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 122880
-                  }, {
-                    "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" : 245760,
-            "vCores" : 240,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 245760
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 240
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : "weight",
-          "queueType" : "leaf",
-          "creationMethod" : "static",
-          "autoCreationEligibility" : "off",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : { },
-          "numActiveApplications" : 0,
-          "numPendingApplications" : 0,
-          "numContainers" : 0,
-          "maxApplications" : 2000,
-          "maxApplicationsPerUser" : 2000,
-          "userLimit" : 100,
-          "users" : { },
-          "userLimitFactor" : 1,
-          "configuredMaxAMResourceLimit" : 0.1,
-          "AMResourceLimit" : {
-            "memory" : 122880,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 122880
-              }, {
-                "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" : 122880,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 122880
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        }, {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.auto1",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 2,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 1,
-          "normalizedWeight" : 0.02,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "auto1",
-          "isAbsoluteResource" : false,
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 2,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 100,
-              "weight" : 1,
-              "normalizedWeight" : 0.02,
-              "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" : 24575,
-                "vCores" : 23,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 24575
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 23
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 1228800,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "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" : 24575,
-            "vCores" : 23,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 24575
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 23
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : "weight",
-          "queueType" : "leaf",
-          "creationMethod" : "dynamicFlexible",
-          "autoCreationEligibility" : "off",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : { },
-          "numActiveApplications" : 0,
-          "numPendingApplications" : 0,
-          "numContainers" : 0,
-          "maxApplications" : 200,
-          "maxApplicationsPerUser" : 200,
-          "userLimit" : 100,
-          "users" : { },
-          "userLimitFactor" : -1,
-          "configuredMaxAMResourceLimit" : 1,
-          "AMResourceLimit" : {
-            "memory" : 1228800,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "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" : 1228800,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        }, {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.auto2",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 2,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 1,
-          "normalizedWeight" : 0.02,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "auto2",
-          "isAbsoluteResource" : false,
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 2,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 100,
-              "weight" : 1,
-              "normalizedWeight" : 0.02,
-              "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" : 24575,
-                "vCores" : 23,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 24575
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 23
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 1228800,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "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" : 24575,
-            "vCores" : 23,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 24575
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 23
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : "weight",
-          "queueType" : "leaf",
-          "creationMethod" : "dynamicFlexible",
-          "autoCreationEligibility" : "off",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : { },
-          "numActiveApplications" : 0,
-          "numPendingApplications" : 0,
-          "numContainers" : 0,
-          "maxApplications" : 200,
-          "maxApplicationsPerUser" : 200,
-          "userLimit" : 100,
-          "users" : { },
-          "userLimitFactor" : -1,
-          "configuredMaxAMResourceLimit" : 1,
-          "AMResourceLimit" : {
-            "memory" : 1228800,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "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" : 1228800,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1
-              } ]
-            }
-          },
-          "preemptionDisabled" : true,
-          "intraQueuePreemptionDisabled" : true,
-          "defaultPriority" : 0,
-          "isAutoCreatedLeafQueue" : false,
-          "maxApplicationLifetime" : -1,
-          "defaultApplicationLifetime" : -1
-        }, {
-          "type" : "capacitySchedulerLeafQueueInfo",
-          "queuePath" : "root.auto3",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 2,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 1,
-          "normalizedWeight" : 0.02,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "auto3",
-          "isAbsoluteResource" : false,
-          "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" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 2,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 100,
-              "weight" : 1,
-              "normalizedWeight" : 0.02,
-              "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" : 24575,
-                "vCores" : 23,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 24575
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 23
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 1228800,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "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" : 24575,
-            "vCores" : 23,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 24575
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 23
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : "weight",
-          "queueType" : "leaf",
-          "creationMethod" : "dynamicFlexible",
-          "autoCreationEligibility" : "off",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : { },
-          "numActiveApplications" : 0,
-          "numPendingApplications" : 0,
-          "numContainers" : 0,
-          "maxApplications" : 200,
-          "maxApplicationsPerUser" : 200,
-          "userLimit" : 100,
-          "users" : { },
-          "userLimitFactor" : -1,
-          "configuredMaxAMResourceLimit" : 1,
-          "AMResourceLimit" : {
-            "memory" : 1228800,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "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" : 1228800,
-            "vCores" : 1,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "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.parent",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 40,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 20,
-          "normalizedWeight" : 0.4,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "parent",
-          "isAbsoluteResource" : false,
-          "state" : "RUNNING",
-          "queues" : {
-            "queue" : [ {
-              "queuePath" : "root.parent.autoParent3",
-              "capacity" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 40,
-              "absoluteMaxCapacity" : 100,
-              "absoluteUsedCapacity" : 0,
-              "weight" : 1,
-              "normalizedWeight" : 1,
-              "numApplications" : 0,
-              "maxParallelApps" : 2147483647,
-              "queueName" : "autoParent3",
-              "isAbsoluteResource" : false,
-              "state" : "RUNNING",
-              "queues" : {
-                "queue" : [ {
-                  "type" : "capacitySchedulerLeafQueueInfo",
-                  "queuePath" : "root.parent.autoParent3.auto6",
-                  "capacity" : 0,
-                  "usedCapacity" : 0,
-                  "maxCapacity" : 100,
-                  "absoluteCapacity" : 40,
-                  "absoluteMaxCapacity" : 100,
-                  "absoluteUsedCapacity" : 0,
-                  "weight" : 1,
-                  "normalizedWeight" : 1,
-                  "numApplications" : 0,
-                  "maxParallelApps" : 2147483647,
-                  "queueName" : "auto6",
-                  "isAbsoluteResource" : false,
-                  "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" : 0,
-                      "usedCapacity" : 0,
-                      "maxCapacity" : 100,
-                      "absoluteCapacity" : 40,
-                      "absoluteUsedCapacity" : 0,
-                      "absoluteMaxCapacity" : 100,
-                      "maxAMLimitPercentage" : 100,
-                      "weight" : 1,
-                      "normalizedWeight" : 1,
-                      "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" : 491520,
-                        "vCores" : 480,
-                        "resourceInformations" : {
-                          "resourceInformation" : [ {
-                            "attributes" : { },
-                            "maximumAllocation" : 9223372036854775807,
-                            "minimumAllocation" : 0,
-                            "name" : "memory-mb",
-                            "resourceType" : "COUNTABLE",
-                            "units" : "Mi",
-                            "value" : 491520
-                          }, {
-                            "attributes" : { },
-                            "maximumAllocation" : 9223372036854775807,
-                            "minimumAllocation" : 0,
-                            "name" : "vcores",
-                            "resourceType" : "COUNTABLE",
-                            "units" : "",
-                            "value" : 480
-                          } ]
-                        }
-                      },
-                      "effectiveMaxResource" : {
-                        "memory" : 1228800,
-                        "vCores" : 1200,
-                        "resourceInformations" : {
-                          "resourceInformation" : [ {
-                            "attributes" : { },
-                            "maximumAllocation" : 9223372036854775807,
-                            "minimumAllocation" : 0,
-                            "name" : "memory-mb",
-                            "resourceType" : "COUNTABLE",
-                            "units" : "Mi",
-                            "value" : 1228800
-                          }, {
-                            "attributes" : { },
-                            "maximumAllocation" : 9223372036854775807,
-                            "minimumAllocation" : 0,
-                            "name" : "vcores",
-                            "resourceType" : "COUNTABLE",
-                            "units" : "",
-                            "value" : 1200
-                          } ]
-                        }
-                      }
-                    } ]
-                  },
-                  "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" : 1228800,
-                        "vCores" : 1,
-                        "resourceInformations" : {
-                          "resourceInformation" : [ {
-                            "attributes" : { },
-                            "maximumAllocation" : 9223372036854775807,
-                            "minimumAllocation" : 0,
-                            "name" : "memory-mb",
-                            "resourceType" : "COUNTABLE",
-                            "units" : "Mi",
-                            "value" : 1228800
-                          }, {
-                            "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" : 491520,
-                    "vCores" : 480,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 491520
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 480
-                      } ]
-                    }
-                  },
-                  "maxEffectiveCapacity" : {
-                    "memory" : 1228800,
-                    "vCores" : 1200,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 1228800
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 1200
-                      } ]
-                    }
-                  },
-                  "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" : "wildAdmin2 "
-                    }, {
-                      "accessType" : "APPLICATION_MAX_PRIORITY",
-                      "accessControlList" : "*"
-                    }, {
-                      "accessType" : "SUBMIT_APP",
-                      "accessControlList" : "wildUser2 "
-                    } ]
-                  },
-                  "queuePriority" : 0,
-                  "orderingPolicyInfo" : "fifo",
-                  "autoCreateChildQueueEnabled" : false,
-                  "leafQueueTemplate" : { },
-                  "mode" : "weight",
-                  "queueType" : "leaf",
-                  "creationMethod" : "dynamicFlexible",
-                  "autoCreationEligibility" : "off",
-                  "autoQueueTemplateProperties" : { },
-                  "autoQueueParentTemplateProperties" : { },
-                  "autoQueueLeafTemplateProperties" : { },
-                  "numActiveApplications" : 0,
-                  "numPendingApplications" : 0,
-                  "numContainers" : 0,
-                  "maxApplications" : 4000,
-                  "maxApplicationsPerUser" : 4000,
-                  "userLimit" : 100,
-                  "users" : { },
-                  "userLimitFactor" : -1,
-                  "configuredMaxAMResourceLimit" : 1,
-                  "AMResourceLimit" : {
-                    "memory" : 1228800,
-                    "vCores" : 1,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 1228800
-                      }, {
-                        "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" : 1228800,
-                    "vCores" : 1,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 1228800
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 1
-                      } ]
-                    }
-                  },
-                  "preemptionDisabled" : true,
-                  "intraQueuePreemptionDisabled" : true,
-                  "defaultPriority" : 0,
-                  "isAutoCreatedLeafQueue" : false,
-                  "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,
-                  "usedCapacity" : 0,
-                  "maxCapacity" : 100,
-                  "absoluteCapacity" : 40,
-                  "absoluteUsedCapacity" : 0,
-                  "absoluteMaxCapacity" : 100,
-                  "maxAMLimitPercentage" : 0,
-                  "weight" : 1,
-                  "normalizedWeight" : 1,
-                  "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" : 491520,
-                    "vCores" : 480,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 491520
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 480
-                      } ]
-                    }
-                  },
-                  "effectiveMaxResource" : {
-                    "memory" : 1228800,
-                    "vCores" : 1200,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 1228800
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 1200
-                      } ]
-                    }
-                  }
-                } ]
-              },
-              "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" : 491520,
-                "vCores" : 480,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 491520
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 480
-                  } ]
-                }
-              },
-              "maxEffectiveCapacity" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              },
-              "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" : "parentAdmin2 "
-                }, {
-                  "accessType" : "APPLICATION_MAX_PRIORITY",
-                  "accessControlList" : "*"
-                }, {
-                  "accessType" : "SUBMIT_APP",
-                  "accessControlList" : "parentUser2 "
-                } ]
-              },
-              "queuePriority" : 0,
-              "orderingPolicyInfo" : "utilization",
-              "autoCreateChildQueueEnabled" : false,
-              "leafQueueTemplate" : { },
-              "mode" : "weight",
-              "queueType" : "parent",
-              "creationMethod" : "dynamicFlexible",
-              "autoCreationEligibility" : "flexible",
-              "autoQueueTemplateProperties" : { },
-              "autoQueueParentTemplateProperties" : { },
-              "autoQueueLeafTemplateProperties" : {
-                "property" : [ {
-                  "name" : "acl_administer_queue",
-                  "value" : "wildAdmin2"
-                }, {
-                  "name" : "acl_submit_applications",
-                  "value" : "wildUser2"
-                } ]
-              }
-            } ]
-          },
-          "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" : 40,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 0,
-              "weight" : 20,
-              "normalizedWeight" : 0.4,
-              "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" : 491520,
-                "vCores" : 480,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 491520
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 480
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 491520,
-            "vCores" : 480,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 491520
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 480
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : false,
-          "leafQueueTemplate" : { },
-          "mode" : "weight",
-          "queueType" : "parent",
-          "creationMethod" : "static",
-          "autoCreationEligibility" : "flexible",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : {
-            "property" : [ {
-              "name" : "acl_administer_queue",
-              "value" : "parentAdmin2"
-            }, {
-              "name" : "acl_submit_applications",
-              "value" : "parentUser2"
-            } ]
-          },
-          "autoQueueLeafTemplateProperties" : {
-            "property" : [ {
-              "name" : "acl_administer_queue",
-              "value" : "wildAdmin1"
-            }, {
-              "name" : "acl_submit_applications",
-              "value" : "wildUser1"
-            } ]
-          }
-        }, {
-          "queuePath" : "root.autoParent1",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 2,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 1,
-          "normalizedWeight" : 0.02,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "autoParent1",
-          "isAbsoluteResource" : false,
-          "state" : "RUNNING",
-          "queues" : {
-            "queue" : [ {
-              "type" : "capacitySchedulerLeafQueueInfo",
-              "queuePath" : "root.autoParent1.auto4",
-              "capacity" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 2,
-              "absoluteMaxCapacity" : 100,
-              "absoluteUsedCapacity" : 0,
-              "weight" : 1,
-              "normalizedWeight" : 1,
-              "numApplications" : 0,
-              "maxParallelApps" : 2147483647,
-              "queueName" : "auto4",
-              "isAbsoluteResource" : false,
-              "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" : 0,
-                  "usedCapacity" : 0,
-                  "maxCapacity" : 100,
-                  "absoluteCapacity" : 2,
-                  "absoluteUsedCapacity" : 0,
-                  "absoluteMaxCapacity" : 100,
-                  "maxAMLimitPercentage" : 100,
-                  "weight" : 1,
-                  "normalizedWeight" : 1,
-                  "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" : 24575,
-                    "vCores" : 23,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 24575
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 23
-                      } ]
-                    }
-                  },
-                  "effectiveMaxResource" : {
-                    "memory" : 1228800,
-                    "vCores" : 1200,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 1228800
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 1200
-                      } ]
-                    }
-                  }
-                } ]
-              },
-              "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" : 1228800,
-                    "vCores" : 1,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 1228800
-                      }, {
-                        "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" : 24575,
-                "vCores" : 23,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 24575
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 23
-                  } ]
-                }
-              },
-              "maxEffectiveCapacity" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              },
-              "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" : "admin1 "
-                }, {
-                  "accessType" : "APPLICATION_MAX_PRIORITY",
-                  "accessControlList" : "*"
-                }, {
-                  "accessType" : "SUBMIT_APP",
-                  "accessControlList" : "user1 "
-                } ]
-              },
-              "queuePriority" : 0,
-              "orderingPolicyInfo" : "fifo",
-              "autoCreateChildQueueEnabled" : false,
-              "leafQueueTemplate" : { },
-              "mode" : "weight",
-              "queueType" : "leaf",
-              "creationMethod" : "dynamicFlexible",
-              "autoCreationEligibility" : "off",
-              "autoQueueTemplateProperties" : { },
-              "autoQueueParentTemplateProperties" : { },
-              "autoQueueLeafTemplateProperties" : { },
-              "numActiveApplications" : 0,
-              "numPendingApplications" : 0,
-              "numContainers" : 0,
-              "maxApplications" : 300,
-              "maxApplicationsPerUser" : 300,
-              "userLimit" : 100,
-              "users" : { },
-              "userLimitFactor" : -1,
-              "configuredMaxAMResourceLimit" : 1,
-              "AMResourceLimit" : {
-                "memory" : 1228800,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "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" : 1228800,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1
-                  } ]
-                }
-              },
-              "preemptionDisabled" : true,
-              "intraQueuePreemptionDisabled" : true,
-              "defaultPriority" : 0,
-              "isAutoCreatedLeafQueue" : false,
-              "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,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 2,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 0,
-              "weight" : 1,
-              "normalizedWeight" : 0.02,
-              "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" : 24575,
-                "vCores" : 23,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 24575
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 23
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 24575,
-            "vCores" : 23,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 24575
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 23
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : "parentAdmin1 "
-            }, {
-              "accessType" : "APPLICATION_MAX_PRIORITY",
-              "accessControlList" : "*"
-            }, {
-              "accessType" : "SUBMIT_APP",
-              "accessControlList" : "parentUser1 "
-            } ]
-          },
-          "queuePriority" : 0,
-          "orderingPolicyInfo" : "utilization",
-          "autoCreateChildQueueEnabled" : false,
-          "leafQueueTemplate" : { },
-          "mode" : "weight",
-          "queueType" : "parent",
-          "creationMethod" : "dynamicFlexible",
-          "autoCreationEligibility" : "flexible",
-          "autoQueueTemplateProperties" : {
-            "property" : [ {
-              "name" : "maximum-applications",
-              "value" : "300"
-            } ]
-          },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : {
-            "property" : [ {
-              "name" : "acl_administer_queue",
-              "value" : "admin1"
-            }, {
-              "name" : "acl_submit_applications",
-              "value" : "user1"
-            } ]
-          }
-        }, {
-          "queuePath" : "root.autoParent2",
-          "capacity" : 0,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 2,
-          "absoluteMaxCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "weight" : 1,
-          "normalizedWeight" : 0.02,
-          "numApplications" : 0,
-          "maxParallelApps" : 2147483647,
-          "queueName" : "autoParent2",
-          "isAbsoluteResource" : false,
-          "state" : "RUNNING",
-          "queues" : {
-            "queue" : [ {
-              "type" : "capacitySchedulerLeafQueueInfo",
-              "queuePath" : "root.autoParent2.auto5",
-              "capacity" : 0,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 2,
-              "absoluteMaxCapacity" : 100,
-              "absoluteUsedCapacity" : 0,
-              "weight" : 1,
-              "normalizedWeight" : 1,
-              "numApplications" : 0,
-              "maxParallelApps" : 2147483647,
-              "queueName" : "auto5",
-              "isAbsoluteResource" : false,
-              "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" : 0,
-                  "usedCapacity" : 0,
-                  "maxCapacity" : 100,
-                  "absoluteCapacity" : 2,
-                  "absoluteUsedCapacity" : 0,
-                  "absoluteMaxCapacity" : 100,
-                  "maxAMLimitPercentage" : 100,
-                  "weight" : 1,
-                  "normalizedWeight" : 1,
-                  "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" : 24575,
-                    "vCores" : 23,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 24575
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 23
-                      } ]
-                    }
-                  },
-                  "effectiveMaxResource" : {
-                    "memory" : 1228800,
-                    "vCores" : 1200,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 1228800
-                      }, {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "vcores",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "",
-                        "value" : 1200
-                      } ]
-                    }
-                  }
-                } ]
-              },
-              "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" : 1228800,
-                    "vCores" : 1,
-                    "resourceInformations" : {
-                      "resourceInformation" : [ {
-                        "attributes" : { },
-                        "maximumAllocation" : 9223372036854775807,
-                        "minimumAllocation" : 0,
-                        "name" : "memory-mb",
-                        "resourceType" : "COUNTABLE",
-                        "units" : "Mi",
-                        "value" : 1228800
-                      }, {
-                        "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" : 24575,
-                "vCores" : 23,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 24575
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 23
-                  } ]
-                }
-              },
-              "maxEffectiveCapacity" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              },
-              "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" : "wildAdmin1 "
-                }, {
-                  "accessType" : "APPLICATION_MAX_PRIORITY",
-                  "accessControlList" : "*"
-                }, {
-                  "accessType" : "SUBMIT_APP",
-                  "accessControlList" : "wildUser1 "
-                } ]
-              },
-              "queuePriority" : 0,
-              "orderingPolicyInfo" : "fifo",
-              "autoCreateChildQueueEnabled" : false,
-              "leafQueueTemplate" : { },
-              "mode" : "weight",
-              "queueType" : "leaf",
-              "creationMethod" : "dynamicFlexible",
-              "autoCreationEligibility" : "off",
-              "autoQueueTemplateProperties" : { },
-              "autoQueueParentTemplateProperties" : { },
-              "autoQueueLeafTemplateProperties" : { },
-              "numActiveApplications" : 0,
-              "numPendingApplications" : 0,
-              "numContainers" : 0,
-              "maxApplications" : 200,
-              "maxApplicationsPerUser" : 200,
-              "userLimit" : 100,
-              "users" : { },
-              "userLimitFactor" : -1,
-              "configuredMaxAMResourceLimit" : 1,
-              "AMResourceLimit" : {
-                "memory" : 1228800,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "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" : 1228800,
-                "vCores" : 1,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1
-                  } ]
-                }
-              },
-              "preemptionDisabled" : true,
-              "intraQueuePreemptionDisabled" : true,
-              "defaultPriority" : 0,
-              "isAutoCreatedLeafQueue" : false,
-              "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,
-              "usedCapacity" : 0,
-              "maxCapacity" : 100,
-              "absoluteCapacity" : 2,
-              "absoluteUsedCapacity" : 0,
-              "absoluteMaxCapacity" : 100,
-              "maxAMLimitPercentage" : 0,
-              "weight" : 1,
-              "normalizedWeight" : 0.02,
-              "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" : 24575,
-                "vCores" : 23,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 24575
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 23
-                  } ]
-                }
-              },
-              "effectiveMaxResource" : {
-                "memory" : 1228800,
-                "vCores" : 1200,
-                "resourceInformations" : {
-                  "resourceInformation" : [ {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "memory-mb",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "Mi",
-                    "value" : 1228800
-                  }, {
-                    "attributes" : { },
-                    "maximumAllocation" : 9223372036854775807,
-                    "minimumAllocation" : 0,
-                    "name" : "vcores",
-                    "resourceType" : "COUNTABLE",
-                    "units" : "",
-                    "value" : 1200
-                  } ]
-                }
-              }
-            } ]
-          },
-          "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" : 24575,
-            "vCores" : 23,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 24575
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 23
-              } ]
-            }
-          },
-          "maxEffectiveCapacity" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "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" : "parentAdmin1 "
-            }, {
-              "accessType" : "APPLICATION_MAX_PRIORITY",
-              "accessControlList" : "*"
-            }, {
-              "accessType" : "SUBMIT_APP",
-              "accessControlList" : "parentUser1 "
-            } ]
-          },
-          "queuePriority" : 0,
-          "orderingPolicyInfo" : "utilization",
-          "autoCreateChildQueueEnabled" : false,
-          "leafQueueTemplate" : { },
-          "mode" : "weight",
-          "queueType" : "parent",
-          "creationMethod" : "dynamicFlexible",
-          "autoCreationEligibility" : "flexible",
-          "autoQueueTemplateProperties" : { },
-          "autoQueueParentTemplateProperties" : { },
-          "autoQueueLeafTemplateProperties" : {
-            "property" : [ {
-              "name" : "acl_administer_queue",
-              "value" : "wildAdmin1"
-            }, {
-              "name" : "acl_submit_applications",
-              "value" : "wildUser1"
-            } ]
-          }
-        } ]
-      },
-      "capacities" : {
-        "queueCapacitiesByPartition" : [ {
-          "partitionName" : "",
-          "capacity" : 100,
-          "usedCapacity" : 0,
-          "maxCapacity" : 100,
-          "absoluteCapacity" : 100,
-          "absoluteUsedCapacity" : 0,
-          "absoluteMaxCapacity" : 100,
-          "maxAMLimitPercentage" : 0,
-          "weight" : 1,
-          "normalizedWeight" : 1,
-          "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" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          },
-          "effectiveMaxResource" : {
-            "memory" : 1228800,
-            "vCores" : 1200,
-            "resourceInformations" : {
-              "resourceInformation" : [ {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "memory-mb",
-                "resourceType" : "COUNTABLE",
-                "units" : "Mi",
-                "value" : 1228800
-              }, {
-                "attributes" : { },
-                "maximumAllocation" : 9223372036854775807,
-                "minimumAllocation" : 0,
-                "name" : "vcores",
-                "resourceType" : "COUNTABLE",
-                "units" : "",
-                "value" : 1200
-              } ]
-            }
-          }
-        } ]
-      },
-      "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" : "weight",
-      "queueType" : "parent",
-      "creationMethod" : "static",
-      "autoCreationEligibility" : "flexible",
-      "autoQueueTemplateProperties" : { },
-      "autoQueueParentTemplateProperties" : {
-        "property" : [ {
-          "name" : "acl_administer_queue",
-          "value" : "parentAdmin1"
-        }, {
-          "name" : "acl_submit_applications",
-          "value" : "parentUser1"
-        } ]
-      },
-      "autoQueueLeafTemplateProperties" : { }
-    }
-  }
-}

File diff suppressed because it is too large
+ 99 - 1662
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.json


File diff suppressed because it is too large
+ 105 - 1732
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response.xml


Some files were not shown because too many files changed in this diff