|
@@ -16,7 +16,9 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter;
|
|
|
|
|
|
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DEFAULT_MAX_PARALLEL_APPLICATIONS;
|
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX;
|
|
|
+import static org.junit.Assert.assertArrayEquals;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
@@ -35,6 +37,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
|
|
@@ -81,12 +85,13 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
private FSQueueConverter converter;
|
|
|
private Configuration yarnConfig;
|
|
|
- private Configuration csConfig;
|
|
|
+ private CapacitySchedulerConfiguration csConfig;
|
|
|
private FairScheduler fs;
|
|
|
private FSQueue rootQueue;
|
|
|
private ConversionOptions conversionOptions;
|
|
|
private DryRunResultHolder dryRunResultHolder;
|
|
|
private FSQueueConverterBuilder builder;
|
|
|
+ private String key;
|
|
|
|
|
|
@Mock
|
|
|
private FSConfigToCSConfigRuleHandler ruleHandler;
|
|
@@ -100,7 +105,8 @@ public class TestFSQueueConverter {
|
|
|
yarnConfig.set(FairSchedulerConfiguration.ALLOCATION_FILE,
|
|
|
FAIR_SCHEDULER_XML);
|
|
|
yarnConfig.setBoolean(FairSchedulerConfiguration.MIGRATION_MODE, true);
|
|
|
- csConfig = new Configuration(false);
|
|
|
+ csConfig = new CapacitySchedulerConfiguration(
|
|
|
+ new Configuration(false));
|
|
|
dryRunResultHolder = new DryRunResultHolder();
|
|
|
conversionOptions =
|
|
|
new ConversionOptions(dryRunResultHolder, false);
|
|
@@ -149,20 +155,20 @@ public class TestFSQueueConverter {
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
// root children
|
|
|
- assertEquals("root children", "admins,users,misc,default",
|
|
|
- csConfig.get(PREFIX + "root.queues"));
|
|
|
+ assertArrayEquals("root children", new String[]{"admins", "users", "misc", "default"},
|
|
|
+ csConfig.getQueues("root"));
|
|
|
|
|
|
// root.admins children
|
|
|
- assertEquals("root.admins children", "bob,alice",
|
|
|
- csConfig.get(PREFIX + "root.admins.queues"));
|
|
|
+ assertArrayEquals("root.admins children", new String[]{"bob", "alice"},
|
|
|
+ csConfig.getQueues("root.admins"));
|
|
|
|
|
|
// root.default children - none
|
|
|
- assertNull("root.default children", csConfig.get(PREFIX + "root.default" +
|
|
|
- ".queues"));
|
|
|
+ assertNull("root.default children",
|
|
|
+ csConfig.getQueues("root.default"));
|
|
|
|
|
|
// root.users children
|
|
|
- assertEquals("root.users children", "john,joe",
|
|
|
- csConfig.get(PREFIX + "root.users.queues"));
|
|
|
+ assertArrayEquals("root.users children", new String[]{"john", "joe"},
|
|
|
+ csConfig.getQueues("root.users"));
|
|
|
|
|
|
Set<String> leafs = Sets.difference(ALL_QUEUES,
|
|
|
Sets.newHashSet("root",
|
|
@@ -171,7 +177,12 @@ public class TestFSQueueConverter {
|
|
|
"root.users",
|
|
|
"root.misc"));
|
|
|
|
|
|
- assertNoValueForQueues(leafs, ".queues", csConfig);
|
|
|
+ for (String queue : leafs) {
|
|
|
+ key = PREFIX + queue + ".queues";
|
|
|
+ assertNull("Key " + key + " has value, but it should be null",
|
|
|
+ csConfig.getQueues(queue));
|
|
|
+ }
|
|
|
+
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -181,18 +192,24 @@ public class TestFSQueueConverter {
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
// root.admins.bob
|
|
|
- assertEquals("root.admins.bob AM share", "1.0",
|
|
|
- csConfig.get(PREFIX + "root.admins.bob.maximum-am-resource-percent"));
|
|
|
+ assertEquals("root.admins.bob AM share", 1.0f,
|
|
|
+ csConfig.getMaximumApplicationMasterResourcePerQueuePercent(
|
|
|
+ "root.admins.bob"), 0.0f);
|
|
|
|
|
|
// root.admins.alice
|
|
|
- assertEquals("root.admins.alice AM share", "0.15",
|
|
|
- csConfig.get(PREFIX +
|
|
|
- "root.admins.alice.maximum-am-resource-percent"));
|
|
|
+ assertEquals("root.admins.alice AM share", 0.15f,
|
|
|
+ csConfig.getMaximumApplicationMasterResourcePerQueuePercent(
|
|
|
+ "root.admins.alice"), 0.0f);
|
|
|
|
|
|
Set<String> remaining = Sets.difference(ALL_QUEUES,
|
|
|
Sets.newHashSet("root.admins.bob", "root.admins.alice"));
|
|
|
- assertNoValueForQueues(remaining, ".maximum-am-resource-percent",
|
|
|
- csConfig);
|
|
|
+
|
|
|
+ for (String queue : remaining) {
|
|
|
+ key = PREFIX + queue + ".maximum-am-resource-percent";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ 0.1f, csConfig
|
|
|
+ .getMaximumApplicationMasterResourcePerQueuePercent(queue), 0.0f);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -202,12 +219,17 @@ public class TestFSQueueConverter {
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
assertEquals("root.admins.alice max apps", 2,
|
|
|
- csConfig.getInt(PREFIX + "root.admins.alice.max-parallel-apps",
|
|
|
- -1));
|
|
|
+ csConfig.getMaxParallelAppsForQueue("root.admins.alice"), 0);
|
|
|
|
|
|
Set<String> remaining = Sets.difference(ALL_QUEUES,
|
|
|
Sets.newHashSet("root.admins.alice"));
|
|
|
- assertNoValueForQueues(remaining, ".max-parallel-apps", csConfig);
|
|
|
+
|
|
|
+ for (String queue : remaining) {
|
|
|
+ key = PREFIX + queue + ".max-parallel-apps";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ DEFAULT_MAX_PARALLEL_APPLICATIONS, csConfig
|
|
|
+ .getMaxParallelAppsForQueue(queue), 0);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -218,21 +240,30 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
// root.admins vcores + mb
|
|
|
assertEquals("root.admins max vcores", 3,
|
|
|
- csConfig.getInt(PREFIX + "root.admins.maximum-allocation-vcores", -1));
|
|
|
+ csConfig.getQueueMaximumAllocationVcores("root.admins"));
|
|
|
assertEquals("root.admins max memory", 4096,
|
|
|
- csConfig.getInt(PREFIX + "root.admins.maximum-allocation-mb", -1));
|
|
|
+ csConfig.getQueueMaximumAllocationMb("root.admins"));
|
|
|
|
|
|
// root.users.john max vcores + mb
|
|
|
assertEquals("root.users.john max vcores", 2,
|
|
|
- csConfig.getInt(PREFIX + "root.users.john.maximum-allocation-vcores",
|
|
|
- -1));
|
|
|
+ csConfig.getQueueMaximumAllocationVcores("root.users.john"));
|
|
|
assertEquals("root.users.john max memory", 8192,
|
|
|
- csConfig.getInt(PREFIX + "root.users.john.maximum-allocation-mb", -1));
|
|
|
+ csConfig.getQueueMaximumAllocationMb("root.users.john"));
|
|
|
|
|
|
Set<String> remaining = Sets.difference(ALL_QUEUES,
|
|
|
Sets.newHashSet("root.admins", "root.users.john"));
|
|
|
- assertNoValueForQueues(remaining, ".maximum-allocation-vcores", csConfig);
|
|
|
- assertNoValueForQueues(remaining, ".maximum-allocation-mb", csConfig);
|
|
|
+
|
|
|
+ for (String queue : remaining) {
|
|
|
+ key = PREFIX + queue + ".maximum-allocation-vcores";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ -1.0, csConfig
|
|
|
+ .getQueueMaximumAllocationVcores(queue), 0.0f);
|
|
|
+
|
|
|
+ key = PREFIX + queue + ".maximum-allocation-mb";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ -1.0, csConfig
|
|
|
+ .getQueueMaximumAllocationMb(queue), 0.0f);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -242,15 +273,20 @@ public class TestFSQueueConverter {
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
assertTrue("root.admins.alice preemption setting",
|
|
|
- csConfig.getBoolean(PREFIX + "root.admins.alice.disable_preemption",
|
|
|
- false));
|
|
|
+ csConfig.getPreemptionDisabled(
|
|
|
+ "root.admins.alice", false));
|
|
|
assertTrue("root.users.joe preemption setting",
|
|
|
- csConfig.getBoolean(PREFIX + "root.users.joe.disable_preemption",
|
|
|
- false));
|
|
|
+ csConfig.getPreemptionDisabled(
|
|
|
+ "root.users.joe", false));
|
|
|
|
|
|
Set<String> remaining = Sets.difference(ALL_QUEUES,
|
|
|
Sets.newHashSet("root.admins.alice", "root.users.joe"));
|
|
|
- assertNoValueForQueues(remaining, ".disable_preemption", csConfig);
|
|
|
+
|
|
|
+ for (String queue : remaining) {
|
|
|
+ key = PREFIX + queue + ".disable_preemption";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ false, csConfig.getPreemptionDisabled(queue, false));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -259,7 +295,11 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
- assertNoValueForQueues(ALL_QUEUES, ".disable_preemption", csConfig);
|
|
|
+ for (String queue : ALL_QUEUES) {
|
|
|
+ key = PREFIX + queue + ".disable_preemption";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ false, csConfig.getPreemptionDisabled(queue, false));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -269,32 +309,42 @@ public class TestFSQueueConverter {
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
// root
|
|
|
- assertEquals("root.default capacity", "33.333",
|
|
|
- csConfig.get(PREFIX + "root.default.capacity"));
|
|
|
- assertEquals("root.admins capacity", "33.333",
|
|
|
- csConfig.get(PREFIX + "root.admins.capacity"));
|
|
|
- assertEquals("root.users capacity", "33.334",
|
|
|
- csConfig.get(PREFIX + "root.users.capacity"));
|
|
|
+ assertEquals("root.default capacity", 33.333f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.default")), 0.0f);
|
|
|
+ assertEquals("root.admins capacity", 33.333f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.admins")), 0.0f);
|
|
|
+ assertEquals("root.users capacity", 33.334f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.users")), 0.0f);
|
|
|
|
|
|
// root.users
|
|
|
- assertEquals("root.users.john capacity", "25.000",
|
|
|
- csConfig.get(PREFIX + "root.users.john.capacity"));
|
|
|
- assertEquals("root.users.joe capacity", "75.000",
|
|
|
- csConfig.get(PREFIX + "root.users.joe.capacity"));
|
|
|
+ assertEquals("root.users.john capacity", 25.000f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.users.john")), 0.0f);
|
|
|
+ assertEquals("root.users.joe capacity", 75.000f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.users.joe")), 0.0f);
|
|
|
|
|
|
// root.admins
|
|
|
- assertEquals("root.admins.alice capacity", "75.000",
|
|
|
- csConfig.get(PREFIX + "root.admins.alice.capacity"));
|
|
|
- assertEquals("root.admins.bob capacity", "25.000",
|
|
|
- csConfig.get(PREFIX + "root.admins.bob.capacity"));
|
|
|
+ assertEquals("root.admins.alice capacity", 75.000f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.admins.alice")), 0.0f);
|
|
|
+ assertEquals("root.admins.bob capacity", 25.000f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.admins.bob")), 0.0f);
|
|
|
|
|
|
// root.misc
|
|
|
- assertEquals("root.misc capacity", "0.000",
|
|
|
- csConfig.get(PREFIX + "root.misc.capacity"));
|
|
|
- assertEquals("root.misc.a capacity", "0.000",
|
|
|
- csConfig.get(PREFIX + "root.misc.a.capacity"));
|
|
|
- assertEquals("root.misc.b capacity", "0.000",
|
|
|
- csConfig.get(PREFIX + "root.misc.b.capacity"));
|
|
|
+ assertEquals("root.misc capacity", 0.000f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.misc")), 0.000f);
|
|
|
+ assertEquals("root.misc.a capacity", 0.000f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.misc.a")), 0.000f);
|
|
|
+ assertEquals("root.misc.b capacity", 0.000f,
|
|
|
+ csConfig.getNonLabeledQueueCapacity(
|
|
|
+ new QueuePath("root.misc.b")), 0.000f);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -304,32 +354,32 @@ public class TestFSQueueConverter {
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
// root
|
|
|
- assertEquals("root.default weight", "1.0w",
|
|
|
- csConfig.get(PREFIX + "root.default.capacity"));
|
|
|
- assertEquals("root.admins weight", "1.0w",
|
|
|
- csConfig.get(PREFIX + "root.admins.capacity"));
|
|
|
- assertEquals("root.users weight", "1.0w",
|
|
|
- csConfig.get(PREFIX + "root.users.capacity"));
|
|
|
+ assertEquals("root.default weight", 1.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.default"), 0.01f);
|
|
|
+ assertEquals("root.admins weight", 1.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.admins"), 0.01f);
|
|
|
+ assertEquals("root.users weight", 1.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.users"), 0.01f);
|
|
|
|
|
|
// root.users
|
|
|
- assertEquals("root.users.john weight", "1.0w",
|
|
|
- csConfig.get(PREFIX + "root.users.john.capacity"));
|
|
|
- assertEquals("root.users.joe weight", "3.0w",
|
|
|
- csConfig.get(PREFIX + "root.users.joe.capacity"));
|
|
|
+ assertEquals("root.users.john weight", 1.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.users.john"), 0.01f);
|
|
|
+ assertEquals("root.users.joe weight", 3.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.users.joe"), 0.01f);
|
|
|
|
|
|
// root.admins
|
|
|
- assertEquals("root.admins.alice weight", "3.0w",
|
|
|
- csConfig.get(PREFIX + "root.admins.alice.capacity"));
|
|
|
- assertEquals("root.admins.bob weight", "1.0w",
|
|
|
- csConfig.get(PREFIX + "root.admins.bob.capacity"));
|
|
|
+ assertEquals("root.admins.alice weight", 3.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.admins.alice"), 0.01f);
|
|
|
+ assertEquals("root.admins.bob weight", 1.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.admins.bob"), 0.01f);
|
|
|
|
|
|
// root.misc
|
|
|
- assertEquals("root.misc weight", "0.0w",
|
|
|
- csConfig.get(PREFIX + "root.misc.capacity"));
|
|
|
- assertEquals("root.misc.a weight", "0.0w",
|
|
|
- csConfig.get(PREFIX + "root.misc.a.capacity"));
|
|
|
- assertEquals("root.misc.b weight", "0.0w",
|
|
|
- csConfig.get(PREFIX + "root.misc.b.capacity"));
|
|
|
+ assertEquals("root.misc weight", 0.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.misc"), 0.00f);
|
|
|
+ assertEquals("root.misc.a weight", 0.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.misc.a"), 0.00f);
|
|
|
+ assertEquals("root.misc.b weight", 0.0f,
|
|
|
+ csConfig.getNonLabeledQueueWeight("root.misc.b"), 0.00f);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -339,21 +389,15 @@ public class TestFSQueueConverter {
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
assertTrue("root autocreate v2 flag",
|
|
|
- csConfig.getBoolean(
|
|
|
- PREFIX + "root.auto-queue-creation-v2.enabled", false));
|
|
|
+ csConfig.isAutoQueueCreationV2Enabled("root"));
|
|
|
assertTrue("root.admins autocreate v2 flag",
|
|
|
- csConfig.getBoolean(
|
|
|
- PREFIX + "root.admins.auto-queue-creation-v2.enabled", false));
|
|
|
+ csConfig.isAutoQueueCreationV2Enabled("root.admins"));
|
|
|
assertTrue("root.admins.alice autocreate v2 flag",
|
|
|
- csConfig.getBoolean(
|
|
|
- PREFIX + "root.admins.alice.auto-queue-creation-v2.enabled",
|
|
|
- false));
|
|
|
+ csConfig.isAutoQueueCreationV2Enabled("root.admins.alice"));
|
|
|
assertTrue("root.users autocreate v2 flag",
|
|
|
- csConfig.getBoolean(
|
|
|
- PREFIX + "root.users.auto-queue-creation-v2.enabled", false));
|
|
|
+ csConfig.isAutoQueueCreationV2Enabled("root.users"));
|
|
|
assertTrue("root.misc autocreate v2 flag",
|
|
|
- csConfig.getBoolean(
|
|
|
- PREFIX + "root.misc.auto-queue-creation-v2.enabled", false));
|
|
|
+ csConfig.isAutoQueueCreationV2Enabled("root.misc"));
|
|
|
|
|
|
//leaf queue root.admins.alice is removed from the below list
|
|
|
//adding reservation to a leaf, it's queueType changes to FSParentQueue
|
|
@@ -363,8 +407,14 @@ public class TestFSQueueConverter {
|
|
|
"root.users",
|
|
|
"root.misc",
|
|
|
"root.admins.alice"));
|
|
|
- assertNoValueForQueues(leafs, ".auto-queue-creation-v2.enabled",
|
|
|
- csConfig);
|
|
|
+
|
|
|
+ for (String queue : leafs) {
|
|
|
+ key = PREFIX + queue + ".auto-queue-creation-v2.enabled";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ false, csConfig
|
|
|
+ .isAutoQueueCreationV2Enabled(queue));
|
|
|
+ }
|
|
|
+
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -375,11 +425,16 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
Set<String> noZeroSumAllowedQueues = Sets.difference(ALL_QUEUES,
|
|
|
Sets.newHashSet("root.misc"));
|
|
|
- assertNoValueForQueues(noZeroSumAllowedQueues, ".allow-zero-capacity-sum",
|
|
|
- csConfig);
|
|
|
|
|
|
- assertTrue("root.misc allow zero capacities", csConfig.getBoolean(
|
|
|
- PREFIX + "root.misc.allow-zero-capacity-sum", false));
|
|
|
+ for (String queue : noZeroSumAllowedQueues) {
|
|
|
+ key = PREFIX + queue + ".allow-zero-capacity-sum";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ false, csConfig
|
|
|
+ .getAllowZeroCapacitySum(queue));
|
|
|
+ }
|
|
|
+
|
|
|
+ assertTrue("root.misc allow zero capacities",
|
|
|
+ csConfig.getAllowZeroCapacitySum("root.misc"));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -388,7 +443,12 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
- assertValueForQueues(ALL_QUEUES, ".maximum-capacity", csConfig, "100");
|
|
|
+ for (String queue : ALL_QUEUES) {
|
|
|
+ key = PREFIX + queue + ".maximum-capacity";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ 100.0, csConfig
|
|
|
+ .getNonLabeledQueueMaximumCapacity(new QueuePath(queue)), 0.0f);
|
|
|
+ }
|
|
|
verify(ruleHandler, times(3)).handleMaxResources();
|
|
|
}
|
|
|
|
|
@@ -409,8 +469,11 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
- assertNoValueForQueues(ALL_QUEUES, ".auto-create-child-queue.enabled",
|
|
|
- csConfig);
|
|
|
+ for (String queue : ALL_QUEUES) {
|
|
|
+ key = PREFIX + queue + ".auto-create-child-queue.enabled";
|
|
|
+ assertEquals("Key " + key + " has different value",
|
|
|
+ false, csConfig.isAutoCreateChildQueueEnabled(queue));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -419,8 +482,11 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
- assertTrueForQueues(ALL_QUEUES,
|
|
|
- ".ordering-policy.fair.enable-size-based-weight", csConfig);
|
|
|
+ for (String queue : ALL_QUEUES) {
|
|
|
+ key = PREFIX + queue + ".ordering-policy.fair.enable-size-based-weight";
|
|
|
+ assertTrue("Key " + key + " has different value",
|
|
|
+ csConfig.getBoolean(key, false));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -429,8 +495,11 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
|
|
|
- assertNoValueForQueues(ALL_QUEUES,
|
|
|
- ".ordering-policy.fair.enable-size-based-weight", csConfig);
|
|
|
+ for (String queue : ALL_QUEUES) {
|
|
|
+ key = PREFIX + queue + ".ordering-policy.fair.enable-size-based-weight";
|
|
|
+ assertNull("Key " + key + " has different value",
|
|
|
+ csConfig.get(key));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -446,28 +515,27 @@ public class TestFSQueueConverter {
|
|
|
rootQueue = fs.getQueueManager().getRootQueue();
|
|
|
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
-
|
|
|
// root
|
|
|
- assertEquals("root ordering policy", null,
|
|
|
- csConfig.get(PREFIX + "root.ordering-policy"));
|
|
|
+ assertEquals("root ordering policy", "fifo",
|
|
|
+ csConfig.getAppOrderingPolicy("root").getConfigName());
|
|
|
assertEquals("root.default ordering policy", "fair",
|
|
|
- csConfig.get(PREFIX + "root.default.ordering-policy"));
|
|
|
- assertEquals("root.admins ordering policy", null,
|
|
|
- csConfig.get(PREFIX + "root.admins.ordering-policy"));
|
|
|
- assertEquals("root.users ordering policy", null,
|
|
|
- csConfig.get(PREFIX + "root.users.ordering-policy"));
|
|
|
+ csConfig.getAppOrderingPolicy("root.default").getConfigName());
|
|
|
+ assertEquals("root.admins ordering policy", "fifo",
|
|
|
+ csConfig.getAppOrderingPolicy("root.admins").getConfigName());
|
|
|
+ assertEquals("root.users ordering policy", "fifo",
|
|
|
+ csConfig.getAppOrderingPolicy("root.users").getConfigName());
|
|
|
|
|
|
// root.users
|
|
|
assertEquals("root.users.joe ordering policy", "fair",
|
|
|
- csConfig.get(PREFIX + "root.users.joe.ordering-policy"));
|
|
|
+ csConfig.getAppOrderingPolicy("root.users.joe").getConfigName());
|
|
|
assertEquals("root.users.john ordering policy", "fifo",
|
|
|
- csConfig.get(PREFIX + "root.users.john.ordering-policy"));
|
|
|
+ csConfig.getAppOrderingPolicy("root.users.john").getConfigName());
|
|
|
|
|
|
// root.admins
|
|
|
assertEquals("root.admins.alice ordering policy", "fifo",
|
|
|
- csConfig.get(PREFIX + "root.admins.alice.ordering-policy"));
|
|
|
+ csConfig.getAppOrderingPolicy("root.admins.alice.").getConfigName());
|
|
|
assertEquals("root.admins.bob ordering policy", "fair",
|
|
|
- csConfig.get(PREFIX + "root.admins.bob.ordering-policy"));
|
|
|
+ csConfig.getAppOrderingPolicy("root.admins.bob").getConfigName());
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -512,31 +580,4 @@ public class TestFSQueueConverter {
|
|
|
|
|
|
converter.convertQueueHierarchy(rootQueue);
|
|
|
}
|
|
|
-
|
|
|
- private void assertNoValueForQueues(Set<String> queues, String postfix,
|
|
|
- Configuration config) {
|
|
|
- for (String queue : queues) {
|
|
|
- String key = PREFIX + queue + postfix;
|
|
|
- assertNull("Key " + key + " has value, but it should be null",
|
|
|
- config.get(key));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void assertValueForQueues(Set<String> queues, String postfix,
|
|
|
- Configuration config, String expectedValue) {
|
|
|
- for (String queue : queues) {
|
|
|
- String key = PREFIX + queue + postfix;
|
|
|
- assertEquals("Key " + key + " has different value",
|
|
|
- expectedValue, config.get(key));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void assertTrueForQueues(Set<String> queues, String postfix,
|
|
|
- Configuration config) {
|
|
|
- for (String queue : queues) {
|
|
|
- String key = PREFIX + queue + postfix;
|
|
|
- assertTrue("Key " + key + " is false, should be true",
|
|
|
- config.getBoolean(key, false));
|
|
|
- }
|
|
|
- }
|
|
|
}
|