Browse Source

SUBMARINE-45. Can't specify queue by using the parameter --queue. Contributed by Ayush Saxena, Zac Zhou.

Zhankun Tang 5 years ago
parent
commit
85a00fd2bb

+ 1 - 0
hadoop-submarine/hadoop-submarine-yarnservice-runtime/src/main/java/org/apache/hadoop/yarn/submarine/runtimes/yarnservice/AbstractServiceSpec.java

@@ -78,6 +78,7 @@ public abstract class AbstractServiceSpec implements ServiceSpec {
     serviceSpec.setName(parameters.getName());
     serviceSpec.setVersion(String.valueOf(System.currentTimeMillis()));
     serviceSpec.setArtifact(getDockerArtifact(parameters.getDockerImageName()));
+    serviceSpec.setQueue(parameters.getQueue());
 
     KerberosPrincipal kerberosPrincipal = KerberosPrincipalFactory
         .create(fsOperations, remoteDirectoryManager, parameters);

+ 6 - 0
hadoop-submarine/hadoop-submarine-yarnservice-runtime/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/yarnservice/ParamBuilderForTest.java

@@ -57,6 +57,12 @@ class ParamBuilderForTest {
     return this;
   }
 
+  ParamBuilderForTest withQueue(String queue) {
+    params.add("--queue");
+    params.add(queue);
+    return this;
+  }
+
   ParamBuilderForTest withNumberOfWorkers(int numWorkers) {
     params.add("--num_workers");
     params.add(String.valueOf(numWorkers));

+ 6 - 0
hadoop-submarine/hadoop-submarine-yarnservice-runtime/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/yarnservice/TestYarnServiceRunJobCli.java

@@ -65,6 +65,7 @@ import static org.apache.hadoop.yarn.submarine.client.cli.yarnservice.TestYarnSe
 import static org.apache.hadoop.yarn.submarine.client.cli.yarnservice.TestYarnServiceRunJobCliCommons.DEFAULT_WORKER_DOCKER_IMAGE;
 import static org.apache.hadoop.yarn.submarine.client.cli.yarnservice.TestYarnServiceRunJobCliCommons.DEFAULT_WORKER_LAUNCH_CMD;
 import static org.apache.hadoop.yarn.submarine.client.cli.yarnservice.TestYarnServiceRunJobCliCommons.DEFAULT_WORKER_RESOURCES;
+import static org.apache.hadoop.yarn.submarine.client.cli.yarnservice.TestYarnServiceRunJobCliCommons.DEFAULT_QUEUE;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -134,6 +135,7 @@ public class TestYarnServiceRunJobCli {
     assertEquals(DEFAULT_WORKER_DOCKER_IMAGE, workerComp.getArtifact().getId());
     assertEquals(DEFAULT_PS_DOCKER_IMAGE, psComp.getArtifact().getId());
 
+    assertEquals(DEFAULT_QUEUE, serviceSpec.getQueue());
     assertTrue(SubmarineLogs.isVerbose());
   }
 
@@ -183,6 +185,7 @@ public class TestYarnServiceRunJobCli {
         .withDockerImage(DEFAULT_DOCKER_IMAGE)
         .withInputPath(DEFAULT_INPUT_PATH)
         .withCheckpointPath(DEFAULT_CHECKPOINT_PATH)
+        .withQueue(DEFAULT_QUEUE)
         .withNumberOfWorkers(3)
         .withWorkerDockerImage(DEFAULT_WORKER_DOCKER_IMAGE)
         .withWorkerLaunchCommand(DEFAULT_WORKER_LAUNCH_CMD)
@@ -219,6 +222,7 @@ public class TestYarnServiceRunJobCli {
         .withDockerImage(DEFAULT_DOCKER_IMAGE)
         .withInputPath(DEFAULT_INPUT_PATH)
         .withCheckpointPath(DEFAULT_CHECKPOINT_PATH)
+        .withQueue(DEFAULT_QUEUE)
         .withNumberOfWorkers(3)
         .withWorkerDockerImage(DEFAULT_WORKER_DOCKER_IMAGE)
         .withWorkerLaunchCommand(DEFAULT_WORKER_LAUNCH_CMD)
@@ -562,6 +566,7 @@ public class TestYarnServiceRunJobCli {
         .withDockerImage(DEFAULT_DOCKER_IMAGE)
         .withInputPath(DEFAULT_INPUT_PATH)
         .withCheckpointPath(DEFAULT_CHECKPOINT_PATH)
+        .withQueue(DEFAULT_QUEUE)
         .withNumberOfWorkers(3)
         .withWorkerDockerImage(DEFAULT_WORKER_DOCKER_IMAGE)
         .withWorkerLaunchCommand(DEFAULT_WORKER_LAUNCH_CMD)
@@ -602,6 +607,7 @@ public class TestYarnServiceRunJobCli {
         .withDockerImage(DEFAULT_DOCKER_IMAGE)
         .withInputPath(DEFAULT_INPUT_PATH)
         .withCheckpointPath(DEFAULT_CHECKPOINT_PATH)
+        .withQueue(DEFAULT_QUEUE)
         .withNumberOfWorkers(3)
         .withWorkerDockerImage(DEFAULT_WORKER_DOCKER_IMAGE)
         .withWorkerLaunchCommand(DEFAULT_WORKER_LAUNCH_CMD)

+ 1 - 0
hadoop-submarine/hadoop-submarine-yarnservice-runtime/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/yarnservice/TestYarnServiceRunJobCliCommons.java

@@ -40,6 +40,7 @@ public class TestYarnServiceRunJobCliCommons {
   static final String DEFAULT_DOCKER_IMAGE = "tf-docker:1.1.0";
   static final String DEFAULT_INPUT_PATH = "s3://input";
   static final String DEFAULT_CHECKPOINT_PATH = "s3://output";
+  static final String DEFAULT_QUEUE = "root.queue";
   static final String DEFAULT_WORKER_DOCKER_IMAGE = "worker.image";
   static final String DEFAULT_PS_DOCKER_IMAGE = "ps.image";
   static final String DEFAULT_WORKER_LAUNCH_CMD = "python run-job.py";