Browse Source

YARN-1315. TestQueueACLs should also test FairScheduler (Sandy Ryza)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1534508 13f79535-47bb-0310-9956-ffa450edef68
Sanford Ryza 11 years ago
parent
commit
755def8fe9

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

@@ -97,6 +97,8 @@ Release 2.2.1 - UNRELEASED
 
     YARN-1288. Make Fair Scheduler ACLs more user friendly (Sandy Ryza)
 
+    YARN-1315. TestQueueACLs should also test FairScheduler (Sandy Ryza)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -50,7 +50,7 @@ public class FairSchedulerConfiguration extends Configuration {
   
   private static final String CONF_PREFIX =  "yarn.scheduler.fair.";
 
-  protected static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
+  public static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
   protected static final String DEFAULT_ALLOCATION_FILE = "fair-scheduler.xml";
   protected static final String EVENT_LOG_DIR = "eventlog.dir";
 
@@ -113,6 +113,10 @@ public class FairSchedulerConfiguration extends Configuration {
   protected static final String MAX_ASSIGN = CONF_PREFIX + "max.assign";
   protected static final int DEFAULT_MAX_ASSIGN = -1;
 
+  public FairSchedulerConfiguration() {
+    super();
+  }
+  
   public FairSchedulerConfiguration(Configuration conf) {
     super(conf);
   }

+ 27 - 65
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestQueueACLs.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java

@@ -42,41 +42,43 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestQueueACLs {
+public abstract class QueueACLsTestBase {
 
-  private static final String COMMON_USER = "common_user";
-  private static final String QUEUE_A_USER = "queueA_user";
-  private static final String QUEUE_B_USER = "queueB_user";
-  private static final String ROOT_ADMIN = "root_admin";
-  private static final String QUEUE_A_ADMIN = "queueA_admin";
-  private static final String QUEUE_B_ADMIN = "queueB_admin";
+  protected static final String COMMON_USER = "common_user";
+  protected static final String QUEUE_A_USER = "queueA_user";
+  protected static final String QUEUE_B_USER = "queueB_user";
+  protected static final String ROOT_ADMIN = "root_admin";
+  protected static final String QUEUE_A_ADMIN = "queueA_admin";
+  protected static final String QUEUE_B_ADMIN = "queueB_admin";
 
-  private static final String QUEUEA = "queueA";
-  private static final String QUEUEB = "queueB";
+  protected static final String QUEUEA = "queueA";
+  protected static final String QUEUEB = "queueB";
 
   private static final Log LOG = LogFactory.getLog(TestApplicationACLs.class);
 
-  static MockRM resourceManager;
-  static Configuration conf = createConfiguration();
-  final static YarnRPC rpc = YarnRPC.create(conf);
-  final static InetSocketAddress rmAddress = conf.getSocketAddr(
-    YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS,
-    YarnConfiguration.DEFAULT_RM_PORT);
-
-  @BeforeClass
-  public static void setup() throws InterruptedException, IOException {
+  MockRM resourceManager;
+  Configuration conf;
+  YarnRPC rpc;
+  InetSocketAddress rmAddress;
+
+  @Before
+  public void setup() throws InterruptedException, IOException {
+    conf = createConfiguration();
+    rpc = YarnRPC.create(conf);
+    rmAddress = conf.getSocketAddr(
+      YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_PORT);
+    
     AccessControlList adminACL = new AccessControlList("");
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, adminACL.getAclString());
 
@@ -109,8 +111,8 @@ public class TestQueueACLs {
     }
   }
 
-  @AfterClass
-  public static void tearDown() {
+  @After
+  public void tearDown() {
     if (resourceManager != null) {
       resourceManager.stop();
     }
@@ -262,45 +264,5 @@ public class TestQueueACLs {
     return userClient;
   }
 
-  private static YarnConfiguration createConfiguration() {
-    CapacitySchedulerConfiguration csConf =
-        new CapacitySchedulerConfiguration();
-    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
-        QUEUEA, QUEUEB });
-
-    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f);
-    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 50f);
-
-    Map<QueueACL, AccessControlList> aclsOnQueueA =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonQueueA = new AccessControlList(QUEUE_A_USER);
-    submitACLonQueueA.addUser(COMMON_USER);
-    AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN);
-    aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA);
-    aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA,
-      aclsOnQueueA);
-
-    Map<QueueACL, AccessControlList> aclsOnQueueB =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonQueueB = new AccessControlList(QUEUE_B_USER);
-    submitACLonQueueB.addUser(COMMON_USER);
-    AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN);
-    aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB);
-    aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB,
-      aclsOnQueueB);
-
-    Map<QueueACL, AccessControlList> aclsOnRootQueue =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonRoot = new AccessControlList("");
-    AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN);
-    aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot);
-    aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue);
-
-    YarnConfiguration conf = new YarnConfiguration(csConf);
-    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
-    return conf;
-  }
+  protected abstract Configuration createConfiguration() throws IOException;
 }

+ 73 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerQueueACLs.java

@@ -0,0 +1,73 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.QueueACLsTestBase;
+
+public class TestCapacitySchedulerQueueACLs extends QueueACLsTestBase {
+  @Override
+  protected Configuration createConfiguration() {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
+        QUEUEA, QUEUEB });
+
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f);
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 50f);
+
+    Map<QueueACL, AccessControlList> aclsOnQueueA =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonQueueA = new AccessControlList(QUEUE_A_USER);
+    submitACLonQueueA.addUser(COMMON_USER);
+    AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN);
+    aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA);
+    aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA,
+      aclsOnQueueA);
+
+    Map<QueueACL, AccessControlList> aclsOnQueueB =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonQueueB = new AccessControlList(QUEUE_B_USER);
+    submitACLonQueueB.addUser(COMMON_USER);
+    AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN);
+    aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB);
+    aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB,
+      aclsOnQueueB);
+
+    Map<QueueACL, AccessControlList> aclsOnRootQueue =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonRoot = new AccessControlList("");
+    AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN);
+    aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot);
+    aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue);
+
+    csConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    csConf.set("yarn.resourcemanager.scheduler.class", CapacityScheduler.class.getName());
+
+    return csConf;
+  }
+}

+ 62 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerQueueACLs.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.QueueACLsTestBase;
+
+public class TestFairSchedulerQueueACLs extends QueueACLsTestBase {
+  @Override
+  protected Configuration createConfiguration() throws IOException {
+    FairSchedulerConfiguration fsConf = new FairSchedulerConfiguration();
+    
+    final String TEST_DIR = new File(System.getProperty("test.build.data",
+        "/tmp")).getAbsolutePath();
+    final String ALLOC_FILE = new File(TEST_DIR, "test-queues.xml")
+        .getAbsolutePath();
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps>root_admin </aclAdministerApps>");
+    out.println("  <queue name=\"queueA\">");
+    out.println("    <aclSubmitApps>queueA_user,common_user </aclSubmitApps>");
+    out.println("    <aclAdministerApps>queueA_admin </aclAdministerApps>");
+    out.println("  </queue>");
+    out.println("  <queue name=\"queueB\">");
+    out.println("    <aclSubmitApps>queueB_user,common_user </aclSubmitApps>");
+    out.println("    <aclAdministerApps>queueB_admin </aclAdministerApps>");
+    out.println("  </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+    fsConf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+
+    fsConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    fsConf.set("yarn.resourcemanager.scheduler.class", FairScheduler.class.getName());
+
+    return fsConf;
+  }
+}