|
@@ -24,7 +24,11 @@ import static org.mockito.Mockito.when;
|
|
|
import static org.mockito.Mockito.isNull;
|
|
|
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
|
|
|
+import com.google.common.collect.Maps;
|
|
|
+import org.apache.commons.compress.utils.Lists;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.security.GroupMappingServiceProvider;
|
|
|
import org.apache.hadoop.security.Groups;
|
|
@@ -35,6 +39,7 @@ 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.AbstractCSQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
|
|
@@ -45,8 +50,147 @@ 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);
|
|
|
+
|
|
|
+ private static class MockQueueHierarchyBuilder {
|
|
|
+ private static final String ROOT = "root";
|
|
|
+ private static final String QUEUE_SEP = ".";
|
|
|
+ private List<String> queuePaths = Lists.newArrayList();
|
|
|
+ private List<String> managedParentQueues = Lists.newArrayList();
|
|
|
+ private CapacitySchedulerQueueManager queueManager;
|
|
|
+
|
|
|
+ public static MockQueueHierarchyBuilder create() {
|
|
|
+ return new MockQueueHierarchyBuilder();
|
|
|
+ }
|
|
|
+
|
|
|
+ public MockQueueHierarchyBuilder withQueueManager(
|
|
|
+ CapacitySchedulerQueueManager queueManager) {
|
|
|
+ this.queueManager = queueManager;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public MockQueueHierarchyBuilder withQueue(String queue) {
|
|
|
+ this.queuePaths.add(queue);
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public MockQueueHierarchyBuilder withManagedParentQueue(
|
|
|
+ String managedQueue) {
|
|
|
+ this.managedParentQueues.add(managedQueue);
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public void build() {
|
|
|
+ if (this.queueManager == null) {
|
|
|
+ throw new IllegalStateException(
|
|
|
+ "QueueManager instance is not provided!");
|
|
|
+ }
|
|
|
+
|
|
|
+ for (String managedParentQueue : managedParentQueues) {
|
|
|
+ if (!queuePaths.contains(managedParentQueue)) {
|
|
|
+ queuePaths.add(managedParentQueue);
|
|
|
+ } else {
|
|
|
+ throw new IllegalStateException("Cannot add a managed parent " +
|
|
|
+ "and a simple queue with the same path");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ Map<String, AbstractCSQueue> queues = Maps.newHashMap();
|
|
|
+ for (String queuePath : queuePaths) {
|
|
|
+ LOG.info("Processing queue path: " + queuePath);
|
|
|
+ addQueues(queues, queuePath);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void addQueues(Map<String, AbstractCSQueue> queues,
|
|
|
+ String queuePath) {
|
|
|
+ final String[] pathComponents = queuePath.split("\\" + QUEUE_SEP);
|
|
|
+
|
|
|
+ String currentQueuePath = "";
|
|
|
+ for (int i = 0; i < pathComponents.length; ++i) {
|
|
|
+ boolean isLeaf = i == pathComponents.length - 1;
|
|
|
+ String queueName = pathComponents[i];
|
|
|
+ String parentPath = currentQueuePath;
|
|
|
+ currentQueuePath += currentQueuePath.equals("") ?
|
|
|
+ queueName : QUEUE_SEP + queueName;
|
|
|
+
|
|
|
+ if (managedParentQueues.contains(parentPath) && !isLeaf) {
|
|
|
+ throw new IllegalStateException("Cannot add a queue under " +
|
|
|
+ "managed parent");
|
|
|
+ }
|
|
|
+ if (!queues.containsKey(currentQueuePath)) {
|
|
|
+ ParentQueue parentQueue = (ParentQueue) queues.get(parentPath);
|
|
|
+ AbstractCSQueue queue = createQueue(parentQueue, queueName,
|
|
|
+ currentQueuePath, isLeaf);
|
|
|
+ queues.put(currentQueuePath, queue);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private AbstractCSQueue createQueue(ParentQueue parentQueue,
|
|
|
+ String queueName, String currentQueuePath, boolean isLeaf) {
|
|
|
+ if (queueName.equals(ROOT)) {
|
|
|
+ return createRootQueue(ROOT);
|
|
|
+ } else if (managedParentQueues.contains(currentQueuePath)) {
|
|
|
+ return addManagedParentQueueAsChildOf(parentQueue, queueName);
|
|
|
+ } else if (isLeaf) {
|
|
|
+ return addLeafQueueAsChildOf(parentQueue, queueName);
|
|
|
+ } else {
|
|
|
+ return addParentQueueAsChildOf(parentQueue, queueName);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private AbstractCSQueue createRootQueue(String rootQueueName) {
|
|
|
+ ParentQueue root = mock(ParentQueue.class);
|
|
|
+ when(root.getQueuePath()).thenReturn(rootQueueName);
|
|
|
+ when(queueManager.getQueue(rootQueueName)).thenReturn(root);
|
|
|
+ return root;
|
|
|
+ }
|
|
|
+
|
|
|
+ private AbstractCSQueue addParentQueueAsChildOf(ParentQueue parent,
|
|
|
+ String queueName) {
|
|
|
+ ParentQueue queue = mock(ParentQueue.class);
|
|
|
+ setQueueFields(parent, queue, queueName);
|
|
|
+ return queue;
|
|
|
+ }
|
|
|
+
|
|
|
+ private AbstractCSQueue addManagedParentQueueAsChildOf(ParentQueue parent,
|
|
|
+ String queueName) {
|
|
|
+ ManagedParentQueue queue = mock(ManagedParentQueue.class);
|
|
|
+ setQueueFields(parent, queue, queueName);
|
|
|
+ return queue;
|
|
|
+ }
|
|
|
+
|
|
|
+ private AbstractCSQueue addLeafQueueAsChildOf(ParentQueue parent,
|
|
|
+ String queueName) {
|
|
|
+ LeafQueue queue = mock(LeafQueue.class);
|
|
|
+ setQueueFields(parent, queue, queueName);
|
|
|
+ return queue;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setQueueFields(ParentQueue parent, AbstractCSQueue newQueue,
|
|
|
+ String queueName) {
|
|
|
+ String fullPathOfQueue = parent.getQueuePath() + QUEUE_SEP + queueName;
|
|
|
+ addQueueToQueueManager(queueName, newQueue, fullPathOfQueue);
|
|
|
+
|
|
|
+ when(newQueue.getParent()).thenReturn(parent);
|
|
|
+ when(newQueue.getQueuePath()).thenReturn(fullPathOfQueue);
|
|
|
+ when(newQueue.getQueueName()).thenReturn(queueName);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void addQueueToQueueManager(String queueName, AbstractCSQueue queue,
|
|
|
+ String fullPathOfQueue) {
|
|
|
+ when(queueManager.getQueue(queueName)).thenReturn(queue);
|
|
|
+ when(queueManager.getQueue(fullPathOfQueue)).thenReturn(queue);
|
|
|
+ when(queueManager.getQueueByFullName(fullPathOfQueue)).thenReturn(queue);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
YarnConfiguration conf = new YarnConfiguration();
|
|
|
|
|
|
@Before
|
|
@@ -71,61 +215,15 @@ public class TestUserGroupMappingPlacementRule {
|
|
|
CapacitySchedulerQueueManager queueManager =
|
|
|
mock(CapacitySchedulerQueueManager.class);
|
|
|
|
|
|
- ParentQueue root = mock(ParentQueue.class);
|
|
|
- when(root.getQueuePath()).thenReturn("root");
|
|
|
-
|
|
|
- ParentQueue agroup = mock(ParentQueue.class);
|
|
|
- when(agroup.getQueuePath()).thenReturn("root.agroup");
|
|
|
- ParentQueue bsubgroup2 = mock(ParentQueue.class);
|
|
|
- when(bsubgroup2.getQueuePath()).thenReturn("root.bsubgroup2");
|
|
|
- when(bsubgroup2.getParent()).thenReturn(root);
|
|
|
-
|
|
|
- ManagedParentQueue managedParent = mock(ManagedParentQueue.class);
|
|
|
- when(managedParent.getQueueName()).thenReturn("managedParent");
|
|
|
- when(managedParent.getQueuePath()).thenReturn("root.managedParent");
|
|
|
-
|
|
|
- LeafQueue a = mock(LeafQueue.class);
|
|
|
- when(a.getQueuePath()).thenReturn("root.agroup.a");
|
|
|
- when(a.getParent()).thenReturn(agroup);
|
|
|
- LeafQueue b = mock(LeafQueue.class);
|
|
|
- when(b.getQueuePath()).thenReturn("root.bsubgroup2.b");
|
|
|
- when(b.getParent()).thenReturn(bsubgroup2);
|
|
|
- LeafQueue asubgroup2 = mock(LeafQueue.class);
|
|
|
- when(asubgroup2.getQueuePath()).thenReturn("root.asubgroup2");
|
|
|
- when(asubgroup2.getParent()).thenReturn(root);
|
|
|
+ MockQueueHierarchyBuilder.create()
|
|
|
+ .withQueueManager(queueManager)
|
|
|
+ .withQueue("root.agroup.a")
|
|
|
+ .withQueue("root.asubgroup2")
|
|
|
+ .withQueue("root.bsubgroup2.b")
|
|
|
+ .withManagedParentQueue("root.managedParent")
|
|
|
+ .build();
|
|
|
|
|
|
when(queueManager.getQueue(isNull())).thenReturn(null);
|
|
|
- when(queueManager.getQueue("a")).thenReturn(a);
|
|
|
- when(a.getParent()).thenReturn(agroup);
|
|
|
- when(queueManager.getQueue("b")).thenReturn(b);
|
|
|
- when(b.getParent()).thenReturn(bsubgroup2);
|
|
|
- when(queueManager.getQueue("agroup")).thenReturn(agroup);
|
|
|
- when(agroup.getParent()).thenReturn(root);
|
|
|
- when(queueManager.getQueue("bsubgroup2")).thenReturn(bsubgroup2);
|
|
|
- when(bsubgroup2.getParent()).thenReturn(root);
|
|
|
- when(queueManager.getQueue("asubgroup2")).thenReturn(asubgroup2);
|
|
|
- when(asubgroup2.getParent()).thenReturn(root);
|
|
|
- when(queueManager.getQueue("managedParent")).thenReturn(managedParent);
|
|
|
- when(managedParent.getParent()).thenReturn(root);
|
|
|
-
|
|
|
- when(queueManager.getQueue("root")).thenReturn(root);
|
|
|
- when(queueManager.getQueue("root.agroup")).thenReturn(agroup);
|
|
|
- when(queueManager.getQueue("root.bsubgroup2")).thenReturn(bsubgroup2);
|
|
|
- when(queueManager.getQueue("root.asubgroup2")).thenReturn(asubgroup2);
|
|
|
- when(queueManager.getQueue("root.agroup.a")).thenReturn(a);
|
|
|
- when(queueManager.getQueue("root.bsubgroup2.b")).thenReturn(b);
|
|
|
- when(queueManager.getQueue("root.managedParent")).thenReturn(managedParent);
|
|
|
-
|
|
|
- when(queueManager.getQueueByFullName("root.agroup")).thenReturn(agroup);
|
|
|
- when(queueManager.getQueueByFullName("root.bsubgroup2"))
|
|
|
- .thenReturn(bsubgroup2);
|
|
|
- when(queueManager.getQueueByFullName("root.asubgroup2"))
|
|
|
- .thenReturn(asubgroup2);
|
|
|
- when(queueManager.getQueueByFullName("root.agroup.a")).thenReturn(a);
|
|
|
- when(queueManager.getQueueByFullName("root.bsubgroup2.b")).thenReturn(b);
|
|
|
- when(queueManager.getQueueByFullName("root.managedParent"))
|
|
|
- .thenReturn(managedParent);
|
|
|
-
|
|
|
|
|
|
rule.setQueueManager(queueManager);
|
|
|
ApplicationSubmissionContext asc = Records.newRecord(
|