|
@@ -37,7 +37,15 @@ import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
public class TestAppNameMappingPlacementRule {
|
|
|
+ private static final String ROOT_QUEUE = "root";
|
|
|
+ private static final String Q2_QUEUE = "q2";
|
|
|
+ private static final String Q1_QUEUE = "q1";
|
|
|
+ private static final String USER_NAME = "user";
|
|
|
+ private static final String DEFAULT_QUEUE = "default";
|
|
|
+ private static final String APPLICATION_PLACEHOLDER = "%application";
|
|
|
+ private static final String AMBIGUOUS_QUEUE = "ambiguousQueue";
|
|
|
private static final String APP_NAME = "DistributedShell";
|
|
|
+ private static final String MAPREDUCE_APP_NAME = "MAPREDUCE";
|
|
|
|
|
|
private YarnConfiguration conf = new YarnConfiguration();
|
|
|
|
|
@@ -62,16 +70,20 @@ public class TestAppNameMappingPlacementRule {
|
|
|
CapacitySchedulerQueueManager qm =
|
|
|
mock(CapacitySchedulerQueueManager.class);
|
|
|
when(qm.isAmbiguous(Mockito.isA(String.class))).thenReturn(false);
|
|
|
+ when(qm.isAmbiguous(AMBIGUOUS_QUEUE)).thenReturn(true);
|
|
|
+
|
|
|
rule.queueManager = qm;
|
|
|
|
|
|
ApplicationSubmissionContext asc = Records.newRecord(
|
|
|
ApplicationSubmissionContext.class);
|
|
|
- if (inputQueue.equals("%application")) {
|
|
|
+ if (inputQueue.equals(APPLICATION_PLACEHOLDER)) {
|
|
|
inputQueue = APP_NAME;
|
|
|
}
|
|
|
asc.setQueue(inputQueue);
|
|
|
String appName = queueMapping.getSource();
|
|
|
- if (appName.equals("%application")) {
|
|
|
+ // to create a scenario when source != appName
|
|
|
+ if (appName.equals(APPLICATION_PLACEHOLDER)
|
|
|
+ || appName.equals(MAPREDUCE_APP_NAME)) {
|
|
|
appName = APP_NAME;
|
|
|
}
|
|
|
asc.setApplicationName(appName);
|
|
@@ -81,31 +93,85 @@ public class TestAppNameMappingPlacementRule {
|
|
|
ctx != null ? ctx.getQueue() : inputQueue);
|
|
|
}
|
|
|
|
|
|
- public QueueMapping queueMappingBuilder(String source, String queue) {
|
|
|
+ public QueueMapping getQueueMapping(String source, String queue) {
|
|
|
+ return getQueueMapping(source, null, queue);
|
|
|
+ }
|
|
|
+
|
|
|
+ public QueueMapping getQueueMapping(String source, String parent,
|
|
|
+ String queue) {
|
|
|
return QueueMapping.QueueMappingBuilder.create()
|
|
|
.type(QueueMapping.MappingType.APPLICATION)
|
|
|
.source(source)
|
|
|
.queue(queue)
|
|
|
+ .parentQueue(parent)
|
|
|
.build();
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testMapping() throws YarnException {
|
|
|
- // simple base case for mapping user to queue
|
|
|
- verifyQueueMapping(queueMappingBuilder(APP_NAME,
|
|
|
- "q1"), "user_1", "q1");
|
|
|
- verifyQueueMapping(queueMappingBuilder("%application", "q2"), "user_1",
|
|
|
- "q2");
|
|
|
- verifyQueueMapping(queueMappingBuilder("%application", "%application"),
|
|
|
- "user_1", APP_NAME);
|
|
|
-
|
|
|
- // specify overwritten, and see if user specified a queue, and it will be
|
|
|
- // overridden
|
|
|
- verifyQueueMapping(queueMappingBuilder(APP_NAME,
|
|
|
- "q1"), "1", "q2", "q1", true);
|
|
|
-
|
|
|
- // if overwritten not specified, it should be which user specified
|
|
|
- verifyQueueMapping(queueMappingBuilder(APP_NAME,
|
|
|
- "q1"), "1", "q2", "q2", false);
|
|
|
+ public void testSpecificAppNameMappedToDefinedQueue() throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APP_NAME, Q1_QUEUE),
|
|
|
+ USER_NAME, Q1_QUEUE);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testPlaceholderAppSourceMappedToQueue() throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APPLICATION_PLACEHOLDER, Q2_QUEUE),
|
|
|
+ USER_NAME, Q2_QUEUE);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testPlaceHolderAppSourceAndQueueMappedToAppNameQueue()
|
|
|
+ throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APPLICATION_PLACEHOLDER,
|
|
|
+ APPLICATION_PLACEHOLDER), USER_NAME, APP_NAME);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testQueueInMappingOverridesSpecifiedQueue()
|
|
|
+ throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APP_NAME,
|
|
|
+ Q1_QUEUE), USER_NAME, Q2_QUEUE, Q1_QUEUE, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testQueueInMappingDoesNotOverrideSpecifiedQueue()
|
|
|
+ throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APP_NAME,
|
|
|
+ Q1_QUEUE), USER_NAME, Q2_QUEUE, Q2_QUEUE, false);
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDefaultQueueInMappingIsNotUsedWithoutOverride()
|
|
|
+ throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APP_NAME,
|
|
|
+ DEFAULT_QUEUE), USER_NAME, Q2_QUEUE, Q2_QUEUE, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDefaultQueueInMappingEqualsToInputQueue()
|
|
|
+ throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APP_NAME,
|
|
|
+ DEFAULT_QUEUE), USER_NAME, DEFAULT_QUEUE, DEFAULT_QUEUE, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMappingSourceDiffersFromInputQueue() throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(MAPREDUCE_APP_NAME,
|
|
|
+ Q1_QUEUE), USER_NAME, DEFAULT_QUEUE, DEFAULT_QUEUE, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(expected = YarnException.class)
|
|
|
+ public void testMappingContainsAmbiguousLeafQueueWithoutParent()
|
|
|
+ throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APP_NAME, AMBIGUOUS_QUEUE),
|
|
|
+ USER_NAME, DEFAULT_QUEUE, DEFAULT_QUEUE, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMappingContainsAmbiguousLeafQueueWithParent()
|
|
|
+ throws YarnException {
|
|
|
+ verifyQueueMapping(getQueueMapping(APP_NAME, ROOT_QUEUE, AMBIGUOUS_QUEUE),
|
|
|
+ USER_NAME, DEFAULT_QUEUE, AMBIGUOUS_QUEUE, false);
|
|
|
+ }
|
|
|
+
|
|
|
}
|