|
@@ -500,9 +500,12 @@ public class TestCapacityScheduler {
|
|
|
public void testParseQueue() throws IOException {
|
|
|
CapacityScheduler cs = new CapacityScheduler();
|
|
|
cs.setConf(new YarnConfiguration());
|
|
|
-
|
|
|
+ cs.setRMContext(resourceManager.getRMContext());
|
|
|
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
|
|
|
setupQueueConfiguration(conf);
|
|
|
+ cs.init(conf);
|
|
|
+ cs.start();
|
|
|
+
|
|
|
conf.setQueues(CapacitySchedulerConfiguration.ROOT + ".a.a1", new String[] {"b1"} );
|
|
|
conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a.a1.b1", 100.0f);
|
|
|
conf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".a.a1.b1", 100.0f);
|
|
@@ -2124,4 +2127,301 @@ public class TestCapacityScheduler {
|
|
|
assertFalse("queue " + B2 + " should have been preemptable",
|
|
|
queueB2.getPreemptionDisabled());
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRefreshQueuesMaxAllocationRefresh() throws Exception {
|
|
|
+ // queue refresh should not allow changing the maximum allocation setting
|
|
|
+ // per queue to be smaller than previous setting
|
|
|
+ CapacityScheduler cs = new CapacityScheduler();
|
|
|
+ CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
|
|
|
+ setupQueueConfiguration(conf);
|
|
|
+ cs.setConf(new YarnConfiguration());
|
|
|
+ cs.setRMContext(resourceManager.getRMContext());
|
|
|
+ cs.init(conf);
|
|
|
+ cs.start();
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
|
|
|
+
|
|
|
+ assertEquals("max allocation in CS",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
+ cs.getMaximumResourceCapability().getMemory());
|
|
|
+ assertEquals("max allocation for A1",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
+ conf.getMaximumAllocationPerQueue(A1).getMemory());
|
|
|
+ assertEquals("max allocation",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
+ conf.getMaximumAllocation().getMemory());
|
|
|
+
|
|
|
+ CSQueue rootQueue = cs.getRootQueue();
|
|
|
+ CSQueue queueA = findQueue(rootQueue, A);
|
|
|
+ CSQueue queueA1 = findQueue(queueA, A1);
|
|
|
+ assertEquals("queue max allocation", ((LeafQueue) queueA1)
|
|
|
+ .getMaximumAllocation().getMemory(), 8192);
|
|
|
+
|
|
|
+ setMaxAllocMb(conf, A1, 4096);
|
|
|
+
|
|
|
+ try {
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ fail("should have thrown exception");
|
|
|
+ } catch (IOException e) {
|
|
|
+ assertTrue("max allocation exception",
|
|
|
+ e.getCause().toString().contains("not be decreased"));
|
|
|
+ }
|
|
|
+
|
|
|
+ setMaxAllocMb(conf, A1, 8192);
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+
|
|
|
+ setMaxAllocVcores(conf, A1,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES - 1);
|
|
|
+ try {
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ fail("should have thrown exception");
|
|
|
+ } catch (IOException e) {
|
|
|
+ assertTrue("max allocation exception",
|
|
|
+ e.getCause().toString().contains("not be decreased"));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRefreshQueuesMaxAllocationPerQueueLarge() throws Exception {
|
|
|
+ // verify we can't set the allocation per queue larger then cluster setting
|
|
|
+ CapacityScheduler cs = new CapacityScheduler();
|
|
|
+ cs.setConf(new YarnConfiguration());
|
|
|
+ cs.setRMContext(resourceManager.getRMContext());
|
|
|
+ CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
|
|
|
+ setupQueueConfiguration(conf);
|
|
|
+ cs.init(conf);
|
|
|
+ cs.start();
|
|
|
+ // change max allocation for B3 queue to be larger then cluster max
|
|
|
+ setMaxAllocMb(conf, B3,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 2048);
|
|
|
+ try {
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ fail("should have thrown exception");
|
|
|
+ } catch (IOException e) {
|
|
|
+ assertTrue("maximum allocation exception",
|
|
|
+ e.getCause().getMessage().contains("maximum allocation"));
|
|
|
+ }
|
|
|
+
|
|
|
+ setMaxAllocMb(conf, B3,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+
|
|
|
+ setMaxAllocVcores(conf, B3,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
|
|
|
+ try {
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ fail("should have thrown exception");
|
|
|
+ } catch (IOException e) {
|
|
|
+ assertTrue("maximum allocation exception",
|
|
|
+ e.getCause().getMessage().contains("maximum allocation"));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRefreshQueuesMaxAllocationRefreshLarger() throws Exception {
|
|
|
+ // queue refresh should allow max allocation per queue to go larger
|
|
|
+ CapacityScheduler cs = new CapacityScheduler();
|
|
|
+ cs.setConf(new YarnConfiguration());
|
|
|
+ cs.setRMContext(resourceManager.getRMContext());
|
|
|
+ CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
|
|
|
+ setupQueueConfiguration(conf);
|
|
|
+ setMaxAllocMb(conf,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
|
|
|
+ setMaxAllocVcores(conf,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
|
|
|
+ setMaxAllocMb(conf, A1, 4096);
|
|
|
+ setMaxAllocVcores(conf, A1, 2);
|
|
|
+ cs.init(conf);
|
|
|
+ cs.start();
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
|
|
|
+
|
|
|
+ assertEquals("max capability MB in CS",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
+ cs.getMaximumResourceCapability().getMemory());
|
|
|
+ assertEquals("max capability vcores in CS",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
|
|
|
+ cs.getMaximumResourceCapability().getVirtualCores());
|
|
|
+ assertEquals("max allocation MB A1",
|
|
|
+ 4096,
|
|
|
+ conf.getMaximumAllocationPerQueue(A1).getMemory());
|
|
|
+ assertEquals("max allocation vcores A1",
|
|
|
+ 2,
|
|
|
+ conf.getMaximumAllocationPerQueue(A1).getVirtualCores());
|
|
|
+ assertEquals("cluster max allocation MB",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
+ conf.getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("cluster max allocation vcores",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
|
|
|
+ conf.getMaximumAllocation().getVirtualCores());
|
|
|
+
|
|
|
+ CSQueue rootQueue = cs.getRootQueue();
|
|
|
+ CSQueue queueA = findQueue(rootQueue, A);
|
|
|
+ CSQueue queueA1 = findQueue(queueA, A1);
|
|
|
+ assertEquals("queue max allocation", ((LeafQueue) queueA1)
|
|
|
+ .getMaximumAllocation().getMemory(), 4096);
|
|
|
+
|
|
|
+ setMaxAllocMb(conf, A1, 6144);
|
|
|
+ setMaxAllocVcores(conf, A1, 3);
|
|
|
+ cs.reinitialize(conf, null);
|
|
|
+ // conf will have changed but we shouldn't be able to change max allocation
|
|
|
+ // for the actual queue
|
|
|
+ assertEquals("max allocation MB A1", 6144,
|
|
|
+ conf.getMaximumAllocationPerQueue(A1).getMemory());
|
|
|
+ assertEquals("max allocation vcores A1", 3,
|
|
|
+ conf.getMaximumAllocationPerQueue(A1).getVirtualCores());
|
|
|
+ assertEquals("max allocation MB cluster",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
+ conf.getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("max allocation vcores cluster",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
|
|
|
+ conf.getMaximumAllocation().getVirtualCores());
|
|
|
+ assertEquals("queue max allocation MB", 6144,
|
|
|
+ ((LeafQueue) queueA1).getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("queue max allocation vcores", 3,
|
|
|
+ ((LeafQueue) queueA1).getMaximumAllocation().getVirtualCores());
|
|
|
+ assertEquals("max capability MB cluster",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
+ cs.getMaximumResourceCapability().getMemory());
|
|
|
+ assertEquals("cluster max capability vcores",
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
|
|
|
+ cs.getMaximumResourceCapability().getVirtualCores());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRefreshQueuesMaxAllocationCSError() throws Exception {
|
|
|
+ // Try to refresh the cluster level max allocation size to be smaller
|
|
|
+ // and it should error out
|
|
|
+ CapacityScheduler cs = new CapacityScheduler();
|
|
|
+ cs.setConf(new YarnConfiguration());
|
|
|
+ cs.setRMContext(resourceManager.getRMContext());
|
|
|
+ CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
|
|
|
+ setupQueueConfiguration(conf);
|
|
|
+ setMaxAllocMb(conf, 10240);
|
|
|
+ setMaxAllocVcores(conf, 10);
|
|
|
+ setMaxAllocMb(conf, A1, 4096);
|
|
|
+ setMaxAllocVcores(conf, A1, 4);
|
|
|
+ cs.init(conf);
|
|
|
+ cs.start();
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
|
|
|
+
|
|
|
+ assertEquals("max allocation MB in CS", 10240,
|
|
|
+ cs.getMaximumResourceCapability().getMemory());
|
|
|
+ assertEquals("max allocation vcores in CS", 10,
|
|
|
+ cs.getMaximumResourceCapability().getVirtualCores());
|
|
|
+
|
|
|
+ setMaxAllocMb(conf, 6144);
|
|
|
+ try {
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ fail("should have thrown exception");
|
|
|
+ } catch (IOException e) {
|
|
|
+ assertTrue("max allocation exception",
|
|
|
+ e.getCause().toString().contains("not be decreased"));
|
|
|
+ }
|
|
|
+
|
|
|
+ setMaxAllocMb(conf, 10240);
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+
|
|
|
+ setMaxAllocVcores(conf, 8);
|
|
|
+ try {
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ fail("should have thrown exception");
|
|
|
+ } catch (IOException e) {
|
|
|
+ assertTrue("max allocation exception",
|
|
|
+ e.getCause().toString().contains("not be decreased"));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRefreshQueuesMaxAllocationCSLarger() throws Exception {
|
|
|
+ // Try to refresh the cluster level max allocation size to be larger
|
|
|
+ // and verify that if there is no setting per queue it uses the
|
|
|
+ // cluster level setting.
|
|
|
+ CapacityScheduler cs = new CapacityScheduler();
|
|
|
+ cs.setConf(new YarnConfiguration());
|
|
|
+ cs.setRMContext(resourceManager.getRMContext());
|
|
|
+ CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
|
|
|
+ setupQueueConfiguration(conf);
|
|
|
+ setMaxAllocMb(conf, 10240);
|
|
|
+ setMaxAllocVcores(conf, 10);
|
|
|
+ setMaxAllocMb(conf, A1, 4096);
|
|
|
+ setMaxAllocVcores(conf, A1, 4);
|
|
|
+ cs.init(conf);
|
|
|
+ cs.start();
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
+ checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
|
|
|
+
|
|
|
+ assertEquals("max allocation MB in CS", 10240,
|
|
|
+ cs.getMaximumResourceCapability().getMemory());
|
|
|
+ assertEquals("max allocation vcores in CS", 10,
|
|
|
+ cs.getMaximumResourceCapability().getVirtualCores());
|
|
|
+
|
|
|
+ CSQueue rootQueue = cs.getRootQueue();
|
|
|
+ CSQueue queueA = findQueue(rootQueue, A);
|
|
|
+ CSQueue queueB = findQueue(rootQueue, B);
|
|
|
+ CSQueue queueA1 = findQueue(queueA, A1);
|
|
|
+ CSQueue queueA2 = findQueue(queueA, A2);
|
|
|
+ CSQueue queueB2 = findQueue(queueB, B2);
|
|
|
+
|
|
|
+ assertEquals("queue A1 max allocation MB", 4096,
|
|
|
+ ((LeafQueue) queueA1).getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("queue A1 max allocation vcores", 4,
|
|
|
+ ((LeafQueue) queueA1).getMaximumAllocation().getVirtualCores());
|
|
|
+ assertEquals("queue A2 max allocation MB", 10240,
|
|
|
+ ((LeafQueue) queueA2).getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("queue A2 max allocation vcores", 10,
|
|
|
+ ((LeafQueue) queueA2).getMaximumAllocation().getVirtualCores());
|
|
|
+ assertEquals("queue B2 max allocation MB", 10240,
|
|
|
+ ((LeafQueue) queueB2).getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("queue B2 max allocation vcores", 10,
|
|
|
+ ((LeafQueue) queueB2).getMaximumAllocation().getVirtualCores());
|
|
|
+
|
|
|
+ setMaxAllocMb(conf, 12288);
|
|
|
+ setMaxAllocVcores(conf, 12);
|
|
|
+ cs.reinitialize(conf, null);
|
|
|
+ // cluster level setting should change and any queues without
|
|
|
+ // per queue setting
|
|
|
+ assertEquals("max allocation MB in CS", 12288,
|
|
|
+ cs.getMaximumResourceCapability().getMemory());
|
|
|
+ assertEquals("max allocation vcores in CS", 12,
|
|
|
+ cs.getMaximumResourceCapability().getVirtualCores());
|
|
|
+ assertEquals("queue A1 max MB allocation", 4096,
|
|
|
+ ((LeafQueue) queueA1).getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("queue A1 max vcores allocation", 4,
|
|
|
+ ((LeafQueue) queueA1).getMaximumAllocation().getVirtualCores());
|
|
|
+ assertEquals("queue A2 max MB allocation", 12288,
|
|
|
+ ((LeafQueue) queueA2).getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("queue A2 max vcores allocation", 12,
|
|
|
+ ((LeafQueue) queueA2).getMaximumAllocation().getVirtualCores());
|
|
|
+ assertEquals("queue B2 max MB allocation", 12288,
|
|
|
+ ((LeafQueue) queueB2).getMaximumAllocation().getMemory());
|
|
|
+ assertEquals("queue B2 max vcores allocation", 12,
|
|
|
+ ((LeafQueue) queueB2).getMaximumAllocation().getVirtualCores());
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setMaxAllocMb(Configuration conf, int maxAllocMb) {
|
|
|
+ conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
+ maxAllocMb);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setMaxAllocMb(CapacitySchedulerConfiguration conf,
|
|
|
+ String queueName, int maxAllocMb) {
|
|
|
+ String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName)
|
|
|
+ + CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_MB;
|
|
|
+ conf.setInt(propName, maxAllocMb);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setMaxAllocVcores(Configuration conf, int maxAllocVcores) {
|
|
|
+ conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
|
|
|
+ maxAllocVcores);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setMaxAllocVcores(CapacitySchedulerConfiguration conf,
|
|
|
+ String queueName, int maxAllocVcores) {
|
|
|
+ String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName)
|
|
|
+ + CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_VCORES;
|
|
|
+ conf.setInt(propName, maxAllocVcores);
|
|
|
+ }
|
|
|
}
|