|
@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
|
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.QueueState;
|
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
@@ -68,6 +69,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.NodeManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
@@ -100,6 +102,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
|
|
|
|
+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.CapacitySchedulerQueueInfoList;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
@@ -1014,4 +1020,782 @@ public class TestCapacityScheduler {
|
|
// Now with updated ResourceRequest, a container is allocated for AM.
|
|
// Now with updated ResourceRequest, a container is allocated for AM.
|
|
Assert.assertTrue(containers.size() == 1);
|
|
Assert.assertTrue(containers.size() == 1);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private MockRM setUpMove() {
|
|
|
|
+ CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
|
|
|
|
+ setupQueueConfiguration(conf);
|
|
|
|
+ conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
|
|
|
+ ResourceScheduler.class);
|
|
|
|
+ MockRM rm = new MockRM(conf);
|
|
|
|
+ rm.start();
|
|
|
|
+ return rm;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMoveAppBasic() throws Exception {
|
|
|
|
+ MockRM rm = setUpMove();
|
|
|
|
+ AbstractYarnScheduler scheduler =
|
|
|
|
+ (AbstractYarnScheduler) rm.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ // submit an app
|
|
|
|
+ RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");
|
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
|
+ rm.getApplicationReport(app.getApplicationId())
|
|
|
|
+ .getCurrentApplicationAttemptId();
|
|
|
|
+
|
|
|
|
+ // check preconditions
|
|
|
|
+ List<ApplicationAttemptId> appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertEquals(1, appsInA1.size());
|
|
|
|
+ String queue =
|
|
|
|
+ scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
|
|
|
|
+ .getQueueName();
|
|
|
|
+ Assert.assertTrue(queue.equals("a1"));
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInB1 = scheduler.getAppsInQueue("b1");
|
|
|
|
+ assertTrue(appsInB1.isEmpty());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInB = scheduler.getAppsInQueue("b");
|
|
|
|
+ assertTrue(appsInB.isEmpty());
|
|
|
|
+
|
|
|
|
+ // now move the app
|
|
|
|
+ scheduler.moveApplication(app.getApplicationId(), "b1");
|
|
|
|
+
|
|
|
|
+ // check postconditions
|
|
|
|
+ appsInB1 = scheduler.getAppsInQueue("b1");
|
|
|
|
+ assertEquals(1, appsInB1.size());
|
|
|
|
+ queue =
|
|
|
|
+ scheduler.getApplicationAttempt(appsInB1.get(0)).getQueue()
|
|
|
|
+ .getQueueName();
|
|
|
|
+ System.out.println(queue);
|
|
|
|
+ Assert.assertTrue(queue.equals("b1"));
|
|
|
|
+
|
|
|
|
+ appsInB = scheduler.getAppsInQueue("b");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInB.size());
|
|
|
|
+
|
|
|
|
+ appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertTrue(appsInA1.isEmpty());
|
|
|
|
+
|
|
|
|
+ appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.isEmpty());
|
|
|
|
+
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMoveAppSameParent() throws Exception {
|
|
|
|
+ MockRM rm = setUpMove();
|
|
|
|
+ AbstractYarnScheduler scheduler =
|
|
|
|
+ (AbstractYarnScheduler) rm.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ // submit an app
|
|
|
|
+ RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");
|
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
|
+ rm.getApplicationReport(app.getApplicationId())
|
|
|
|
+ .getCurrentApplicationAttemptId();
|
|
|
|
+
|
|
|
|
+ // check preconditions
|
|
|
|
+ List<ApplicationAttemptId> appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertEquals(1, appsInA1.size());
|
|
|
|
+ String queue =
|
|
|
|
+ scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
|
|
|
|
+ .getQueueName();
|
|
|
|
+ Assert.assertTrue(queue.equals("a1"));
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInA2 = scheduler.getAppsInQueue("a2");
|
|
|
|
+ assertTrue(appsInA2.isEmpty());
|
|
|
|
+
|
|
|
|
+ // now move the app
|
|
|
|
+ scheduler.moveApplication(app.getApplicationId(), "a2");
|
|
|
|
+
|
|
|
|
+ // check postconditions
|
|
|
|
+ appsInA2 = scheduler.getAppsInQueue("a2");
|
|
|
|
+ assertEquals(1, appsInA2.size());
|
|
|
|
+ queue =
|
|
|
|
+ scheduler.getApplicationAttempt(appsInA2.get(0)).getQueue()
|
|
|
|
+ .getQueueName();
|
|
|
|
+ Assert.assertTrue(queue.equals("a2"));
|
|
|
|
+
|
|
|
|
+ appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertTrue(appsInA1.isEmpty());
|
|
|
|
+
|
|
|
|
+ appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
|
+
|
|
|
|
+ appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMoveAppForMoveToQueueWithFreeCap() throws Exception {
|
|
|
|
+
|
|
|
|
+ ResourceScheduler scheduler = resourceManager.getResourceScheduler();
|
|
|
|
+ // Register node1
|
|
|
|
+ String host_0 = "host_0";
|
|
|
|
+ NodeManager nm_0 =
|
|
|
|
+ registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
|
+ Resources.createResource(4 * GB, 1));
|
|
|
|
+
|
|
|
|
+ // Register node2
|
|
|
|
+ String host_1 = "host_1";
|
|
|
|
+ NodeManager nm_1 =
|
|
|
|
+ registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
|
+ Resources.createResource(2 * GB, 1));
|
|
|
|
+
|
|
|
|
+ // ResourceRequest priorities
|
|
|
|
+ Priority priority_0 =
|
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
|
|
|
|
+ .create(0);
|
|
|
|
+ Priority priority_1 =
|
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
|
|
|
|
+ .create(1);
|
|
|
|
+
|
|
|
|
+ // Submit application_0
|
|
|
|
+ Application application_0 =
|
|
|
|
+ new Application("user_0", "a1", resourceManager);
|
|
|
|
+ application_0.submit(); // app + app attempt event sent to scheduler
|
|
|
|
+
|
|
|
|
+ application_0.addNodeManager(host_0, 1234, nm_0);
|
|
|
|
+ application_0.addNodeManager(host_1, 1234, nm_1);
|
|
|
|
+
|
|
|
|
+ Resource capability_0_0 = Resources.createResource(1 * GB, 1);
|
|
|
|
+ application_0.addResourceRequestSpec(priority_1, capability_0_0);
|
|
|
|
+
|
|
|
|
+ Resource capability_0_1 = Resources.createResource(2 * GB, 1);
|
|
|
|
+ application_0.addResourceRequestSpec(priority_0, capability_0_1);
|
|
|
|
+
|
|
|
|
+ Task task_0_0 =
|
|
|
|
+ new Task(application_0, priority_1, new String[] { host_0, host_1 });
|
|
|
|
+ application_0.addTask(task_0_0);
|
|
|
|
+
|
|
|
|
+ // Submit application_1
|
|
|
|
+ Application application_1 =
|
|
|
|
+ new Application("user_1", "b2", resourceManager);
|
|
|
|
+ application_1.submit(); // app + app attempt event sent to scheduler
|
|
|
|
+
|
|
|
|
+ application_1.addNodeManager(host_0, 1234, nm_0);
|
|
|
|
+ application_1.addNodeManager(host_1, 1234, nm_1);
|
|
|
|
+
|
|
|
|
+ Resource capability_1_0 = Resources.createResource(1 * GB, 1);
|
|
|
|
+ application_1.addResourceRequestSpec(priority_1, capability_1_0);
|
|
|
|
+
|
|
|
|
+ Resource capability_1_1 = Resources.createResource(2 * GB, 1);
|
|
|
|
+ application_1.addResourceRequestSpec(priority_0, capability_1_1);
|
|
|
|
+
|
|
|
|
+ Task task_1_0 =
|
|
|
|
+ new Task(application_1, priority_1, new String[] { host_0, host_1 });
|
|
|
|
+ application_1.addTask(task_1_0);
|
|
|
|
+
|
|
|
|
+ // Send resource requests to the scheduler
|
|
|
|
+ application_0.schedule(); // allocate
|
|
|
|
+ application_1.schedule(); // allocate
|
|
|
|
+
|
|
|
|
+ // task_0_0 task_1_0 allocated, used=2G
|
|
|
|
+ nodeUpdate(nm_0);
|
|
|
|
+
|
|
|
|
+ // nothing allocated
|
|
|
|
+ nodeUpdate(nm_1);
|
|
|
|
+
|
|
|
|
+ // Get allocations from the scheduler
|
|
|
|
+ application_0.schedule(); // task_0_0
|
|
|
|
+ checkApplicationResourceUsage(1 * GB, application_0);
|
|
|
|
+
|
|
|
|
+ application_1.schedule(); // task_1_0
|
|
|
|
+ checkApplicationResourceUsage(1 * GB, application_1);
|
|
|
|
+
|
|
|
|
+ checkNodeResourceUsage(2 * GB, nm_0); // task_0_0 (1G) and task_1_0 (1G) 2G
|
|
|
|
+ // available
|
|
|
|
+ checkNodeResourceUsage(0 * GB, nm_1); // no tasks, 2G available
|
|
|
|
+
|
|
|
|
+ // move app from a1(30% cap of total 10.5% cap) to b1(79,2% cap of 89,5%
|
|
|
|
+ // total cap)
|
|
|
|
+ scheduler.moveApplication(application_0.getApplicationId(), "b1");
|
|
|
|
+
|
|
|
|
+ // 2GB 1C
|
|
|
|
+ Task task_1_1 =
|
|
|
|
+ new Task(application_1, priority_0,
|
|
|
|
+ new String[] { ResourceRequest.ANY });
|
|
|
|
+ application_1.addTask(task_1_1);
|
|
|
|
+
|
|
|
|
+ application_1.schedule();
|
|
|
|
+
|
|
|
|
+ // 2GB 1C
|
|
|
|
+ Task task_0_1 =
|
|
|
|
+ new Task(application_0, priority_0, new String[] { host_0, host_1 });
|
|
|
|
+ application_0.addTask(task_0_1);
|
|
|
|
+
|
|
|
|
+ application_0.schedule();
|
|
|
|
+
|
|
|
|
+ // prev 2G used free 2G
|
|
|
|
+ nodeUpdate(nm_0);
|
|
|
|
+
|
|
|
|
+ // prev 0G used free 2G
|
|
|
|
+ nodeUpdate(nm_1);
|
|
|
|
+
|
|
|
|
+ // Get allocations from the scheduler
|
|
|
|
+ application_1.schedule();
|
|
|
|
+ checkApplicationResourceUsage(3 * GB, application_1);
|
|
|
|
+
|
|
|
|
+ // Get allocations from the scheduler
|
|
|
|
+ application_0.schedule();
|
|
|
|
+ checkApplicationResourceUsage(3 * GB, application_0);
|
|
|
|
+
|
|
|
|
+ checkNodeResourceUsage(4 * GB, nm_0);
|
|
|
|
+ checkNodeResourceUsage(2 * GB, nm_1);
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMoveAppSuccess() throws Exception {
|
|
|
|
+
|
|
|
|
+ ResourceScheduler scheduler = resourceManager.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ // Register node1
|
|
|
|
+ String host_0 = "host_0";
|
|
|
|
+ NodeManager nm_0 =
|
|
|
|
+ registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
|
+ Resources.createResource(5 * GB, 1));
|
|
|
|
+
|
|
|
|
+ // Register node2
|
|
|
|
+ String host_1 = "host_1";
|
|
|
|
+ NodeManager nm_1 =
|
|
|
|
+ registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
|
+ Resources.createResource(5 * GB, 1));
|
|
|
|
+
|
|
|
|
+ // ResourceRequest priorities
|
|
|
|
+ Priority priority_0 =
|
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
|
|
|
|
+ .create(0);
|
|
|
|
+ Priority priority_1 =
|
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
|
|
|
|
+ .create(1);
|
|
|
|
+
|
|
|
|
+ // Submit application_0
|
|
|
|
+ Application application_0 =
|
|
|
|
+ new Application("user_0", "a1", resourceManager);
|
|
|
|
+ application_0.submit(); // app + app attempt event sent to scheduler
|
|
|
|
+
|
|
|
|
+ application_0.addNodeManager(host_0, 1234, nm_0);
|
|
|
|
+ application_0.addNodeManager(host_1, 1234, nm_1);
|
|
|
|
+
|
|
|
|
+ Resource capability_0_0 = Resources.createResource(3 * GB, 1);
|
|
|
|
+ application_0.addResourceRequestSpec(priority_1, capability_0_0);
|
|
|
|
+
|
|
|
|
+ Resource capability_0_1 = Resources.createResource(2 * GB, 1);
|
|
|
|
+ application_0.addResourceRequestSpec(priority_0, capability_0_1);
|
|
|
|
+
|
|
|
|
+ Task task_0_0 =
|
|
|
|
+ new Task(application_0, priority_1, new String[] { host_0, host_1 });
|
|
|
|
+ application_0.addTask(task_0_0);
|
|
|
|
+
|
|
|
|
+ // Submit application_1
|
|
|
|
+ Application application_1 =
|
|
|
|
+ new Application("user_1", "b2", resourceManager);
|
|
|
|
+ application_1.submit(); // app + app attempt event sent to scheduler
|
|
|
|
+
|
|
|
|
+ application_1.addNodeManager(host_0, 1234, nm_0);
|
|
|
|
+ application_1.addNodeManager(host_1, 1234, nm_1);
|
|
|
|
+
|
|
|
|
+ Resource capability_1_0 = Resources.createResource(1 * GB, 1);
|
|
|
|
+ application_1.addResourceRequestSpec(priority_1, capability_1_0);
|
|
|
|
+
|
|
|
|
+ Resource capability_1_1 = Resources.createResource(2 * GB, 1);
|
|
|
|
+ application_1.addResourceRequestSpec(priority_0, capability_1_1);
|
|
|
|
+
|
|
|
|
+ Task task_1_0 =
|
|
|
|
+ new Task(application_1, priority_1, new String[] { host_0, host_1 });
|
|
|
|
+ application_1.addTask(task_1_0);
|
|
|
|
+
|
|
|
|
+ // Send resource requests to the scheduler
|
|
|
|
+ application_0.schedule(); // allocate
|
|
|
|
+ application_1.schedule(); // allocate
|
|
|
|
+
|
|
|
|
+ // b2 can only run 1 app at a time
|
|
|
|
+ scheduler.moveApplication(application_0.getApplicationId(), "b2");
|
|
|
|
+
|
|
|
|
+ nodeUpdate(nm_0);
|
|
|
|
+
|
|
|
|
+ nodeUpdate(nm_1);
|
|
|
|
+
|
|
|
|
+ // Get allocations from the scheduler
|
|
|
|
+ application_0.schedule(); // task_0_0
|
|
|
|
+ checkApplicationResourceUsage(0 * GB, application_0);
|
|
|
|
+
|
|
|
|
+ application_1.schedule(); // task_1_0
|
|
|
|
+ checkApplicationResourceUsage(1 * GB, application_1);
|
|
|
|
+
|
|
|
|
+ // task_1_0 (1G) application_0 moved to b2 with max running app 1 so it is
|
|
|
|
+ // not scheduled
|
|
|
|
+ checkNodeResourceUsage(1 * GB, nm_0);
|
|
|
|
+ checkNodeResourceUsage(0 * GB, nm_1);
|
|
|
|
+
|
|
|
|
+ // lets move application_0 to a queue where it can run
|
|
|
|
+ scheduler.moveApplication(application_0.getApplicationId(), "a2");
|
|
|
|
+ application_0.schedule();
|
|
|
|
+
|
|
|
|
+ nodeUpdate(nm_1);
|
|
|
|
+
|
|
|
|
+ // Get allocations from the scheduler
|
|
|
|
+ application_0.schedule(); // task_0_0
|
|
|
|
+ checkApplicationResourceUsage(3 * GB, application_0);
|
|
|
|
+
|
|
|
|
+ checkNodeResourceUsage(1 * GB, nm_0);
|
|
|
|
+ checkNodeResourceUsage(3 * GB, nm_1);
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test(expected = YarnException.class)
|
|
|
|
+ public void testMoveAppViolateQueueState() throws Exception {
|
|
|
|
+
|
|
|
|
+ resourceManager = new ResourceManager();
|
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
|
+ setupQueueConfiguration(csConf);
|
|
|
|
+ StringBuilder qState = new StringBuilder();
|
|
|
|
+ qState.append(CapacitySchedulerConfiguration.PREFIX).append(B)
|
|
|
|
+ .append(CapacitySchedulerConfiguration.DOT)
|
|
|
|
+ .append(CapacitySchedulerConfiguration.STATE);
|
|
|
|
+ csConf.set(qState.toString(), QueueState.STOPPED.name());
|
|
|
|
+ YarnConfiguration conf = new YarnConfiguration(csConf);
|
|
|
|
+ conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
|
|
|
+ ResourceScheduler.class);
|
|
|
|
+ resourceManager.init(conf);
|
|
|
|
+ resourceManager.getRMContext().getContainerTokenSecretManager()
|
|
|
|
+ .rollMasterKey();
|
|
|
|
+ resourceManager.getRMContext().getNMTokenSecretManager().rollMasterKey();
|
|
|
|
+ ((AsyncDispatcher) resourceManager.getRMContext().getDispatcher()).start();
|
|
|
|
+ mockContext = mock(RMContext.class);
|
|
|
|
+ when(mockContext.getConfigurationProvider()).thenReturn(
|
|
|
|
+ new LocalConfigurationProvider());
|
|
|
|
+
|
|
|
|
+ ResourceScheduler scheduler = resourceManager.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ // Register node1
|
|
|
|
+ String host_0 = "host_0";
|
|
|
|
+ NodeManager nm_0 =
|
|
|
|
+ registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
|
+ Resources.createResource(6 * GB, 1));
|
|
|
|
+
|
|
|
|
+ // ResourceRequest priorities
|
|
|
|
+ Priority priority_0 =
|
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
|
|
|
|
+ .create(0);
|
|
|
|
+ Priority priority_1 =
|
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
|
|
|
|
+ .create(1);
|
|
|
|
+
|
|
|
|
+ // Submit application_0
|
|
|
|
+ Application application_0 =
|
|
|
|
+ new Application("user_0", "a1", resourceManager);
|
|
|
|
+ application_0.submit(); // app + app attempt event sent to scheduler
|
|
|
|
+
|
|
|
|
+ application_0.addNodeManager(host_0, 1234, nm_0);
|
|
|
|
+
|
|
|
|
+ Resource capability_0_0 = Resources.createResource(3 * GB, 1);
|
|
|
|
+ application_0.addResourceRequestSpec(priority_1, capability_0_0);
|
|
|
|
+
|
|
|
|
+ Resource capability_0_1 = Resources.createResource(2 * GB, 1);
|
|
|
|
+ application_0.addResourceRequestSpec(priority_0, capability_0_1);
|
|
|
|
+
|
|
|
|
+ Task task_0_0 =
|
|
|
|
+ new Task(application_0, priority_1, new String[] { host_0 });
|
|
|
|
+ application_0.addTask(task_0_0);
|
|
|
|
+
|
|
|
|
+ // Send resource requests to the scheduler
|
|
|
|
+ application_0.schedule(); // allocate
|
|
|
|
+
|
|
|
|
+ // task_0_0 allocated
|
|
|
|
+ nodeUpdate(nm_0);
|
|
|
|
+
|
|
|
|
+ // Get allocations from the scheduler
|
|
|
|
+ application_0.schedule(); // task_0_0
|
|
|
|
+ checkApplicationResourceUsage(3 * GB, application_0);
|
|
|
|
+
|
|
|
|
+ checkNodeResourceUsage(3 * GB, nm_0);
|
|
|
|
+ // b2 queue contains 3GB consumption app,
|
|
|
|
+ // add another 3GB will hit max capacity limit on queue b
|
|
|
|
+ scheduler.moveApplication(application_0.getApplicationId(), "b1");
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMoveAppQueueMetricsCheck() throws Exception {
|
|
|
|
+ ResourceScheduler scheduler = resourceManager.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ // Register node1
|
|
|
|
+ String host_0 = "host_0";
|
|
|
|
+ NodeManager nm_0 =
|
|
|
|
+ registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
|
+ Resources.createResource(5 * GB, 1));
|
|
|
|
+
|
|
|
|
+ // Register node2
|
|
|
|
+ String host_1 = "host_1";
|
|
|
|
+ NodeManager nm_1 =
|
|
|
|
+ registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
|
|
|
+ Resources.createResource(5 * GB, 1));
|
|
|
|
+
|
|
|
|
+ // ResourceRequest priorities
|
|
|
|
+ Priority priority_0 =
|
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
|
|
|
|
+ .create(0);
|
|
|
|
+ Priority priority_1 =
|
|
|
|
+ org.apache.hadoop.yarn.server.resourcemanager.resource.Priority
|
|
|
|
+ .create(1);
|
|
|
|
+
|
|
|
|
+ // Submit application_0
|
|
|
|
+ Application application_0 =
|
|
|
|
+ new Application("user_0", "a1", resourceManager);
|
|
|
|
+ application_0.submit(); // app + app attempt event sent to scheduler
|
|
|
|
+
|
|
|
|
+ application_0.addNodeManager(host_0, 1234, nm_0);
|
|
|
|
+ application_0.addNodeManager(host_1, 1234, nm_1);
|
|
|
|
+
|
|
|
|
+ Resource capability_0_0 = Resources.createResource(3 * GB, 1);
|
|
|
|
+ application_0.addResourceRequestSpec(priority_1, capability_0_0);
|
|
|
|
+
|
|
|
|
+ Resource capability_0_1 = Resources.createResource(2 * GB, 1);
|
|
|
|
+ application_0.addResourceRequestSpec(priority_0, capability_0_1);
|
|
|
|
+
|
|
|
|
+ Task task_0_0 =
|
|
|
|
+ new Task(application_0, priority_1, new String[] { host_0, host_1 });
|
|
|
|
+ application_0.addTask(task_0_0);
|
|
|
|
+
|
|
|
|
+ // Submit application_1
|
|
|
|
+ Application application_1 =
|
|
|
|
+ new Application("user_1", "b2", resourceManager);
|
|
|
|
+ application_1.submit(); // app + app attempt event sent to scheduler
|
|
|
|
+
|
|
|
|
+ application_1.addNodeManager(host_0, 1234, nm_0);
|
|
|
|
+ application_1.addNodeManager(host_1, 1234, nm_1);
|
|
|
|
+
|
|
|
|
+ Resource capability_1_0 = Resources.createResource(1 * GB, 1);
|
|
|
|
+ application_1.addResourceRequestSpec(priority_1, capability_1_0);
|
|
|
|
+
|
|
|
|
+ Resource capability_1_1 = Resources.createResource(2 * GB, 1);
|
|
|
|
+ application_1.addResourceRequestSpec(priority_0, capability_1_1);
|
|
|
|
+
|
|
|
|
+ Task task_1_0 =
|
|
|
|
+ new Task(application_1, priority_1, new String[] { host_0, host_1 });
|
|
|
|
+ application_1.addTask(task_1_0);
|
|
|
|
+
|
|
|
|
+ // Send resource requests to the scheduler
|
|
|
|
+ application_0.schedule(); // allocate
|
|
|
|
+ application_1.schedule(); // allocate
|
|
|
|
+
|
|
|
|
+ nodeUpdate(nm_0);
|
|
|
|
+
|
|
|
|
+ nodeUpdate(nm_1);
|
|
|
|
+
|
|
|
|
+ CapacityScheduler cs =
|
|
|
|
+ (CapacityScheduler) resourceManager.getResourceScheduler();
|
|
|
|
+ CSQueue origRootQ = cs.getRootQueue();
|
|
|
|
+ CapacitySchedulerInfo oldInfo = new CapacitySchedulerInfo(origRootQ);
|
|
|
|
+ int origNumAppsA = getNumAppsInQueue("a", origRootQ.getChildQueues());
|
|
|
|
+ int origNumAppsRoot = origRootQ.getNumApplications();
|
|
|
|
+
|
|
|
|
+ scheduler.moveApplication(application_0.getApplicationId(), "a2");
|
|
|
|
+
|
|
|
|
+ CSQueue newRootQ = cs.getRootQueue();
|
|
|
|
+ int newNumAppsA = getNumAppsInQueue("a", newRootQ.getChildQueues());
|
|
|
|
+ int newNumAppsRoot = newRootQ.getNumApplications();
|
|
|
|
+ CapacitySchedulerInfo newInfo = new CapacitySchedulerInfo(newRootQ);
|
|
|
|
+ CapacitySchedulerLeafQueueInfo origOldA1 =
|
|
|
|
+ (CapacitySchedulerLeafQueueInfo) getQueueInfo("a1", oldInfo.getQueues());
|
|
|
|
+ CapacitySchedulerLeafQueueInfo origNewA1 =
|
|
|
|
+ (CapacitySchedulerLeafQueueInfo) getQueueInfo("a1", newInfo.getQueues());
|
|
|
|
+ CapacitySchedulerLeafQueueInfo targetOldA2 =
|
|
|
|
+ (CapacitySchedulerLeafQueueInfo) getQueueInfo("a2", oldInfo.getQueues());
|
|
|
|
+ CapacitySchedulerLeafQueueInfo targetNewA2 =
|
|
|
|
+ (CapacitySchedulerLeafQueueInfo) getQueueInfo("a2", newInfo.getQueues());
|
|
|
|
+ // originally submitted here
|
|
|
|
+ assertEquals(1, origOldA1.getNumApplications());
|
|
|
|
+ assertEquals(1, origNumAppsA);
|
|
|
|
+ assertEquals(2, origNumAppsRoot);
|
|
|
|
+ // after the move
|
|
|
|
+ assertEquals(0, origNewA1.getNumApplications());
|
|
|
|
+ assertEquals(1, newNumAppsA);
|
|
|
|
+ assertEquals(2, newNumAppsRoot);
|
|
|
|
+ // original consumption on a1
|
|
|
|
+ assertEquals(3 * GB, origOldA1.getResourcesUsed().getMemory());
|
|
|
|
+ assertEquals(1, origOldA1.getResourcesUsed().getvCores());
|
|
|
|
+ assertEquals(0, origNewA1.getResourcesUsed().getMemory()); // after the move
|
|
|
|
+ assertEquals(0, origNewA1.getResourcesUsed().getvCores()); // after the move
|
|
|
|
+ // app moved here with live containers
|
|
|
|
+ assertEquals(3 * GB, targetNewA2.getResourcesUsed().getMemory());
|
|
|
|
+ assertEquals(1, targetNewA2.getResourcesUsed().getvCores());
|
|
|
|
+ // it was empty before the move
|
|
|
|
+ assertEquals(0, targetOldA2.getNumApplications());
|
|
|
|
+ assertEquals(0, targetOldA2.getResourcesUsed().getMemory());
|
|
|
|
+ assertEquals(0, targetOldA2.getResourcesUsed().getvCores());
|
|
|
|
+ // after the app moved here
|
|
|
|
+ assertEquals(1, targetNewA2.getNumApplications());
|
|
|
|
+ // 1 container on original queue before move
|
|
|
|
+ assertEquals(1, origOldA1.getNumContainers());
|
|
|
|
+ // after the move the resource released
|
|
|
|
+ assertEquals(0, origNewA1.getNumContainers());
|
|
|
|
+ // and moved to the new queue
|
|
|
|
+ assertEquals(1, targetNewA2.getNumContainers());
|
|
|
|
+ // which originally didn't have any
|
|
|
|
+ assertEquals(0, targetOldA2.getNumContainers());
|
|
|
|
+ // 1 user with 3GB
|
|
|
|
+ assertEquals(3 * GB, origOldA1.getUsers().getUsersList().get(0)
|
|
|
|
+ .getResourcesUsed().getMemory());
|
|
|
|
+ // 1 user with 1 core
|
|
|
|
+ assertEquals(1, origOldA1.getUsers().getUsersList().get(0)
|
|
|
|
+ .getResourcesUsed().getvCores());
|
|
|
|
+ // user ha no more running app in the orig queue
|
|
|
|
+ assertEquals(0, origNewA1.getUsers().getUsersList().size());
|
|
|
|
+ // 1 user with 3GB
|
|
|
|
+ assertEquals(3 * GB, targetNewA2.getUsers().getUsersList().get(0)
|
|
|
|
+ .getResourcesUsed().getMemory());
|
|
|
|
+ // 1 user with 1 core
|
|
|
|
+ assertEquals(1, targetNewA2.getUsers().getUsersList().get(0)
|
|
|
|
+ .getResourcesUsed().getvCores());
|
|
|
|
+
|
|
|
|
+ // Get allocations from the scheduler
|
|
|
|
+ application_0.schedule(); // task_0_0
|
|
|
|
+ checkApplicationResourceUsage(3 * GB, application_0);
|
|
|
|
+
|
|
|
|
+ application_1.schedule(); // task_1_0
|
|
|
|
+ checkApplicationResourceUsage(1 * GB, application_1);
|
|
|
|
+
|
|
|
|
+ // task_1_0 (1G) application_0 moved to b2 with max running app 1 so it is
|
|
|
|
+ // not scheduled
|
|
|
|
+ checkNodeResourceUsage(4 * GB, nm_0);
|
|
|
|
+ checkNodeResourceUsage(0 * GB, nm_1);
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private int getNumAppsInQueue(String name, List<CSQueue> queues) {
|
|
|
|
+ for (CSQueue queue : queues) {
|
|
|
|
+ if (queue.getQueueName().equals(name)) {
|
|
|
|
+ return queue.getNumApplications();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return -1;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private CapacitySchedulerQueueInfo getQueueInfo(String name,
|
|
|
|
+ CapacitySchedulerQueueInfoList info) {
|
|
|
|
+ if (info != null) {
|
|
|
|
+ for (CapacitySchedulerQueueInfo queueInfo : info.getQueueInfoList()) {
|
|
|
|
+ if (queueInfo.getQueueName().equals(name)) {
|
|
|
|
+ return queueInfo;
|
|
|
|
+ } else {
|
|
|
|
+ CapacitySchedulerQueueInfo result =
|
|
|
|
+ getQueueInfo(name, queueInfo.getQueues());
|
|
|
|
+ if (result == null) {
|
|
|
|
+ continue;
|
|
|
|
+ }
|
|
|
|
+ return result;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMoveAllApps() throws Exception {
|
|
|
|
+ MockRM rm = setUpMove();
|
|
|
|
+ AbstractYarnScheduler scheduler =
|
|
|
|
+ (AbstractYarnScheduler) rm.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ // submit an app
|
|
|
|
+ RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");
|
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
|
+ rm.getApplicationReport(app.getApplicationId())
|
|
|
|
+ .getCurrentApplicationAttemptId();
|
|
|
|
+
|
|
|
|
+ // check preconditions
|
|
|
|
+ List<ApplicationAttemptId> appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertEquals(1, appsInA1.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
|
+ String queue =
|
|
|
|
+ scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
|
|
|
|
+ .getQueueName();
|
|
|
|
+ Assert.assertTrue(queue.equals("a1"));
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInB1 = scheduler.getAppsInQueue("b1");
|
|
|
|
+ assertTrue(appsInB1.isEmpty());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInB = scheduler.getAppsInQueue("b");
|
|
|
|
+ assertTrue(appsInB.isEmpty());
|
|
|
|
+
|
|
|
|
+ // now move the app
|
|
|
|
+ scheduler.moveAllApps("a1", "b1");
|
|
|
|
+
|
|
|
|
+ // check postconditions
|
|
|
|
+ Thread.sleep(1000);
|
|
|
|
+ appsInB1 = scheduler.getAppsInQueue("b1");
|
|
|
|
+ assertEquals(1, appsInB1.size());
|
|
|
|
+ queue =
|
|
|
|
+ scheduler.getApplicationAttempt(appsInB1.get(0)).getQueue()
|
|
|
|
+ .getQueueName();
|
|
|
|
+ Assert.assertTrue(queue.equals("b1"));
|
|
|
|
+
|
|
|
|
+ appsInB = scheduler.getAppsInQueue("b");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInB.size());
|
|
|
|
+
|
|
|
|
+ appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertTrue(appsInA1.isEmpty());
|
|
|
|
+
|
|
|
|
+ appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.isEmpty());
|
|
|
|
+
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMoveAllAppsInvalidDestination() throws Exception {
|
|
|
|
+ MockRM rm = setUpMove();
|
|
|
|
+ AbstractYarnScheduler scheduler =
|
|
|
|
+ (AbstractYarnScheduler) rm.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ // submit an app
|
|
|
|
+ RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");
|
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
|
+ rm.getApplicationReport(app.getApplicationId())
|
|
|
|
+ .getCurrentApplicationAttemptId();
|
|
|
|
+
|
|
|
|
+ // check preconditions
|
|
|
|
+ List<ApplicationAttemptId> appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertEquals(1, appsInA1.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInB1 = scheduler.getAppsInQueue("b1");
|
|
|
|
+ assertTrue(appsInB1.isEmpty());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInB = scheduler.getAppsInQueue("b");
|
|
|
|
+ assertTrue(appsInB.isEmpty());
|
|
|
|
+
|
|
|
|
+ // now move the app
|
|
|
|
+ try {
|
|
|
|
+ scheduler.moveAllApps("a1", "DOES_NOT_EXIST");
|
|
|
|
+ Assert.fail();
|
|
|
|
+ } catch (YarnException e) {
|
|
|
|
+ // expected
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // check postconditions, app should still be in a1
|
|
|
|
+ appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertEquals(1, appsInA1.size());
|
|
|
|
+
|
|
|
|
+ appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
|
+
|
|
|
|
+ appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ appsInB1 = scheduler.getAppsInQueue("b1");
|
|
|
|
+ assertTrue(appsInB1.isEmpty());
|
|
|
|
+
|
|
|
|
+ appsInB = scheduler.getAppsInQueue("b");
|
|
|
|
+ assertTrue(appsInB.isEmpty());
|
|
|
|
+
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMoveAllAppsInvalidSource() throws Exception {
|
|
|
|
+ MockRM rm = setUpMove();
|
|
|
|
+ AbstractYarnScheduler scheduler =
|
|
|
|
+ (AbstractYarnScheduler) rm.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ // submit an app
|
|
|
|
+ RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");
|
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
|
+ rm.getApplicationReport(app.getApplicationId())
|
|
|
|
+ .getCurrentApplicationAttemptId();
|
|
|
|
+
|
|
|
|
+ // check preconditions
|
|
|
|
+ List<ApplicationAttemptId> appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertEquals(1, appsInA1.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInB1 = scheduler.getAppsInQueue("b1");
|
|
|
|
+ assertTrue(appsInB1.isEmpty());
|
|
|
|
+
|
|
|
|
+ List<ApplicationAttemptId> appsInB = scheduler.getAppsInQueue("b");
|
|
|
|
+ assertTrue(appsInB.isEmpty());
|
|
|
|
+
|
|
|
|
+ // now move the app
|
|
|
|
+ try {
|
|
|
|
+ scheduler.moveAllApps("DOES_NOT_EXIST", "b1");
|
|
|
|
+ Assert.fail();
|
|
|
|
+ } catch (YarnException e) {
|
|
|
|
+ // expected
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // check postconditions, app should still be in a1
|
|
|
|
+ appsInA1 = scheduler.getAppsInQueue("a1");
|
|
|
|
+ assertEquals(1, appsInA1.size());
|
|
|
|
+
|
|
|
|
+ appsInA = scheduler.getAppsInQueue("a");
|
|
|
|
+ assertTrue(appsInA.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
|
+
|
|
|
|
+ appsInRoot = scheduler.getAppsInQueue("root");
|
|
|
|
+ assertTrue(appsInRoot.contains(appAttemptId));
|
|
|
|
+ assertEquals(1, appsInRoot.size());
|
|
|
|
+
|
|
|
|
+ appsInB1 = scheduler.getAppsInQueue("b1");
|
|
|
|
+ assertTrue(appsInB1.isEmpty());
|
|
|
|
+
|
|
|
|
+ appsInB = scheduler.getAppsInQueue("b");
|
|
|
|
+ assertTrue(appsInB.isEmpty());
|
|
|
|
+
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|