|
@@ -18,6 +18,8 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.apache.hadoop.util.Time;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
@@ -26,12 +28,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
|
|
|
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.SchedulerEvent;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
@@ -39,6 +44,9 @@ import org.junit.Test;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import java.util.Set;
|
|
|
+import java.util.HashSet;
|
|
|
+
|
|
|
public class TestCapacitySchedulerNewQueueAutoCreation
|
|
|
extends TestCapacitySchedulerAutoCreatedQueueBase {
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
@@ -50,6 +58,16 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|
|
private CapacityScheduler cs;
|
|
|
private CapacitySchedulerConfiguration csConf;
|
|
|
private CapacitySchedulerAutoQueueHandler autoQueueHandler;
|
|
|
+ private AutoCreatedQueueDeletionPolicy policy = new
|
|
|
+ AutoCreatedQueueDeletionPolicy();
|
|
|
+
|
|
|
+ public CapacityScheduler getCs() {
|
|
|
+ return cs;
|
|
|
+ }
|
|
|
+
|
|
|
+ public AutoCreatedQueueDeletionPolicy getPolicy() {
|
|
|
+ return policy;
|
|
|
+ }
|
|
|
|
|
|
/*
|
|
|
Create the following structure:
|
|
@@ -75,9 +93,12 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|
|
csConf.setAutoQueueCreationV2Enabled("root", true);
|
|
|
csConf.setAutoQueueCreationV2Enabled("root.a", true);
|
|
|
csConf.setAutoQueueCreationV2Enabled("root.e", true);
|
|
|
+ csConf.setAutoQueueCreationV2Enabled(PARENT_QUEUE, true);
|
|
|
+ // Test for auto deletion when expired
|
|
|
+ csConf.setAutoExpiredDeletionTime(1);
|
|
|
}
|
|
|
|
|
|
- private void startScheduler() throws Exception {
|
|
|
+ protected void startScheduler() throws Exception {
|
|
|
RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
|
|
|
mgr.init(csConf);
|
|
|
mockRM = new MockRM(csConf) {
|
|
@@ -87,6 +108,8 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|
|
};
|
|
|
cs = (CapacityScheduler) mockRM.getResourceScheduler();
|
|
|
cs.updatePlacementRules();
|
|
|
+ // Policy for new auto created queue's auto deletion when expired
|
|
|
+ policy.init(cs.getConfiguration(), cs.getRMContext(), cs);
|
|
|
mockRM.start();
|
|
|
cs.start();
|
|
|
autoQueueHandler = new CapacitySchedulerAutoQueueHandler(
|
|
@@ -506,7 +529,7 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|
|
Assert.assertTrue(user0.isDynamicQueue());
|
|
|
Assert.assertTrue(user0 instanceof LeafQueue);
|
|
|
|
|
|
- LeafQueue user0LeafQueue = (LeafQueue)user0;
|
|
|
+ LeafQueue user0LeafQueue = (LeafQueue) user0;
|
|
|
|
|
|
// Assert user limit factor is -1
|
|
|
Assert.assertTrue(user0LeafQueue.getUserLimitFactor() == -1);
|
|
@@ -517,10 +540,11 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|
|
|
|
|
// Assert AM Resource
|
|
|
Assert.assertEquals(user0LeafQueue.getAMResourceLimit().getMemorySize(),
|
|
|
- user0LeafQueue.getMaxAMResourcePerQueuePercent()*MAX_MEMORY*GB, 1e-6);
|
|
|
+ user0LeafQueue.
|
|
|
+ getMaxAMResourcePerQueuePercent() * MAX_MEMORY * GB, 1e-6);
|
|
|
|
|
|
// Assert user limit (no limit) when limit factor is -1
|
|
|
- Assert.assertEquals(MAX_MEMORY*GB,
|
|
|
+ Assert.assertEquals(MAX_MEMORY * GB,
|
|
|
user0LeafQueue.getEffectiveMaxCapacityDown("",
|
|
|
user0LeafQueue.getMinimumAllocation()).getMemorySize(), 1e-6);
|
|
|
}
|
|
@@ -585,7 +609,274 @@ public class TestCapacitySchedulerNewQueueAutoCreation
|
|
|
|
|
|
}
|
|
|
|
|
|
- private LeafQueue createQueue(String queuePath) throws YarnException {
|
|
|
+ @Test
|
|
|
+ public void testCapacitySchedulerAutoQueueDeletion() throws Exception {
|
|
|
+ startScheduler();
|
|
|
+ csConf.setBoolean(
|
|
|
+ YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
|
|
|
+ csConf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
|
|
|
+ AutoCreatedQueueDeletionPolicy.class.getCanonicalName());
|
|
|
+ csConf.setAutoExpiredDeletionTime(1);
|
|
|
+ cs.reinitialize(csConf, mockRM.getRMContext());
|
|
|
+
|
|
|
+ Set<String> policies = new HashSet<>();
|
|
|
+ policies.add(
|
|
|
+ AutoCreatedQueueDeletionPolicy.class.getCanonicalName());
|
|
|
+
|
|
|
+ Assert.assertTrue(
|
|
|
+ "No AutoCreatedQueueDeletionPolicy " +
|
|
|
+ "is present in running monitors",
|
|
|
+ cs.getSchedulingMonitorManager().
|
|
|
+ isSameConfiguredPolicies(policies));
|
|
|
+
|
|
|
+ ApplicationAttemptId a2App = submitApp(cs, USER0,
|
|
|
+ "a2-auto", "root.a.a1-auto");
|
|
|
+
|
|
|
+ // Wait a2 created successfully.
|
|
|
+ GenericTestUtils.waitFor(()-> cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto") != null,
|
|
|
+ 100, 2000);
|
|
|
+
|
|
|
+ AbstractCSQueue a1 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ Assert.assertNotNull("a1 is not present", a1);
|
|
|
+ AbstractCSQueue a2 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto");
|
|
|
+ Assert.assertNotNull("a2 is not present", a2);
|
|
|
+ Assert.assertTrue("a2 is not a dynamic queue",
|
|
|
+ a2.isDynamicQueue());
|
|
|
+
|
|
|
+ // Now there are still 1 app in a2 queue.
|
|
|
+ Assert.assertEquals(1, a2.getNumApplications());
|
|
|
+
|
|
|
+ // Wait the time expired.
|
|
|
+ long l1 = a2.getLastSubmittedTimestamp();
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ long duration = (Time.monotonicNow() - l1)/1000;
|
|
|
+ return duration > csConf.getAutoExpiredDeletionTime();
|
|
|
+ }, 100, 2000);
|
|
|
+
|
|
|
+ // Make sure the queue will not be deleted
|
|
|
+ // when expired with remaining apps.
|
|
|
+ a2 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto");
|
|
|
+ Assert.assertNotNull("a2 is not present", a2);
|
|
|
+
|
|
|
+ // Make app finished.
|
|
|
+ AppAttemptRemovedSchedulerEvent event =
|
|
|
+ new AppAttemptRemovedSchedulerEvent(a2App,
|
|
|
+ RMAppAttemptState.FINISHED, false);
|
|
|
+ cs.handle(event);
|
|
|
+ AppRemovedSchedulerEvent rEvent = new AppRemovedSchedulerEvent(
|
|
|
+ a2App.getApplicationId(), RMAppState.FINISHED);
|
|
|
+ cs.handle(rEvent);
|
|
|
+
|
|
|
+ // Now there are no apps in a2 queue.
|
|
|
+ Assert.assertEquals(0, a2.getNumApplications());
|
|
|
+
|
|
|
+ // Wait the a2 deleted.
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ AbstractCSQueue a2Tmp = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto");
|
|
|
+ return a2Tmp == null;
|
|
|
+ }, 100, 3000);
|
|
|
+
|
|
|
+ a2 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto");
|
|
|
+ Assert.assertNull("a2 is not deleted", a2);
|
|
|
+
|
|
|
+ // The parent will not be deleted with child queues
|
|
|
+ a1 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ Assert.assertNotNull("a1 is not present", a1);
|
|
|
+
|
|
|
+ // Now the parent queue without child
|
|
|
+ // will be deleted for expired.
|
|
|
+ // Wait a1 deleted.
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ AbstractCSQueue a1Tmp = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ return a1Tmp == null;
|
|
|
+ }, 100, 3000);
|
|
|
+ a1 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ Assert.assertNull("a1 is not deleted", a1);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCapacitySchedulerAutoQueueDeletionDisabled()
|
|
|
+ throws Exception {
|
|
|
+ startScheduler();
|
|
|
+ // Test for disabled auto deletion
|
|
|
+ csConf.setAutoExpiredDeletionEnabled(
|
|
|
+ "root.a.a1-auto.a2-auto", false);
|
|
|
+ csConf.setBoolean(
|
|
|
+ YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
|
|
|
+ csConf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
|
|
|
+ AutoCreatedQueueDeletionPolicy.class.getCanonicalName());
|
|
|
+ csConf.setAutoExpiredDeletionTime(1);
|
|
|
+ cs.reinitialize(csConf, mockRM.getRMContext());
|
|
|
+
|
|
|
+ Set<String> policies = new HashSet<>();
|
|
|
+ policies.add(
|
|
|
+ AutoCreatedQueueDeletionPolicy.class.getCanonicalName());
|
|
|
+
|
|
|
+ Assert.assertTrue(
|
|
|
+ "No AutoCreatedQueueDeletionPolicy " +
|
|
|
+ "is present in running monitors",
|
|
|
+ cs.getSchedulingMonitorManager().
|
|
|
+ isSameConfiguredPolicies(policies));
|
|
|
+
|
|
|
+ ApplicationAttemptId a2App = submitApp(cs, USER0,
|
|
|
+ "a2-auto", "root.a.a1-auto");
|
|
|
+
|
|
|
+ // Wait a2 created successfully.
|
|
|
+ GenericTestUtils.waitFor(()-> cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto") != null,
|
|
|
+ 100, 2000);
|
|
|
+
|
|
|
+ AbstractCSQueue a1 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ Assert.assertNotNull("a1 is not present", a1);
|
|
|
+ AbstractCSQueue a2 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto");
|
|
|
+ Assert.assertNotNull("a2 is not present", a2);
|
|
|
+ Assert.assertTrue("a2 is not a dynamic queue",
|
|
|
+ a2.isDynamicQueue());
|
|
|
+
|
|
|
+ // Make app finished.
|
|
|
+ AppAttemptRemovedSchedulerEvent event =
|
|
|
+ new AppAttemptRemovedSchedulerEvent(a2App,
|
|
|
+ RMAppAttemptState.FINISHED, false);
|
|
|
+ cs.handle(event);
|
|
|
+ AppRemovedSchedulerEvent rEvent = new AppRemovedSchedulerEvent(
|
|
|
+ a2App.getApplicationId(), RMAppState.FINISHED);
|
|
|
+ cs.handle(rEvent);
|
|
|
+
|
|
|
+ // Now there are no apps in a2 queue.
|
|
|
+ Assert.assertEquals(0, a2.getNumApplications());
|
|
|
+
|
|
|
+ // Wait the time expired.
|
|
|
+ long l1 = a2.getLastSubmittedTimestamp();
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ long duration = (Time.monotonicNow() - l1)/1000;
|
|
|
+ return duration > csConf.getAutoExpiredDeletionTime();
|
|
|
+ }, 100, 2000);
|
|
|
+
|
|
|
+ // The auto deletion is no enabled for a2-auto
|
|
|
+ a1 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ Assert.assertNotNull("a1 is not present", a1);
|
|
|
+ a2 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto");
|
|
|
+ Assert.assertNotNull("a2 is not present", a2);
|
|
|
+ Assert.assertTrue("a2 is not a dynamic queue",
|
|
|
+ a2.isDynamicQueue());
|
|
|
+
|
|
|
+ // Enabled now
|
|
|
+ // The auto deletion will work.
|
|
|
+ csConf.setAutoExpiredDeletionEnabled(
|
|
|
+ "root.a.a1-auto.a2-auto", true);
|
|
|
+ cs.reinitialize(csConf, mockRM.getRMContext());
|
|
|
+
|
|
|
+ // Wait the a2 deleted.
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ AbstractCSQueue a2Tmp = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto.a2-auto");
|
|
|
+ return a2Tmp == null;
|
|
|
+ }, 100, 3000);
|
|
|
+
|
|
|
+ a2 = (AbstractCSQueue) cs.
|
|
|
+ getQueue("root.a.a1-auto.a2-auto");
|
|
|
+ Assert.assertNull("a2 is not deleted", a2);
|
|
|
+ // The parent will not be deleted with child queues
|
|
|
+ a1 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ Assert.assertNotNull("a1 is not present", a1);
|
|
|
+
|
|
|
+ // Now the parent queue without child
|
|
|
+ // will be deleted for expired.
|
|
|
+ // Wait a1 deleted.
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ AbstractCSQueue a1Tmp = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ return a1Tmp == null;
|
|
|
+ }, 100, 3000);
|
|
|
+ a1 = (AbstractCSQueue) cs.getQueue(
|
|
|
+ "root.a.a1-auto");
|
|
|
+ Assert.assertNull("a1 is not deleted", a1);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testAutoCreateQueueAfterRemoval() throws Exception {
|
|
|
+ // queue's weights are 1
|
|
|
+ // root
|
|
|
+ // - a (w=1)
|
|
|
+ // - b (w=1)
|
|
|
+ // - c-auto (w=1)
|
|
|
+ // - d-auto (w=1)
|
|
|
+ // - e-auto (w=1)
|
|
|
+ // - e1-auto (w=1)
|
|
|
+ startScheduler();
|
|
|
+
|
|
|
+ createBasicQueueStructureAndValidate();
|
|
|
+
|
|
|
+ // Under e, there's only one queue, so e1/e have same capacity
|
|
|
+ CSQueue e1 = cs.getQueue("root.e-auto.e1-auto");
|
|
|
+ Assert.assertEquals(1 / 5f, e1.getAbsoluteCapacity(), 1e-6);
|
|
|
+ Assert.assertEquals(1f, e1.getQueueCapacities().getWeight(), 1e-6);
|
|
|
+ Assert.assertEquals(240 * GB,
|
|
|
+ e1.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize());
|
|
|
+
|
|
|
+ // Check after removal e1.
|
|
|
+ cs.removeQueue(e1);
|
|
|
+ CSQueue e = cs.getQueue("root.e-auto");
|
|
|
+ Assert.assertEquals(1 / 5f, e.getAbsoluteCapacity(), 1e-6);
|
|
|
+ Assert.assertEquals(1f, e.getQueueCapacities().getWeight(), 1e-6);
|
|
|
+ Assert.assertEquals(240 * GB,
|
|
|
+ e.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize());
|
|
|
+
|
|
|
+ // Check after removal e.
|
|
|
+ cs.removeQueue(e);
|
|
|
+ CSQueue d = cs.getQueue("root.d-auto");
|
|
|
+ Assert.assertEquals(1 / 4f, d.getAbsoluteCapacity(), 1e-6);
|
|
|
+ Assert.assertEquals(1f, d.getQueueCapacities().getWeight(), 1e-6);
|
|
|
+ Assert.assertEquals(300 * GB,
|
|
|
+ d.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize());
|
|
|
+
|
|
|
+ // Check after removal d.
|
|
|
+ cs.removeQueue(d);
|
|
|
+ CSQueue c = cs.getQueue("root.c-auto");
|
|
|
+ Assert.assertEquals(1 / 3f, c.getAbsoluteCapacity(), 1e-6);
|
|
|
+ Assert.assertEquals(1f, c.getQueueCapacities().getWeight(), 1e-6);
|
|
|
+ Assert.assertEquals(400 * GB,
|
|
|
+ c.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize());
|
|
|
+
|
|
|
+ // Check after removal c.
|
|
|
+ cs.removeQueue(c);
|
|
|
+ CSQueue b = cs.getQueue("root.b");
|
|
|
+ Assert.assertEquals(1 / 2f, b.getAbsoluteCapacity(), 1e-6);
|
|
|
+ Assert.assertEquals(1f, b.getQueueCapacities().getWeight(), 1e-6);
|
|
|
+ Assert.assertEquals(600 * GB,
|
|
|
+ b.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize());
|
|
|
+
|
|
|
+ // Check can't remove static queue b.
|
|
|
+ try {
|
|
|
+ cs.removeQueue(b);
|
|
|
+ Assert.fail("Can't remove static queue b!");
|
|
|
+ } catch (Exception ex) {
|
|
|
+ Assert.assertTrue(ex
|
|
|
+ instanceof SchedulerDynamicEditException);
|
|
|
+ }
|
|
|
+ // Check a.
|
|
|
+ CSQueue a = cs.getQueue("root.a");
|
|
|
+ Assert.assertEquals(1 / 2f, a.getAbsoluteCapacity(), 1e-6);
|
|
|
+ Assert.assertEquals(1f, a.getQueueCapacities().getWeight(), 1e-6);
|
|
|
+ Assert.assertEquals(600 * GB,
|
|
|
+ b.getQueueResourceQuotas().getEffectiveMinResource().getMemorySize());
|
|
|
+ }
|
|
|
+
|
|
|
+ protected LeafQueue createQueue(String queuePath) throws YarnException {
|
|
|
return autoQueueHandler.autoCreateQueue(
|
|
|
CSQueueUtils.extractQueuePath(queuePath));
|
|
|
}
|