You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by bt...@apache.org on 2022/03/25 20:55:28 UTC

[hadoop] branch trunk updated: YARN-11069. Dynamic Queue ACL handling in Legacy and Flexible Auto Created Queues. Contributed by Tamas Domok

This is an automated email from the ASF dual-hosted git repository.

bteke pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new da09d68  YARN-11069. Dynamic Queue ACL handling in Legacy and Flexible Auto Created Queues. Contributed by Tamas Domok
da09d68 is described below

commit da09d68056d4e6a9490ddc6d9ae816b65217e117
Author: Tamas Domok <td...@cloudera.com>
AuthorDate: Thu Jan 27 09:57:52 2022 +0100

    YARN-11069. Dynamic Queue ACL handling in Legacy and Flexible Auto Created Queues. Contributed by Tamas Domok
---
 .../hadoop/yarn/security/PrivilegedEntity.java     |    5 +
 .../yarn/server/resourcemanager/RMAppManager.java  |   70 +-
 .../scheduler/capacity/AbstractCSQueue.java        |    8 +-
 .../scheduler/capacity/AbstractLeafQueue.java      |   15 +
 .../capacity/CapacitySchedulerConfiguration.java   |   84 +
 .../capacity/CapacitySchedulerQueueManager.java    |   43 +-
 .../scheduler/capacity/ParentQueue.java            |   12 +
 .../webapp/dao/CapacitySchedulerInfo.java          |    2 +-
 .../webapp/dao/CapacitySchedulerQueueInfo.java     |   10 +-
 .../server/resourcemanager/TestAppManager.java     |  701 +++++-
 ...estRMWebServicesCapacitySchedDynamicConfig.java |   51 +-
 ...er-response-AbsoluteModeLegacyAutoCreation.json |   24 +-
 .../webapp/scheduler-response-WeightMode.json      |  531 ++++-
 ...onse-WeightModeWithAutoCreatedQueues-After.json | 2397 ++++++++++++++++++--
 ...nse-WeightModeWithAutoCreatedQueues-Before.json |  449 +++-
 .../src/site/markdown/CapacityScheduler.md         |    8 +-
 16 files changed, 4045 insertions(+), 365 deletions(-)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java
index 580bdf4..3539c8f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java
@@ -45,6 +45,11 @@ public class PrivilegedEntity {
     this.name = name;
   }
 
+  public PrivilegedEntity(String name) {
+    this.type = EntityType.QUEUE;
+    this.name = name;
+  }
+
   public EntityType getType() {
     return type;
   }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 5bedef4..f847152 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -28,6 +28,10 @@ import java.util.concurrent.Future;
 
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.security.ConfiguredYarnAuthorizer;
+import org.apache.hadoop.yarn.security.Permission;
+import org.apache.hadoop.yarn.security.PrivilegedEntity;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -473,32 +477,33 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       if (scheduler instanceof CapacityScheduler) {
         String queueName = placementContext == null ?
             submissionContext.getQueue() : placementContext.getFullQueuePath();
-
-        String appName = submissionContext.getApplicationName();
-        CSQueue csqueue = ((CapacityScheduler) scheduler).getQueue(queueName);
-
-        if (csqueue == null && placementContext != null) {
-          //could be an auto created queue through queue mapping. Validate
-          // parent queue exists and has valid acls
-          String parentQueueName = placementContext.getParentQueue();
-          csqueue = ((CapacityScheduler) scheduler).getQueue(parentQueueName);
+        CapacityScheduler cs = (CapacityScheduler) scheduler;
+        CSQueue csqueue = cs.getQueue(queueName);
+        PrivilegedEntity privilegedEntity = new PrivilegedEntity(
+            csqueue == null ? queueName : csqueue.getQueuePath());
+
+        YarnAuthorizationProvider dynamicAuthorizer = null;
+        if (csqueue == null) {
+          List<Permission> permissions =
+              cs.getCapacitySchedulerQueueManager().getPermissionsForDynamicQueue(
+                  new QueuePath(queueName), cs.getConfiguration());
+          if (!permissions.isEmpty()) {
+            dynamicAuthorizer = new ConfiguredYarnAuthorizer();
+            dynamicAuthorizer.setPermission(permissions, userUgi);
+          }
         }
 
-        if (csqueue != null
-            && !authorizer.checkPermission(
-            new AccessRequest(csqueue.getPrivilegedEntity(), userUgi,
-                SchedulerUtils.toAccessType(QueueACL.SUBMIT_APPLICATIONS),
-                applicationId.toString(), appName, Server.getRemoteAddress(),
-                null))
-            && !authorizer.checkPermission(
-            new AccessRequest(csqueue.getPrivilegedEntity(), userUgi,
-                SchedulerUtils.toAccessType(QueueACL.ADMINISTER_QUEUE),
-                applicationId.toString(), appName, Server.getRemoteAddress(),
-                null))) {
-          throw RPCUtil.getRemoteException(new AccessControlException(
-              "User " + user + " does not have permission to submit "
-                  + applicationId + " to queue "
-                  + submissionContext.getQueue()));
+        if (csqueue != null || dynamicAuthorizer != null) {
+          String appName = submissionContext.getApplicationName();
+          if (!checkPermission(createAccessRequest(privilegedEntity, userUgi, applicationId,
+                  appName, QueueACL.SUBMIT_APPLICATIONS), dynamicAuthorizer) &&
+              !checkPermission(createAccessRequest(privilegedEntity, userUgi, applicationId,
+                  appName, QueueACL.ADMINISTER_QUEUE), dynamicAuthorizer)) {
+            throw RPCUtil.getRemoteException(new AccessControlException(
+                "User " + user + " does not have permission to submit "
+                    + applicationId + " to queue "
+                    + submissionContext.getQueue()));
+          }
         }
       }
       if (scheduler instanceof FairScheduler) {
@@ -572,6 +577,23 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     return application;
   }
 
+  private boolean checkPermission(AccessRequest accessRequest,
+                                  YarnAuthorizationProvider dynamicAuthorizer) {
+    return authorizer.checkPermission(accessRequest) ||
+        (dynamicAuthorizer != null && dynamicAuthorizer.checkPermission(accessRequest));
+  }
+
+  private static AccessRequest createAccessRequest(PrivilegedEntity privilegedEntity,
+                                                   UserGroupInformation userUgi,
+                                                   ApplicationId applicationId,
+                                                   String appName,
+                                                   QueueACL submitApplications) {
+    return new AccessRequest(privilegedEntity, userUgi,
+        SchedulerUtils.toAccessType(submitApplications),
+        applicationId.toString(), appName, Server.getRemoteAddress(),
+        null);
+  }
+
   private List<ResourceRequest> validateAndCreateResourceRequest(
       ApplicationSubmissionContext submissionContext, boolean isRecovery)
       throws InvalidResourceRequestException {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index 87fa9da..6caca65 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -341,8 +341,11 @@ public abstract class AbstractCSQueue implements CSQueue {
     writeLock.lock();
     try {
       CapacitySchedulerConfiguration configuration = queueContext.getConfiguration();
+      this.acls = configuration.getAcls(getQueuePath());
+
       if (isDynamicQueue() || this instanceof AbstractAutoCreatedLeafQueue) {
         setDynamicQueueProperties();
+        setDynamicQueueACLProperties();
       }
 
       // Collect and set the Node label configuration
@@ -369,8 +372,6 @@ public abstract class AbstractCSQueue implements CSQueue {
 
       authorizer = YarnAuthorizationProvider.getInstance(configuration);
 
-      this.acls = configuration.getAcls(getQueuePath());
-
       this.userWeights = getUserWeightsFromHierarchy();
 
       this.reservationsContinueLooking =
@@ -426,6 +427,9 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
   }
 
+  protected void setDynamicQueueACLProperties() {
+  }
+
   private UserWeights getUserWeightsFromHierarchy() {
     UserWeights unionInheritedWeights = UserWeights.createEmpty();
     CSQueue parentQ = parent;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java
index e194800..3d68633 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java
@@ -86,6 +86,8 @@ import org.apache.hadoop.classification.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedLeafQueue;
+
 public class AbstractLeafQueue extends AbstractCSQueue {
   private static final Logger LOG =
       LoggerFactory.getLogger(AbstractLeafQueue.class);
@@ -1697,6 +1699,19 @@ public class AbstractLeafQueue extends AbstractCSQueue {
     super.setDynamicQueueProperties();
   }
 
+  @Override
+  protected void setDynamicQueueACLProperties() {
+    super.setDynamicQueueACLProperties();
+
+    if (parent instanceof AbstractManagedParentQueue) {
+      acls.putAll(queueContext.getConfiguration().getACLsForLegacyAutoCreatedLeafQueue(
+          parent.getQueuePath()));
+    } else if (parent instanceof ParentQueue) {
+      acls.putAll(getACLsForFlexibleAutoCreatedLeafQueue(
+          ((ParentQueue) parent).getAutoCreatedQueueTemplate()));
+    }
+  }
+
   private void updateSchedulerHealthForCompletedContainer(
       RMContainer rmContainer, ContainerStatus containerStatus) {
     // Update SchedulerHealth for released / preempted container
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
index a9e8423..51616da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
@@ -941,6 +941,90 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     return "acl_" + StringUtils.toLowerCase(acl.toString());
   }
 
+  /**
+   * Creates a mapping of queue ACLs for a Legacy Auto Created Leaf Queue.
+   *
+   * @param parentQueuePath the parent's queue path
+   * @return A mapping of the queue ACLs.
+   */
+  public Map<AccessType, AccessControlList> getACLsForLegacyAutoCreatedLeafQueue(
+      String parentQueuePath) {
+    final String prefix =
+        getQueuePrefix(getAutoCreatedQueueTemplateConfPrefix(
+            parentQueuePath));
+
+    Map<String, String> properties = new HashMap<>();
+    for (QueueACL acl : QueueACL.values()) {
+      final String key = getAclKey(acl);
+      final String value = get(prefix + key);
+      if (value != null) {
+        properties.put(key, get(prefix + key));
+      }
+    }
+    return getACLsFromProperties(properties);
+  }
+
+  /**
+   * Creates a mapping of queue ACLs for a Flexible Auto Created Parent Queue.
+   * The .parent-template is preferred to .template ACLs.
+   *
+   * @param aqc The AQC templates to use.
+   * @return A mapping of the queue ACLs.
+   */
+  public static Map<AccessType, AccessControlList> getACLsForFlexibleAutoCreatedParentQueue(
+      AutoCreatedQueueTemplate aqc) {
+    return getACLsFromProperties(aqc.getParentOnlyProperties(),
+        aqc.getTemplateProperties());
+  }
+
+  /**
+   * Creates a mapping of queue ACLs for a Flexible Auto Created Leaf Queue.
+   * The .leaf-template is preferred to .template ACLs.
+   *
+   * @param aqc The AQC templates to use.
+   * @return A mapping of the queue ACLs.
+   */
+  public static Map<AccessType, AccessControlList> getACLsForFlexibleAutoCreatedLeafQueue(
+      AutoCreatedQueueTemplate aqc) {
+    return getACLsFromProperties(aqc.getLeafOnlyProperties(),
+        aqc.getTemplateProperties());
+  }
+
+  /**
+   * Transforms the string ACL properties to AccessType and AccessControlList mapping.
+   *
+   * @param properties The ACL properties.
+   * @return A mapping of the queue ACLs.
+   */
+  private static Map<AccessType, AccessControlList> getACLsFromProperties(
+      Map<String, String> properties) {
+    return getACLsFromProperties(properties, new HashMap<>());
+  }
+
+  /**
+   * Transforms the string ACL properties to AccessType and AccessControlList mapping.
+   *
+   * @param properties The ACL properties.
+   * @param fallbackProperties The fallback properties to use.
+   * @return A mapping of the queue ACLs.
+   */
+  private static Map<AccessType, AccessControlList> getACLsFromProperties(
+      Map<String, String> properties, Map<String, String> fallbackProperties) {
+    Map<AccessType, AccessControlList> acls = new HashMap<>();
+    for (QueueACL acl : QueueACL.values()) {
+      String aclStr = properties.get(getAclKey(acl));
+      if (aclStr == null) {
+        aclStr = fallbackProperties.get(getAclKey(acl));
+        if (aclStr == null) {
+          aclStr = NONE_ACL;
+        }
+      }
+      acls.put(SchedulerUtils.toAccessType(acl),
+          new AccessControlList(aclStr));
+    }
+    return acls;
+  }
+
   @Override
   public Map<ReservationACL, AccessControlList> getReservationAcls(String
         queue) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java
index 29e5de0..ba6849c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java
@@ -23,13 +23,13 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -52,6 +52,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsMan
 
 import org.apache.hadoop.classification.VisibleForTesting;
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedLeafQueue;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedParentQueue;
+
 /**
  *
  * Context of the Queues in Capacity Scheduler.
@@ -596,6 +599,44 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
     return parentsToCreate;
   }
 
+  public List<Permission> getPermissionsForDynamicQueue(
+      QueuePath queuePath,
+      CapacitySchedulerConfiguration csConf) {
+    List<Permission> permissions = new ArrayList<>();
+
+    try {
+      PrivilegedEntity privilegedEntity = new PrivilegedEntity(queuePath.getFullPath());
+
+      CSQueue parentQueue = getQueueByFullName(queuePath.getParent());
+      if (parentQueue == null) {
+        for (String missingParent : determineMissingParents(queuePath)) {
+          String parentOfMissingParent = new QueuePath(missingParent).getParent();
+          permissions.add(new Permission(new PrivilegedEntity(missingParent),
+              getACLsForFlexibleAutoCreatedParentQueue(
+                  new AutoCreatedQueueTemplate(csConf,
+                      new QueuePath(parentOfMissingParent)))));
+        }
+      }
+
+      if (parentQueue instanceof AbstractManagedParentQueue) {
+        // An AbstractManagedParentQueue must have been found for Legacy AQC
+        permissions.add(new Permission(privilegedEntity,
+            csConf.getACLsForLegacyAutoCreatedLeafQueue(queuePath.getParent())));
+      } else {
+        // Every other case must be a Flexible Leaf Queue
+        permissions.add(new Permission(privilegedEntity,
+            getACLsForFlexibleAutoCreatedLeafQueue(
+                new AutoCreatedQueueTemplate(csConf, new QueuePath(queuePath.getParent())))));
+      }
+
+    } catch (SchedulerDynamicEditException e) {
+      LOG.debug("Could not determine missing parents for queue {} reason {}",
+          queuePath.getFullPath(), e.getMessage());
+    }
+
+    return permissions;
+  }
+
   /**
    * Get {@code ConfiguredNodeLabels} which contains the configured node labels
    * for all queues.
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
index 671ed21..0949d51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
@@ -78,6 +78,8 @@ import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedParentQueue;
+
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -188,6 +190,16 @@ public class ParentQueue extends AbstractCSQueue {
     }
   }
 
+  @Override
+  protected void setDynamicQueueACLProperties() {
+    super.setDynamicQueueACLProperties();
+
+    if (parent instanceof ParentQueue) {
+      acls.putAll(getACLsForFlexibleAutoCreatedParentQueue(
+          ((ParentQueue) parent).getAutoCreatedQueueTemplate()));
+    }
+  }
+
   private static float PRECISION = 0.0005f; // 0.05% precision
 
   // Check weight configuration, throw exception when configuration is invalid
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
index 3d410ec..83394b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
@@ -97,7 +97,7 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
 
     CapacitySchedulerConfiguration conf = cs.getConfiguration();
     queueAcls = new QueueAclsInfo();
-    queueAcls.addAll(getSortedQueueAclInfoList(queueName, conf));
+    queueAcls.addAll(getSortedQueueAclInfoList(parent, queueName, conf));
 
     queuePriority = parent.getPriority().getPriority();
     if (parent instanceof ParentQueue) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index 78b5392..bda69c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -157,7 +157,7 @@ public class CapacitySchedulerQueueInfo {
 
     CapacitySchedulerConfiguration conf = cs.getConfiguration();
     queueAcls = new QueueAclsInfo();
-    queueAcls.addAll(getSortedQueueAclInfoList(queuePath, conf));
+    queueAcls.addAll(getSortedQueueAclInfoList(q, queuePath, conf));
 
     queuePriority = q.getPriority().getPriority();
     if (q instanceof ParentQueue) {
@@ -183,11 +183,11 @@ public class CapacitySchedulerQueueInfo {
     leafQueueTemplate = new LeafQueueTemplateInfo(conf, queuePath);
   }
 
-  public static ArrayList<QueueAclInfo> getSortedQueueAclInfoList(String queuePath,
-                                                             CapacitySchedulerConfiguration conf) {
+  public static ArrayList<QueueAclInfo> getSortedQueueAclInfoList(
+      CSQueue queue, String queuePath, CapacitySchedulerConfiguration conf) {
     ArrayList<QueueAclInfo> queueAclsInfo = new ArrayList<>();
-    for (Map.Entry<AccessType, AccessControlList> e : conf
-        .getAcls(queuePath).entrySet()) {
+    for (Map.Entry<AccessType, AccessControlList> e :
+        ((AbstractCSQueue) queue).getACLs().entrySet()) {
       QueueAclInfo queueAcl = new QueueAclInfo(e.getKey().toString(),
           e.getValue().getAclString());
       queueAclsInfo.add(queueAcl);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
index f9ecc85..cc5e8af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
@@ -68,9 +68,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessM
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedLeafQueue;
 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.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -108,7 +112,11 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentMap;
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.AUTO_QUEUE_LEAF_TEMPLATE_PREFIX;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.AUTO_QUEUE_PARENT_TEMPLATE_PREFIX;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueuePrefix;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
@@ -141,6 +149,7 @@ public class TestAppManager extends AppManagerTestBase{
   private ResourceScheduler scheduler;
 
   private static final String USER_ID_PREFIX = "userid=";
+  private static final String ROOT_PARENT =  PREFIX + "root.parent.";
 
   public synchronized RMAppEventType getAppEventType() {
     return appEventType;
@@ -307,11 +316,7 @@ public class TestAppManager extends AppManagerTestBase{
   @Test
   public void testQueueSubmitWithACLsEnabledWithQueueMapping()
       throws YarnException {
-    YarnConfiguration conf = new YarnConfiguration(new Configuration(false));
-    conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
     CapacitySchedulerConfiguration csConf = new
         CapacitySchedulerConfiguration(conf, false);
     csConf.set(PREFIX + "root.queues", "default,test");
@@ -331,35 +336,29 @@ public class TestAppManager extends AppManagerTestBase{
     csConf.set(PREFIX + "root.test.acl_submit_applications", "test");
     csConf.set(PREFIX + "root.test.acl_administer_queue", "test");
 
-    asContext.setQueue("oldQueue");
-
     MockRM newMockRM = new MockRM(csConf);
     RMContext newMockRMContext = newMockRM.getRMContext();
-    newMockRMContext.setQueuePlacementManager(createMockPlacementManager("test", "test", null));
+    newMockRMContext.setQueuePlacementManager(
+        createMockPlacementManager("test", "root.test", null));
     TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
 
-    newAppMonitor.submitApplication(asContext, "test");
-    RMApp app = newMockRMContext.getRMApps().get(appId);
-    Assert.assertNotNull("app should not be null", app);
-    Assert.assertEquals("the queue should be placed on 'test' queue", "test", app.getQueue());
-
-    try {
-      asContext.setApplicationId(appId = MockApps.newAppID(2));
-      newAppMonitor.submitApplication(asContext, "test1");
-      Assert.fail("should fail since test1 does not have permission to submit to queue");
-    } catch(YarnException e) {
-      assertTrue(e.getCause() instanceof AccessControlException);
-    }
+    ApplicationSubmissionContext submission = createAppSubmissionContext(MockApps.newAppID(1));
+    submission.setQueue("oldQueue");
+    verifyAppSubmission(submission,
+        newAppMonitor,
+        newMockRMContext,
+        "test",
+        "root.test");
+
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(2)),
+        "test1");
   }
 
   @Test
-  public void testQueueSubmitWithACLsEnabledWithQueueMappingForAutoCreatedQueue()
+  public void testQueueSubmitWithACLsEnabledWithQueueMappingForLegacyAutoCreatedQueue()
       throws IOException, YarnException {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
     CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(
         conf, false);
     csConf.set(PREFIX + "root.queues", "default,managedparent");
@@ -383,8 +382,6 @@ public class TestAppManager extends AppManagerTestBase{
     csConf.setAutoCreatedLeafQueueConfigCapacity("root.managedparent", 30f);
     csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.managedparent", 100f);
 
-    asContext.setQueue("oldQueue");
-
     MockRM newMockRM = new MockRM(csConf);
     CapacityScheduler cs =
         ((CapacityScheduler) newMockRM.getResourceScheduler());
@@ -395,25 +392,653 @@ public class TestAppManager extends AppManagerTestBase{
 
     RMContext newMockRMContext = newMockRM.getRMContext();
     newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
-        "user1|user2", "user1", "managedparent"));
+        "user1|user2", "user1", "root.managedparent"));
     TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
 
-    newAppMonitor.submitApplication(asContext, "user1");
-    RMApp app = newMockRMContext.getRMApps().get(appId);
-    Assert.assertNotNull("app should not be null", app);
-    Assert.assertEquals("the queue should be placed on 'managedparent.user1' queue",
-        "managedparent.user1",
-        app.getQueue());
+    ApplicationSubmissionContext submission = createAppSubmissionContext(MockApps.newAppID(1));
+    submission.setQueue("oldQueue");
+    verifyAppSubmission(submission,
+        newAppMonitor,
+        newMockRMContext,
+        "user1",
+        "root.managedparent.user1");
+
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(2)),
+        "user2");
+  }
+
+  @Test
+  public void testLegacyAutoCreatedQueuesWithACLTemplates()
+      throws IOException, YarnException {
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(
+        conf, false);
+    csConf.set(PREFIX + "root.queues", "parent");
+    csConf.set(PREFIX + "root.acl_submit_applications", " ");
+    csConf.set(PREFIX + "root.acl_administer_queue", " ");
+
+    csConf.setCapacity("root.parent", 100.0f);
+    csConf.set(PREFIX + "root.parent.acl_administer_queue", "user1,user4");
+    csConf.set(PREFIX + "root.parent.acl_submit_applications", "user1,user4");
+
+    csConf.setAutoCreateChildQueueEnabled("root.parent", true);
+    csConf.setAutoCreatedLeafQueueConfigCapacity("root.parent", 50f);
+    csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.parent", 100f);
+    csConf.set(getQueuePrefix(csConf.getAutoCreatedQueueTemplateConfPrefix("root.parent")) +
+        "acl_administer_queue", "user2,user4");
+    csConf.set(getQueuePrefix(csConf.getAutoCreatedQueueTemplateConfPrefix("root.parent")) +
+        "acl_submit_applications", "user2,user4");
+
+    MockRM newMockRM = new MockRM(csConf);
+
+    RMContext newMockRMContext = newMockRM.getRMContext();
+    TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
+
+    // user1 has permission on root.parent so a queue would be created
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user1", "user1", "root.parent"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)),
+        newAppMonitor,
+        newMockRMContext,
+        "user1",
+        "root.parent.user1");
+
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user1|user2|user3|user4", "user2", "root.parent"));
+
+    // user2 has permission (due to ACL templates)
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)),
+        newAppMonitor,
+        newMockRMContext,
+        "user2",
+        "root.parent.user2");
+
+    // user3 doesn't have permission
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(3)),
+        "user3");
+
+    // user4 has permission on root.parent
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(4)),
+        newAppMonitor,
+        newMockRMContext,
+        "user4",
+        "root.parent.user2");
+
+    // create the root.parent.user2 manually
+    CapacityScheduler cs =
+        ((CapacityScheduler) newMockRM.getResourceScheduler());
+    cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2"));
+    AutoCreatedLeafQueue autoCreatedLeafQueue = (AutoCreatedLeafQueue) cs.getQueue("user2");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue("parent");
+    assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
+    // reinitialize to load the ACLs for the queue
+    cs.reinitialize(csConf, newMockRMContext);
+
+    // template ACLs do work after reinitialize
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(5)),
+        newAppMonitor,
+        newMockRMContext,
+        "user2",
+        "root.parent.user2");
+
+    // user3 doesn't have permission for root.parent.user2 queue
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(6)),
+        "user3");
+
+    // user1 doesn't have permission for root.parent.user2 queue, but it has for root.parent
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(7)),
+        newAppMonitor,
+        newMockRMContext,
+        "user1",
+        "root.parent.user2");
+  }
+
+  @Test
+  public void testFlexibleAutoCreatedQueuesWithSpecializedACLTemplatesAndDynamicParentQueue()
+      throws IOException, YarnException {
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
+    CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf);
+
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user2");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user2");
+
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user3");
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user3");
+
+    MockRM newMockRM = new MockRM(csConf);
+
+    RMContext newMockRMContext = newMockRM.getRMContext();
+    TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
+
+    // user1 has permission on root.parent so a queue would be created
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user1", "user1", "root.parent"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)),
+        newAppMonitor,
+        newMockRMContext,
+        "user1",
+        "root.parent.user1");
+
+    // user2 doesn't have permission to create a dynamic leaf queue (parent only template)
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user2", "user2", "root.parent"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(2)),
+        "user2");
+
+    // user3 has permission on root.parent.user2.user3 due to ACL templates
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "user3", "root.parent.user2"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(3)),
+        newAppMonitor,
+        newMockRMContext,
+        "user3",
+        "root.parent.user2.user3");
+
+    // user4 doesn't have permission
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user4", "user4", "root.parent.user2"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(4)),
+        "user4");
+
+    // create the root.parent.user2.user3 manually
+    CapacityScheduler cs =
+        ((CapacityScheduler) newMockRM.getResourceScheduler());
+    cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2.user3"));
+
+    ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue);
+    ParentQueue parentQueue = (ParentQueue) cs.getQueue("parent");
+    assertEquals(parentQueue, autoCreatedParentQueue.getParent());
+
+    LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent());
+
+    // reinitialize to load the ACLs for the queue
+    cs.reinitialize(csConf, newMockRMContext);
+
+    // template ACLs do work after reinitialize
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "user3", "root.parent.user2"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(5)),
+        newAppMonitor,
+        newMockRMContext,
+        "user3",
+        "root.parent.user2.user3");
+
+    // user4 doesn't have permission
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user4", "user4", "root.parent.user2"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(6)),
+        "user4");
+  }
+
+  @Test
+  public void testFlexibleAutoCreatedQueuesWithMixedCommonLeafACLTemplatesAndDynamicParentQueue()
+      throws IOException, YarnException {
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
+    CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf);
+
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user2");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user2");
+
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user3");
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user3");
+
+    testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario(conf, csConf);
+  }
+
+  @Test
+  public void testFlexibleAutoCreatedQueuesWithMixedCommonCommonACLTemplatesAndDynamicParentQueue()
+      throws IOException, YarnException {
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
+    CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf);
+
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user2");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user2");
+
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user3");
+    csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user3");
+
+    testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario(conf, csConf);
+  }
+
+  private void testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario(
+      YarnConfiguration conf, CapacitySchedulerConfiguration csConf)
+      throws YarnException, IOException {
+    MockRM newMockRM = new MockRM(csConf);
+
+    RMContext newMockRMContext = newMockRM.getRMContext();
+    TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
+
+    // user1 has permission on root.parent so a queue would be created
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user1", "user1", "root.parent"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)),
+        newAppMonitor,
+        newMockRMContext,
+        "user1",
+        "root.parent.user1");
+
+    // user2 has permission on root.parent a dynamic leaf queue would be created
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user2", "user2", "root.parent"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)),
+        newAppMonitor,
+        newMockRMContext,
+        "user2",
+        "root.parent.user2");
+
+    // user3 has permission on root.parent.user2.user3 a dynamic parent and leaf queue
+    // would be created
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "user3", "root.parent.user2"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(3)),
+        newAppMonitor,
+        newMockRMContext,
+        "user3",
+        "root.parent.user2.user3");
+
+    // user4 doesn't have permission
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user4", "user4", "root.parent.user2"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(4)),
+        "user4");
+
+    // create the root.parent.user2.user3 manually
+    CapacityScheduler cs =
+        ((CapacityScheduler) newMockRM.getResourceScheduler());
+    cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2.user3"));
+
+    ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue);
+    ParentQueue parentQueue = (ParentQueue) cs.getQueue("parent");
+    assertEquals(parentQueue, autoCreatedParentQueue.getParent());
+
+    LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent());
+
+    // reinitialize to load the ACLs for the queue
+    cs.reinitialize(csConf, newMockRMContext);
+
+    // template ACLs do work after reinitialize
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "user3", "root.parent.user2"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(5)),
+        newAppMonitor,
+        newMockRMContext,
+        "user3",
+        "root.parent.user2.user3");
+
+    // user4 doesn't have permission
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user4", "user4", "root.parent.user2"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(6)),
+        "user4");
+  }
+
+  @Test
+  public void testFlexibleAutoCreatedQueuesWithACLTemplatesALeafOnly()
+      throws IOException, YarnException {
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
+    CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf);
+
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user2");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user2");
+
+    testFlexibleAQCLeafOnly(conf, csConf);
+  }
+
+  @Test
+  public void testFlexibleAutoCreatedQueuesWithSpecialisedACLTemplatesALeafOnly()
+      throws IOException, YarnException {
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
+    CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf);
+
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user2");
+    csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user2");
+
+    testFlexibleAQCLeafOnly(conf, csConf);
+  }
+
+  private void testFlexibleAQCLeafOnly(
+      YarnConfiguration conf,
+      CapacitySchedulerConfiguration csConf)
+      throws YarnException, IOException {
+    MockRM newMockRM = new MockRM(csConf);
+    RMContext newMockRMContext = newMockRM.getRMContext();
+    TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
+
+    // user1 has permission on root.parent so a queue would be created
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user1", "user1", "root.parent"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)),
+        newAppMonitor,
+        newMockRMContext,
+        "user1",
+        "root.parent.user1");
+
+    // user2 has permission on root.parent.user2 due to ACL templates
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user2", "user2", "root.parent"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)),
+        newAppMonitor,
+        newMockRMContext,
+        "user2",
+        "root.parent.user2");
+
+    // user3 doesn't have permission
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "user3", "root.parent"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(3)),
+        "user3");
+
+    // create the root.parent.user2 manually
+    CapacityScheduler cs =
+        ((CapacityScheduler) newMockRM.getResourceScheduler());
+    cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2"));
 
+    ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("parent");
+    LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user2");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent());
+
+    // reinitialize to load the ACLs for the queue
+    cs.reinitialize(csConf, newMockRMContext);
+
+    // template ACLs do work after reinitialize
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user2", "user2", "root.parent"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(4)),
+        newAppMonitor,
+        newMockRMContext,
+        "user2",
+        "root.parent.user2");
+
+    // user3 doesn't have permission
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "user3", "root.parent"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(5)),
+        "user3");
+  }
+
+  @Test
+  public void testFlexibleAutoCreatedQueuesWithSpecializedACLTemplatesAndDynamicRootParentQueue()
+      throws IOException, YarnException {
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
+
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(
+        conf, false);
+    csConf.set(PREFIX + "root.queues", "");
+    csConf.set(PREFIX + "root.acl_submit_applications", "user1");
+    csConf.set(PREFIX + "root.acl_administer_queue", "admin1");
+
+    csConf.setAutoQueueCreationV2Enabled("root", true);
+
+    csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user2");
+    csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user2");
+
+    csConf.set(PREFIX + "root." + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(PREFIX + "root." + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user3");
+    csConf.set(PREFIX + "root." + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX +
+            "acl_submit_applications",
+        "user3");
+
+    MockRM newMockRM = new MockRM(csConf);
+
+    RMContext newMockRMContext = newMockRM.getRMContext();
+    TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
+
+    // user1 has permission on root so a queue would be created
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user1", "user1", "root"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)),
+        newAppMonitor,
+        newMockRMContext,
+        "user1",
+        "root.user1");
+
+    // user2 doesn't have permission to create a dynamic leaf queue (parent only template)
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user2", "user2", "root"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(2)),
+        "user2");
+
+    // user3 has permission on root.user2.user3 due to ACL templates
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "user3", "root.user2"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(3)),
+        newAppMonitor,
+        newMockRMContext,
+        "user3",
+        "root.user2.user3");
+
+    // user4 doesn't have permission
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user4", "user4", "root.user2"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(4)),
+        "user4");
+
+    // create the root.user2.user3 manually
+    CapacityScheduler cs =
+        ((CapacityScheduler) newMockRM.getResourceScheduler());
+    cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.user2.user3"));
+
+    ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue);
+    ParentQueue parentQueue = (ParentQueue) cs.getQueue("root");
+    assertEquals(parentQueue, autoCreatedParentQueue.getParent());
+
+    LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent());
+
+    // reinitialize to load the ACLs for the queue
+    cs.reinitialize(csConf, newMockRMContext);
+
+    // template ACLs do work after reinitialize
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "user3", "root.user2"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(5)),
+        newAppMonitor,
+        newMockRMContext,
+        "user3",
+        "root.user2.user3");
+
+    // user4 doesn't have permission
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user4", "user4", "root.user2"));
+    verifyAppSubmissionFailure(newAppMonitor,
+        createAppSubmissionContext(MockApps.newAppID(6)),
+        "user4");
+  }
+
+  @Test
+  public void testFlexibleAutoCreatedQueuesMultiLevelDynamicParentACL()
+      throws IOException, YarnException {
+    YarnConfiguration conf = createYarnACLEnabledConfiguration();
+
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(
+        conf, false);
+    csConf.set(PREFIX + "root.queues", "");
+    csConf.set(PREFIX + "root.acl_submit_applications", "user1");
+    csConf.set(PREFIX + "root.acl_administer_queue", "admin1");
+
+    csConf.setAutoQueueCreationV2Enabled("root", true);
+
+    csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_administer_queue",
+        "user2");
+    csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_submit_applications",
+        "user2");
+
+    csConf.set(PREFIX + "root." + "user2.user3." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity",
+        "1w");
+    csConf.set(PREFIX + "root." + "user2.user3." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX +
+            "acl_administer_queue",
+        "user3");
+    csConf.set(PREFIX + "root." + "user2.user3." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX +
+            "acl_submit_applications",
+        "user3");
+    csConf.setMaximumAutoCreatedQueueDepth(4);
+
+    MockRM newMockRM = new MockRM(csConf);
+
+    RMContext newMockRMContext = newMockRM.getRMContext();
+    TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
+
+    // user3 has permission on root.user2.user3.queue due to ACL templates
+    newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
+        "user3", "queue", "root.user2.user3"));
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)),
+        newAppMonitor,
+        newMockRMContext,
+        "user3",
+        "root.user2.user3.queue");
+
+    // create the root.user2.user3.queue manually
+    CapacityScheduler cs =
+        ((CapacityScheduler) newMockRM.getResourceScheduler());
+    cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.user2.user3.queue"));
+
+    ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue);
+    ParentQueue parentQueue = (ParentQueue) cs.getQueue("root");
+    assertEquals(parentQueue, autoCreatedParentQueue.getParent());
+
+    ParentQueue autoCreatedParentQueue2 = (ParentQueue) cs.getQueue("user3");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue2);
+    assertEquals(autoCreatedParentQueue, autoCreatedParentQueue2.getParent());
+
+    LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("queue");
+    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertEquals(autoCreatedParentQueue, autoCreatedParentQueue2.getParent());
+
+    // reinitialize to load the ACLs for the queue
+    cs.reinitialize(csConf, newMockRMContext);
+
+    // template ACLs do work after reinitialize
+    verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)),
+        newAppMonitor,
+        newMockRMContext,
+        "user3",
+        "root.user2.user3.queue");
+  }
+
+  private YarnConfiguration createYarnACLEnabledConfiguration() {
+    YarnConfiguration conf = new YarnConfiguration(new Configuration(false));
+    conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    return conf;
+  }
+
+  private CapacitySchedulerConfiguration createFlexibleAQCBaseACLConfiguration(
+      YarnConfiguration conf) {
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(
+        conf, false);
+    csConf.set(PREFIX + "root.queues", "parent");
+    csConf.set(PREFIX + "root.acl_submit_applications", " ");
+    csConf.set(PREFIX + "root.acl_administer_queue", " ");
+
+    csConf.setCapacity("root.parent", "1w");
+    csConf.set(PREFIX + "root.parent.acl_administer_queue", "user1");
+    csConf.set(PREFIX + "root.parent.acl_submit_applications", "user1");
+
+    csConf.setAutoQueueCreationV2Enabled("root.parent", true);
+    return csConf;
+  }
+
+  private static void verifyAppSubmissionFailure(TestRMAppManager appManager,
+                                                 ApplicationSubmissionContext submission,
+                                                 String user) {
     try {
-      asContext.setApplicationId(appId = MockApps.newAppID(2));
-      newAppMonitor.submitApplication(asContext, "user2");
-      Assert.fail("should fail since user2 does not have permission to submit to queue");
+      appManager.submitApplication(submission, user);
+      Assert.fail(
+          String.format("should fail since %s does not have permission to submit to queue", user));
     } catch (YarnException e) {
       assertTrue(e.getCause() instanceof AccessControlException);
     }
   }
 
+  private static void verifyAppSubmission(ApplicationSubmissionContext submission,
+                                          TestRMAppManager appManager,
+                                          RMContext rmContext,
+                                          String user,
+                                          String expectedQueue) throws YarnException {
+    appManager.submitApplication(submission, user);
+    RMApp app = rmContext.getRMApps().get(submission.getApplicationId());
+    Assert.assertNotNull("app should not be null", app);
+    Assert.assertEquals(String.format("the queue should be placed on '%s' queue", expectedQueue),
+        expectedQueue,
+        app.getQueue());
+  }
+
+  private static ApplicationSubmissionContext createAppSubmissionContext(ApplicationId id) {
+    RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+    ApplicationSubmissionContext appSubmission =
+        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
+    appSubmission.setApplicationId(id);
+    appSubmission.setAMContainerSpec(mockContainerLaunchContext(recordFactory));
+    appSubmission.setResource(mockResource());
+    appSubmission.setPriority(Priority.newInstance(0));
+    appSubmission.setQueue("default");
+    return appSubmission;
+  }
+
   @After
   public void tearDown() {
     setAppEventType(RMAppEventType.KILL);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java
index 4cc50c4..f2336e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java
@@ -162,14 +162,9 @@ public class TestRMWebServicesCapacitySchedDynamicConfig extends
     createQueue("root.auto2");
     createQueue("root.auto3");
     createQueue("root.autoParent1.auto4");
+    createQueue("root.autoParent2.auto5");
+    createQueue("root.parent.autoParent3.auto6");
 
-    /*
-     *                         root   default  test1  test2  autoParent1  auto1  auto2  auto3  auto4
-     * weight:                  1        10      4     6         1          1      1     1      1
-     * normalizedWeight:        1        0.41    0.16  0.25      1          0.04   0.04  0.04   0.04
-     * autoCreationEligibility: flexible off     off   off     flexible     off    off   off    off
-     * queueType:               parent   leaf    leaf  leaf    parent       leaf   leaf  leaf   leaf
-     */
     assertJsonResponse(sendRequest(),
         "webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json");
   }
@@ -227,6 +222,10 @@ public class TestRMWebServicesCapacitySchedDynamicConfig extends
       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);
     }
 
@@ -256,17 +255,45 @@ public class TestRMWebServicesCapacitySchedDynamicConfig extends
 
     private static Configuration createWeightConfigInternal(boolean enableAqc) {
       Map<String, String> conf = new HashMap<>();
-      conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2");
+      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", "4w");
-      conf.put("yarn.scheduler.capacity.root.test2.capacity", "6w");
+      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) {
-        conf.put("yarn.scheduler.capacity.root.auto-queue-creation-v2.enabled",
-            "true");
+        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);
     }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json
index 7580005..719aa81 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json
@@ -885,7 +885,7 @@
         "queueAcls": {"queueAcl": [
           {
             "accessType": "ADMINISTER_QUEUE",
-            "accessControlList": " "
+            "accessControlList": "admin "
           },
           {
             "accessType": "APPLICATION_MAX_PRIORITY",
@@ -893,7 +893,7 @@
           },
           {
             "accessType": "SUBMIT_APP",
-            "accessControlList": " "
+            "accessControlList": "user "
           }
         ]},
         "queuePriority": 0,
@@ -1296,10 +1296,20 @@
       "queuePriority": 0,
       "orderingPolicyInfo": "utilization",
       "autoCreateChildQueueEnabled": true,
-      "leafQueueTemplate": {"property": [{
-        "name": "leaf-queue-template.capacity",
-        "value": "[memory=2048,vcores=2]"
-      }]},
+      "leafQueueTemplate": {"property": [
+        {
+          "name": "leaf-queue-template.acl_administer_queue",
+          "value": "admin"
+        },
+        {
+          "name": "leaf-queue-template.capacity",
+          "value": "[memory=2048,vcores=2]"
+        },
+        {
+          "name": "leaf-queue-template.acl_submit_applications",
+          "value": "user"
+        }
+      ]},
       "mode": "absolute",
       "queueType": "parent",
       "creationMethod": "static",
@@ -1579,4 +1589,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json
index b1894c58..1f0d07f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json
@@ -16,11 +16,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 30.000002,
+      "absoluteCapacity": 22.222223,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
-      "weight": 6,
-      "normalizedWeight": 0.3,
+      "weight": 10,
+      "normalizedWeight": 0.22222222,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "test2",
@@ -60,12 +60,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 30.000002,
+        "absoluteCapacity": 22.222223,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
-        "weight": 6,
-        "normalizedWeight": 0.3,
+        "weight": 10,
+        "normalizedWeight": 0.22222222,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -410,8 +410,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 3000,
-      "maxApplicationsPerUser": 3000,
+      "maxApplications": 2222,
+      "maxApplicationsPerUser": 2222,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -501,11 +501,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 20,
+      "absoluteCapacity": 11.111112,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
-      "weight": 4,
-      "normalizedWeight": 0.2,
+      "weight": 5,
+      "normalizedWeight": 0.11111111,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "test1",
@@ -545,12 +545,497 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 20,
+        "absoluteCapacity": 11.111112,
+        "absoluteUsedCapacity": 0,
+        "absoluteMaxCapacity": 100,
+        "maxAMLimitPercentage": 10,
+        "weight": 5,
+        "normalizedWeight": 0.11111111,
+        "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": "weight",
+      "queueType": "leaf",
+      "creationMethod": "static",
+      "autoCreationEligibility": "off",
+      "autoQueueTemplateProperties": {},
+      "autoQueueParentTemplateProperties": {},
+      "autoQueueLeafTemplateProperties": {},
+      "numActiveApplications": 0,
+      "numPendingApplications": 0,
+      "numContainers": 0,
+      "maxApplications": 1111,
+      "maxApplicationsPerUser": 1111,
+      "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.parent",
+      "capacity": 0,
+      "usedCapacity": 0,
+      "maxCapacity": 100,
+      "absoluteCapacity": 44.444447,
+      "absoluteMaxCapacity": 100,
+      "absoluteUsedCapacity": 0,
+      "weight": 20,
+      "normalizedWeight": 0.44444445,
+      "numApplications": 0,
+      "maxParallelApps": 2147483647,
+      "queueName": "parent",
+      "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": 44.444447,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
-        "weight": 4,
-        "normalizedWeight": 0.2,
+        "weight": 20,
+        "normalizedWeight": 0.44444445,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -895,8 +1380,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 2000,
-      "maxApplicationsPerUser": 2000,
+      "maxApplications": 4444,
+      "maxApplicationsPerUser": 4444,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -986,11 +1471,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 50,
+      "absoluteCapacity": 22.222223,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "weight": 10,
-      "normalizedWeight": 0.5,
+      "normalizedWeight": 0.22222222,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "default",
@@ -1030,12 +1515,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 50,
+        "absoluteCapacity": 22.222223,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "weight": 10,
-        "normalizedWeight": 0.5,
+        "normalizedWeight": 0.22222222,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -1380,8 +1865,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 5000,
-      "maxApplicationsPerUser": 5000,
+      "maxApplications": 2222,
+      "maxApplicationsPerUser": 2222,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -1736,4 +2221,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json
index 188b72a..a9c30d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json
@@ -16,11 +16,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 25,
+      "absoluteCapacity": 20,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
-      "weight": 6,
-      "normalizedWeight": 0.25,
+      "weight": 10,
+      "normalizedWeight": 0.2,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "test2",
@@ -60,12 +60,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 25,
+        "absoluteCapacity": 20,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
-        "weight": 6,
-        "normalizedWeight": 0.25,
+        "weight": 10,
+        "normalizedWeight": 0.2,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -115,8 +115,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 307200,
-          "vCores": 300,
+          "memory": 245760,
+          "vCores": 240,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -125,7 +125,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 307200
+              "value": 245760
             },
             {
               "attributes": {},
@@ -134,7 +134,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 300
+              "value": 240
             }
           ]}
         },
@@ -311,8 +311,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 307200,
-        "vCores": 300,
+        "memory": 245760,
+        "vCores": 240,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -321,7 +321,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 307200
+            "value": 245760
           },
           {
             "attributes": {},
@@ -330,7 +330,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 300
+            "value": 240
           }
         ]}
       },
@@ -410,8 +410,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 2500,
-      "maxApplicationsPerUser": 2500,
+      "maxApplications": 2000,
+      "maxApplicationsPerUser": 2000,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -501,11 +501,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 16.666668,
+      "absoluteCapacity": 10,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
-      "weight": 4,
-      "normalizedWeight": 0.16666667,
+      "weight": 5,
+      "normalizedWeight": 0.1,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "test1",
@@ -545,12 +545,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 16.666668,
+        "absoluteCapacity": 10,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
-        "weight": 4,
-        "normalizedWeight": 0.16666667,
+        "weight": 5,
+        "normalizedWeight": 0.1,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -600,8 +600,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 204800,
-          "vCores": 200,
+          "memory": 122880,
+          "vCores": 120,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -610,7 +610,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 204800
+              "value": 122880
             },
             {
               "attributes": {},
@@ -619,7 +619,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 200
+              "value": 120
             }
           ]}
         },
@@ -796,8 +796,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 204800,
-        "vCores": 200,
+        "memory": 122880,
+        "vCores": 120,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -806,7 +806,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 204800
+            "value": 122880
           },
           {
             "attributes": {},
@@ -815,7 +815,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 200
+            "value": 120
           }
         ]}
       },
@@ -895,8 +895,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 1666,
-      "maxApplicationsPerUser": 1666,
+      "maxApplications": 1000,
+      "maxApplicationsPerUser": 1000,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -986,11 +986,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 41.666664,
+      "absoluteCapacity": 20,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "weight": 10,
-      "normalizedWeight": 0.41666666,
+      "normalizedWeight": 0.2,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "default",
@@ -1030,12 +1030,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 41.666664,
+        "absoluteCapacity": 20,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "weight": 10,
-        "normalizedWeight": 0.41666666,
+        "normalizedWeight": 0.2,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -1085,8 +1085,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 511999,
-          "vCores": 499,
+          "memory": 245760,
+          "vCores": 240,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -1095,7 +1095,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 511999
+              "value": 245760
             },
             {
               "attributes": {},
@@ -1104,7 +1104,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 499
+              "value": 240
             }
           ]}
         },
@@ -1281,8 +1281,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 511999,
-        "vCores": 499,
+        "memory": 245760,
+        "vCores": 240,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -1291,7 +1291,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 511999
+            "value": 245760
           },
           {
             "attributes": {},
@@ -1300,7 +1300,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 499
+            "value": 240
           }
         ]}
       },
@@ -1380,8 +1380,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 4166,
-      "maxApplicationsPerUser": 4166,
+      "maxApplications": 2000,
+      "maxApplicationsPerUser": 2000,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -1471,11 +1471,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 4.166667,
+      "absoluteCapacity": 2,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "weight": 1,
-      "normalizedWeight": 0.041666668,
+      "normalizedWeight": 0.02,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "auto1",
@@ -1515,12 +1515,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 4.166667,
+        "absoluteCapacity": 2,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 100,
         "weight": 1,
-        "normalizedWeight": 0.041666668,
+        "normalizedWeight": 0.02,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -1570,8 +1570,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 51200,
-          "vCores": 50,
+          "memory": 24575,
+          "vCores": 23,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -1580,7 +1580,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 51200
+              "value": 24575
             },
             {
               "attributes": {},
@@ -1589,7 +1589,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 50
+              "value": 23
             }
           ]}
         },
@@ -1766,8 +1766,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 51200,
-        "vCores": 50,
+        "memory": 24575,
+        "vCores": 23,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -1776,7 +1776,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 51200
+            "value": 24575
           },
           {
             "attributes": {},
@@ -1785,7 +1785,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 50
+            "value": 23
           }
         ]}
       },
@@ -1865,8 +1865,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 416,
-      "maxApplicationsPerUser": 416,
+      "maxApplications": 200,
+      "maxApplicationsPerUser": 200,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": -1,
@@ -1956,11 +1956,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 4.166667,
+      "absoluteCapacity": 2,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "weight": 1,
-      "normalizedWeight": 0.041666668,
+      "normalizedWeight": 0.02,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "auto2",
@@ -2000,12 +2000,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 4.166667,
+        "absoluteCapacity": 2,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 100,
         "weight": 1,
-        "normalizedWeight": 0.041666668,
+        "normalizedWeight": 0.02,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -2055,8 +2055,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 51200,
-          "vCores": 50,
+          "memory": 24575,
+          "vCores": 23,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -2065,7 +2065,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 51200
+              "value": 24575
             },
             {
               "attributes": {},
@@ -2074,7 +2074,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 50
+              "value": 23
             }
           ]}
         },
@@ -2251,8 +2251,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 51200,
-        "vCores": 50,
+        "memory": 24575,
+        "vCores": 23,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -2261,7 +2261,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 51200
+            "value": 24575
           },
           {
             "attributes": {},
@@ -2270,7 +2270,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 50
+            "value": 23
           }
         ]}
       },
@@ -2350,8 +2350,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 416,
-      "maxApplicationsPerUser": 416,
+      "maxApplications": 200,
+      "maxApplicationsPerUser": 200,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": -1,
@@ -2441,11 +2441,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 4.166667,
+      "absoluteCapacity": 2,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "weight": 1,
-      "normalizedWeight": 0.041666668,
+      "normalizedWeight": 0.02,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "auto3",
@@ -2485,12 +2485,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 4.166667,
+        "absoluteCapacity": 2,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 100,
         "weight": 1,
-        "normalizedWeight": 0.041666668,
+        "normalizedWeight": 0.02,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -2540,8 +2540,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 51200,
-          "vCores": 50,
+          "memory": 24575,
+          "vCores": 23,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -2550,7 +2550,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 51200
+              "value": 24575
             },
             {
               "attributes": {},
@@ -2559,7 +2559,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 50
+              "value": 23
             }
           ]}
         },
@@ -2736,8 +2736,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 51200,
-        "vCores": 50,
+        "memory": 24575,
+        "vCores": 23,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -2746,7 +2746,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 51200
+            "value": 24575
           },
           {
             "attributes": {},
@@ -2755,7 +2755,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 50
+            "value": 23
           }
         ]}
       },
@@ -2835,8 +2835,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 416,
-      "maxApplicationsPerUser": 416,
+      "maxApplications": 200,
+      "maxApplicationsPerUser": 200,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": -1,
@@ -2921,84 +2921,59 @@
       "defaultApplicationLifetime": -1
     },
     {
-      "queuePath": "root.autoParent1",
+      "queuePath": "root.parent",
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 4.166667,
+      "absoluteCapacity": 40,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
-      "weight": 1,
-      "normalizedWeight": 0.041666668,
+      "weight": 20,
+      "normalizedWeight": 0.4,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
-      "queueName": "autoParent1",
+      "queueName": "parent",
       "isAbsoluteResource": false,
       "state": "RUNNING",
       "queues": {"queue": [{
-        "type": "capacitySchedulerLeafQueueInfo",
-        "queuePath": "root.autoParent1.auto4",
+        "queuePath": "root.parent.autoParent3",
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 4.166667,
+        "absoluteCapacity": 40,
         "absoluteMaxCapacity": 100,
         "absoluteUsedCapacity": 0,
         "weight": 1,
         "normalizedWeight": 1,
         "numApplications": 0,
         "maxParallelApps": 2147483647,
-        "queueName": "auto4",
+        "queueName": "autoParent3",
         "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": "",
+        "queues": {"queue": [{
+          "type": "capacitySchedulerLeafQueueInfo",
+          "queuePath": "root.parent.autoParent3.auto6",
           "capacity": 0,
           "usedCapacity": 0,
           "maxCapacity": 100,
-          "absoluteCapacity": 4.166667,
-          "absoluteUsedCapacity": 0,
+          "absoluteCapacity": 40,
           "absoluteMaxCapacity": 100,
-          "maxAMLimitPercentage": 100,
+          "absoluteUsedCapacity": 0,
           "weight": 1,
           "normalizedWeight": 1,
-          "configuredMinResource": {
+          "numApplications": 0,
+          "maxParallelApps": 2147483647,
+          "queueName": "auto6",
+          "isAbsoluteResource": false,
+          "state": "RUNNING",
+          "resourcesUsed": {
             "memory": 0,
             "vCores": 0,
             "resourceInformations": {"resourceInformation": [
               {
                 "attributes": {},
-                "maximumAllocation": 8192,
-                "minimumAllocation": 1024,
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
                 "name": "memory-mb",
                 "resourceType": "COUNTABLE",
                 "units": "Mi",
@@ -3006,8 +2981,8 @@
               },
               {
                 "attributes": {},
-                "maximumAllocation": 4,
-                "minimumAllocation": 1,
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
                 "name": "vcores",
                 "resourceType": "COUNTABLE",
                 "units": "",
@@ -3015,33 +2990,293 @@
               }
             ]}
           },
-          "configuredMaxResource": {
-            "memory": 0,
-            "vCores": 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": 8192,
-                "minimumAllocation": 1024,
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
                 "name": "memory-mb",
                 "resourceType": "COUNTABLE",
                 "units": "Mi",
-                "value": 0
+                "value": 491520
               },
               {
                 "attributes": {},
-                "maximumAllocation": 4,
-                "minimumAllocation": 1,
+                "maximumAllocation": 9223372036854775807,
+                "minimumAllocation": 0,
                 "name": "vcores",
                 "resourceType": "COUNTABLE",
                 "units": "",
-                "value": 0
+                "value": 480
               }
             ]}
           },
-          "effectiveMinResource": {
-            "memory": 51200,
-            "vCores": 50,
+          "maxEffectiveCapacity": {
+            "memory": 1228800,
+            "vCores": 1200,
             "resourceInformations": {"resourceInformation": [
               {
                 "attributes": {},
@@ -3050,7 +3285,7 @@
                 "name": "memory-mb",
                 "resourceType": "COUNTABLE",
                 "units": "Mi",
-                "value": 51200
+                "value": 1228800
               },
               {
                 "attributes": {},
@@ -3059,13 +3294,71 @@
                 "name": "vcores",
                 "resourceType": "COUNTABLE",
                 "units": "",
-                "value": 50
+                "value": 1200
               }
             ]}
           },
-          "effectiveMaxResource": {
+          "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": 1200,
+            "vCores": 1,
             "resourceInformations": {"resourceInformation": [
               {
                 "attributes": {},
@@ -3083,14 +3376,11 @@
                 "name": "vcores",
                 "resourceType": "COUNTABLE",
                 "units": "",
-                "value": 1200
+                "value": 1
               }
             ]}
-          }
-        }]},
-        "resources": {"resourceUsagesByPartition": [{
-          "partitionName": "",
-          "used": {
+          },
+          "usedAMResource": {
             "memory": 0,
             "vCores": 0,
             "resourceInformations": {"resourceInformation": [
@@ -3114,9 +3404,9 @@
               }
             ]}
           },
-          "reserved": {
-            "memory": 0,
-            "vCores": 0,
+          "userAMResourceLimit": {
+            "memory": 1228800,
+            "vCores": 1,
             "resourceInformations": {"resourceInformation": [
               {
                 "attributes": {},
@@ -3125,7 +3415,7 @@
                 "name": "memory-mb",
                 "resourceType": "COUNTABLE",
                 "units": "Mi",
-                "value": 0
+                "value": 1228800
               },
               {
                 "attributes": {},
@@ -3134,22 +3424,117 @@
                 "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
               }
             ]}
           },
-          "pending": {
+          "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": 0
+                "value": 491520
               },
               {
                 "attributes": {},
@@ -3158,11 +3543,38 @@
                 "name": "vcores",
                 "resourceType": "COUNTABLE",
                 "units": "",
-                "value": 0
+                "value": 480
               }
             ]}
           },
-          "amUsed": {
+          "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": [
@@ -3186,9 +3598,9 @@
               }
             ]}
           },
-          "amLimit": {
-            "memory": 1228800,
-            "vCores": 1,
+          "reserved": {
+            "memory": 0,
+            "vCores": 0,
             "resourceInformations": {"resourceInformation": [
               {
                 "attributes": {},
@@ -3197,7 +3609,7 @@
                 "name": "memory-mb",
                 "resourceType": "COUNTABLE",
                 "units": "Mi",
-                "value": 1228800
+                "value": 0
               },
               {
                 "attributes": {},
@@ -3206,11 +3618,11 @@
                 "name": "vcores",
                 "resourceType": "COUNTABLE",
                 "units": "",
-                "value": 1
+                "value": 0
               }
             ]}
           },
-          "userAmLimit": {
+          "pending": {
             "memory": 0,
             "vCores": 0,
             "resourceInformations": {"resourceInformation": [
@@ -3236,8 +3648,8 @@
           }
         }]},
         "minEffectiveCapacity": {
-          "memory": 51200,
-          "vCores": 50,
+          "memory": 491520,
+          "vCores": 480,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -3246,7 +3658,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 51200
+              "value": 491520
             },
             {
               "attributes": {},
@@ -3255,7 +3667,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 50
+              "value": 480
             }
           ]}
         },
@@ -3310,7 +3722,7 @@
         "queueAcls": {"queueAcl": [
           {
             "accessType": "ADMINISTER_QUEUE",
-            "accessControlList": " "
+            "accessControlList": "parentAdmin2 "
           },
           {
             "accessType": "APPLICATION_MAX_PRIORITY",
@@ -3318,7 +3730,1579 @@
           },
           {
             "accessType": "SUBMIT_APP",
-            "accessControlList": " "
+            "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,
@@ -3335,8 +5319,8 @@
         "numActiveApplications": 0,
         "numPendingApplications": 0,
         "numContainers": 0,
-        "maxApplications": 300,
-        "maxApplicationsPerUser": 300,
+        "maxApplications": 200,
+        "maxApplicationsPerUser": 200,
         "userLimit": 100,
         "users": {},
         "userLimitFactor": -1,
@@ -3454,12 +5438,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 4.166667,
+        "absoluteCapacity": 2,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 0,
         "weight": 1,
-        "normalizedWeight": 0.041666668,
+        "normalizedWeight": 0.02,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -3509,8 +5493,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 51200,
-          "vCores": 50,
+          "memory": 24575,
+          "vCores": 23,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -3519,7 +5503,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 51200
+              "value": 24575
             },
             {
               "attributes": {},
@@ -3528,7 +5512,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 50
+              "value": 23
             }
           ]}
         },
@@ -3633,8 +5617,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 51200,
-        "vCores": 50,
+        "memory": 24575,
+        "vCores": 23,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -3643,7 +5627,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 51200
+            "value": 24575
           },
           {
             "attributes": {},
@@ -3652,7 +5636,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 50
+            "value": 23
           }
         ]}
       },
@@ -3707,7 +5691,7 @@
       "queueAcls": {"queueAcl": [
         {
           "accessType": "ADMINISTER_QUEUE",
-          "accessControlList": " "
+          "accessControlList": "parentAdmin1 "
         },
         {
           "accessType": "APPLICATION_MAX_PRIORITY",
@@ -3715,7 +5699,7 @@
         },
         {
           "accessType": "SUBMIT_APP",
-          "accessControlList": " "
+          "accessControlList": "parentUser1 "
         }
       ]},
       "queuePriority": 0,
@@ -3726,12 +5710,18 @@
       "queueType": "parent",
       "creationMethod": "dynamicFlexible",
       "autoCreationEligibility": "flexible",
-      "autoQueueTemplateProperties": {"property": [{
-        "name": "maximum-applications",
-        "value": "300"
-      }]},
+      "autoQueueTemplateProperties": {},
       "autoQueueParentTemplateProperties": {},
-      "autoQueueLeafTemplateProperties": {}
+      "autoQueueLeafTemplateProperties": {"property": [
+        {
+          "name": "acl_administer_queue",
+          "value": "wildAdmin1"
+        },
+        {
+          "name": "acl_submit_applications",
+          "value": "wildUser1"
+        }
+      ]}
     }
   ]},
   "capacities": {"queueCapacitiesByPartition": [{
@@ -4002,6 +5992,15 @@
   "creationMethod": "static",
   "autoCreationEligibility": "flexible",
   "autoQueueTemplateProperties": {},
-  "autoQueueParentTemplateProperties": {},
+  "autoQueueParentTemplateProperties": {"property": [
+    {
+      "name": "acl_administer_queue",
+      "value": "parentAdmin1"
+    },
+    {
+      "name": "acl_submit_applications",
+      "value": "parentUser1"
+    }
+  ]},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json
index ee4cd14..84fd173 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json
@@ -16,11 +16,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 30.000002,
+      "absoluteCapacity": 22.222223,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
-      "weight": 6,
-      "normalizedWeight": 0.3,
+      "weight": 10,
+      "normalizedWeight": 0.22222222,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "test2",
@@ -60,12 +60,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 30.000002,
+        "absoluteCapacity": 22.222223,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
-        "weight": 6,
-        "normalizedWeight": 0.3,
+        "weight": 10,
+        "normalizedWeight": 0.22222222,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -115,8 +115,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 368640,
-          "vCores": 360,
+          "memory": 273066,
+          "vCores": 266,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -125,7 +125,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 368640
+              "value": 273066
             },
             {
               "attributes": {},
@@ -134,7 +134,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 360
+              "value": 266
             }
           ]}
         },
@@ -311,8 +311,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 368640,
-        "vCores": 360,
+        "memory": 273066,
+        "vCores": 266,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -321,7 +321,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 368640
+            "value": 273066
           },
           {
             "attributes": {},
@@ -330,7 +330,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 360
+            "value": 266
           }
         ]}
       },
@@ -410,8 +410,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 3000,
-      "maxApplicationsPerUser": 3000,
+      "maxApplications": 2222,
+      "maxApplicationsPerUser": 2222,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -501,11 +501,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 20,
+      "absoluteCapacity": 11.111112,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
-      "weight": 4,
-      "normalizedWeight": 0.2,
+      "weight": 5,
+      "normalizedWeight": 0.11111111,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "test1",
@@ -545,12 +545,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 20,
+        "absoluteCapacity": 11.111112,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
-        "weight": 4,
-        "normalizedWeight": 0.2,
+        "weight": 5,
+        "normalizedWeight": 0.11111111,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -600,8 +600,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 245760,
-          "vCores": 240,
+          "memory": 136533,
+          "vCores": 133,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -610,7 +610,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 245760
+              "value": 136533
             },
             {
               "attributes": {},
@@ -619,7 +619,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 240
+              "value": 133
             }
           ]}
         },
@@ -796,8 +796,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 245760,
-        "vCores": 240,
+        "memory": 136533,
+        "vCores": 133,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -806,7 +806,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 245760
+            "value": 136533
           },
           {
             "attributes": {},
@@ -815,7 +815,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 240
+            "value": 133
           }
         ]}
       },
@@ -895,8 +895,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 2000,
-      "maxApplicationsPerUser": 2000,
+      "maxApplications": 1111,
+      "maxApplicationsPerUser": 1111,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -986,11 +986,11 @@
       "capacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
-      "absoluteCapacity": 50,
+      "absoluteCapacity": 22.222223,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "weight": 10,
-      "normalizedWeight": 0.5,
+      "normalizedWeight": 0.22222222,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "queueName": "default",
@@ -1030,12 +1030,12 @@
         "capacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
-        "absoluteCapacity": 50,
+        "absoluteCapacity": 22.222223,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "weight": 10,
-        "normalizedWeight": 0.5,
+        "normalizedWeight": 0.22222222,
         "configuredMinResource": {
           "memory": 0,
           "vCores": 0,
@@ -1085,8 +1085,8 @@
           ]}
         },
         "effectiveMinResource": {
-          "memory": 614400,
-          "vCores": 600,
+          "memory": 273066,
+          "vCores": 266,
           "resourceInformations": {"resourceInformation": [
             {
               "attributes": {},
@@ -1095,7 +1095,7 @@
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "units": "Mi",
-              "value": 614400
+              "value": 273066
             },
             {
               "attributes": {},
@@ -1104,7 +1104,7 @@
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "units": "",
-              "value": 600
+              "value": 266
             }
           ]}
         },
@@ -1281,8 +1281,8 @@
         }
       }]},
       "minEffectiveCapacity": {
-        "memory": 614400,
-        "vCores": 600,
+        "memory": 273066,
+        "vCores": 266,
         "resourceInformations": {"resourceInformation": [
           {
             "attributes": {},
@@ -1291,7 +1291,7 @@
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "units": "Mi",
-            "value": 614400
+            "value": 273066
           },
           {
             "attributes": {},
@@ -1300,7 +1300,7 @@
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "units": "",
-            "value": 600
+            "value": 266
           }
         ]}
       },
@@ -1380,8 +1380,8 @@
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
-      "maxApplications": 5000,
-      "maxApplicationsPerUser": 5000,
+      "maxApplications": 2222,
+      "maxApplicationsPerUser": 2222,
       "userLimit": 100,
       "users": {},
       "userLimitFactor": 1,
@@ -1464,6 +1464,350 @@
       "isAutoCreatedLeafQueue": false,
       "maxApplicationLifetime": -1,
       "defaultApplicationLifetime": -1
+    },
+    {
+      "queuePath": "root.parent",
+      "capacity": 0,
+      "usedCapacity": 0,
+      "maxCapacity": 100,
+      "absoluteCapacity": 44.444447,
+      "absoluteMaxCapacity": 100,
+      "absoluteUsedCapacity": 0,
+      "weight": 20,
+      "normalizedWeight": 0.44444445,
+      "numApplications": 0,
+      "maxParallelApps": 2147483647,
+      "queueName": "parent",
+      "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": 44.444447,
+        "absoluteUsedCapacity": 0,
+        "absoluteMaxCapacity": 100,
+        "maxAMLimitPercentage": 0,
+        "weight": 20,
+        "normalizedWeight": 0.44444445,
+        "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": 546133,
+          "vCores": 533,
+          "resourceInformations": {"resourceInformation": [
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "memory-mb",
+              "resourceType": "COUNTABLE",
+              "units": "Mi",
+              "value": 546133
+            },
+            {
+              "attributes": {},
+              "maximumAllocation": 9223372036854775807,
+              "minimumAllocation": 0,
+              "name": "vcores",
+              "resourceType": "COUNTABLE",
+              "units": "",
+              "value": 533
+            }
+          ]}
+        },
+        "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": 546133,
+        "vCores": 533,
+        "resourceInformations": {"resourceInformation": [
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "memory-mb",
+            "resourceType": "COUNTABLE",
+            "units": "Mi",
+            "value": 546133
+          },
+          {
+            "attributes": {},
+            "maximumAllocation": 9223372036854775807,
+            "minimumAllocation": 0,
+            "name": "vcores",
+            "resourceType": "COUNTABLE",
+            "units": "",
+            "value": 533
+          }
+        ]}
+      },
+      "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"
+        }
+      ]}
     }
   ]},
   "capacities": {"queueCapacitiesByPartition": [{
@@ -1734,6 +2078,15 @@
   "creationMethod": "static",
   "autoCreationEligibility": "flexible",
   "autoQueueTemplateProperties": {},
-  "autoQueueParentTemplateProperties": {},
+  "autoQueueParentTemplateProperties": {"property": [
+    {
+      "name": "acl_administer_queue",
+      "value": "parentAdmin1"
+    },
+    {
+      "name": "acl_submit_applications",
+      "value": "parentUser1"
+    }
+  ]},
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
index 9173cb0..3a694e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
@@ -642,9 +642,7 @@ support other pre-configured queues to co-exist along with auto-created queues.
 
 The parent queue which has been enabled for auto leaf queue creation,supports
  the configuration of template parameters for automatic configuration of the auto-created leaf queues. The auto-created queues support all of the
- leaf queue configuration parameters except for **Queue ACL**, **Absolute
- Resource** configurations. Queue ACLs are
- currently inherited from the parent queue i.e they are not configurable on the leaf queue template
+ leaf queue configuration parameters except for **Absolute Resource** configurations.
 
 | Property | Description |
 |:---- |:---- |
@@ -730,11 +728,11 @@ The `Flexible Dynamic Queue Auto-Creation and Management` feature allows a **Par
 
 * Configuring **flexible** `Auto-Created Leaf Queues` with `CapacityScheduler`
 
-The parent queue which has the flexible auto queue creation enabled supports the configuration of dynamically created leaf and parent queues through template parameters. The auto-created queues support all of the leaf queue configuration parameters except for **Queue ACL**, **Absolute Resource** configurations. Queue ACLs are currently inherited from the parent queue i.e they are not configurable on the leaf queue template
+The parent queue which has the flexible auto queue creation enabled supports the configuration of dynamically created leaf and parent queues through template parameters. The auto-created queues support all of the leaf queue configuration parameters except for **Absolute Resource** configurations.
 
 | Property | Description |
 |:---- |:---- |
-| `yarn.scheduler.capacity.<queue-path>.auto-queue-creation-v2.template.<queue-property>` | *Optional* parameter: Specifies a queue property (like capacity, maximum-capacity, user-limit-factor, maximum-am-resource-percent ...  - Refer **Queue Properties** section) inherited by the auto-created **parent** and **leaf** queues. |
+| `yarn.scheduler.capacity.<queue-path>.auto-queue-creation-v2.template.<queue-property>` | *Optional* parameter: Specifies a queue property (like capacity, maximum-capacity, user-limit-factor, maximum-am-resource-percent ...  - Refer **Queue Properties** section) inherited by the auto-created **parent** and **leaf** queues. Dynamic Queue ACLs set here can be overwritten by the parent-template for dynamic parent queues and with the leaf-template for dynamic leaf queues.  |
 | `yarn.scheduler.capacity.<queue-path>.auto-queue-creation-v2.leaf-template.<queue-property>` | *Optional* parameter: Specifies a queue property inherited by auto-created **leaf** queues. |
 | `yarn.scheduler.capacity.<queue-path>.auto-queue-creation-v2.parent-template.<queue-property>` |  *Optional* parameter: Specifies a queue property inherited by auto-created **parent** queues. |
 

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