|
@@ -274,10 +274,41 @@ public class TestAppManager extends AppManagerTestBase{
|
|
|
setupDispatcher(rmContext, conf);
|
|
|
}
|
|
|
|
|
|
+ private static PlacementManager createMockPlacementManager(
|
|
|
+ String userRegex, String placementQueue, String placementParentQueue
|
|
|
+ ) throws YarnException {
|
|
|
+ PlacementManager placementMgr = mock(PlacementManager.class);
|
|
|
+ doAnswer(new Answer<ApplicationPlacementContext>() {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public ApplicationPlacementContext answer(InvocationOnMock invocation)
|
|
|
+ throws Throwable {
|
|
|
+ return new ApplicationPlacementContext(placementQueue, placementParentQueue);
|
|
|
+ }
|
|
|
+
|
|
|
+ }).when(placementMgr).placeApplication(
|
|
|
+ any(ApplicationSubmissionContext.class),
|
|
|
+ matches(userRegex),
|
|
|
+ any(Boolean.class));
|
|
|
+
|
|
|
+ return placementMgr;
|
|
|
+ }
|
|
|
+
|
|
|
+ private TestRMAppManager createAppManager(RMContext context, Configuration configuration) {
|
|
|
+ ApplicationMasterService masterService = new ApplicationMasterService(context,
|
|
|
+ context.getScheduler());
|
|
|
+
|
|
|
+ return new TestRMAppManager(context,
|
|
|
+ new ClientToAMTokenSecretManagerInRM(),
|
|
|
+ context.getScheduler(), masterService,
|
|
|
+ new ApplicationACLsManager(configuration), configuration);
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testQueueSubmitWithACLsEnabledWithQueueMapping()
|
|
|
- throws IOException, YarnException, InterruptedException {
|
|
|
- YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ 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);
|
|
|
|
|
@@ -285,11 +316,11 @@ public class TestAppManager extends AppManagerTestBase{
|
|
|
CapacitySchedulerConfiguration(conf, false);
|
|
|
csConf.set(PREFIX + "root.queues", "default,test");
|
|
|
|
|
|
- csConf.setFloat(PREFIX + "root.default.capacity", 50.0f);
|
|
|
- csConf.setFloat(PREFIX + "root.default.maximum-capacity", 100.0f);
|
|
|
+ csConf.setCapacity("root.default", 50.0f);
|
|
|
+ csConf.setMaximumCapacity("root.default", 100.0f);
|
|
|
|
|
|
- csConf.setFloat(PREFIX + "root.test.capacity", 50.0f);
|
|
|
- csConf.setFloat(PREFIX + "root.test.maximum-capacity", 100.0f);
|
|
|
+ csConf.setCapacity("root.test", 50.0f);
|
|
|
+ csConf.setMaximumCapacity("root.test", 100.0f);
|
|
|
|
|
|
csConf.set(PREFIX + "root.acl_submit_applications", " ");
|
|
|
csConf.set(PREFIX + "root.acl_administer_queue", " ");
|
|
@@ -300,55 +331,30 @@ public class TestAppManager extends AppManagerTestBase{
|
|
|
csConf.set(PREFIX + "root.test.acl_submit_applications", "test");
|
|
|
csConf.set(PREFIX + "root.test.acl_administer_queue", "test");
|
|
|
|
|
|
- csConf.set(PREFIX + "root.test.acl_submit_applications", "test");
|
|
|
- csConf.set(PREFIX + "root.test.acl_administer_queue", "test");
|
|
|
-
|
|
|
- csConf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
|
|
|
-
|
|
|
- // Setup a PlacementManager returns a new queue
|
|
|
- PlacementManager placementMgr = mock(PlacementManager.class);
|
|
|
- doAnswer(new Answer<ApplicationPlacementContext>() {
|
|
|
-
|
|
|
- @Override
|
|
|
- public ApplicationPlacementContext answer(InvocationOnMock invocation)
|
|
|
- throws Throwable {
|
|
|
- return new ApplicationPlacementContext("test");
|
|
|
- }
|
|
|
-
|
|
|
- }).when(placementMgr).placeApplication(
|
|
|
- any(ApplicationSubmissionContext.class), matches("test"));
|
|
|
-
|
|
|
- asContext.setQueue("oldQueue");
|
|
|
+ asContext.setQueue("test");
|
|
|
|
|
|
- MockRM newMockRM = new MockRM(conf);
|
|
|
+ MockRM newMockRM = new MockRM(csConf);
|
|
|
RMContext newMockRMContext = newMockRM.getRMContext();
|
|
|
- newMockRMContext.setQueuePlacementManager(placementMgr);
|
|
|
- ApplicationMasterService masterService =
|
|
|
- new ApplicationMasterService(newMockRMContext,
|
|
|
- newMockRMContext.getScheduler());
|
|
|
-
|
|
|
- TestRMAppManager newAppMonitor = new TestRMAppManager(newMockRMContext,
|
|
|
- new ClientToAMTokenSecretManagerInRM(),
|
|
|
- newMockRMContext.getScheduler(), masterService,
|
|
|
- new ApplicationACLsManager(conf), conf);
|
|
|
+ newMockRMContext.setQueuePlacementManager(createMockPlacementManager("test", "test", null));
|
|
|
+ TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
|
|
|
|
|
|
- //only user test has permission to submit to 'test' queue
|
|
|
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 {
|
|
|
- //should fail since user does not have permission to submit to queue
|
|
|
- // 'test'
|
|
|
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);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void
|
|
|
- testQueueSubmitWithACLsEnabledWithQueueMappingForAutoCreatedQueue()
|
|
|
- throws IOException, YarnException, InterruptedException {
|
|
|
+ public void testQueueSubmitWithACLsEnabledWithQueueMappingForAutoCreatedQueue()
|
|
|
+ throws IOException, YarnException {
|
|
|
YarnConfiguration conf = new YarnConfiguration();
|
|
|
conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
|
|
|
conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
|
@@ -358,8 +364,11 @@ public class TestAppManager extends AppManagerTestBase{
|
|
|
conf, false);
|
|
|
csConf.set(PREFIX + "root.queues", "default,managedparent");
|
|
|
|
|
|
- csConf.setFloat(PREFIX + "root.default.capacity", 50.0f);
|
|
|
- csConf.setFloat(PREFIX + "root.default.maximum-capacity", 100.0f);
|
|
|
+ csConf.setCapacity("root.default", 50.0f);
|
|
|
+ csConf.setMaximumCapacity("root.default", 100.0f);
|
|
|
+
|
|
|
+ csConf.setCapacity("root.managedparent", 50.0f);
|
|
|
+ csConf.setMaximumCapacity("root.managedparent", 100.0f);
|
|
|
|
|
|
csConf.set(PREFIX + "root.acl_submit_applications", " ");
|
|
|
csConf.set(PREFIX + "root.acl_administer_queue", " ");
|
|
@@ -374,22 +383,9 @@ public class TestAppManager extends AppManagerTestBase{
|
|
|
csConf.setAutoCreatedLeafQueueConfigCapacity("root.managedparent", 30f);
|
|
|
csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.managedparent", 100f);
|
|
|
|
|
|
- // Setup a PlacementManager returns a new queue
|
|
|
- PlacementManager placementMgr = mock(PlacementManager.class);
|
|
|
- doAnswer(new Answer<ApplicationPlacementContext>() {
|
|
|
-
|
|
|
- @Override
|
|
|
- public ApplicationPlacementContext answer(InvocationOnMock invocation)
|
|
|
- throws Throwable {
|
|
|
- return new ApplicationPlacementContext("user1", "managedparent");
|
|
|
- }
|
|
|
-
|
|
|
- }).when(placementMgr).placeApplication(
|
|
|
- any(ApplicationSubmissionContext.class), matches("user1|user2"));
|
|
|
-
|
|
|
asContext.setQueue("oldQueue");
|
|
|
|
|
|
- MockRM newMockRM = new MockRM(conf);
|
|
|
+ MockRM newMockRM = new MockRM(csConf);
|
|
|
CapacityScheduler cs =
|
|
|
((CapacityScheduler) newMockRM.getResourceScheduler());
|
|
|
ManagedParentQueue managedParentQueue = new ManagedParentQueue(cs,
|
|
@@ -398,22 +394,21 @@ public class TestAppManager extends AppManagerTestBase{
|
|
|
managedParentQueue);
|
|
|
|
|
|
RMContext newMockRMContext = newMockRM.getRMContext();
|
|
|
- newMockRMContext.setQueuePlacementManager(placementMgr);
|
|
|
- ApplicationMasterService masterService = new ApplicationMasterService(
|
|
|
- newMockRMContext, newMockRMContext.getScheduler());
|
|
|
-
|
|
|
- TestRMAppManager newAppMonitor = new TestRMAppManager(newMockRMContext,
|
|
|
- new ClientToAMTokenSecretManagerInRM(), newMockRMContext.getScheduler(),
|
|
|
- masterService, new ApplicationACLsManager(conf), conf);
|
|
|
+ newMockRMContext.setQueuePlacementManager(createMockPlacementManager(
|
|
|
+ "user1|user2", "user1", "managedparent"));
|
|
|
+ TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf);
|
|
|
|
|
|
- //only user test has permission to submit to 'user1' queue
|
|
|
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());
|
|
|
|
|
|
try {
|
|
|
- //should fail since user does not have permission to submit to queue
|
|
|
- // 'managedparent'
|
|
|
asContext.setApplicationId(appId = MockApps.newAppID(2));
|
|
|
newAppMonitor.submitApplication(asContext, "user2");
|
|
|
+ Assert.fail("should fail since user2 does not have permission to submit to queue");
|
|
|
} catch (YarnException e) {
|
|
|
assertTrue(e.getCause() instanceof AccessControlException);
|
|
|
}
|