|
@@ -20,15 +20,22 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
|
|
import static org.junit.Assert.*;
|
|
|
import static org.mockito.Mockito.*;
|
|
|
|
|
|
+import java.util.Collections;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Set;
|
|
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
|
|
|
+import com.google.common.collect.ImmutableSet;
|
|
|
import com.google.common.collect.Sets;
|
|
|
|
|
|
public class TestQueueManager {
|
|
@@ -305,4 +312,334 @@ public class TestQueueManager {
|
|
|
assertEquals("createQueue() returned wrong queue",
|
|
|
"root.queue1.queue2", q2.getName());
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRemovalOfDynamicLeafQueue() {
|
|
|
+ AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
|
|
|
+
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ FSQueue q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", true);
|
|
|
+
|
|
|
+ assertNotNull("Queue root.test.childB.dynamic1 was not created", q1);
|
|
|
+ assertEquals("createQueue() returned wrong queue",
|
|
|
+ "root.test.childB.dynamic1", q1.getName());
|
|
|
+ assertTrue("root.test.childB.dynamic1 is not a dynamic queue",
|
|
|
+ q1.isDynamic());
|
|
|
+
|
|
|
+ // an application is submitted to root.test.childB.dynamic1
|
|
|
+ notEmptyQueues.add(q1);
|
|
|
+
|
|
|
+ // root.test.childB.dynamic1 is not empty and should not be removed
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", false);
|
|
|
+ assertNotNull("Queue root.test.childB.dynamic1 was deleted", q1);
|
|
|
+
|
|
|
+ // the application finishes, the next removeEmptyDynamicQueues() should
|
|
|
+ // clean root.test.childB.dynamic1 up, but keep its static parent
|
|
|
+ notEmptyQueues.remove(q1);
|
|
|
+
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", false);
|
|
|
+ assertNull("Queue root.test.childB.dynamic1 was not deleted", q1);
|
|
|
+ assertNotNull("The static parent of root.test.childB.dynamic1 was deleted",
|
|
|
+ queueManager.getParentQueue("root.test.childB", false));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRemovalOfDynamicParentQueue() {
|
|
|
+ AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
|
|
|
+
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ FSQueue q1 = queueManager.getLeafQueue("root.parent1.dynamic1", true);
|
|
|
+
|
|
|
+ assertNotNull("Queue root.parent1.dynamic1 was not created", q1);
|
|
|
+ assertEquals("createQueue() returned wrong queue",
|
|
|
+ "root.parent1.dynamic1", q1.getName());
|
|
|
+ assertTrue("root.parent1.dynamic1 is not a dynamic queue", q1.isDynamic());
|
|
|
+
|
|
|
+ FSQueue p1 = queueManager.getParentQueue("root.parent1", false);
|
|
|
+ assertNotNull("Queue root.parent1 was not created", p1);
|
|
|
+ assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic());
|
|
|
+
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q1 = queueManager.getLeafQueue("root.parent1.dynamic1", false);
|
|
|
+ p1 = queueManager.getParentQueue("root.parent1", false);
|
|
|
+
|
|
|
+ assertNull("Queue root.parent1.dynamic1 was not deleted", q1);
|
|
|
+ assertNull("Queue root.parent1 was not deleted", p1);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testNonEmptyDynamicQueueBecomingStaticQueue() {
|
|
|
+ AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
|
|
|
+
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ FSLeafQueue q1 = queueManager.getLeafQueue("root.leaf1", true);
|
|
|
+
|
|
|
+ assertNotNull("Queue root.leaf1 was not created", q1);
|
|
|
+ assertEquals("createQueue() returned wrong queue",
|
|
|
+ "root.leaf1", q1.getName());
|
|
|
+ assertTrue("root.leaf1 is not a dynamic queue", q1.isDynamic());
|
|
|
+
|
|
|
+ // pretend that we submitted an app to the queue
|
|
|
+ notEmptyQueues.add(q1);
|
|
|
+
|
|
|
+ // non-empty queues should not be deleted
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q1 = queueManager.getLeafQueue("root.leaf1", false);
|
|
|
+ assertNotNull("Queue root.leaf1 was deleted", q1);
|
|
|
+
|
|
|
+ // next we add leaf1 under root in the allocation config
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.leaf1");
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ // updateAllocationConfiguration() should make root.leaf1 a dynamic queue
|
|
|
+ assertFalse("root.leaf1 is not a static queue", q1.isDynamic());
|
|
|
+
|
|
|
+ // application finished now and the queue is empty, but since leaf1 is a
|
|
|
+ // static queue at this point, hence not affected by
|
|
|
+ // removeEmptyDynamicQueues()
|
|
|
+ notEmptyQueues.clear();
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q1 = queueManager.getLeafQueue("root.leaf1", false);
|
|
|
+ assertNotNull("Queue root.leaf1 was deleted", q1);
|
|
|
+ assertFalse("root.leaf1 is not a static queue", q1.isDynamic());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testNonEmptyStaticQueueBecomingDynamicQueue() {
|
|
|
+ AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ FSLeafQueue q1 = queueManager.getLeafQueue("root.test.childA", false);
|
|
|
+
|
|
|
+ assertNotNull("Queue root.test.childA does not exist", q1);
|
|
|
+ assertEquals("createQueue() returned wrong queue",
|
|
|
+ "root.test.childA", q1.getName());
|
|
|
+ assertFalse("root.test.childA is not a static queue", q1.isDynamic());
|
|
|
+
|
|
|
+ // we submitted an app to the queue
|
|
|
+ notEmptyQueues.add(q1);
|
|
|
+
|
|
|
+ // the next removeEmptyDynamicQueues() call should not modify
|
|
|
+ // root.test.childA
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q1 = queueManager.getLeafQueue("root.test.childA", false);
|
|
|
+ assertNotNull("Queue root.test.childA was deleted", q1);
|
|
|
+ assertFalse("root.test.childA is not a dynamic queue", q1.isDynamic());
|
|
|
+
|
|
|
+ // next we remove all queues from the allocation config,
|
|
|
+ // this causes all queues to change to dynamic
|
|
|
+ for (Set<String> queueNames : allocConf.configuredQueues.values()) {
|
|
|
+ queueManager.setQueuesToDynamic(queueNames);
|
|
|
+ queueNames.clear();
|
|
|
+ }
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ q1 = queueManager.getLeafQueue("root.test.childA", false);
|
|
|
+ assertNotNull("Queue root.test.childA was deleted", q1);
|
|
|
+ assertTrue("root.test.childA is not a dynamic queue", q1.isDynamic());
|
|
|
+
|
|
|
+ // application finished - the queue does not have runnable app
|
|
|
+ // the next removeEmptyDynamicQueues() call should remove the queues
|
|
|
+ notEmptyQueues.remove(q1);
|
|
|
+
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+
|
|
|
+ q1 = queueManager.getLeafQueue("root.test.childA", false);
|
|
|
+ assertNull("Queue root.test.childA was not deleted", q1);
|
|
|
+
|
|
|
+ FSParentQueue p1 = queueManager.getParentQueue("root.test", false);
|
|
|
+ assertNull("Queue root.test was not deleted", p1);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRemovalOfChildlessParentQueue() {
|
|
|
+ AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ FSParentQueue q1 = queueManager.getParentQueue("root.test.childB", false);
|
|
|
+
|
|
|
+ assertNotNull("Queue root.test.childB was not created", q1);
|
|
|
+ assertEquals("createQueue() returned wrong queue",
|
|
|
+ "root.test.childB", q1.getName());
|
|
|
+ assertFalse("root.test.childB is a dynamic queue", q1.isDynamic());
|
|
|
+
|
|
|
+ // static queues should not be deleted
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q1 = queueManager.getParentQueue("root.test.childB", false);
|
|
|
+ assertNotNull("Queue root.test.childB was deleted", q1);
|
|
|
+
|
|
|
+ // next we remove root.test.childB from the allocation config
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.PARENT)
|
|
|
+ .remove("root.test.childB");
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+ queueManager.setQueuesToDynamic(Collections.singleton("root.test.childB"));
|
|
|
+
|
|
|
+ // the next removeEmptyDynamicQueues() call should clean
|
|
|
+ // root.test.childB up
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q1 = queueManager.getParentQueue("root.leaf1", false);
|
|
|
+ assertNull("Queue root.leaf1 was not deleted", q1);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testQueueTypeChange() {
|
|
|
+ AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ FSQueue q1 = queueManager.getLeafQueue("root.parent1.leaf1", true);
|
|
|
+ assertNotNull("Queue root.parent1.leaf1 was not created", q1);
|
|
|
+ assertEquals("createQueue() returned wrong queue",
|
|
|
+ "root.parent1.leaf1", q1.getName());
|
|
|
+ assertTrue("root.parent1.leaf1 is not a dynamic queue", q1.isDynamic());
|
|
|
+
|
|
|
+ FSQueue p1 = queueManager.getParentQueue("root.parent1", false);
|
|
|
+ assertNotNull("Queue root.parent1 was not created", p1);
|
|
|
+ assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic());
|
|
|
+
|
|
|
+ // adding root.parent1.leaf1 and root.parent1 to the allocation config
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.parent1");
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.LEAF)
|
|
|
+ .add("root.parent1.leaf1");
|
|
|
+
|
|
|
+ // updateAllocationConfiguration() should change both queues over to static
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+ q1 = queueManager.getLeafQueue("root.parent1.leaf1", false);
|
|
|
+ assertFalse("root.parent1.leaf1 is not a static queue", q1.isDynamic());
|
|
|
+ p1 = queueManager.getParentQueue("root.parent1", false);
|
|
|
+ assertFalse("root.parent1 is not a static queue", p1.isDynamic());
|
|
|
+
|
|
|
+ // removing root.parent1.leaf1 and root.parent1 from the allocation
|
|
|
+ // config
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.PARENT).remove("root.parent1");
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.LEAF)
|
|
|
+ .remove("root.parent1.leaf1");
|
|
|
+
|
|
|
+ // updateAllocationConfiguration() should change both queues
|
|
|
+ // to dynamic
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+ queueManager.setQueuesToDynamic(
|
|
|
+ ImmutableSet.of("root.parent1", "root.parent1.leaf1"));
|
|
|
+ q1 = queueManager.getLeafQueue("root.parent1.leaf1", false);
|
|
|
+ assertTrue("root.parent1.leaf1 is not a dynamic queue", q1.isDynamic());
|
|
|
+ p1 = queueManager.getParentQueue("root.parent1", false);
|
|
|
+ assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testApplicationAssignmentPreventsRemovalOfDynamicQueue()
|
|
|
+ throws Exception {
|
|
|
+ AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
|
|
|
+ queueManager = new QueueManager(scheduler);
|
|
|
+ queueManager.initialize(conf);
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ FSLeafQueue q = queueManager.getLeafQueue("root.leaf1", true);
|
|
|
+ assertNotNull("root.leaf1 does not exist", q);
|
|
|
+ assertTrue("root.leaf1 is not empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ // assigning an application (without an appAttempt so far) to the queue
|
|
|
+ // removeEmptyDynamicQueues() should not remove the queue
|
|
|
+ ApplicationId applicationId = ApplicationId.newInstance(1L, 0);
|
|
|
+ q.addAssignedApp(applicationId);
|
|
|
+ q = queueManager.getLeafQueue("root.leaf1", false);
|
|
|
+ assertFalse("root.leaf1 is empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q = queueManager.getLeafQueue("root.leaf1", false);
|
|
|
+ assertNotNull("root.leaf1 has been removed", q);
|
|
|
+ assertFalse("root.leaf1 is empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ ApplicationAttemptId applicationAttemptId =
|
|
|
+ ApplicationAttemptId.newInstance(applicationId, 0);
|
|
|
+ ActiveUsersManager activeUsersManager =
|
|
|
+ Mockito.mock(ActiveUsersManager.class);
|
|
|
+ RMContext rmContext = Mockito.mock(RMContext.class);
|
|
|
+
|
|
|
+ // the appAttempt is created
|
|
|
+ // removeEmptyDynamicQueues() should not remove the queue
|
|
|
+ FSAppAttempt appAttempt = new FSAppAttempt(scheduler, applicationAttemptId,
|
|
|
+ "a_user", q, activeUsersManager, rmContext);
|
|
|
+ q.addApp(appAttempt, true);
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q = queueManager.getLeafQueue("root.leaf1", false);
|
|
|
+ assertNotNull("root.leaf1 has been removed", q);
|
|
|
+ assertFalse("root.leaf1 is empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ // the appAttempt finished, the queue should be empty
|
|
|
+ q.removeApp(appAttempt);
|
|
|
+ q = queueManager.getLeafQueue("root.leaf1", false);
|
|
|
+ assertTrue("root.leaf1 is not empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ // removeEmptyDynamicQueues() should remove the queue
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q = queueManager.getLeafQueue("root.leaf1", false);
|
|
|
+ assertNull("root.leaf1 has not been removed", q);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRemovalOfIncompatibleNonEmptyQueue()
|
|
|
+ throws Exception {
|
|
|
+ AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.a");
|
|
|
+ scheduler.allocConf = allocConf;
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ FSLeafQueue q = queueManager.getLeafQueue("root.a", true);
|
|
|
+ assertNotNull("root.a does not exist", q);
|
|
|
+ assertTrue("root.a is not empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ // we start to run an application on root.a
|
|
|
+ notEmptyQueues.add(q);
|
|
|
+ q = queueManager.getLeafQueue("root.a", false);
|
|
|
+ assertNotNull("root.a does not exist", q);
|
|
|
+ assertFalse("root.a is empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ // root.a should not be removed by removeEmptyDynamicQueues or by
|
|
|
+ // removePendingIncompatibleQueues
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ q = queueManager.getLeafQueue("root.a", false);
|
|
|
+ assertNotNull("root.a does not exist", q);
|
|
|
+
|
|
|
+ // let's introduce queue incompatibility
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.LEAF).remove("root.a");
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.a");
|
|
|
+ allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.a.b");
|
|
|
+ queueManager.updateAllocationConfiguration(allocConf);
|
|
|
+
|
|
|
+ // since root.a has running applications, it should be still a leaf queue
|
|
|
+ q = queueManager.getLeafQueue("root.a", false);
|
|
|
+ assertNotNull("root.a has been removed", q);
|
|
|
+ assertFalse("root.a is empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ // removePendingIncompatibleQueues should still keep root.a as a leaf queue
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ q = queueManager.getLeafQueue("root.a", false);
|
|
|
+ assertNotNull("root.a has been removed", q);
|
|
|
+ assertFalse("root.a is empty", queueManager.isEmpty(q));
|
|
|
+
|
|
|
+ // when the application finishes, root.a should be a parent queue
|
|
|
+ notEmptyQueues.clear();
|
|
|
+ queueManager.removePendingIncompatibleQueues();
|
|
|
+ queueManager.removeEmptyDynamicQueues();
|
|
|
+ FSParentQueue p = queueManager.getParentQueue("root.a", false);
|
|
|
+ assertNotNull("root.a does not exist", p);
|
|
|
+ }
|
|
|
+
|
|
|
}
|