1
0
Преглед на файлове

YARN-11069. Dynamic Queue ACL handling in Legacy and Flexible Auto Created Queues. Contributed by Tamas Domok

Tamas Domok преди 3 години
родител
ревизия
da09d68056
променени са 16 файла, в които са добавени 2230 реда и са изтрити 314 реда
  1. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java
  2. 46 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  3. 6 2
      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
  4. 15 0
      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
  5. 84 0
      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
  6. 42 1
      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
  7. 12 0
      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
  8. 1 1
      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
  9. 5 5
      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
  10. 663 38
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
  11. 39 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java
  12. 17 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json
  13. 508 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json
  14. 383 148
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json
  15. 401 48
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json
  16. 3 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md

+ 5 - 0
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;
     this.name = name;
   }
   }
 
 
+  public PrivilegedEntity(String name) {
+    this.type = EntityType.QUEUE;
+    this.name = name;
+  }
+
   public EntityType getType() {
   public EntityType getType() {
     return type;
     return type;
   }
   }

+ 46 - 24
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.Container;
 import org.apache.hadoop.yarn.api.records.NodeId;
 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.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -473,32 +477,33 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       if (scheduler instanceof CapacityScheduler) {
       if (scheduler instanceof CapacityScheduler) {
         String queueName = placementContext == null ?
         String queueName = placementContext == null ?
             submissionContext.getQueue() : placementContext.getFullQueuePath();
             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) {
       if (scheduler instanceof FairScheduler) {
@@ -572,6 +577,23 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     return application;
     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(
   private List<ResourceRequest> validateAndCreateResourceRequest(
       ApplicationSubmissionContext submissionContext, boolean isRecovery)
       ApplicationSubmissionContext submissionContext, boolean isRecovery)
       throws InvalidResourceRequestException {
       throws InvalidResourceRequestException {

+ 6 - 2
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();
     writeLock.lock();
     try {
     try {
       CapacitySchedulerConfiguration configuration = queueContext.getConfiguration();
       CapacitySchedulerConfiguration configuration = queueContext.getConfiguration();
+      this.acls = configuration.getAcls(getQueuePath());
+
       if (isDynamicQueue() || this instanceof AbstractAutoCreatedLeafQueue) {
       if (isDynamicQueue() || this instanceof AbstractAutoCreatedLeafQueue) {
         setDynamicQueueProperties();
         setDynamicQueueProperties();
+        setDynamicQueueACLProperties();
       }
       }
 
 
       // Collect and set the Node label configuration
       // Collect and set the Node label configuration
@@ -369,8 +372,6 @@ public abstract class AbstractCSQueue implements CSQueue {
 
 
       authorizer = YarnAuthorizationProvider.getInstance(configuration);
       authorizer = YarnAuthorizationProvider.getInstance(configuration);
 
 
-      this.acls = configuration.getAcls(getQueuePath());
-
       this.userWeights = getUserWeightsFromHierarchy();
       this.userWeights = getUserWeightsFromHierarchy();
 
 
       this.reservationsContinueLooking =
       this.reservationsContinueLooking =
@@ -426,6 +427,9 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
     }
   }
   }
 
 
+  protected void setDynamicQueueACLProperties() {
+  }
+
   private UserWeights getUserWeightsFromHierarchy() {
   private UserWeights getUserWeightsFromHierarchy() {
     UserWeights unionInheritedWeights = UserWeights.createEmpty();
     UserWeights unionInheritedWeights = UserWeights.createEmpty();
     CSQueue parentQ = parent;
     CSQueue parentQ = parent;

+ 15 - 0
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.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedLeafQueue;
+
 public class AbstractLeafQueue extends AbstractCSQueue {
 public class AbstractLeafQueue extends AbstractCSQueue {
   private static final Logger LOG =
   private static final Logger LOG =
       LoggerFactory.getLogger(AbstractLeafQueue.class);
       LoggerFactory.getLogger(AbstractLeafQueue.class);
@@ -1697,6 +1699,19 @@ public class AbstractLeafQueue extends AbstractCSQueue {
     super.setDynamicQueueProperties();
     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(
   private void updateSchedulerHealthForCompletedContainer(
       RMContainer rmContainer, ContainerStatus containerStatus) {
       RMContainer rmContainer, ContainerStatus containerStatus) {
     // Update SchedulerHealth for released / preempted container
     // Update SchedulerHealth for released / preempted container

+ 84 - 0
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());
     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
   @Override
   public Map<ReservationACL, AccessControlList> getReservationAcls(String
   public Map<ReservationACL, AccessControlList> getReservationAcls(String
         queue) {
         queue) {

+ 42 - 1
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.Arrays;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Set;
 import java.util.Set;
 
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 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 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.
  * Context of the Queues in Capacity Scheduler.
@@ -596,6 +599,44 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
     return parentsToCreate;
     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
    * Get {@code ConfiguredNodeLabels} which contains the configured node labels
    * for all queues.
    * for all queues.

+ 12 - 0
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.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedParentQueue;
+
 @Private
 @Private
 @Evolving
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
 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
   private static float PRECISION = 0.0005f; // 0.05% precision
 
 
   // Check weight configuration, throw exception when configuration is invalid
   // Check weight configuration, throw exception when configuration is invalid

+ 1 - 1
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();
     CapacitySchedulerConfiguration conf = cs.getConfiguration();
     queueAcls = new QueueAclsInfo();
     queueAcls = new QueueAclsInfo();
-    queueAcls.addAll(getSortedQueueAclInfoList(queueName, conf));
+    queueAcls.addAll(getSortedQueueAclInfoList(parent, queueName, conf));
 
 
     queuePriority = parent.getPriority().getPriority();
     queuePriority = parent.getPriority().getPriority();
     if (parent instanceof ParentQueue) {
     if (parent instanceof ParentQueue) {

+ 5 - 5
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();
     CapacitySchedulerConfiguration conf = cs.getConfiguration();
     queueAcls = new QueueAclsInfo();
     queueAcls = new QueueAclsInfo();
-    queueAcls.addAll(getSortedQueueAclInfoList(queuePath, conf));
+    queueAcls.addAll(getSortedQueueAclInfoList(q, queuePath, conf));
 
 
     queuePriority = q.getPriority().getPriority();
     queuePriority = q.getPriority().getPriority();
     if (q instanceof ParentQueue) {
     if (q instanceof ParentQueue) {
@@ -183,11 +183,11 @@ public class CapacitySchedulerQueueInfo {
     leafQueueTemplate = new LeafQueueTemplateInfo(conf, queuePath);
     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<>();
     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(),
       QueueAclInfo queueAcl = new QueueAclInfo(e.getKey().toString(),
           e.getValue().getAclString());
           e.getValue().getAclString());
       queueAclsInfo.add(queueAcl);
       queueAclsInfo.add(queueAcl);

+ 663 - 38
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.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 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.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.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 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.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.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -108,7 +112,11 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentMap;
 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.PREFIX;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueuePrefix;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.any;
@@ -141,6 +149,7 @@ public class TestAppManager extends AppManagerTestBase{
   private ResourceScheduler scheduler;
   private ResourceScheduler scheduler;
 
 
   private static final String USER_ID_PREFIX = "userid=";
   private static final String USER_ID_PREFIX = "userid=";
+  private static final String ROOT_PARENT =  PREFIX + "root.parent.";
 
 
   public synchronized RMAppEventType getAppEventType() {
   public synchronized RMAppEventType getAppEventType() {
     return appEventType;
     return appEventType;
@@ -307,11 +316,7 @@ public class TestAppManager extends AppManagerTestBase{
   @Test
   @Test
   public void testQueueSubmitWithACLsEnabledWithQueueMapping()
   public void testQueueSubmitWithACLsEnabledWithQueueMapping()
       throws YarnException {
       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 csConf = new
         CapacitySchedulerConfiguration(conf, false);
         CapacitySchedulerConfiguration(conf, false);
     csConf.set(PREFIX + "root.queues", "default,test");
     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_submit_applications", "test");
     csConf.set(PREFIX + "root.test.acl_administer_queue", "test");
     csConf.set(PREFIX + "root.test.acl_administer_queue", "test");
 
 
-    asContext.setQueue("oldQueue");
-
     MockRM newMockRM = new MockRM(csConf);
     MockRM newMockRM = new MockRM(csConf);
     RMContext newMockRMContext = newMockRM.getRMContext();
     RMContext newMockRMContext = newMockRM.getRMContext();
-    newMockRMContext.setQueuePlacementManager(createMockPlacementManager("test", "test", null));
+    newMockRMContext.setQueuePlacementManager(
+        createMockPlacementManager("test", "root.test", null));
     TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
     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
   @Test
-  public void testQueueSubmitWithACLsEnabledWithQueueMappingForAutoCreatedQueue()
+  public void testQueueSubmitWithACLsEnabledWithQueueMappingForLegacyAutoCreatedQueue()
       throws IOException, YarnException {
       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(
     CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(
         conf, false);
         conf, false);
     csConf.set(PREFIX + "root.queues", "default,managedparent");
     csConf.set(PREFIX + "root.queues", "default,managedparent");
@@ -383,8 +382,6 @@ public class TestAppManager extends AppManagerTestBase{
     csConf.setAutoCreatedLeafQueueConfigCapacity("root.managedparent", 30f);
     csConf.setAutoCreatedLeafQueueConfigCapacity("root.managedparent", 30f);
     csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.managedparent", 100f);
     csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.managedparent", 100f);
 
 
-    asContext.setQueue("oldQueue");
-
     MockRM newMockRM = new MockRM(csConf);
     MockRM newMockRM = new MockRM(csConf);
     CapacityScheduler cs =
     CapacityScheduler cs =
         ((CapacityScheduler) newMockRM.getResourceScheduler());
         ((CapacityScheduler) newMockRM.getResourceScheduler());
@@ -395,25 +392,653 @@ public class TestAppManager extends AppManagerTestBase{
 
 
     RMContext newMockRMContext = newMockRM.getRMContext();
     RMContext newMockRMContext = newMockRM.getRMContext();
     newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
     newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
-        "user1|user2", "user1", "managedparent"));
+        "user1|user2", "user1", "root.managedparent"));
     TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
     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 {
     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) {
     } catch (YarnException e) {
       assertTrue(e.getCause() instanceof AccessControlException);
       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
   @After
   public void tearDown() {
   public void tearDown() {
     setAppEventType(RMAppEventType.KILL);
     setAppEventType(RMAppEventType.KILL);

+ 39 - 12
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.auto2");
     createQueue("root.auto3");
     createQueue("root.auto3");
     createQueue("root.autoParent1.auto4");
     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(),
     assertJsonResponse(sendRequest(),
         "webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json");
         "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.state", "RUNNING");
       conf.put("yarn.scheduler.capacity.root.managed." +
       conf.put("yarn.scheduler.capacity.root.managed." +
           "auto-create-child-queue.enabled", "true");
           "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);
       return createConfiguration(conf);
     }
     }
 
 
@@ -256,17 +255,45 @@ public class TestRMWebServicesCapacitySchedDynamicConfig extends
 
 
     private static Configuration createWeightConfigInternal(boolean enableAqc) {
     private static Configuration createWeightConfigInternal(boolean enableAqc) {
       Map<String, String> conf = new HashMap<>();
       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.capacity", "1w");
       conf.put("yarn.scheduler.capacity.root.default.capacity", "10w");
       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.test1.state", "RUNNING");
       conf.put("yarn.scheduler.capacity.root.test2.state", "RUNNING");
       conf.put("yarn.scheduler.capacity.root.test2.state", "RUNNING");
 
 
       if (enableAqc) {
       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);
       return createConfiguration(conf);
     }
     }

+ 17 - 7
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": [
         "queueAcls": {"queueAcl": [
           {
           {
             "accessType": "ADMINISTER_QUEUE",
             "accessType": "ADMINISTER_QUEUE",
-            "accessControlList": " "
+            "accessControlList": "admin "
           },
           },
           {
           {
             "accessType": "APPLICATION_MAX_PRIORITY",
             "accessType": "APPLICATION_MAX_PRIORITY",
@@ -893,7 +893,7 @@
           },
           },
           {
           {
             "accessType": "SUBMIT_APP",
             "accessType": "SUBMIT_APP",
-            "accessControlList": " "
+            "accessControlList": "user "
           }
           }
         ]},
         ]},
         "queuePriority": 0,
         "queuePriority": 0,
@@ -1296,10 +1296,20 @@
       "queuePriority": 0,
       "queuePriority": 0,
       "orderingPolicyInfo": "utilization",
       "orderingPolicyInfo": "utilization",
       "autoCreateChildQueueEnabled": true,
       "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",
       "mode": "absolute",
       "queueType": "parent",
       "queueType": "parent",
       "creationMethod": "static",
       "creationMethod": "static",
@@ -1579,4 +1589,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

+ 508 - 23
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,
       "capacity": 0,
       "usedCapacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
       "maxCapacity": 100,
-      "absoluteCapacity": 30.000002,
+      "absoluteCapacity": 22.222223,
       "absoluteMaxCapacity": 100,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "absoluteUsedCapacity": 0,
-      "weight": 6,
-      "normalizedWeight": 0.3,
+      "weight": 10,
+      "normalizedWeight": 0.22222222,
       "numApplications": 0,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "maxParallelApps": 2147483647,
       "queueName": "test2",
       "queueName": "test2",
@@ -60,12 +60,12 @@
         "capacity": 0,
         "capacity": 0,
         "usedCapacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
         "maxCapacity": 100,
-        "absoluteCapacity": 30.000002,
+        "absoluteCapacity": 22.222223,
         "absoluteUsedCapacity": 0,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "maxAMLimitPercentage": 10,
-        "weight": 6,
-        "normalizedWeight": 0.3,
+        "weight": 10,
+        "normalizedWeight": 0.22222222,
         "configuredMinResource": {
         "configuredMinResource": {
           "memory": 0,
           "memory": 0,
           "vCores": 0,
           "vCores": 0,
@@ -410,8 +410,8 @@
       "numActiveApplications": 0,
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
       "numContainers": 0,
-      "maxApplications": 3000,
-      "maxApplicationsPerUser": 3000,
+      "maxApplications": 2222,
+      "maxApplicationsPerUser": 2222,
       "userLimit": 100,
       "userLimit": 100,
       "users": {},
       "users": {},
       "userLimitFactor": 1,
       "userLimitFactor": 1,
@@ -501,11 +501,11 @@
       "capacity": 0,
       "capacity": 0,
       "usedCapacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
       "maxCapacity": 100,
-      "absoluteCapacity": 20,
+      "absoluteCapacity": 11.111112,
       "absoluteMaxCapacity": 100,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "absoluteUsedCapacity": 0,
-      "weight": 4,
-      "normalizedWeight": 0.2,
+      "weight": 5,
+      "normalizedWeight": 0.11111111,
       "numApplications": 0,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "maxParallelApps": 2147483647,
       "queueName": "test1",
       "queueName": "test1",
@@ -545,12 +545,497 @@
         "capacity": 0,
         "capacity": 0,
         "usedCapacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
         "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,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "maxAMLimitPercentage": 10,
-        "weight": 4,
-        "normalizedWeight": 0.2,
+        "weight": 20,
+        "normalizedWeight": 0.44444445,
         "configuredMinResource": {
         "configuredMinResource": {
           "memory": 0,
           "memory": 0,
           "vCores": 0,
           "vCores": 0,
@@ -895,8 +1380,8 @@
       "numActiveApplications": 0,
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
       "numContainers": 0,
-      "maxApplications": 2000,
-      "maxApplicationsPerUser": 2000,
+      "maxApplications": 4444,
+      "maxApplicationsPerUser": 4444,
       "userLimit": 100,
       "userLimit": 100,
       "users": {},
       "users": {},
       "userLimitFactor": 1,
       "userLimitFactor": 1,
@@ -986,11 +1471,11 @@
       "capacity": 0,
       "capacity": 0,
       "usedCapacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
       "maxCapacity": 100,
-      "absoluteCapacity": 50,
+      "absoluteCapacity": 22.222223,
       "absoluteMaxCapacity": 100,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "absoluteUsedCapacity": 0,
       "weight": 10,
       "weight": 10,
-      "normalizedWeight": 0.5,
+      "normalizedWeight": 0.22222222,
       "numApplications": 0,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "maxParallelApps": 2147483647,
       "queueName": "default",
       "queueName": "default",
@@ -1030,12 +1515,12 @@
         "capacity": 0,
         "capacity": 0,
         "usedCapacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
         "maxCapacity": 100,
-        "absoluteCapacity": 50,
+        "absoluteCapacity": 22.222223,
         "absoluteUsedCapacity": 0,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "maxAMLimitPercentage": 10,
         "weight": 10,
         "weight": 10,
-        "normalizedWeight": 0.5,
+        "normalizedWeight": 0.22222222,
         "configuredMinResource": {
         "configuredMinResource": {
           "memory": 0,
           "memory": 0,
           "vCores": 0,
           "vCores": 0,
@@ -1380,8 +1865,8 @@
       "numActiveApplications": 0,
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
       "numContainers": 0,
-      "maxApplications": 5000,
-      "maxApplicationsPerUser": 5000,
+      "maxApplications": 2222,
+      "maxApplicationsPerUser": 2222,
       "userLimit": 100,
       "userLimit": 100,
       "users": {},
       "users": {},
       "userLimitFactor": 1,
       "userLimitFactor": 1,
@@ -1736,4 +2221,4 @@
   "autoQueueTemplateProperties": {},
   "autoQueueTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueParentTemplateProperties": {},
   "autoQueueLeafTemplateProperties": {}
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

Файловите разлики са ограничени, защото са твърде много
+ 383 - 148
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json


+ 401 - 48
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,
       "capacity": 0,
       "usedCapacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
       "maxCapacity": 100,
-      "absoluteCapacity": 30.000002,
+      "absoluteCapacity": 22.222223,
       "absoluteMaxCapacity": 100,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "absoluteUsedCapacity": 0,
-      "weight": 6,
-      "normalizedWeight": 0.3,
+      "weight": 10,
+      "normalizedWeight": 0.22222222,
       "numApplications": 0,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "maxParallelApps": 2147483647,
       "queueName": "test2",
       "queueName": "test2",
@@ -60,12 +60,12 @@
         "capacity": 0,
         "capacity": 0,
         "usedCapacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
         "maxCapacity": 100,
-        "absoluteCapacity": 30.000002,
+        "absoluteCapacity": 22.222223,
         "absoluteUsedCapacity": 0,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "maxAMLimitPercentage": 10,
-        "weight": 6,
-        "normalizedWeight": 0.3,
+        "weight": 10,
+        "normalizedWeight": 0.22222222,
         "configuredMinResource": {
         "configuredMinResource": {
           "memory": 0,
           "memory": 0,
           "vCores": 0,
           "vCores": 0,
@@ -115,8 +115,8 @@
           ]}
           ]}
         },
         },
         "effectiveMinResource": {
         "effectiveMinResource": {
-          "memory": 368640,
-          "vCores": 360,
+          "memory": 273066,
+          "vCores": 266,
           "resourceInformations": {"resourceInformation": [
           "resourceInformations": {"resourceInformation": [
             {
             {
               "attributes": {},
               "attributes": {},
@@ -125,7 +125,7 @@
               "name": "memory-mb",
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "resourceType": "COUNTABLE",
               "units": "Mi",
               "units": "Mi",
-              "value": 368640
+              "value": 273066
             },
             },
             {
             {
               "attributes": {},
               "attributes": {},
@@ -134,7 +134,7 @@
               "name": "vcores",
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "resourceType": "COUNTABLE",
               "units": "",
               "units": "",
-              "value": 360
+              "value": 266
             }
             }
           ]}
           ]}
         },
         },
@@ -311,8 +311,8 @@
         }
         }
       }]},
       }]},
       "minEffectiveCapacity": {
       "minEffectiveCapacity": {
-        "memory": 368640,
-        "vCores": 360,
+        "memory": 273066,
+        "vCores": 266,
         "resourceInformations": {"resourceInformation": [
         "resourceInformations": {"resourceInformation": [
           {
           {
             "attributes": {},
             "attributes": {},
@@ -321,7 +321,7 @@
             "name": "memory-mb",
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "resourceType": "COUNTABLE",
             "units": "Mi",
             "units": "Mi",
-            "value": 368640
+            "value": 273066
           },
           },
           {
           {
             "attributes": {},
             "attributes": {},
@@ -330,7 +330,7 @@
             "name": "vcores",
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "resourceType": "COUNTABLE",
             "units": "",
             "units": "",
-            "value": 360
+            "value": 266
           }
           }
         ]}
         ]}
       },
       },
@@ -410,8 +410,8 @@
       "numActiveApplications": 0,
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
       "numContainers": 0,
-      "maxApplications": 3000,
-      "maxApplicationsPerUser": 3000,
+      "maxApplications": 2222,
+      "maxApplicationsPerUser": 2222,
       "userLimit": 100,
       "userLimit": 100,
       "users": {},
       "users": {},
       "userLimitFactor": 1,
       "userLimitFactor": 1,
@@ -501,11 +501,11 @@
       "capacity": 0,
       "capacity": 0,
       "usedCapacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
       "maxCapacity": 100,
-      "absoluteCapacity": 20,
+      "absoluteCapacity": 11.111112,
       "absoluteMaxCapacity": 100,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "absoluteUsedCapacity": 0,
-      "weight": 4,
-      "normalizedWeight": 0.2,
+      "weight": 5,
+      "normalizedWeight": 0.11111111,
       "numApplications": 0,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "maxParallelApps": 2147483647,
       "queueName": "test1",
       "queueName": "test1",
@@ -545,12 +545,12 @@
         "capacity": 0,
         "capacity": 0,
         "usedCapacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
         "maxCapacity": 100,
-        "absoluteCapacity": 20,
+        "absoluteCapacity": 11.111112,
         "absoluteUsedCapacity": 0,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "maxAMLimitPercentage": 10,
-        "weight": 4,
-        "normalizedWeight": 0.2,
+        "weight": 5,
+        "normalizedWeight": 0.11111111,
         "configuredMinResource": {
         "configuredMinResource": {
           "memory": 0,
           "memory": 0,
           "vCores": 0,
           "vCores": 0,
@@ -600,8 +600,8 @@
           ]}
           ]}
         },
         },
         "effectiveMinResource": {
         "effectiveMinResource": {
-          "memory": 245760,
-          "vCores": 240,
+          "memory": 136533,
+          "vCores": 133,
           "resourceInformations": {"resourceInformation": [
           "resourceInformations": {"resourceInformation": [
             {
             {
               "attributes": {},
               "attributes": {},
@@ -610,7 +610,7 @@
               "name": "memory-mb",
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "resourceType": "COUNTABLE",
               "units": "Mi",
               "units": "Mi",
-              "value": 245760
+              "value": 136533
             },
             },
             {
             {
               "attributes": {},
               "attributes": {},
@@ -619,7 +619,7 @@
               "name": "vcores",
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "resourceType": "COUNTABLE",
               "units": "",
               "units": "",
-              "value": 240
+              "value": 133
             }
             }
           ]}
           ]}
         },
         },
@@ -796,8 +796,8 @@
         }
         }
       }]},
       }]},
       "minEffectiveCapacity": {
       "minEffectiveCapacity": {
-        "memory": 245760,
-        "vCores": 240,
+        "memory": 136533,
+        "vCores": 133,
         "resourceInformations": {"resourceInformation": [
         "resourceInformations": {"resourceInformation": [
           {
           {
             "attributes": {},
             "attributes": {},
@@ -806,7 +806,7 @@
             "name": "memory-mb",
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "resourceType": "COUNTABLE",
             "units": "Mi",
             "units": "Mi",
-            "value": 245760
+            "value": 136533
           },
           },
           {
           {
             "attributes": {},
             "attributes": {},
@@ -815,7 +815,7 @@
             "name": "vcores",
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "resourceType": "COUNTABLE",
             "units": "",
             "units": "",
-            "value": 240
+            "value": 133
           }
           }
         ]}
         ]}
       },
       },
@@ -895,8 +895,8 @@
       "numActiveApplications": 0,
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
       "numContainers": 0,
-      "maxApplications": 2000,
-      "maxApplicationsPerUser": 2000,
+      "maxApplications": 1111,
+      "maxApplicationsPerUser": 1111,
       "userLimit": 100,
       "userLimit": 100,
       "users": {},
       "users": {},
       "userLimitFactor": 1,
       "userLimitFactor": 1,
@@ -986,11 +986,11 @@
       "capacity": 0,
       "capacity": 0,
       "usedCapacity": 0,
       "usedCapacity": 0,
       "maxCapacity": 100,
       "maxCapacity": 100,
-      "absoluteCapacity": 50,
+      "absoluteCapacity": 22.222223,
       "absoluteMaxCapacity": 100,
       "absoluteMaxCapacity": 100,
       "absoluteUsedCapacity": 0,
       "absoluteUsedCapacity": 0,
       "weight": 10,
       "weight": 10,
-      "normalizedWeight": 0.5,
+      "normalizedWeight": 0.22222222,
       "numApplications": 0,
       "numApplications": 0,
       "maxParallelApps": 2147483647,
       "maxParallelApps": 2147483647,
       "queueName": "default",
       "queueName": "default",
@@ -1030,12 +1030,12 @@
         "capacity": 0,
         "capacity": 0,
         "usedCapacity": 0,
         "usedCapacity": 0,
         "maxCapacity": 100,
         "maxCapacity": 100,
-        "absoluteCapacity": 50,
+        "absoluteCapacity": 22.222223,
         "absoluteUsedCapacity": 0,
         "absoluteUsedCapacity": 0,
         "absoluteMaxCapacity": 100,
         "absoluteMaxCapacity": 100,
         "maxAMLimitPercentage": 10,
         "maxAMLimitPercentage": 10,
         "weight": 10,
         "weight": 10,
-        "normalizedWeight": 0.5,
+        "normalizedWeight": 0.22222222,
         "configuredMinResource": {
         "configuredMinResource": {
           "memory": 0,
           "memory": 0,
           "vCores": 0,
           "vCores": 0,
@@ -1085,8 +1085,8 @@
           ]}
           ]}
         },
         },
         "effectiveMinResource": {
         "effectiveMinResource": {
-          "memory": 614400,
-          "vCores": 600,
+          "memory": 273066,
+          "vCores": 266,
           "resourceInformations": {"resourceInformation": [
           "resourceInformations": {"resourceInformation": [
             {
             {
               "attributes": {},
               "attributes": {},
@@ -1095,7 +1095,7 @@
               "name": "memory-mb",
               "name": "memory-mb",
               "resourceType": "COUNTABLE",
               "resourceType": "COUNTABLE",
               "units": "Mi",
               "units": "Mi",
-              "value": 614400
+              "value": 273066
             },
             },
             {
             {
               "attributes": {},
               "attributes": {},
@@ -1104,7 +1104,7 @@
               "name": "vcores",
               "name": "vcores",
               "resourceType": "COUNTABLE",
               "resourceType": "COUNTABLE",
               "units": "",
               "units": "",
-              "value": 600
+              "value": 266
             }
             }
           ]}
           ]}
         },
         },
@@ -1281,8 +1281,8 @@
         }
         }
       }]},
       }]},
       "minEffectiveCapacity": {
       "minEffectiveCapacity": {
-        "memory": 614400,
-        "vCores": 600,
+        "memory": 273066,
+        "vCores": 266,
         "resourceInformations": {"resourceInformation": [
         "resourceInformations": {"resourceInformation": [
           {
           {
             "attributes": {},
             "attributes": {},
@@ -1291,7 +1291,7 @@
             "name": "memory-mb",
             "name": "memory-mb",
             "resourceType": "COUNTABLE",
             "resourceType": "COUNTABLE",
             "units": "Mi",
             "units": "Mi",
-            "value": 614400
+            "value": 273066
           },
           },
           {
           {
             "attributes": {},
             "attributes": {},
@@ -1300,7 +1300,7 @@
             "name": "vcores",
             "name": "vcores",
             "resourceType": "COUNTABLE",
             "resourceType": "COUNTABLE",
             "units": "",
             "units": "",
-            "value": 600
+            "value": 266
           }
           }
         ]}
         ]}
       },
       },
@@ -1380,8 +1380,8 @@
       "numActiveApplications": 0,
       "numActiveApplications": 0,
       "numPendingApplications": 0,
       "numPendingApplications": 0,
       "numContainers": 0,
       "numContainers": 0,
-      "maxApplications": 5000,
-      "maxApplicationsPerUser": 5000,
+      "maxApplications": 2222,
+      "maxApplicationsPerUser": 2222,
       "userLimit": 100,
       "userLimit": 100,
       "users": {},
       "users": {},
       "userLimitFactor": 1,
       "userLimitFactor": 1,
@@ -1464,6 +1464,350 @@
       "isAutoCreatedLeafQueue": false,
       "isAutoCreatedLeafQueue": false,
       "maxApplicationLifetime": -1,
       "maxApplicationLifetime": -1,
       "defaultApplicationLifetime": -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": [{
   "capacities": {"queueCapacitiesByPartition": [{
@@ -1734,6 +2078,15 @@
   "creationMethod": "static",
   "creationMethod": "static",
   "autoCreationEligibility": "flexible",
   "autoCreationEligibility": "flexible",
   "autoQueueTemplateProperties": {},
   "autoQueueTemplateProperties": {},
-  "autoQueueParentTemplateProperties": {},
+  "autoQueueParentTemplateProperties": {"property": [
+    {
+      "name": "acl_administer_queue",
+      "value": "parentAdmin1"
+    },
+    {
+      "name": "acl_submit_applications",
+      "value": "parentUser1"
+    }
+  ]},
   "autoQueueLeafTemplateProperties": {}
   "autoQueueLeafTemplateProperties": {}
-}}}
+}}}

+ 3 - 5
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 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
  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 |
 | 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`
 * 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 |
 | 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.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. |
 | `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. |
 
 

Някои файлове не бяха показани, защото твърде много файлове са промени