|
@@ -390,6 +390,204 @@ public class TestApplicationLifetimeMonitor {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ static final String CQ1 = "child1";
|
|
|
|
+ @Test(timeout = 120000)
|
|
|
|
+ public void testInheritAppLifetimeFromParentQueue() throws Exception {
|
|
|
|
+ YarnConfiguration yarnConf = conf;
|
|
|
|
+ long maxRootLifetime = 20L;
|
|
|
|
+ long defaultRootLifetime = 10L;
|
|
|
|
+ if (scheduler.equals(CapacityScheduler.class)) {
|
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
|
+ csConf.setQueues(CapacitySchedulerConfiguration.ROOT,
|
|
|
|
+ new String[] {CQ1});
|
|
|
|
+ csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + CQ1, 100);
|
|
|
|
+ csConf.setMaximumLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT, maxRootLifetime);
|
|
|
|
+ csConf.setDefaultLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT, defaultRootLifetime);
|
|
|
|
+ yarnConf = new YarnConfiguration(csConf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ MockRM rm = null;
|
|
|
|
+ try {
|
|
|
|
+ rm = new MockRM(yarnConf);
|
|
|
|
+ rm.start();
|
|
|
|
+
|
|
|
|
+ Priority appPriority = Priority.newInstance(0);
|
|
|
|
+ MockNM nm1 = rm.registerNode("127.0.0.1:1234", 16 * 1024);
|
|
|
|
+
|
|
|
|
+ // user not set lifetime, so queue max lifetime will be considered.
|
|
|
|
+ RMApp app1 = MockRMAppSubmitter.submit(rm,
|
|
|
|
+ MockRMAppSubmissionData.Builder.createWithMemory(1024, rm)
|
|
|
|
+ .withAppPriority(appPriority)
|
|
|
|
+ .withApplicationTimeouts(Collections.emptyMap())
|
|
|
|
+ .withQueue(CQ1)
|
|
|
|
+ .build());
|
|
|
|
+
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+
|
|
|
|
+ if (scheduler.equals(CapacityScheduler.class)) {
|
|
|
|
+ // Supported only on capacity scheduler
|
|
|
|
+ CapacityScheduler csched =
|
|
|
|
+ (CapacityScheduler) rm.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ rm.waitForState(app1.getApplicationId(), RMAppState.KILLED);
|
|
|
|
+ long totalTimeRun = app1.getFinishTime() - app1.getSubmitTime();
|
|
|
|
+ // Child queue should have inherited parent max and default lifetimes.
|
|
|
|
+ Assert.assertEquals("Child queue max lifetime should have overridden"
|
|
|
|
+ + " parent value",
|
|
|
|
+ maxRootLifetime,
|
|
|
|
+ csched.getQueue(CQ1).getMaximumApplicationLifetime());
|
|
|
|
+ Assert.assertEquals("Child queue default lifetime should have"
|
|
|
|
+ + " overridden parent value",
|
|
|
|
+ defaultRootLifetime,
|
|
|
|
+ csched.getQueue(CQ1).getDefaultApplicationLifetime());
|
|
|
|
+ // app1 (run in the 'child1' queue) should have run longer than the
|
|
|
|
+ // default lifetime but less than the max lifetime.
|
|
|
|
+ Assert.assertTrue("Application killed before default lifetime value",
|
|
|
|
+ totalTimeRun > (defaultRootLifetime * 1000));
|
|
|
|
+ Assert.assertTrue(
|
|
|
|
+ "Application killed after max lifetime value " + totalTimeRun,
|
|
|
|
+ totalTimeRun < (maxRootLifetime * 1000));
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ stopRM(rm);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test(timeout = 120000)
|
|
|
|
+ public void testOverrideParentQueueMaxAppLifetime() throws Exception {
|
|
|
|
+ YarnConfiguration yarnConf = conf;
|
|
|
|
+ long maxRootLifetime = 20L;
|
|
|
|
+ long maxChildLifetime = 40L;
|
|
|
|
+ long defaultRootLifetime = 10L;
|
|
|
|
+ if (scheduler.equals(CapacityScheduler.class)) {
|
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
|
+ csConf.setQueues(CapacitySchedulerConfiguration.ROOT,
|
|
|
|
+ new String[] {CQ1});
|
|
|
|
+ csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + CQ1, 100);
|
|
|
|
+ csConf.setMaximumLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT, maxRootLifetime);
|
|
|
|
+ csConf.setMaximumLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT + "." + CQ1, maxChildLifetime);
|
|
|
|
+ csConf.setDefaultLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT, defaultRootLifetime);
|
|
|
|
+ csConf.setDefaultLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT + "." + CQ1, maxChildLifetime);
|
|
|
|
+ yarnConf = new YarnConfiguration(csConf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ MockRM rm = null;
|
|
|
|
+ try {
|
|
|
|
+ rm = new MockRM(yarnConf);
|
|
|
|
+ rm.start();
|
|
|
|
+
|
|
|
|
+ Priority appPriority = Priority.newInstance(0);
|
|
|
|
+ MockNM nm1 = rm.registerNode("127.0.0.1:1234", 16 * 1024);
|
|
|
|
+
|
|
|
|
+ // user not set lifetime, so queue max lifetime will be considered.
|
|
|
|
+ RMApp app1 = MockRMAppSubmitter.submit(rm,
|
|
|
|
+ MockRMAppSubmissionData.Builder.createWithMemory(1024, rm)
|
|
|
|
+ .withAppPriority(appPriority)
|
|
|
|
+ .withApplicationTimeouts(Collections.emptyMap())
|
|
|
|
+ .withQueue(CQ1)
|
|
|
|
+ .build());
|
|
|
|
+
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+
|
|
|
|
+ if (scheduler.equals(CapacityScheduler.class)) {
|
|
|
|
+ // Supported only on capacity scheduler
|
|
|
|
+ CapacityScheduler csched =
|
|
|
|
+ (CapacityScheduler) rm.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ rm.waitForState(app1.getApplicationId(), RMAppState.KILLED);
|
|
|
|
+ long totalTimeRun = app1.getFinishTime() - app1.getSubmitTime();
|
|
|
|
+ // Child queue's max lifetime can override parent's and be larger.
|
|
|
|
+ Assert.assertTrue("Application killed before default lifetime value",
|
|
|
|
+ (maxRootLifetime < maxChildLifetime)
|
|
|
|
+ && (totalTimeRun > (maxChildLifetime * 1000)));
|
|
|
|
+ Assert.assertEquals("Root queue max lifetime property set incorrectly",
|
|
|
|
+ maxRootLifetime,
|
|
|
|
+ csched.getRootQueue().getMaximumApplicationLifetime());
|
|
|
|
+ Assert.assertEquals("Child queue max lifetime should have overridden"
|
|
|
|
+ + " parent value", maxChildLifetime,
|
|
|
|
+ csched.getQueue(CQ1).getMaximumApplicationLifetime());
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ stopRM(rm);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test(timeout = 120000)
|
|
|
|
+ public void testOverrideParentQueueDefaultAppLifetime() throws Exception {
|
|
|
|
+ YarnConfiguration yarnConf = conf;
|
|
|
|
+ long maxRootLifetime = -1L;
|
|
|
|
+ long maxChildLifetime = -1L;
|
|
|
|
+ long defaultChildLifetime = 10L;
|
|
|
|
+ if (scheduler.equals(CapacityScheduler.class)) {
|
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
|
+ csConf.setQueues(CapacitySchedulerConfiguration.ROOT,
|
|
|
|
+ new String[] {CQ1});
|
|
|
|
+ csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + CQ1, 100);
|
|
|
|
+ csConf.setMaximumLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT, maxRootLifetime);
|
|
|
|
+ csConf.setMaximumLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT + "." + CQ1, maxChildLifetime);
|
|
|
|
+ csConf.setDefaultLifetimePerQueue(
|
|
|
|
+ CapacitySchedulerConfiguration.ROOT + "." + CQ1,
|
|
|
|
+ defaultChildLifetime);
|
|
|
|
+ yarnConf = new YarnConfiguration(csConf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ MockRM rm = null;
|
|
|
|
+ try {
|
|
|
|
+ rm = new MockRM(yarnConf);
|
|
|
|
+ rm.start();
|
|
|
|
+
|
|
|
|
+ Priority appPriority = Priority.newInstance(0);
|
|
|
|
+ MockNM nm1 = rm.registerNode("127.0.0.1:1234", 16 * 1024);
|
|
|
|
+
|
|
|
|
+ // user not set lifetime, so queue max lifetime will be considered.
|
|
|
|
+ RMApp app1 = MockRMAppSubmitter.submit(rm,
|
|
|
|
+ MockRMAppSubmissionData.Builder.createWithMemory(1024, rm)
|
|
|
|
+ .withAppPriority(appPriority)
|
|
|
|
+ .withApplicationTimeouts(Collections.emptyMap())
|
|
|
|
+ .withQueue(CQ1)
|
|
|
|
+ .build());
|
|
|
|
+
|
|
|
|
+ nm1.nodeHeartbeat(true);
|
|
|
|
+
|
|
|
|
+ if (scheduler.equals(CapacityScheduler.class)) {
|
|
|
|
+ // Supported only on capacity scheduler
|
|
|
|
+ CapacityScheduler csched =
|
|
|
|
+ (CapacityScheduler) rm.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ rm.waitForState(app1.getApplicationId(), RMAppState.KILLED);
|
|
|
|
+ long totalTimeRun = app1.getFinishTime() - app1.getSubmitTime();
|
|
|
|
+ // app1 (run in 'child1' queue) should have overridden the parent's
|
|
|
|
+ // default lifetime.
|
|
|
|
+ Assert.assertTrue("Application killed before default lifetime value",
|
|
|
|
+ totalTimeRun > (defaultChildLifetime * 1000));
|
|
|
|
+ // Root and child queue's max lifetime should be -1.
|
|
|
|
+ Assert.assertEquals("Root queue max lifetime property set incorrectly",
|
|
|
|
+ maxRootLifetime,
|
|
|
|
+ csched.getRootQueue().getMaximumApplicationLifetime());
|
|
|
|
+ Assert.assertEquals("Child queue max lifetime property set incorrectly",
|
|
|
|
+ maxChildLifetime,
|
|
|
|
+ csched.getQueue(CQ1).getMaximumApplicationLifetime());
|
|
|
|
+ // 'child1' queue's default lifetime should have overridden parent's.
|
|
|
|
+ Assert.assertEquals("Child queue default lifetime should have"
|
|
|
|
+ + " overridden parent value", defaultChildLifetime,
|
|
|
|
+ csched.getQueue(CQ1).getDefaultApplicationLifetime());
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ stopRM(rm);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private CapacitySchedulerConfiguration setUpCSQueue(long maxLifetime,
|
|
private CapacitySchedulerConfiguration setUpCSQueue(long maxLifetime,
|
|
long defaultLifetime) {
|
|
long defaultLifetime) {
|
|
CapacitySchedulerConfiguration csConf =
|
|
CapacitySchedulerConfiguration csConf =
|