|
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertFalse;
|
|
|
|
+import static org.junit.Assert.assertNull;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.fail;
|
|
import static org.junit.Assert.fail;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.mock;
|
|
@@ -115,13 +116,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdate
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.
|
|
|
|
- ContainerExpiredSchedulerEvent;
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
|
-
|
|
|
|
-import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
|
|
@@ -131,6 +128,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.
|
|
|
|
+ ContainerExpiredSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
|
@@ -144,6 +143,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedule
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerLeafQueueInfo;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerLeafQueueInfo;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfoList;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfoList;
|
|
|
|
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
|
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
|
|
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
|
|
@@ -395,6 +395,16 @@ public class TestCapacityScheduler {
|
|
resourceManager.getResourceScheduler().handle(nodeUpdate);
|
|
resourceManager.getResourceScheduler().handle(nodeUpdate);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * @param conf
|
|
|
|
+ * @return
|
|
|
|
+ * root
|
|
|
|
+ * / \
|
|
|
|
+ * a b
|
|
|
|
+ * / \ / | \
|
|
|
|
+ * a1 a2 b1 b2 b3
|
|
|
|
+ *
|
|
|
|
+ */
|
|
private CapacitySchedulerConfiguration setupQueueConfiguration(
|
|
private CapacitySchedulerConfiguration setupQueueConfiguration(
|
|
CapacitySchedulerConfiguration conf) {
|
|
CapacitySchedulerConfiguration conf) {
|
|
|
|
|
|
@@ -423,6 +433,67 @@ public class TestCapacityScheduler {
|
|
return conf;
|
|
return conf;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * @param conf, to be modified
|
|
|
|
+ * @return, CS configuration which has deleted a queue(b1)
|
|
|
|
+ * root
|
|
|
|
+ * / \
|
|
|
|
+ * a b
|
|
|
|
+ * / \ | \
|
|
|
|
+ * a1 a2 b2 b3
|
|
|
|
+ */
|
|
|
|
+ private CapacitySchedulerConfiguration setupQueueConfigurationWithOutB1(
|
|
|
|
+ CapacitySchedulerConfiguration conf) {
|
|
|
|
+
|
|
|
|
+ // Define top-level queues
|
|
|
|
+ conf.setQueues(CapacitySchedulerConfiguration.ROOT,
|
|
|
|
+ new String[] { "a", "b" });
|
|
|
|
+
|
|
|
|
+ conf.setCapacity(A, A_CAPACITY);
|
|
|
|
+ conf.setCapacity(B, B_CAPACITY);
|
|
|
|
+
|
|
|
|
+ // Define 2nd-level queues
|
|
|
|
+ conf.setQueues(A, new String[] { "a1", "a2" });
|
|
|
|
+ conf.setCapacity(A1, A1_CAPACITY);
|
|
|
|
+ conf.setUserLimitFactor(A1, 100.0f);
|
|
|
|
+ conf.setCapacity(A2, A2_CAPACITY);
|
|
|
|
+ conf.setUserLimitFactor(A2, 100.0f);
|
|
|
|
+
|
|
|
|
+ conf.setQueues(B, new String[] { "b2", "b3" });
|
|
|
|
+ conf.setCapacity(B2, B2_CAPACITY + B1_CAPACITY); //as B1 is deleted
|
|
|
|
+ conf.setUserLimitFactor(B2, 100.0f);
|
|
|
|
+ conf.setCapacity(B3, B3_CAPACITY);
|
|
|
|
+ conf.setUserLimitFactor(B3, 100.0f);
|
|
|
|
+
|
|
|
|
+ LOG.info("Setup top-level queues a and b (without b3)");
|
|
|
|
+ return conf;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * @param conf, to be modified
|
|
|
|
+ * @return, CS configuration which has deleted a
|
|
|
|
+ * Parent queue(b)
|
|
|
|
+ */
|
|
|
|
+ private CapacitySchedulerConfiguration setupQueueConfigurationWithOutB(
|
|
|
|
+ CapacitySchedulerConfiguration conf) {
|
|
|
|
+
|
|
|
|
+ // Define top-level queues
|
|
|
|
+ conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a" });
|
|
|
|
+
|
|
|
|
+ conf.setCapacity(A, A_CAPACITY + B_CAPACITY);
|
|
|
|
+
|
|
|
|
+ // Define 2nd-level queues
|
|
|
|
+ conf.setQueues(A, new String[] { "a1", "a2" });
|
|
|
|
+ conf.setCapacity(A1, A1_CAPACITY);
|
|
|
|
+ conf.setUserLimitFactor(A1, 100.0f);
|
|
|
|
+ conf.setCapacity(A2, A2_CAPACITY);
|
|
|
|
+ conf.setUserLimitFactor(A2, 100.0f);
|
|
|
|
+
|
|
|
|
+ LOG.info("Setup top-level queues a");
|
|
|
|
+ return conf;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
private CapacitySchedulerConfiguration setupBlockedQueueConfiguration(
|
|
private CapacitySchedulerConfiguration setupBlockedQueueConfiguration(
|
|
CapacitySchedulerConfiguration conf) {
|
|
CapacitySchedulerConfiguration conf) {
|
|
|
|
|
|
@@ -3758,4 +3829,117 @@ public class TestCapacityScheduler {
|
|
Assert.assertArrayEquals(new int[][] { { 1, 0, 0 }, { 0, 1, 0 }, { 0, 0, 1 } },
|
|
Assert.assertArrayEquals(new int[][] { { 1, 0, 0 }, { 0, 1, 0 }, { 0, 0, 1 } },
|
|
attemptMetrics.getLocalityStatistics());
|
|
attemptMetrics.getLocalityStatistics());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test for queue deletion.
|
|
|
|
+ * @throws Exception
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testRefreshQueuesWithQueueDelete() throws Exception {
|
|
|
|
+ CapacityScheduler cs = new CapacityScheduler();
|
|
|
|
+ CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
|
|
|
|
+ RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
|
|
|
|
+ null, new RMContainerTokenSecretManager(conf),
|
|
|
|
+ new NMTokenSecretManagerInRM(conf),
|
|
|
|
+ new ClientToAMTokenSecretManagerInRM(), null);
|
|
|
|
+ setupQueueConfiguration(conf);
|
|
|
|
+ cs.setConf(new YarnConfiguration());
|
|
|
|
+ cs.setRMContext(resourceManager.getRMContext());
|
|
|
|
+ cs.init(conf);
|
|
|
|
+ cs.start();
|
|
|
|
+ cs.reinitialize(conf, rmContext);
|
|
|
|
+ checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
|
|
|
|
+
|
|
|
|
+ // test delete leaf queue when there is application running.
|
|
|
|
+ Map<String, CSQueue> queues =
|
|
|
|
+ cs.getCapacitySchedulerQueueManager().getQueues();
|
|
|
|
+ String b1QTobeDeleted = "b1";
|
|
|
|
+ LeafQueue csB1Queue = Mockito.spy((LeafQueue) queues.get(b1QTobeDeleted));
|
|
|
|
+ when(csB1Queue.getState()).thenReturn(QueueState.DRAINING)
|
|
|
|
+ .thenReturn(QueueState.STOPPED);
|
|
|
|
+ queues.put(b1QTobeDeleted, csB1Queue);
|
|
|
|
+ conf = new CapacitySchedulerConfiguration();
|
|
|
|
+ setupQueueConfigurationWithOutB1(conf);
|
|
|
|
+ try {
|
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
|
+ fail("Expected to throw exception when refresh queue tries to delete a"
|
|
|
|
+ + " queue with running apps");
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ // ignore
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // test delete leaf queue(root.b.b1) when there is no application running.
|
|
|
|
+ conf = new CapacitySchedulerConfiguration();
|
|
|
|
+ setupQueueConfigurationWithOutB1(conf);
|
|
|
|
+ try {
|
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ fail("Expected to NOT throw exception when refresh queue tries to delete"
|
|
|
|
+ + " a queue WITHOUT running apps");
|
|
|
|
+ }
|
|
|
|
+ CSQueue rootQueue = cs.getRootQueue();
|
|
|
|
+ CSQueue queueB = findQueue(rootQueue, B);
|
|
|
|
+ CSQueue queueB3 = findQueue(queueB, B1);
|
|
|
|
+ assertNull("Refresh needs to support delete of leaf queue ", queueB3);
|
|
|
|
+
|
|
|
|
+ // reset back to default configuration for testing parent queue delete
|
|
|
|
+ conf = new CapacitySchedulerConfiguration();
|
|
|
|
+ setupQueueConfiguration(conf);
|
|
|
|
+ cs.reinitialize(conf, rmContext);
|
|
|
|
+ checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
|
|
|
|
+
|
|
|
|
+ // set the configurations such that it fails once but should be successfull
|
|
|
|
+ // next time
|
|
|
|
+ queues = cs.getCapacitySchedulerQueueManager().getQueues();
|
|
|
|
+ CSQueue bQueue = Mockito.spy((ParentQueue) queues.get("b"));
|
|
|
|
+ when(bQueue.getState()).thenReturn(QueueState.DRAINING)
|
|
|
|
+ .thenReturn(QueueState.STOPPED);
|
|
|
|
+ queues.put("b", bQueue);
|
|
|
|
+
|
|
|
|
+ bQueue = Mockito.spy((LeafQueue) queues.get("b1"));
|
|
|
|
+ when(bQueue.getState()).thenReturn(QueueState.STOPPED);
|
|
|
|
+ queues.put("b1", bQueue);
|
|
|
|
+
|
|
|
|
+ bQueue = Mockito.spy((LeafQueue) queues.get("b2"));
|
|
|
|
+ when(bQueue.getState()).thenReturn(QueueState.STOPPED);
|
|
|
|
+ queues.put("b2", bQueue);
|
|
|
|
+
|
|
|
|
+ bQueue = Mockito.spy((LeafQueue) queues.get("b3"));
|
|
|
|
+ when(bQueue.getState()).thenReturn(QueueState.STOPPED);
|
|
|
|
+ queues.put("b3", bQueue);
|
|
|
|
+
|
|
|
|
+ // test delete Parent queue when there is application running.
|
|
|
|
+ conf = new CapacitySchedulerConfiguration();
|
|
|
|
+ setupQueueConfigurationWithOutB(conf);
|
|
|
|
+ try {
|
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
|
+ fail("Expected to throw exception when refresh queue tries to delete a"
|
|
|
|
+ + " parent queue with running apps in children queue");
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ // ignore
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // test delete Parent queue when there is no application running.
|
|
|
|
+ conf = new CapacitySchedulerConfiguration();
|
|
|
|
+ setupQueueConfigurationWithOutB(conf);
|
|
|
|
+ try {
|
|
|
|
+ cs.reinitialize(conf, mockContext);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ fail("Expected to not throw exception when refresh queue tries to delete"
|
|
|
|
+ + " a queue without running apps");
|
|
|
|
+ }
|
|
|
|
+ rootQueue = cs.getRootQueue();
|
|
|
|
+ queueB = findQueue(rootQueue, B);
|
|
|
|
+ String message =
|
|
|
|
+ "Refresh needs to support delete of Parent queue and its children.";
|
|
|
|
+ assertNull(message, queueB);
|
|
|
|
+ assertNull(message,
|
|
|
|
+ cs.getCapacitySchedulerQueueManager().getQueues().get("b"));
|
|
|
|
+ assertNull(message,
|
|
|
|
+ cs.getCapacitySchedulerQueueManager().getQueues().get("b1"));
|
|
|
|
+ assertNull(message,
|
|
|
|
+ cs.getCapacitySchedulerQueueManager().getQueues().get("b2"));
|
|
|
|
+
|
|
|
|
+ cs.stop();
|
|
|
|
+ }
|
|
}
|
|
}
|