|
@@ -34,6 +34,8 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
+import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
+import org.apache.hadoop.yarn.event.DrainDispatcher;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
|
@@ -567,4 +569,166 @@ public class TestApplicationPriority {
|
|
|
Assert.assertEquals(6, schedulerAppAttemptApp1.getLiveContainers().size());
|
|
|
rm.stop();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * <p>
|
|
|
+ * Test case verifies the order of applications activated after RM Restart.
|
|
|
+ * </p>
|
|
|
+ * <li>App-1 and app-2 submitted and scheduled and running with a priority
|
|
|
+ * 5 and 6 Respectively</li>
|
|
|
+ * <li>App-3 submitted and scheduled with a priority 7. This
|
|
|
+ * is not activated since AMResourceLimit is reached</li>
|
|
|
+ * <li>RM restarted</li>
|
|
|
+ * <li>App-1 get activated nevertheless of AMResourceLimit</li>
|
|
|
+ * <li>App-2 and app-3 put in pendingOrderingPolicy</li>
|
|
|
+ * <li>After NM registration, app-3 is activated</li>
|
|
|
+ * <p>
|
|
|
+ * Expected Output : App-2 must get activated since app-2 was running earlier
|
|
|
+ * </p>
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testOrderOfActivatingThePriorityApplicationOnRMRestart()
|
|
|
+ throws Exception {
|
|
|
+ conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
|
|
|
+ conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
|
|
|
+ conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
|
|
|
+ conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
|
|
|
+ YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
|
|
|
+ conf.setInt(YarnConfiguration.MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY, 10);
|
|
|
+ final DrainDispatcher dispatcher = new DrainDispatcher();
|
|
|
+
|
|
|
+ MemoryRMStateStore memStore = new MemoryRMStateStore();
|
|
|
+ memStore.init(conf);
|
|
|
+
|
|
|
+ MockRM rm1 = new MockRM(conf, memStore) {
|
|
|
+ @Override
|
|
|
+ protected Dispatcher createDispatcher() {
|
|
|
+ return dispatcher;
|
|
|
+ }
|
|
|
+ };
|
|
|
+ rm1.start();
|
|
|
+
|
|
|
+ MockNM nm1 =
|
|
|
+ new MockNM("127.0.0.1:1234", 16384, rm1.getResourceTrackerService());
|
|
|
+ nm1.registerNode();
|
|
|
+
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ ResourceScheduler scheduler = rm1.getRMContext().getScheduler();
|
|
|
+ LeafQueue defaultQueue =
|
|
|
+ (LeafQueue) ((CapacityScheduler) scheduler).getQueue("default");
|
|
|
+ int memory = defaultQueue.getAMResourceLimit().getMemory() / 2;
|
|
|
+
|
|
|
+ // App-1 with priority 5 submitted and running
|
|
|
+ Priority appPriority1 = Priority.newInstance(5);
|
|
|
+ RMApp app1 = rm1.submitApp(memory, appPriority1);
|
|
|
+ MockAM am1 = MockRM.launchAM(app1, rm1, nm1);
|
|
|
+ am1.registerAppAttempt();
|
|
|
+
|
|
|
+ // App-2 with priority 6 submitted and running
|
|
|
+ Priority appPriority2 = Priority.newInstance(6);
|
|
|
+ RMApp app2 = rm1.submitApp(memory, appPriority2);
|
|
|
+ MockAM am2 = MockRM.launchAM(app2, rm1, nm1);
|
|
|
+ am2.registerAppAttempt();
|
|
|
+
|
|
|
+ dispatcher.await();
|
|
|
+ Assert.assertEquals(2, defaultQueue.getNumActiveApplications());
|
|
|
+ Assert.assertEquals(0, defaultQueue.getNumPendingApplications());
|
|
|
+
|
|
|
+ // App-3 with priority 7 submitted and scheduled. But not activated since
|
|
|
+ // AMResourceLimit threshold
|
|
|
+ Priority appPriority3 = Priority.newInstance(7);
|
|
|
+ RMApp app3 = rm1.submitApp(memory, appPriority3);
|
|
|
+
|
|
|
+ dispatcher.await();
|
|
|
+ Assert.assertEquals(2, defaultQueue.getNumActiveApplications());
|
|
|
+ Assert.assertEquals(1, defaultQueue.getNumPendingApplications());
|
|
|
+
|
|
|
+ Iterator<FiCaSchedulerApp> iterator =
|
|
|
+ defaultQueue.getOrderingPolicy().getSchedulableEntities().iterator();
|
|
|
+ FiCaSchedulerApp fcApp2 = iterator.next();
|
|
|
+ Assert.assertEquals(app2.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
+ fcApp2.getApplicationAttemptId());
|
|
|
+
|
|
|
+ FiCaSchedulerApp fcApp1 = iterator.next();
|
|
|
+ Assert.assertEquals(app1.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
+ fcApp1.getApplicationAttemptId());
|
|
|
+
|
|
|
+ iterator = defaultQueue.getPendingApplications().iterator();
|
|
|
+ FiCaSchedulerApp fcApp3 = iterator.next();
|
|
|
+ Assert.assertEquals(app3.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
+ fcApp3.getApplicationAttemptId());
|
|
|
+
|
|
|
+ final DrainDispatcher dispatcher1 = new DrainDispatcher();
|
|
|
+ // create new RM to represent restart and recover state
|
|
|
+ MockRM rm2 = new MockRM(conf, memStore) {
|
|
|
+ @Override
|
|
|
+ protected Dispatcher createDispatcher() {
|
|
|
+ return dispatcher1;
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ // start new RM
|
|
|
+ rm2.start();
|
|
|
+ // change NM to point to new RM
|
|
|
+ nm1.setResourceTrackerService(rm2.getResourceTrackerService());
|
|
|
+
|
|
|
+ // Verify RM Apps after this restart
|
|
|
+ Assert.assertEquals(3, rm2.getRMContext().getRMApps().size());
|
|
|
+
|
|
|
+ dispatcher1.await();
|
|
|
+ scheduler = rm2.getRMContext().getScheduler();
|
|
|
+ defaultQueue =
|
|
|
+ (LeafQueue) ((CapacityScheduler) scheduler).getQueue("default");
|
|
|
+
|
|
|
+ // wait for all applications to get added to scheduler
|
|
|
+ int count = 5;
|
|
|
+ while (count-- > 0) {
|
|
|
+ if ((defaultQueue.getNumActiveApplications() + defaultQueue
|
|
|
+ .getNumPendingApplications()) == 3) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ Thread.sleep(500);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Before NM registration, AMResourceLimit threshold is 0. So 1st
|
|
|
+ // applications get activated nevertheless of AMResourceLimit threshold
|
|
|
+ // Two applications are in pending
|
|
|
+ Assert.assertEquals(1, defaultQueue.getNumActiveApplications());
|
|
|
+ Assert.assertEquals(2, defaultQueue.getNumPendingApplications());
|
|
|
+
|
|
|
+ // NM resync to new RM
|
|
|
+ nm1.registerNode();
|
|
|
+ dispatcher1.await();
|
|
|
+
|
|
|
+ // wait for activating one applications
|
|
|
+ count = 5;
|
|
|
+ while (count-- > 0) {
|
|
|
+ if (defaultQueue.getOrderingPolicy().getSchedulableEntities().size() == 2) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ Thread.sleep(500);
|
|
|
+ }
|
|
|
+
|
|
|
+ // verify for order of activated applications iterator
|
|
|
+ iterator =
|
|
|
+ defaultQueue.getOrderingPolicy().getSchedulableEntities().iterator();
|
|
|
+ fcApp2 = iterator.next();
|
|
|
+ Assert.assertEquals(app2.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
+ fcApp2.getApplicationAttemptId());
|
|
|
+
|
|
|
+ fcApp1 = iterator.next();
|
|
|
+ Assert.assertEquals(app1.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
+ fcApp1.getApplicationAttemptId());
|
|
|
+
|
|
|
+ // verify for pending application iterator. It should be app-3 attempt
|
|
|
+ iterator = defaultQueue.getPendingApplications().iterator();
|
|
|
+ fcApp3 = iterator.next();
|
|
|
+ Assert.assertEquals(app3.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
+ fcApp3.getApplicationAttemptId());
|
|
|
+
|
|
|
+ rm2.stop();
|
|
|
+ rm1.stop();
|
|
|
+ }
|
|
|
}
|