|
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
|
import static org.junit.Assert.assertNull;
|
|
|
+import static org.junit.Assert.assertSame;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
@@ -56,10 +57,12 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
+import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
@@ -2547,4 +2550,138 @@ public class TestFairScheduler {
|
|
|
TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(
|
|
|
scheduler.getSchedulerApplications(), scheduler, "default");
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMoveRunnableApp() throws Exception {
|
|
|
+ scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
+
|
|
|
+ QueueManager queueMgr = scheduler.getQueueManager();
|
|
|
+ FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true);
|
|
|
+ FSLeafQueue targetQueue = queueMgr.getLeafQueue("queue2", true);
|
|
|
+
|
|
|
+ ApplicationAttemptId appAttId =
|
|
|
+ createSchedulingRequest(1024, 1, "queue1", "user1", 3);
|
|
|
+ ApplicationId appId = appAttId.getApplicationId();
|
|
|
+ RMNode node = MockNodes.newNodeInfo(1, Resources.createResource(1024));
|
|
|
+ NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node);
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node);
|
|
|
+ scheduler.handle(nodeEvent);
|
|
|
+ scheduler.handle(updateEvent);
|
|
|
+
|
|
|
+ assertEquals(Resource.newInstance(1024, 1), oldQueue.getResourceUsage());
|
|
|
+ scheduler.update();
|
|
|
+ assertEquals(Resource.newInstance(3072, 3), oldQueue.getDemand());
|
|
|
+
|
|
|
+ scheduler.moveApplication(appId, "queue2");
|
|
|
+ FSSchedulerApp app = scheduler.getSchedulerApp(appAttId);
|
|
|
+ assertSame(targetQueue, app.getQueue());
|
|
|
+ assertFalse(oldQueue.getRunnableAppSchedulables()
|
|
|
+ .contains(app.getAppSchedulable()));
|
|
|
+ assertTrue(targetQueue.getRunnableAppSchedulables()
|
|
|
+ .contains(app.getAppSchedulable()));
|
|
|
+ assertEquals(Resource.newInstance(0, 0), oldQueue.getResourceUsage());
|
|
|
+ assertEquals(Resource.newInstance(1024, 1), targetQueue.getResourceUsage());
|
|
|
+ assertEquals(0, oldQueue.getNumRunnableApps());
|
|
|
+ assertEquals(1, targetQueue.getNumRunnableApps());
|
|
|
+ assertEquals(1, queueMgr.getRootQueue().getNumRunnableApps());
|
|
|
+
|
|
|
+ scheduler.update();
|
|
|
+ assertEquals(Resource.newInstance(0, 0), oldQueue.getDemand());
|
|
|
+ assertEquals(Resource.newInstance(3072, 3), targetQueue.getDemand());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMoveNonRunnableApp() throws Exception {
|
|
|
+ scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
+
|
|
|
+ QueueManager queueMgr = scheduler.getQueueManager();
|
|
|
+ FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true);
|
|
|
+ FSLeafQueue targetQueue = queueMgr.getLeafQueue("queue2", true);
|
|
|
+ scheduler.getAllocationConfiguration().queueMaxApps.put("root.queue1", 0);
|
|
|
+ scheduler.getAllocationConfiguration().queueMaxApps.put("root.queue2", 0);
|
|
|
+
|
|
|
+ ApplicationAttemptId appAttId =
|
|
|
+ createSchedulingRequest(1024, 1, "queue1", "user1", 3);
|
|
|
+
|
|
|
+ assertEquals(0, oldQueue.getNumRunnableApps());
|
|
|
+ scheduler.moveApplication(appAttId.getApplicationId(), "queue2");
|
|
|
+ assertEquals(0, oldQueue.getNumRunnableApps());
|
|
|
+ assertEquals(0, targetQueue.getNumRunnableApps());
|
|
|
+ assertEquals(0, queueMgr.getRootQueue().getNumRunnableApps());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMoveMakesAppRunnable() throws Exception {
|
|
|
+ scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
+
|
|
|
+ QueueManager queueMgr = scheduler.getQueueManager();
|
|
|
+ FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true);
|
|
|
+ FSLeafQueue targetQueue = queueMgr.getLeafQueue("queue2", true);
|
|
|
+ scheduler.getAllocationConfiguration().queueMaxApps.put("root.queue1", 0);
|
|
|
+
|
|
|
+ ApplicationAttemptId appAttId =
|
|
|
+ createSchedulingRequest(1024, 1, "queue1", "user1", 3);
|
|
|
+
|
|
|
+ FSSchedulerApp app = scheduler.getSchedulerApp(appAttId);
|
|
|
+ assertTrue(oldQueue.getNonRunnableAppSchedulables()
|
|
|
+ .contains(app.getAppSchedulable()));
|
|
|
+
|
|
|
+ scheduler.moveApplication(appAttId.getApplicationId(), "queue2");
|
|
|
+ assertFalse(oldQueue.getNonRunnableAppSchedulables()
|
|
|
+ .contains(app.getAppSchedulable()));
|
|
|
+ assertFalse(targetQueue.getNonRunnableAppSchedulables()
|
|
|
+ .contains(app.getAppSchedulable()));
|
|
|
+ assertTrue(targetQueue.getRunnableAppSchedulables()
|
|
|
+ .contains(app.getAppSchedulable()));
|
|
|
+ assertEquals(1, targetQueue.getNumRunnableApps());
|
|
|
+ assertEquals(1, queueMgr.getRootQueue().getNumRunnableApps());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test (expected = YarnException.class)
|
|
|
+ public void testMoveWouldViolateMaxAppsConstraints() throws Exception {
|
|
|
+ scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
+
|
|
|
+ QueueManager queueMgr = scheduler.getQueueManager();
|
|
|
+ queueMgr.getLeafQueue("queue2", true);
|
|
|
+ scheduler.getAllocationConfiguration().queueMaxApps.put("root.queue2", 0);
|
|
|
+
|
|
|
+ ApplicationAttemptId appAttId =
|
|
|
+ createSchedulingRequest(1024, 1, "queue1", "user1", 3);
|
|
|
+
|
|
|
+ scheduler.moveApplication(appAttId.getApplicationId(), "queue2");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test (expected = YarnException.class)
|
|
|
+ public void testMoveWouldViolateMaxResourcesConstraints() throws Exception {
|
|
|
+ scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
+
|
|
|
+ QueueManager queueMgr = scheduler.getQueueManager();
|
|
|
+ FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true);
|
|
|
+ queueMgr.getLeafQueue("queue2", true);
|
|
|
+ scheduler.getAllocationConfiguration().maxQueueResources.put("root.queue2",
|
|
|
+ Resource.newInstance(1024, 1));
|
|
|
+
|
|
|
+ ApplicationAttemptId appAttId =
|
|
|
+ createSchedulingRequest(1024, 1, "queue1", "user1", 3);
|
|
|
+ RMNode node = MockNodes.newNodeInfo(1, Resources.createResource(2048, 2));
|
|
|
+ NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node);
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node);
|
|
|
+ scheduler.handle(nodeEvent);
|
|
|
+ scheduler.handle(updateEvent);
|
|
|
+ scheduler.handle(updateEvent);
|
|
|
+
|
|
|
+ assertEquals(Resource.newInstance(2048, 2), oldQueue.getResourceUsage());
|
|
|
+ scheduler.moveApplication(appAttId.getApplicationId(), "queue2");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test (expected = YarnException.class)
|
|
|
+ public void testMoveToNonexistentQueue() throws Exception {
|
|
|
+ scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
+
|
|
|
+ scheduler.getQueueManager().getLeafQueue("queue1", true);
|
|
|
+
|
|
|
+ ApplicationAttemptId appAttId =
|
|
|
+ createSchedulingRequest(1024, 1, "queue1", "user1", 3);
|
|
|
+ scheduler.moveApplication(appAttId.getApplicationId(), "queue2");
|
|
|
+ }
|
|
|
}
|