فهرست منبع

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

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1534316 13f79535-47bb-0310-9956-ffa450edef68
Sanford Ryza 11 سال پیش
والد
کامیت
829fc289cf
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

@@ -77,6 +77,8 @@ Release 2.2.1 - UNRELEASED
 
     YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
 
+    YARN-1288. Make Fair Scheduler ACLs more user friendly (Sandy Ryza)
+
   OPTIMIZATIONS
 
   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;
 
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 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.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
@@ -44,12 +42,6 @@ public interface Queue {
    */
   QueueMetrics getMetrics();
 
-  /**
-   * Get ACLs for the queue.
-   * @return ACLs for the queue
-   */
-  public Map<QueueACL, AccessControlList> getQueueAcls();
-  
   /**
    * Get queue information
    * @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;
   }
 
-  @Override
-  public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
   @Override
   public synchronized QueueInfo getQueueInfo(
       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;
   }
 
-  @Override
-  public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
   @Override
   public synchronized QueueInfo getQueueInfo( 
       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.Iterator;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 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.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -177,8 +175,7 @@ public class FSLeafQueue extends FSQueue {
       recordFactory.newRecordInstance(QueueUserACLInfo.class);
     List<QueueACL> operations = new ArrayList<QueueACL>();
     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);
       }
     }

+ 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.Collection;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 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.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -135,12 +132,6 @@ public abstract class FSQueue extends Schedulable implements Queue {
     return queueInfo;
   }
   
-  @Override
-  public Map<QueueACL, AccessControlList> getQueueAcls() {
-    Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-  
   @Override
   public FSQueueMetrics getMetrics() {
     return metrics;
@@ -154,7 +145,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
   
   public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
     // Check if the leaf-queue allows access
-    if (queueMgr.getQueueAcls(getName()).get(acl).isUserAllowed(user)) {
+    if (queueMgr.getQueueAcl(getName(), acl).isUserAllowed(user)) {
       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).
    */
   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;
 
@@ -381,15 +384,6 @@ public class QueueManager {
         queueMetrics.setMinShare(queue.getMinShare());
         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
       for (String name: queueNamesInAllocFile) {
@@ -454,10 +448,10 @@ public class QueueManager {
         policy.initialize(scheduler.getClusterCapacity());
         queuePolicies.put(queueName, policy);
       } 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));
       } 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));
       } else if ("queue".endsWith(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
-   * 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 {

+ 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;
     }
 
-    @Override
     public Map<QueueACL, AccessControlList> getQueueAcls() {
       Map<QueueACL, AccessControlList> acls =
         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(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
-    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
-    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." + 
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -1063,21 +1066,19 @@ public class TestFairScheduler {
     assertEquals(5, queueManager.getUserMaxApps("user2"));
 
     // 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
-    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." +
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -1664,9 +1665,13 @@ public class TestFairScheduler {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     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("</allocations>");
     out.close();
@@ -1893,9 +1898,13 @@ public class TestFairScheduler {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     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("</allocations>");
     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
      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
      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:
 
+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"?>
 <allocations>
@@ -256,6 +278,7 @@ Allocation file format
     <weight>2.0</weight>
     <schedulingPolicy>fair</schedulingPolicy>
     <queue name="sample_sub_queue">
+      <aclSubmitApps>charlie</aclSubmitApps>
       <minResources>5000 mb,0vcores</minResources>
     </queue>
   </queue>