|
@@ -23,7 +23,8 @@ import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
import static org.mockito.Mockito.isNull;
|
|
|
|
|
|
-import java.util.Arrays;
|
|
|
+import java.io.IOException;
|
|
|
+import java.util.Collections;
|
|
|
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.security.GroupMappingServiceProvider;
|
|
@@ -35,6 +36,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.placement.QueueMapping.MappingType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.placement.QueueMapping.QueueMappingBuilder;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.placement.TestUserGroupMappingPlacementRule.QueueMappingTestData.QueueMappingTestDataBuilder;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.PrimaryGroupMapping;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SimpleGroupsMapping;
|
|
@@ -42,14 +45,9 @@ import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
-import org.slf4j.Logger;
|
|
|
-import org.slf4j.LoggerFactory;
|
|
|
|
|
|
public class TestUserGroupMappingPlacementRule {
|
|
|
- private static final Logger LOG =
|
|
|
- LoggerFactory.getLogger(TestUserGroupMappingPlacementRule.class);
|
|
|
-
|
|
|
- YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ private final YarnConfiguration conf = new YarnConfiguration();
|
|
|
|
|
|
@Before
|
|
|
public void setup() {
|
|
@@ -57,97 +55,128 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
|
|
|
}
|
|
|
|
|
|
- private void verifyQueueMapping(QueueMappingTestData queueMappingTestData)
|
|
|
- throws YarnException {
|
|
|
-
|
|
|
- QueueMapping queueMapping = queueMappingTestData.queueMapping;
|
|
|
- String inputUser = queueMappingTestData.inputUser;
|
|
|
- String inputQueue = queueMappingTestData.inputQueue;
|
|
|
- String expectedQueue = queueMappingTestData.expectedQueue;
|
|
|
- boolean overwrite = queueMappingTestData.overwrite;
|
|
|
- String expectedParentQueue = queueMappingTestData.expectedParentQueue;
|
|
|
-
|
|
|
- Groups groups = new Groups(conf);
|
|
|
- UserGroupMappingPlacementRule rule = new UserGroupMappingPlacementRule(
|
|
|
- overwrite, Arrays.asList(queueMapping), groups);
|
|
|
- CapacitySchedulerQueueManager queueManager =
|
|
|
- mock(CapacitySchedulerQueueManager.class);
|
|
|
-
|
|
|
+ private void createQueueHierarchy(
|
|
|
+ CapacitySchedulerQueueManager queueManager) {
|
|
|
MockQueueHierarchyBuilder.create()
|
|
|
.withQueueManager(queueManager)
|
|
|
+ .withQueue("root.default")
|
|
|
.withQueue("root.agroup.a")
|
|
|
+ .withQueue("root.bgroup")
|
|
|
+ .withQueue("root.usergroup.c")
|
|
|
.withQueue("root.asubgroup2")
|
|
|
.withQueue("root.bsubgroup2.b")
|
|
|
.withQueue("root.users.primarygrouponly")
|
|
|
+ .withQueue("root.devs.primarygrouponly")
|
|
|
.withQueue("root.admins.primarygrouponly")
|
|
|
.withManagedParentQueue("root.managedParent")
|
|
|
.build();
|
|
|
|
|
|
when(queueManager.getQueue(isNull())).thenReturn(null);
|
|
|
- rule.setQueueManager(queueManager);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void verifyQueueMapping(QueueMappingTestData queueMappingTestData)
|
|
|
+ throws IOException, YarnException {
|
|
|
+ QueueMapping queueMapping = queueMappingTestData.queueMapping;
|
|
|
+ String inputUser = queueMappingTestData.inputUser;
|
|
|
+ String inputQueue = queueMappingTestData.inputQueue;
|
|
|
+ String expectedQueue = queueMappingTestData.expectedQueue;
|
|
|
+ boolean overwrite = queueMappingTestData.overwrite;
|
|
|
+ String expectedParentQueue = queueMappingTestData.expectedParentQueue;
|
|
|
+
|
|
|
+ MappingRule rule = MappingRule.createLegacyRule(
|
|
|
+ queueMapping.getType().toString(),
|
|
|
+ queueMapping.getSource(),
|
|
|
+ queueMapping.getFullPath());
|
|
|
+
|
|
|
+ CSMappingPlacementRule engine = setupEngine(rule, overwrite);
|
|
|
+
|
|
|
ApplicationSubmissionContext asc = Records.newRecord(
|
|
|
ApplicationSubmissionContext.class);
|
|
|
asc.setQueue(inputQueue);
|
|
|
- ApplicationPlacementContext ctx = rule.getPlacementForApp(asc, inputUser);
|
|
|
+ ApplicationPlacementContext ctx = engine.getPlacementForApp(asc, inputUser);
|
|
|
Assert.assertEquals("Queue", expectedQueue,
|
|
|
ctx != null ? ctx.getQueue() : inputQueue);
|
|
|
- if (expectedParentQueue != null) {
|
|
|
+ if (ctx != null && expectedParentQueue != null) {
|
|
|
Assert.assertEquals("Parent Queue", expectedParentQueue,
|
|
|
ctx.getParentQueue());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ CSMappingPlacementRule setupEngine(MappingRule rule,
|
|
|
+ boolean override)
|
|
|
+ throws IOException {
|
|
|
+
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
+ mock(CapacitySchedulerConfiguration.class);
|
|
|
+ when(csConf.getMappingRules()).thenReturn(Collections.singletonList(rule));
|
|
|
+ when(csConf.getOverrideWithQueueMappings())
|
|
|
+ .thenReturn(override);
|
|
|
+ CapacitySchedulerQueueManager queueManager =
|
|
|
+ mock(CapacitySchedulerQueueManager.class);
|
|
|
+ createQueueHierarchy(queueManager);
|
|
|
+
|
|
|
+ CSMappingPlacementRule engine = new CSMappingPlacementRule();
|
|
|
+ Groups groups = new Groups(conf);
|
|
|
+
|
|
|
+ CapacityScheduler cs = mock(CapacityScheduler.class);
|
|
|
+ when(cs.getConfiguration()).thenReturn(csConf);
|
|
|
+ when(cs.getCapacitySchedulerQueueManager()).thenReturn(queueManager);
|
|
|
+
|
|
|
+ engine.setGroups(groups);
|
|
|
+ engine.setFailOnConfigError(false);
|
|
|
+ engine.initialize(cs);
|
|
|
+
|
|
|
+ return engine;
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
- public void testSecondaryGroupMapping() throws YarnException {
|
|
|
+ public void testSecondaryGroupMapping() throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
- .queueMapping(QueueMappingBuilder.create()
|
|
|
- .type(MappingType.USER)
|
|
|
- .source("%user")
|
|
|
- .queue("%secondary_group").build())
|
|
|
- .inputUser("a")
|
|
|
- .expectedQueue("asubgroup2")
|
|
|
- .expectedParentQueue("root")
|
|
|
- .build());
|
|
|
+ .queueMapping(QueueMappingBuilder.create()
|
|
|
+ .type(MappingType.USER)
|
|
|
+ .source("%user")
|
|
|
+ .queue("%secondary_group").build())
|
|
|
+ .inputUser("a")
|
|
|
+ .expectedQueue("asubgroup2")
|
|
|
+ .expectedParentQueue("root")
|
|
|
+ .build());
|
|
|
|
|
|
// PrimaryGroupMapping.class returns only primary group, no secondary groups
|
|
|
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
|
|
|
PrimaryGroupMapping.class, GroupMappingServiceProvider.class);
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
- .queueMapping(QueueMappingBuilder.create()
|
|
|
- .type(MappingType.USER)
|
|
|
- .source("%user")
|
|
|
- .queue("%secondary_group")
|
|
|
- .build())
|
|
|
- .inputUser("a")
|
|
|
- .expectedQueue("default")
|
|
|
- .build());
|
|
|
+ .queueMapping(QueueMappingBuilder.create()
|
|
|
+ .type(MappingType.USER)
|
|
|
+ .source("%user")
|
|
|
+ .queue("%secondary_group")
|
|
|
+ .build())
|
|
|
+ .inputUser("a")
|
|
|
+ .expectedQueue("default")
|
|
|
+ .build());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testNullGroupMapping() {
|
|
|
+ @Test(expected = YarnException.class)
|
|
|
+ public void testNullGroupMapping() throws IOException, YarnException {
|
|
|
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
|
|
|
NullGroupsMapping.class, GroupMappingServiceProvider.class);
|
|
|
- try {
|
|
|
- verifyQueueMapping(
|
|
|
- QueueMappingTestDataBuilder.create()
|
|
|
- .queueMapping(QueueMappingBuilder.create()
|
|
|
- .type(MappingType.USER)
|
|
|
- .source("%user")
|
|
|
- .queue("%secondary_group")
|
|
|
- .build())
|
|
|
- .inputUser("a")
|
|
|
- .expectedQueue("default")
|
|
|
- .build());
|
|
|
- fail("No Groups for user 'a'");
|
|
|
- } catch (YarnException e) {
|
|
|
- // Exception is expected as there are no groups for given user
|
|
|
- }
|
|
|
+ verifyQueueMapping(
|
|
|
+ QueueMappingTestDataBuilder.create()
|
|
|
+ .queueMapping(QueueMappingBuilder.create()
|
|
|
+ .type(MappingType.USER)
|
|
|
+ .source("%user")
|
|
|
+ .queue("%secondary_group")
|
|
|
+ .build())
|
|
|
+ .inputUser("a")
|
|
|
+ .expectedQueue("default")
|
|
|
+ .build());
|
|
|
+ fail("No Groups for user 'a'");
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testSimpleUserMappingToSpecificQueue() throws YarnException {
|
|
|
+ public void testSimpleUserMappingToSpecificQueue()
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -161,7 +190,8 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testSimpleGroupMappingToSpecificQueue() throws YarnException {
|
|
|
+ public void testSimpleGroupMappingToSpecificQueue()
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -175,7 +205,8 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testUserMappingToSpecificQueueForEachUser() throws YarnException {
|
|
|
+ public void testUserMappingToSpecificQueueForEachUser()
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -189,7 +220,8 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testUserMappingToQueueNamedAsUsername() throws YarnException {
|
|
|
+ public void testUserMappingToQueueNamedAsUsername()
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -203,7 +235,8 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testUserMappingToQueueNamedGroupOfTheUser() throws YarnException {
|
|
|
+ public void testUserMappingToQueueNamedGroupOfTheUser()
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -211,15 +244,15 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
.source("%user")
|
|
|
.queue("%primary_group")
|
|
|
.build())
|
|
|
- .inputUser("a")
|
|
|
- .expectedQueue("agroup")
|
|
|
+ .inputUser("b")
|
|
|
+ .expectedQueue("bgroup")
|
|
|
.expectedParentQueue("root")
|
|
|
.build());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToQueueNamedAsUsernameWithPrimaryGroupAsParentQueue()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -236,7 +269,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToPrimaryGroupInvalidNestedPlaceholder()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:%user:%primary_group.%random, no matching queue
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -253,7 +286,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToSecondaryGroupInvalidNestedPlaceholder()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:%user:%secondary_group.%random, no matching queue
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -270,7 +303,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingDiffersFromSubmitterQueueDoesNotExist()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:a:%random, submitter: xyz, no matching queue
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -285,23 +318,24 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testSpecificUserMappingToPrimaryGroup() throws YarnException {
|
|
|
+ public void testSpecificUserMappingToPrimaryGroup()
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:a:%primary_group
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
|
.type(MappingType.USER)
|
|
|
- .source("a")
|
|
|
+ .source("b")
|
|
|
.queue("%primary_group")
|
|
|
.build())
|
|
|
- .inputUser("a")
|
|
|
- .expectedQueue("agroup")
|
|
|
+ .inputUser("b")
|
|
|
+ .expectedQueue("bgroup")
|
|
|
.build());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testSpecificUserMappingToSecondaryGroup()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:a:%secondary_group
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -317,7 +351,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testSpecificUserMappingWithNoSecondaryGroup()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:nosecondarygroupuser:%secondary_group, no matching queue
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -333,7 +367,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testGenericUserMappingWithNoSecondaryGroup()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:%user:%user, no matching queue
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -350,7 +384,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToNestedUserPrimaryGroupWithAmbiguousQueues()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:%user:%user, submitter nosecondarygroupuser, queue is ambiguous
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -365,9 +399,9 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
.build());
|
|
|
}
|
|
|
|
|
|
- @Test(expected = YarnException.class)
|
|
|
+ @Test
|
|
|
public void testResolvedQueueIsNotManaged()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:%user:%primary_group.%user, "admins" group will be "root",
|
|
|
// resulting parent queue will be "root" which is not managed
|
|
|
verifyQueueMapping(
|
|
@@ -379,12 +413,13 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
.parentQueue("%primary_group")
|
|
|
.build())
|
|
|
.inputUser("admins")
|
|
|
+ .expectedQueue("default")
|
|
|
.build());
|
|
|
}
|
|
|
|
|
|
- @Test(expected = YarnException.class)
|
|
|
+ @Test
|
|
|
public void testUserMappingToPrimaryGroupWithAmbiguousQueues()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// u:%user:%primary_group, submitter nosecondarygroupuser,
|
|
|
// queue is ambiguous
|
|
|
verifyQueueMapping(
|
|
@@ -401,7 +436,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToQueueNamedAsUsernameWithSecondaryGroupAsParentQueue()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -417,7 +452,8 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testGroupMappingToStaticQueue() throws YarnException {
|
|
|
+ public void testGroupMappingToStaticQueue()
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -432,7 +468,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToQueueNamedAsGroupNameWithRootAsParentQueue()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -441,15 +477,15 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
.queue("%primary_group")
|
|
|
.parentQueue("root")
|
|
|
.build())
|
|
|
- .inputUser("a")
|
|
|
- .expectedQueue("agroup")
|
|
|
+ .inputUser("b")
|
|
|
+ .expectedQueue("bgroup")
|
|
|
.expectedParentQueue("root")
|
|
|
.build());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToPrimaryGroupQueueDoesNotExistUnmanagedParent()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// "abcgroup" queue doesn't exist, %primary_group queue, not managed parent
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -466,7 +502,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToPrimaryGroupQueueDoesNotExistManagedParent()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// "abcgroup" queue doesn't exist, %primary_group queue, managed parent
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -484,7 +520,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToSecondaryGroupQueueDoesNotExist()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// "abcgroup" queue doesn't exist, %secondary_group queue
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -501,7 +537,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToSecondaryGroupQueueUnderParent()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// "asubgroup2" queue exists, %secondary_group queue
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -519,7 +555,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testUserMappingToSpecifiedQueueOverwritesInputQueueFromMapping()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// specify overwritten, and see if user specified a queue, and it will be
|
|
|
// overridden
|
|
|
verifyQueueMapping(
|
|
@@ -537,8 +573,9 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testUserMappingToExplicitlySpecifiedQueue() throws YarnException {
|
|
|
- // if overwritten not specified, it should be which user specified
|
|
|
+ public void testUserMappingToExplicitlySpecifiedQueue()
|
|
|
+ throws IOException, YarnException {
|
|
|
+ // if overwritten not specified, it should be which user specified
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
|
.queueMapping(QueueMappingBuilder.create()
|
|
@@ -554,7 +591,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testGroupMappingToExplicitlySpecifiedQueue()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// if overwritten not specified, it should be which user specified
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -565,14 +602,14 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
.parentQueue("usergroup")
|
|
|
.build())
|
|
|
.inputUser("user")
|
|
|
- .inputQueue("a")
|
|
|
- .expectedQueue("a")
|
|
|
+ .inputQueue("c")
|
|
|
+ .expectedQueue("c")
|
|
|
.build());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testGroupMappingToSpecifiedQueueOverwritesInputQueueFromMapping()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// if overwritten not specified, it should be which user specified
|
|
|
verifyQueueMapping(
|
|
|
QueueMappingTestDataBuilder.create()
|
|
@@ -591,7 +628,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testGroupMappingToSpecifiedQueueUnderAGivenParentQueue()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// If user specific queue is enabled for a specified group under a given
|
|
|
// parent queue
|
|
|
verifyQueueMapping(
|
|
@@ -609,7 +646,7 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
|
|
|
@Test
|
|
|
public void testGroupMappingToSpecifiedQueueWithoutParentQueue()
|
|
|
- throws YarnException {
|
|
|
+ throws IOException, YarnException {
|
|
|
// If user specific queue is enabled for a specified group without parent
|
|
|
// queue
|
|
|
verifyQueueMapping(
|