|
@@ -44,10 +44,15 @@ import java.util.concurrent.CyclicBarrier;
|
|
|
import com.google.common.collect.ImmutableMap;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
@@ -59,9 +64,13 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
+import org.apache.hadoop.yarn.security.AccessType;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.RMAppManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
@@ -77,8 +86,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueStateManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicyWithExclusivePartitions;
|
|
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
|
|
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UsersManager.User;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
|
@@ -4140,4 +4151,140 @@ public class TestLeafQueue {
|
|
|
cs.stop();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private static class TestRMAppManager extends RMAppManager {
|
|
|
+ TestRMAppManager(RMContext context, YarnScheduler scheduler,
|
|
|
+ ApplicationMasterService masterService,
|
|
|
+ ApplicationACLsManager applicationACLsManager, Configuration conf) {
|
|
|
+ super(context, scheduler, masterService, applicationACLsManager, conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void submitApplication(
|
|
|
+ ApplicationSubmissionContext submissionContext, long submitTime,
|
|
|
+ UserGroupInformation userUgi) throws YarnException {
|
|
|
+ super.submitApplication(submissionContext, submitTime, userUgi);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testSubmitUsingRealUserAcls() throws Exception {
|
|
|
+ final String realUser = "AdminUser";
|
|
|
+ final String user0 = "user0";
|
|
|
+ final String user1 = "user1";
|
|
|
+ final String queue = "default";
|
|
|
+
|
|
|
+ YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ MockRM rm = new MockRM();
|
|
|
+ rm.init(conf);
|
|
|
+ rm.start();
|
|
|
+ rm.getConfig().setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
|
|
|
+
|
|
|
+ UserGroupInformation realUserUgi =
|
|
|
+ UserGroupInformation.createRemoteUser(realUser);
|
|
|
+ UserGroupInformation ugi0 =
|
|
|
+ UserGroupInformation.createProxyUserForTesting("user0", realUserUgi,
|
|
|
+ new String[] {"group1"});
|
|
|
+ UserGroupInformation ugi1 =
|
|
|
+ UserGroupInformation.createProxyUserForTesting("user1", realUserUgi,
|
|
|
+ new String[] {"group1"});
|
|
|
+ ApplicationId applicationId0 = TestUtils.getMockApplicationId(0);
|
|
|
+ ApplicationId applicationId1 = TestUtils.getMockApplicationId(1);
|
|
|
+ CapacityScheduler cSched = (CapacityScheduler) rm.getResourceScheduler();
|
|
|
+
|
|
|
+ ParentQueue rootQueue = (ParentQueue) cSched.getRootQueue();
|
|
|
+ Map<AccessType, AccessControlList> rootAcls = rootQueue.acls;
|
|
|
+ rootAcls.put(AccessType.SUBMIT_APP, new AccessControlList(realUser));
|
|
|
+ rootAcls.put(AccessType.ADMINISTER_QUEUE, new AccessControlList(realUser));
|
|
|
+
|
|
|
+ LeafQueue defaultQueue = (LeafQueue)cSched.getQueue(queue);
|
|
|
+ Map<AccessType, AccessControlList> a = defaultQueue.acls;
|
|
|
+ a.put(AccessType.SUBMIT_APP, new AccessControlList(realUser));
|
|
|
+ a.put(AccessType.ADMINISTER_QUEUE, new AccessControlList(realUser));
|
|
|
+
|
|
|
+ TestRMAppManager testRmAppManager =
|
|
|
+ new TestRMAppManager(spyRMContext, cSched,
|
|
|
+ rm.getApplicationMasterService(),
|
|
|
+ rm.getApplicationACLsManager(), rm.getConfig());
|
|
|
+ ContainerLaunchContext clc =
|
|
|
+ mock(ContainerLaunchContext.class);
|
|
|
+ ApplicationSubmissionContext asc =
|
|
|
+ ApplicationSubmissionContext.newInstance(
|
|
|
+ applicationId0, "myApp0", "default", Priority.newInstance(0),
|
|
|
+ clc, false, false, 1, Resource.newInstance(1024, 1));
|
|
|
+
|
|
|
+ // Each of the following test cases has a proxied user and a real user.
|
|
|
+ // The proxied users are user0 and user1, depending on the test. The real
|
|
|
+ // user is always AdminUser.
|
|
|
+
|
|
|
+ // Ensure that user0 is not allowed to submit to the default queue when only
|
|
|
+ // the admin user is in the submit ACL and the admin user does not have the
|
|
|
+ // USE_REAL_ACLS character prepended.
|
|
|
+ try {
|
|
|
+ testRmAppManager.submitApplication(asc, System.currentTimeMillis(), ugi0);
|
|
|
+ Assert.fail(user0 + " should not be allowed to submit to the "
|
|
|
+ + queue + " queue when only admin user is in submit ACL.");
|
|
|
+ } catch (YarnException e) {
|
|
|
+ // This is the expected behavior.
|
|
|
+ assertTrue("Should have received an AccessControlException.",
|
|
|
+ e.getCause() instanceof AccessControlException);
|
|
|
+ }
|
|
|
+
|
|
|
+ // With only user0 in the list of users authorized to submit apps to the
|
|
|
+ // queue, ensure that user0 is allowed to submit to the default queue.
|
|
|
+ a.put(AccessType.SUBMIT_APP, new AccessControlList(user0));
|
|
|
+ a.put(AccessType.ADMINISTER_QUEUE, new AccessControlList(realUser));
|
|
|
+ try {
|
|
|
+ testRmAppManager.submitApplication(asc, System.currentTimeMillis(), ugi0);
|
|
|
+ } catch (YarnException e) {
|
|
|
+ Assert.fail(user0 + " should be allowed to submit to the "
|
|
|
+ + queue + " queue.");
|
|
|
+ }
|
|
|
+
|
|
|
+ // With only user0 in the list of users authorized to submit apps to the
|
|
|
+ // queue, ensure that user1 is NOT allowed to submit to the default queue
|
|
|
+ try {
|
|
|
+ testRmAppManager.submitApplication(asc, System.currentTimeMillis(), ugi1);
|
|
|
+ Assert.fail(user1 + " should not be allowed to submit to the "
|
|
|
+ + queue + " queue.");
|
|
|
+ } catch (YarnException e) {
|
|
|
+ // This is the expected behavior.
|
|
|
+ assertTrue("Should have received an AccessControlException.",
|
|
|
+ e.getCause() instanceof AccessControlException);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Even though the admin user is in the list of users allowed to submit to
|
|
|
+ // the default queue and user1's real user is the admin user, user1 should
|
|
|
+ // not be allowed to submit to queue because the ACL entry does not have the
|
|
|
+ // special character prepended in the list.
|
|
|
+ a.put(AccessType.SUBMIT_APP,
|
|
|
+ new AccessControlList(user0 + "," + realUser));
|
|
|
+ try {
|
|
|
+ testRmAppManager.submitApplication(asc, System.currentTimeMillis(), ugi1);
|
|
|
+ Assert.fail(user1 + " should not be allowed to submit to the "
|
|
|
+ + queue + " queue.");
|
|
|
+ } catch (YarnException e) {
|
|
|
+ // This is the expected behavior.
|
|
|
+ assertTrue("Should have received an AccessControlException.",
|
|
|
+ e.getCause() instanceof AccessControlException);
|
|
|
+ }
|
|
|
+
|
|
|
+ // user1 should now be allowed to submit to the default queue because the
|
|
|
+ // admin user is in the ACL list and has the USE_REAL_ACLS character
|
|
|
+ // prepended.
|
|
|
+ a.put(AccessType.SUBMIT_APP,
|
|
|
+ new AccessControlList(user0 + ","
|
|
|
+ + AccessControlList.USE_REAL_ACLS + realUser));
|
|
|
+ asc.setApplicationId(applicationId1);
|
|
|
+ try {
|
|
|
+ testRmAppManager.submitApplication(asc, System.currentTimeMillis(), ugi1);
|
|
|
+ } catch (YarnException e) {
|
|
|
+ LOG.error("failed to submit", e);
|
|
|
+ Assert.fail(user1 + " should be allowed to submit to the "
|
|
|
+ + queue + " queue when real user is" + realUser + ".");
|
|
|
+ }
|
|
|
+
|
|
|
+ rm.stop();
|
|
|
+ rm.close();
|
|
|
+ }
|
|
|
}
|