|
@@ -106,6 +106,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.C
|
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.nodeUpdate;
|
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.registerNode;
|
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.setUpMove;
|
|
|
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.setUpMoveAmbiguousQueue;
|
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.stopResourceManager;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
@@ -347,6 +348,46 @@ public class TestCapacitySchedulerApps {
|
|
|
rm.stop();
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testMoveAppAmbiguousQueue() throws Exception {
|
|
|
+ MockRM rm = setUpMoveAmbiguousQueue();
|
|
|
+ AbstractYarnScheduler scheduler =
|
|
|
+ (AbstractYarnScheduler) rm.getResourceScheduler();
|
|
|
+ QueueMetrics metrics = scheduler.getRootQueueMetrics();
|
|
|
+ Assert.assertEquals(0, metrics.getAppsPending());
|
|
|
+ // submit an app
|
|
|
+ MockRMAppSubmissionData data =
|
|
|
+ MockRMAppSubmissionData.Builder.createWithMemory(GB, rm)
|
|
|
+ .withAppName("test-move-1")
|
|
|
+ .withUser("user_0")
|
|
|
+ .withAcls(null)
|
|
|
+ .withQueue("root.a.a")
|
|
|
+ .withUnmanagedAM(false)
|
|
|
+ .build();
|
|
|
+ RMApp app = MockRMAppSubmitter.submit(rm, data);
|
|
|
+ // check preconditions
|
|
|
+ List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("root.a.a");
|
|
|
+ assertEquals(1, appsInA.size());
|
|
|
+ String queue =
|
|
|
+ scheduler.getApplicationAttempt(appsInA.get(0)).getQueue()
|
|
|
+ .getQueueName();
|
|
|
+ Assert.assertEquals("a", queue);
|
|
|
+
|
|
|
+ // now move the app
|
|
|
+ scheduler.moveApplication(app.getApplicationId(), "a1");
|
|
|
+ List<ApplicationAttemptId> appsInA1 = scheduler.getAppsInQueue("root.a.a1");
|
|
|
+ assertEquals(1, appsInA1.size());
|
|
|
+ queue =
|
|
|
+ scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
|
|
|
+ .getQueueName();
|
|
|
+ Assert.assertEquals("a1", queue);
|
|
|
+
|
|
|
+ appsInA = scheduler.getAppsInQueue("root.a.a");
|
|
|
+ assertTrue(appsInA.isEmpty());
|
|
|
+
|
|
|
+ rm.stop();
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testMoveAppBasic() throws Exception {
|
|
|
MockRM rm = setUpMove();
|