瀏覽代碼

YARN-1288. Make Fair Scheduler ACLs more user friendly (Sandy Ryza)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1534315 13f79535-47bb-0310-9956-ffa450edef68
Sanford Ryza 11 年之前
父節點
當前提交
dc523bd182
共有 10 個文件被更改,包括 83 次插入91 次删除
  1. 2 0
      hadoop-yarn-project/CHANGES.txt
  2. 0 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
  3. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  4. 0 5
      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
  5. 1 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
  6. 1 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  7. 13 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  8. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  9. 39 30
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  10. 27 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

+ 2 - 0
hadoop-yarn-project/CHANGES.txt

@@ -95,6 +95,8 @@ Release 2.2.1 - UNRELEASED
 
 
     YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
     YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
 
 
+    YARN-1288. Make Fair Scheduler ACLs more user friendly (Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 0 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java

@@ -19,12 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 
 import java.util.List;
 import java.util.List;
-import java.util.Map;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
@@ -44,12 +42,6 @@ public interface Queue {
    */
    */
   QueueMetrics getMetrics();
   QueueMetrics getMetrics();
 
 
-  /**
-   * Get ACLs for the queue.
-   * @return ACLs for the queue
-   */
-  public Map<QueueACL, AccessControlList> getQueueAcls();
-  
   /**
   /**
    * Get queue information
    * Get queue information
    * @param includeChildQueues include child queues?
    * @param includeChildQueues include child queues?

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -526,11 +526,6 @@ public class LeafQueue implements CSQueue {
     return userLimitFactor;
     return userLimitFactor;
   }
   }
 
 
-  @Override
-  public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
   @Override
   @Override
   public synchronized QueueInfo getQueueInfo(
   public synchronized QueueInfo getQueueInfo(
       boolean includeChildQueues, boolean recursive) {
       boolean includeChildQueues, boolean recursive) {

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

@@ -299,11 +299,6 @@ public class ParentQueue implements CSQueue {
     return state;
     return state;
   }
   }
 
 
-  @Override
-  public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
   @Override
   @Override
   public synchronized QueueInfo getQueueInfo( 
   public synchronized QueueInfo getQueueInfo( 
       boolean includeChildQueues, boolean recursive) {
       boolean includeChildQueues, boolean recursive) {

+ 1 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -24,14 +24,12 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
-import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -177,8 +175,7 @@ public class FSLeafQueue extends FSQueue {
       recordFactory.newRecordInstance(QueueUserACLInfo.class);
       recordFactory.newRecordInstance(QueueUserACLInfo.class);
     List<QueueACL> operations = new ArrayList<QueueACL>();
     List<QueueACL> operations = new ArrayList<QueueACL>();
     for (QueueACL operation : QueueACL.values()) {
     for (QueueACL operation : QueueACL.values()) {
-      Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
-      if (acls.get(operation).isUserAllowed(user)) {
+      if (hasAccess(operation, user)) {
         operations.add(operation);
         operations.add(operation);
       }
       }
     }
     }

+ 1 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -20,13 +20,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -135,12 +132,6 @@ public abstract class FSQueue extends Schedulable implements Queue {
     return queueInfo;
     return queueInfo;
   }
   }
   
   
-  @Override
-  public Map<QueueACL, AccessControlList> getQueueAcls() {
-    Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-  
   @Override
   @Override
   public FSQueueMetrics getMetrics() {
   public FSQueueMetrics getMetrics() {
     return metrics;
     return metrics;
@@ -154,7 +145,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
   
   
   public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
   public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
     // Check if the leaf-queue allows access
     // Check if the leaf-queue allows access
-    if (queueMgr.getQueueAcls(getName()).get(acl).isUserAllowed(user)) {
+    if (queueMgr.getQueueAcl(getName(), acl).isUserAllowed(user)) {
       return true;
       return true;
     }
     }
 
 

+ 13 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -72,6 +72,9 @@ public class QueueManager {
    * (this is done to prevent loading a file that hasn't been fully written).
    * (this is done to prevent loading a file that hasn't been fully written).
    */
    */
   public static final long ALLOC_RELOAD_WAIT = 5 * 1000;
   public static final long ALLOC_RELOAD_WAIT = 5 * 1000;
+  
+  private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
+  private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
 
 
   private final FairScheduler scheduler;
   private final FairScheduler scheduler;
 
 
@@ -381,15 +384,6 @@ public class QueueManager {
         queueMetrics.setMinShare(queue.getMinShare());
         queueMetrics.setMinShare(queue.getMinShare());
         queueMetrics.setMaxShare(queue.getMaxShare());
         queueMetrics.setMaxShare(queue.getMaxShare());
       }
       }
-      
-      // Root queue should have empty ACLs.  As a queue's ACL is the union of
-      // its ACL and all its parents' ACLs, setting the roots' to empty will
-      // neither allow nor prohibit more access to its children.
-      Map<QueueACL, AccessControlList> rootAcls =
-          new HashMap<QueueACL, AccessControlList>();
-      rootAcls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(" "));
-      rootAcls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(" "));
-      queueAcls.put(ROOT_QUEUE, rootAcls);
  
  
       // Create all queus
       // Create all queus
       for (String name: queueNamesInAllocFile) {
       for (String name: queueNamesInAllocFile) {
@@ -454,10 +448,10 @@ public class QueueManager {
         policy.initialize(scheduler.getClusterCapacity());
         policy.initialize(scheduler.getClusterCapacity());
         queuePolicies.put(queueName, policy);
         queuePolicies.put(queueName, policy);
       } else if ("aclSubmitApps".equals(field.getTagName())) {
       } else if ("aclSubmitApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
+        String text = ((Text)field.getFirstChild()).getData();
         acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
         acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
       } else if ("aclAdministerApps".equals(field.getTagName())) {
       } else if ("aclAdministerApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
+        String text = ((Text)field.getFirstChild()).getData();
         acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
         acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
       } else if ("queue".endsWith(field.getTagName()) || 
       } else if ("queue".endsWith(field.getTagName()) || 
           "pool".equals(field.getTagName())) {
           "pool".equals(field.getTagName())) {
@@ -577,21 +571,16 @@ public class QueueManager {
 
 
   /**
   /**
    * Get the ACLs associated with this queue. If a given ACL is not explicitly
    * Get the ACLs associated with this queue. If a given ACL is not explicitly
-   * configured, include the default value for that ACL.
+   * configured, include the default value for that ACL.  The default for the
+   * root queue is everybody ("*") and the default for all other queues is
+   * nobody ("")
    */
    */
-  public Map<QueueACL, AccessControlList> getQueueAcls(String queue) {
-    HashMap<QueueACL, AccessControlList> out = new HashMap<QueueACL, AccessControlList>();
-    Map<QueueACL, AccessControlList> queueAcl = info.queueAcls.get(queue);
-    if (queueAcl != null) {
-      out.putAll(queueAcl);
-    }
-    if (!out.containsKey(QueueACL.ADMINISTER_QUEUE)) {
-      out.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList("*"));
-    }
-    if (!out.containsKey(QueueACL.SUBMIT_APPLICATIONS)) {
-      out.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList("*"));
+  public AccessControlList getQueueAcl(String queue, QueueACL operation) {
+    Map<QueueACL, AccessControlList> queueAcls = info.queueAcls.get(queue);
+    if (queueAcls == null || !queueAcls.containsKey(operation)) {
+      return (queue.equals(ROOT_QUEUE)) ? EVERYBODY_ACL : NOBODY_ACL;
     }
     }
-    return out;
+    return queueAcls.get(operation);
   }
   }
   
   
   static class QueueManagerInfo {
   static class QueueManagerInfo {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -156,7 +156,6 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
       return queueInfo;
       return queueInfo;
     }
     }
 
 
-    @Override
     public Map<QueueACL, AccessControlList> getQueueAcls() {
     public Map<QueueACL, AccessControlList> getQueueAcls() {
       Map<QueueACL, AccessControlList> acls =
       Map<QueueACL, AccessControlList> acls =
         new HashMap<QueueACL, AccessControlList>();
         new HashMap<QueueACL, AccessControlList>();

+ 39 - 30
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -865,22 +865,25 @@ public class TestFairScheduler {
     assertEquals(10, queueManager.getUserMaxApps("user1"));
     assertEquals(10, queueManager.getUserMaxApps("user1"));
     assertEquals(5, queueManager.getUserMaxApps("user2"));
     assertEquals(5, queueManager.getUserMaxApps("user2"));
 
 
+    // Root should get * ACL
+    assertEquals("*",queueManager.getQueueAcl("root",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("*", queueManager.getQueueAcl("root",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
+
     // Unspecified queues should get default ACL
     // Unspecified queues should get default ACL
-    Map<QueueACL, AccessControlList> aclsA = queueManager.getQueueAcls("root.queueA");
-    assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    assertEquals(" ",queueManager.getQueueAcl("root.queueA",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
     // Queue B ACL
     // Queue B ACL
-    Map<QueueACL, AccessControlList> aclsB = queueManager.getQueueAcls("root.queueB");
-    assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueB",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
 
 
-    // Queue c ACL
-    Map<QueueACL, AccessControlList> aclsC = queueManager.getQueueAcls("root.queueC");
-    assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    // Queue C ACL
+    assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueC",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." + 
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." + 
         YarnConfiguration.DEFAULT_QUEUE_NAME));
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -1063,21 +1066,19 @@ public class TestFairScheduler {
     assertEquals(5, queueManager.getUserMaxApps("user2"));
     assertEquals(5, queueManager.getUserMaxApps("user2"));
 
 
     // Unspecified queues should get default ACL
     // Unspecified queues should get default ACL
-    Map<QueueACL, AccessControlList> aclsA = queueManager.getQueueAcls("queueA");
-    assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
     // Queue B ACL
     // Queue B ACL
-    Map<QueueACL, AccessControlList> aclsB = queueManager.getQueueAcls("root.queueB");
-    assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueB",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+
+    // Queue C ACL
+    assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueC",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
-    // Queue c ACL
-    Map<QueueACL, AccessControlList> aclsC = queueManager.getQueueAcls("root.queueC");
-    assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." +
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." +
         YarnConfiguration.DEFAULT_QUEUE_NAME));
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -1664,9 +1665,13 @@ public class TestFairScheduler {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
     out.println("<allocations>");
-    out.println("<queue name=\"queue1\">");
-    out.println("<aclSubmitApps>norealuserhasthisname</aclSubmitApps>");
-    out.println("<aclAdministerApps>norealuserhasthisname</aclAdministerApps>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps> </aclAdministerApps>");
+    out.println("  <queue name=\"queue1\">");
+    out.println("    <aclSubmitApps>norealuserhasthisname</aclSubmitApps>");
+    out.println("    <aclAdministerApps>norealuserhasthisname</aclAdministerApps>");
+    out.println("  </queue>");
     out.println("</queue>");
     out.println("</queue>");
     out.println("</allocations>");
     out.println("</allocations>");
     out.close();
     out.close();
@@ -1893,9 +1898,13 @@ public class TestFairScheduler {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
     out.println("<allocations>");
-    out.println("<queue name=\"queue1\">");
-    out.println("<aclSubmitApps>userallow</aclSubmitApps>");
-    out.println("<aclAdministerApps>userallow</aclAdministerApps>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps> </aclAdministerApps>");
+    out.println("  <queue name=\"queue1\">");
+    out.println("    <aclSubmitApps>userallow</aclSubmitApps>");
+    out.println("    <aclAdministerApps>userallow</aclAdministerApps>");
+    out.println("  </queue>");
     out.println("</queue>");
     out.println("</queue>");
     out.println("</allocations>");
     out.println("</allocations>");
     out.close();
     out.close();

+ 27 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

@@ -221,10 +221,14 @@ Allocation file format
      for containers, but apps submitted later may run concurrently if there is
      for containers, but apps submitted later may run concurrently if there is
      leftover space on the cluster after satisfying the earlier app's requests.
      leftover space on the cluster after satisfying the earlier app's requests.
 
 
-   * aclSubmitApps: a list of users that can submit apps to the queue. A (default)
-     value of "*" means that any users can submit apps. A queue inherits the ACL of
-     its parent, so if a queue2 descends from queue1, and user1 is in queue1's ACL,
-     and user2 is in queue2's ACL, then both users may submit to queue2.
+   * aclSubmitApps: a list of users and/or groups that can submit apps to the
+     queue. Refer to the ACLs section below for more info on the format of this
+     list and how queue ACLs work.
+
+   * aclAdministerApps: a list of users and/or groups that can administer a
+     queue.  Currently the only administrative action is killing an application.
+     Refer to the ACLs section below for more info on the format of this list
+     and how queue ACLs work.
 
 
    * minSharePreemptionTimeout: number of seconds the queue is under its minimum share
    * minSharePreemptionTimeout: number of seconds the queue is under its minimum share
      before it will try to preempt containers to take resources from other queues.
      before it will try to preempt containers to take resources from other queues.
@@ -246,6 +250,24 @@ Allocation file format
 
 
   An example allocation file is given here:
   An example allocation file is given here:
 
 
+Queue Access Control Lists (ACLs)
+
+  Queue Access Control Lists (ACLs) allow administrators to control who may
+  take actions on particular queues. They are configured with the aclSubmitApps
+  and aclAdministerApps properties, which can be set per queue. Currently the
+  only supported administrative action is killing an application. Anybody who
+  may administer a queue may also submit applications to it. These properties
+  take values in a format like "user1,user2 group1,group2" or " group1,group2".
+  An action on a queue will be permitted if its user or group is in the ACL of
+  that queue or in the ACL of any of that queue's ancestors. So if queue2
+  is inside queue1, and user1 is in queue1's ACL, and user2 is in queue2's
+  ACL, then both users may submit to queue2.
+  
+  The root queue's ACLs are "*" by default which, because ACLs are passed down,
+  means that everybody may submit to and kill applications from every queue.
+  To start restricting access, change the root queue's ACLs to something other
+  than "*". 
+
 ---
 ---
 <?xml version="1.0"?>
 <?xml version="1.0"?>
 <allocations>
 <allocations>
@@ -256,6 +278,7 @@ Allocation file format
     <weight>2.0</weight>
     <weight>2.0</weight>
     <schedulingPolicy>fair</schedulingPolicy>
     <schedulingPolicy>fair</schedulingPolicy>
     <queue name="sample_sub_queue">
     <queue name="sample_sub_queue">
+      <aclSubmitApps>charlie</aclSubmitApps>
       <minResources>5000 mb,0vcores</minResources>
       <minResources>5000 mb,0vcores</minResources>
     </queue>
     </queue>
   </queue>
   </queue>