|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager;
|
|
|
|
|
|
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Matchers.anyLong;
|
|
|
import static org.mockito.Matchers.isA;
|
|
@@ -84,6 +85,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptI
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
|
|
+ .CapacityScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
|
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
@@ -108,7 +111,10 @@ import com.google.common.collect.Maps;
|
|
|
|
|
|
public class TestAppManager{
|
|
|
private Log LOG = LogFactory.getLog(TestAppManager.class);
|
|
|
- private static RMAppEventType appEventType = RMAppEventType.KILL;
|
|
|
+ private static RMAppEventType appEventType = RMAppEventType.KILL;
|
|
|
+
|
|
|
+ private static String USER = "user_";
|
|
|
+ private static String USER0 = USER + 0;
|
|
|
|
|
|
public synchronized RMAppEventType getAppEventType() {
|
|
|
return appEventType;
|
|
@@ -267,6 +273,70 @@ public class TestAppManager{
|
|
|
setupDispatcher(rmContext, conf);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testQueueSubmitWithACLsEnabledWithQueueMapping()
|
|
|
+ throws IOException, YarnException, InterruptedException {
|
|
|
+ YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
|
|
+ ResourceScheduler.class);
|
|
|
+ conf.set(PREFIX + "root.queues", "default,test");
|
|
|
+
|
|
|
+ conf.setFloat(PREFIX + "root.default.capacity", 50.0f);
|
|
|
+ conf.setFloat(PREFIX + "root.default.maximum-capacity", 100.0f);
|
|
|
+
|
|
|
+ conf.setFloat(PREFIX + "root.test.capacity", 50.0f);
|
|
|
+ conf.setFloat(PREFIX + "root.test.maximum-capacity", 100.0f);
|
|
|
+
|
|
|
+ conf.set(PREFIX + "root.acl_submit_applications", " ");
|
|
|
+ conf.set(PREFIX + "root.acl_administer_queue", " ");
|
|
|
+
|
|
|
+ conf.set(PREFIX + "root.default.acl_submit_applications", " ");
|
|
|
+ conf.set(PREFIX + "root.default.acl_administer_queue", " ");
|
|
|
+
|
|
|
+ conf.set(PREFIX + "root.test.acl_submit_applications", "test");
|
|
|
+ conf.set(PREFIX + "root.test.acl_administer_queue", "test");
|
|
|
+
|
|
|
+ conf.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), any(String.class));
|
|
|
+
|
|
|
+ asContext.setQueue("oldQueue");
|
|
|
+
|
|
|
+ MockRM newMockRM = new MockRM(conf);
|
|
|
+ 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);
|
|
|
+
|
|
|
+ //only user test has permission to submit to 'test' queue
|
|
|
+ newAppMonitor.submitApplication(asContext, "test");
|
|
|
+
|
|
|
+ try {
|
|
|
+ //should fail since user does not have permission to submit to queue
|
|
|
+ // 'test'
|
|
|
+ newAppMonitor.submitApplication(asContext, "test1");
|
|
|
+ } catch(YarnException e) {
|
|
|
+ assertTrue(e.getCause() instanceof AccessControlException);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@After
|
|
|
public void tearDown() {
|
|
|
setAppEventType(RMAppEventType.KILL);
|
|
@@ -328,7 +398,7 @@ public class TestAppManager{
|
|
|
} catch (Exception e) {
|
|
|
e.printStackTrace();
|
|
|
if (e instanceof YarnException) {
|
|
|
- Assert.assertTrue(e.getCause() instanceof AccessControlException);
|
|
|
+ assertTrue(e.getCause() instanceof AccessControlException);
|
|
|
} else {
|
|
|
Assert.fail("Yarn exception is expected : " + e.getMessage());
|
|
|
}
|
|
@@ -718,7 +788,7 @@ public class TestAppManager{
|
|
|
" Tokens are invalid.");
|
|
|
} catch (YarnException e) {
|
|
|
// Exception is expected
|
|
|
- Assert.assertTrue("The thrown exception is not" +
|
|
|
+ assertTrue("The thrown exception is not" +
|
|
|
" java.io.EOFException",
|
|
|
e.getMessage().contains("java.io.EOFException"));
|
|
|
}
|
|
@@ -778,22 +848,25 @@ public class TestAppManager{
|
|
|
ApplicationId appId = MockApps.newAppID(0);
|
|
|
asContext.setApplicationId(appId);
|
|
|
RMApp appOrig = rmContext.getRMApps().get(appId);
|
|
|
- Assert.assertTrue("app name matches but shouldn't", "testApp1" != appOrig.getName());
|
|
|
+ assertTrue("app name matches "
|
|
|
+ + "but shouldn't", "testApp1" != appOrig.getName());
|
|
|
|
|
|
// our testApp1 should be rejected and original app with same id should be left in place
|
|
|
try {
|
|
|
appMonitor.submitApplication(asContext, "test");
|
|
|
Assert.fail("Exception is expected when applicationId is duplicate.");
|
|
|
} catch (YarnException e) {
|
|
|
- Assert.assertTrue("The thrown exception is not the expectd one.",
|
|
|
+ assertTrue("The thrown exception is not the expectd one.",
|
|
|
e.getMessage().contains("Cannot add a duplicate!"));
|
|
|
}
|
|
|
|
|
|
// make sure original app didn't get removed
|
|
|
RMApp app = rmContext.getRMApps().get(appId);
|
|
|
Assert.assertNotNull("app is null", app);
|
|
|
- Assert.assertEquals("app id doesn't match", appId, app.getApplicationId());
|
|
|
- Assert.assertEquals("app state doesn't match", RMAppState.FINISHED, app.getState());
|
|
|
+ Assert.assertEquals("app id doesn't match",
|
|
|
+ appId, app.getApplicationId());
|
|
|
+ Assert.assertEquals("app state doesn't match",
|
|
|
+ RMAppState.FINISHED, app.getState());
|
|
|
}
|
|
|
|
|
|
@SuppressWarnings("deprecation")
|
|
@@ -811,7 +884,7 @@ public class TestAppManager{
|
|
|
// Exception is expected
|
|
|
// TODO Change this to assert the expected exception type - post YARN-142
|
|
|
// sub-task related to specialized exceptions.
|
|
|
- Assert.assertTrue("The thrown exception is not" +
|
|
|
+ assertTrue("The thrown exception is not" +
|
|
|
" InvalidResourceRequestException",
|
|
|
e.getMessage().contains("Invalid resource request"));
|
|
|
}
|
|
@@ -844,16 +917,16 @@ public class TestAppManager{
|
|
|
Assert.assertFalse(msg.contains("\r"));
|
|
|
|
|
|
String escaped = "\\n\\n\\r\\r";
|
|
|
- Assert.assertTrue(msg.contains("Multiline" + escaped +"AppName"));
|
|
|
- Assert.assertTrue(msg.contains("Multiline" + escaped +"UserName"));
|
|
|
- Assert.assertTrue(msg.contains("Multiline" + escaped +"QueueName"));
|
|
|
- Assert.assertTrue(msg.contains("submitTime=1000"));
|
|
|
- Assert.assertTrue(msg.contains("memorySeconds=16384"));
|
|
|
- Assert.assertTrue(msg.contains("vcoreSeconds=64"));
|
|
|
- Assert.assertTrue(msg.contains("preemptedAMContainers=1"));
|
|
|
- Assert.assertTrue(msg.contains("preemptedNonAMContainers=10"));
|
|
|
- Assert.assertTrue(msg.contains("preemptedResources=<memory:1234\\, vCores:56>"));
|
|
|
- Assert.assertTrue(msg.contains("applicationType=MAPREDUCE"));
|
|
|
+ assertTrue(msg.contains("Multiline" + escaped +"AppName"));
|
|
|
+ assertTrue(msg.contains("Multiline" + escaped +"UserName"));
|
|
|
+ assertTrue(msg.contains("Multiline" + escaped +"QueueName"));
|
|
|
+ assertTrue(msg.contains("submitTime=1000"));
|
|
|
+ assertTrue(msg.contains("memorySeconds=16384"));
|
|
|
+ assertTrue(msg.contains("vcoreSeconds=64"));
|
|
|
+ assertTrue(msg.contains("preemptedAMContainers=1"));
|
|
|
+ assertTrue(msg.contains("preemptedNonAMContainers=10"));
|
|
|
+ assertTrue(msg.contains("preemptedResources=<memory:1234\\, vCores:56>"));
|
|
|
+ assertTrue(msg.contains("applicationType=MAPREDUCE"));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -952,4 +1025,5 @@ public class TestAppManager{
|
|
|
}
|
|
|
return cloneReqs;
|
|
|
}
|
|
|
+
|
|
|
}
|