You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by su...@apache.org on 2019/04/06 02:09:37 UTC

[hadoop] branch trunk updated: SUBMARINE-47. Provide an implementation to parse configuration values from a YAML file for submarine run CLI. Contributed by Szilard Nemeth.

This is an automated email from the ASF dual-hosted git repository.

sunilg pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 1b9ba0e  SUBMARINE-47. Provide an implementation to parse configuration values from a YAML file for submarine run CLI. Contributed by Szilard Nemeth.
1b9ba0e is described below

commit 1b9ba0ebb20952d848d205269f4dffe7985c5107
Author: Sunil G <su...@apache.org>
AuthorDate: Sat Apr 6 07:39:13 2019 +0530

    SUBMARINE-47. Provide an implementation to parse configuration values from a YAML file for submarine run CLI. Contributed by Szilard Nemeth.
---
 hadoop-submarine/hadoop-submarine-core/pom.xml     |  10 +
 .../yarn/submarine/client/cli/CliConstants.java    |   1 +
 .../yarn/submarine/client/cli/RunJobCli.java       |  77 ++++-
 .../yarn/submarine/client/cli/ShowJobCli.java      |   7 +-
 .../submarine/client/cli/param/BaseParameters.java |  10 +-
 .../client/cli/param/ParametersHolder.java         | 315 +++++++++++++++++
 .../client/cli/param/RunJobParameters.java         |  83 +++--
 .../submarine/client/cli/param/RunParameters.java  |  37 +-
 .../submarine/client/cli/param/yaml/Configs.java   | 107 ++++++
 .../submarine/client/cli/param/yaml/PsRole.java    |  25 ++
 .../yarn/submarine/client/cli/param/yaml/Role.java |  91 +++++
 .../submarine/client/cli/param/yaml/Roles.java     |  41 +++
 .../client/cli/param/yaml/Scheduling.java          |  32 ++
 .../submarine/client/cli/param/yaml/Security.java  |  50 +++
 .../yarn/submarine/client/cli/param/yaml/Spec.java |  41 +++
 .../client/cli/param/yaml/TensorBoard.java         |  41 +++
 .../client/cli/param/yaml/WorkerRole.java          |  25 ++
 .../client/cli/param/yaml/YamlConfigFile.java      |  77 +++++
 .../client/cli/param/yaml/YamlParseException.java  |  27 ++
 .../client/cli/param/yaml/package-info.java        |  19 ++
 .../submarine/client/cli/TestRunJobCliParsing.java |  50 +--
 .../client/cli/TestRunJobCliParsingYaml.java       | 380 +++++++++++++++++++++
 .../cli/TestRunJobCliParsingYamlStandalone.java    | 205 +++++++++++
 .../submarine/client/cli/YamlConfigTestUtils.java  |  65 ++++
 .../runjobcliparsing/envs-are-missing.yaml         |  59 ++++
 .../runjobcliparsing/missing-configs.yaml          |  41 +++
 .../security-principal-is-missing.yaml             |  61 ++++
 .../runjobcliparsing/some-sections-missing.yaml    |  48 +++
 .../tensorboard-dockerimage-is-missing.yaml        |  61 ++++
 .../runjobcliparsing/test-false-values.yaml        |  56 +++
 .../valid-config-with-overrides.yaml               |  81 +++++
 .../resources/runjobcliparsing/valid-config.yaml   |  62 ++++
 .../runjobcliparsing/wrong-indentation.yaml        |  60 ++++
 .../runjobcliparsing/wrong-property-name.yaml      |  60 ++++
 34 files changed, 2316 insertions(+), 89 deletions(-)

diff --git a/hadoop-submarine/hadoop-submarine-core/pom.xml b/hadoop-submarine/hadoop-submarine-core/pom.xml
index f104d91..4a387fc 100644
--- a/hadoop-submarine/hadoop-submarine-core/pom.xml
+++ b/hadoop-submarine/hadoop-submarine-core/pom.xml
@@ -63,6 +63,10 @@
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.yaml</groupId>
+      <artifactId>snakeyaml</artifactId>
+    </dependency>
 
     <!-- Dependencies for Hadoop commons -->
 
@@ -92,6 +96,12 @@
       <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/CliConstants.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/CliConstants.java
index da4253b..f952aff 100644
--- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/CliConstants.java
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/CliConstants.java
@@ -56,4 +56,5 @@ public class CliConstants {
   public static final String KEYTAB = "keytab";
   public static final String PRINCIPAL = "principal";
   public static final String DISTRIBUTE_KEYTAB = "distribute_keytab";
+  public static final String YAML_CONFIG = "f";
 }
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/RunJobCli.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/RunJobCli.java
index b764d6d..f9583c6 100644
--- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/RunJobCli.java
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/RunJobCli.java
@@ -20,9 +20,14 @@ import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.submarine.client.cli.param.ParametersHolder;
 import org.apache.hadoop.yarn.submarine.client.cli.param.RunJobParameters;
+import org.apache.hadoop.yarn.submarine.client.cli.param.RunJobParameters.UnderscoreConverterPropertyUtils;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.YamlConfigFile;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.YamlParseException;
 import org.apache.hadoop.yarn.submarine.common.ClientContext;
 import org.apache.hadoop.yarn.submarine.common.exception.SubmarineException;
 import org.apache.hadoop.yarn.submarine.runtimes.common.JobMonitor;
@@ -30,7 +35,11 @@ import org.apache.hadoop.yarn.submarine.runtimes.common.JobSubmitter;
 import org.apache.hadoop.yarn.submarine.runtimes.common.StorageKeyConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.Yaml;
+import org.yaml.snakeyaml.constructor.Constructor;
 
+import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -38,6 +47,8 @@ import java.util.Map;
 public class RunJobCli extends AbstractCli {
   private static final Logger LOG =
       LoggerFactory.getLogger(RunJobCli.class);
+  private static final String YAML_PARSE_FAILED = "Failed to parse " +
+      "YAML config";
 
   private Options options;
   private RunJobParameters parameters = new RunJobParameters();
@@ -51,10 +62,10 @@ public class RunJobCli extends AbstractCli {
   }
 
   @VisibleForTesting
-  public RunJobCli(ClientContext cliContext, JobSubmitter jobSubmitter,
+  RunJobCli(ClientContext cliContext, JobSubmitter jobSubmitter,
       JobMonitor jobMonitor) {
     super(cliContext);
-    options = generateOptions();
+    this.options = generateOptions();
     this.jobSubmitter = jobSubmitter;
     this.jobMonitor = jobMonitor;
   }
@@ -65,6 +76,8 @@ public class RunJobCli extends AbstractCli {
 
   private Options generateOptions() {
     Options options = new Options();
+    options.addOption(CliConstants.YAML_CONFIG, true,
+        "Config file (in YAML format)");
     options.addOption(CliConstants.NAME, true, "Name of the job");
     options.addOption(CliConstants.INPUT_PATH, true,
         "Input of the job, could be local or other FS directory");
@@ -77,7 +90,7 @@ public class RunJobCli extends AbstractCli {
             + "exported model is not placed under ${checkpoint_path}"
             + "could be local or other FS directory. This will be used to serve.");
     options.addOption(CliConstants.N_WORKERS, true,
-        "Numnber of worker tasks of the job, by default it's 1");
+        "Number of worker tasks of the job, by default it's 1");
     options.addOption(CliConstants.N_PS, true,
         "Number of PS tasks of the job, by default it's 0");
     options.addOption(CliConstants.WORKER_RES, true,
@@ -119,7 +132,7 @@ public class RunJobCli extends AbstractCli {
             + "uses --" + CliConstants.DOCKER_IMAGE + " as default.");
     options.addOption(CliConstants.QUICKLINK, true, "Specify quicklink so YARN"
         + "web UI shows link to given role instance and port. When "
-        + "--tensorboard is speciied, quicklink to tensorboard instance will "
+        + "--tensorboard is specified, quicklink to tensorboard instance will "
         + "be added automatically. The format of quick link is: "
         + "Quick_link_label=http(or https)://role-name:port. For example, "
         + "if want to link to first worker's 7070 port, and text of quicklink "
@@ -149,7 +162,7 @@ public class RunJobCli extends AbstractCli {
         "by the job under security environment");
     options.addOption(CliConstants.DISTRIBUTE_KEYTAB, false, "Distribute " +
         "local keytab to cluster machines for service authentication. If not " +
-        "sepcified, pre-destributed keytab of which path specified by" +
+        "specified, pre-distributed keytab of which path specified by" +
         " parameter" + CliConstants.KEYTAB + " on cluster machines will be " +
         "used");
     options.addOption("h", "help", false, "Print help");
@@ -180,10 +193,10 @@ public class RunJobCli extends AbstractCli {
       // Do parsing
       GnuParser parser = new GnuParser();
       CommandLine cli = parser.parse(options, args);
-      parameters.updateParametersByParsedCommandline(cli, options,
-          clientContext);
+      ParametersHolder parametersHolder = createParametersHolder(cli);
+      parameters.updateParameters(parametersHolder, clientContext);
     } catch (ParseException e) {
-      LOG.error("Exception in parse:", e.getMessage());
+      LOG.error("Exception in parse: {}", e.getMessage());
       printUsages();
       throw e;
     }
@@ -195,6 +208,51 @@ public class RunJobCli extends AbstractCli {
     replacePatternsInParameters();
   }
 
+  private ParametersHolder createParametersHolder(CommandLine cli) {
+    String yamlConfigFile =
+        cli.getOptionValue(CliConstants.YAML_CONFIG);
+    if (yamlConfigFile != null) {
+      YamlConfigFile yamlConfig = readYamlConfigFile(yamlConfigFile);
+      if (yamlConfig == null) {
+        throw new YamlParseException(String.format(
+            YAML_PARSE_FAILED + ", file is empty: %s", yamlConfigFile));
+      } else if (yamlConfig.getConfigs() == null) {
+        throw new YamlParseException(String.format(YAML_PARSE_FAILED +
+            ", config section should be defined, but it cannot be found in " +
+            "YAML file '%s'!", yamlConfigFile));
+      }
+      LOG.info("Using YAML configuration!");
+      return ParametersHolder.createWithCmdLineAndYaml(cli, yamlConfig);
+    } else {
+      LOG.info("Using CLI configuration!");
+      return ParametersHolder.createWithCmdLine(cli);
+    }
+  }
+
+  private YamlConfigFile readYamlConfigFile(String filename) {
+    Constructor constructor = new Constructor(YamlConfigFile.class);
+    constructor.setPropertyUtils(new UnderscoreConverterPropertyUtils());
+    try {
+      LOG.info("Reading YAML configuration from file: {}", filename);
+      Yaml yaml = new Yaml(constructor);
+      return yaml.loadAs(FileUtils.openInputStream(new File(filename)),
+          YamlConfigFile.class);
+    } catch (FileNotFoundException e) {
+      logExceptionOfYamlParse(filename, e);
+      throw new YamlParseException(YAML_PARSE_FAILED +
+          ", file does not exist!");
+    } catch (Exception e) {
+      logExceptionOfYamlParse(filename, e);
+      throw new YamlParseException(
+          String.format(YAML_PARSE_FAILED + ", details: %s", e.getMessage()));
+    }
+  }
+
+  private void logExceptionOfYamlParse(String filename, Exception e) {
+    LOG.error(String.format("Exception while parsing YAML file %s", filename),
+        e);
+  }
+
   private void setDefaultDirs() throws IOException {
     // Create directories if needed
     String jobDir = parameters.getCheckpointPath();
@@ -248,8 +306,7 @@ public class RunJobCli extends AbstractCli {
 
   @Override
   public int run(String[] args)
-      throws ParseException, IOException, YarnException, InterruptedException,
-      SubmarineException {
+      throws ParseException, IOException, YarnException, SubmarineException {
     if (CliUtils.argsForHelp(args)) {
       printUsages();
       return 0;
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/ShowJobCli.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/ShowJobCli.java
index 6b76192..fd9b860 100644
--- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/ShowJobCli.java
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/ShowJobCli.java
@@ -21,6 +21,7 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.submarine.client.cli.param.ParametersHolder;
 import org.apache.hadoop.yarn.submarine.client.cli.param.ShowJobParameters;
 import org.apache.hadoop.yarn.submarine.common.ClientContext;
 import org.apache.hadoop.yarn.submarine.common.exception.SubmarineException;
@@ -61,8 +62,9 @@ public class ShowJobCli extends AbstractCli {
     CommandLine cli;
     try {
       cli = parser.parse(options, args);
-      parameters.updateParametersByParsedCommandline(cli, options,
-          clientContext);
+      ParametersHolder parametersHolder = ParametersHolder
+          .createWithCmdLine(cli);
+      parameters.updateParameters(parametersHolder, clientContext);
     } catch (ParseException e) {
       printUsages();
     }
@@ -117,7 +119,6 @@ public class ShowJobCli extends AbstractCli {
       printUsages();
       return 0;
     }
-
     parseCommandLineAndGetShowJobParameters(args);
     getAndPrintJobInfo();
     return 0;
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/BaseParameters.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/BaseParameters.java
index 609e868..468306c 100644
--- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/BaseParameters.java
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/BaseParameters.java
@@ -14,8 +14,6 @@
 
 package org.apache.hadoop.yarn.submarine.client.cli.param;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.submarine.client.cli.CliConstants;
@@ -30,15 +28,15 @@ import java.io.IOException;
 public abstract class BaseParameters {
   private String name;
 
-  public void updateParametersByParsedCommandline(CommandLine parsedCommandLine,
-      Options options, ClientContext clientContext)
+  public void updateParameters(ParametersHolder parametersHolder,
+      ClientContext clientContext)
       throws ParseException, IOException, YarnException {
-    String name = parsedCommandLine.getOptionValue(CliConstants.NAME);
+    String name = parametersHolder.getOptionValue(CliConstants.NAME);
     if (name == null) {
       throw new ParseException("--name is absent");
     }
 
-    if (parsedCommandLine.hasOption(CliConstants.VERBOSE)) {
+    if (parametersHolder.hasOption(CliConstants.VERBOSE)) {
       SubmarineLogs.verboseOn();
     }
 
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/ParametersHolder.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/ParametersHolder.java
new file mode 100644
index 0000000..18024af
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/ParametersHolder.java
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.submarine.client.cli.CliConstants;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Configs;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Role;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Roles;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Scheduling;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Security;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.TensorBoard;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.YamlConfigFile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * This class acts as a wrapper of {@code CommandLine} values along with
+ * YAML configuration values.
+ * YAML configuration is only stored if the -f &lt;filename&gt;
+ * option is specified along the CLI arguments.
+ * Using this wrapper class makes easy to deal with
+ * any form of configuration source potentially added into Submarine,
+ * in the future.
+ * If both YAML and CLI value is found for a config, this is an error case.
+ */
+public final class ParametersHolder {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ParametersHolder.class);
+
+  private final CommandLine parsedCommandLine;
+  private final Map<String, String> yamlStringConfigs;
+  private final Map<String, List<String>> yamlListConfigs;
+  private final ImmutableSet onlyDefinedWithCliArgs = ImmutableSet.of(
+      CliConstants.VERBOSE);
+
+  private ParametersHolder(CommandLine parsedCommandLine,
+      YamlConfigFile yamlConfig) {
+    this.parsedCommandLine = parsedCommandLine;
+    this.yamlStringConfigs = initStringConfigValues(yamlConfig);
+    this.yamlListConfigs = initListConfigValues(yamlConfig);
+  }
+
+  /**
+   * Maps every value coming from the passed yamlConfig to {@code CliConstants}.
+   * @param yamlConfig Parsed YAML config
+   * @return A map of config values, keys are {@code CliConstants}
+   * and values are Strings.
+   */
+  private Map<String, String> initStringConfigValues(
+      YamlConfigFile yamlConfig) {
+    if (yamlConfig == null) {
+      return Collections.emptyMap();
+    }
+    Map<String, String> yamlConfigValues = Maps.newHashMap();
+    Roles roles = yamlConfig.getRoles();
+
+    initGenericConfigs(yamlConfig, yamlConfigValues);
+    initPs(yamlConfigValues, roles.getPs());
+    initWorker(yamlConfigValues, roles.getWorker());
+    initScheduling(yamlConfigValues, yamlConfig.getScheduling());
+    initSecurity(yamlConfigValues, yamlConfig.getSecurity());
+    initTensorBoard(yamlConfigValues, yamlConfig.getTensorBoard());
+
+    return yamlConfigValues;
+  }
+
+  private Map<String, List<String>> initListConfigValues(
+      YamlConfigFile yamlConfig) {
+    if (yamlConfig == null) {
+      return Collections.emptyMap();
+    }
+
+    Map<String, List<String>> yamlConfigValues = Maps.newHashMap();
+    Configs configs = yamlConfig.getConfigs();
+    yamlConfigValues.put(CliConstants.LOCALIZATION, configs.getLocalizations());
+    yamlConfigValues.put(CliConstants.ENV,
+        convertToEnvsList(configs.getEnvs()));
+    yamlConfigValues.put(CliConstants.QUICKLINK, configs.getQuicklinks());
+
+    return yamlConfigValues;
+  }
+
+  private void initGenericConfigs(YamlConfigFile yamlConfig,
+      Map<String, String> yamlConfigs) {
+    yamlConfigs.put(CliConstants.NAME, yamlConfig.getSpec().getName());
+
+    Configs configs = yamlConfig.getConfigs();
+    yamlConfigs.put(CliConstants.INPUT_PATH, configs.getInputPath());
+    yamlConfigs.put(CliConstants.CHECKPOINT_PATH, configs.getCheckpointPath());
+    yamlConfigs.put(CliConstants.SAVED_MODEL_PATH, configs.getSavedModelPath());
+    yamlConfigs.put(CliConstants.DOCKER_IMAGE, configs.getDockerImage());
+    yamlConfigs.put(CliConstants.WAIT_JOB_FINISH, configs.getWaitJobFinish());
+  }
+
+  private void initPs(Map<String, String> yamlConfigs, Role ps) {
+    if (ps == null) {
+      return;
+    }
+    yamlConfigs.put(CliConstants.N_PS, String.valueOf(ps.getReplicas()));
+    yamlConfigs.put(CliConstants.PS_RES, ps.getResources());
+    yamlConfigs.put(CliConstants.PS_DOCKER_IMAGE, ps.getDockerImage());
+    yamlConfigs.put(CliConstants.PS_LAUNCH_CMD, ps.getLaunchCmd());
+  }
+
+  private void initWorker(Map<String, String> yamlConfigs, Role worker) {
+    if (worker == null) {
+      return;
+    }
+    yamlConfigs.put(CliConstants.N_WORKERS,
+        String.valueOf(worker.getReplicas()));
+    yamlConfigs.put(CliConstants.WORKER_RES, worker.getResources());
+    yamlConfigs.put(CliConstants.WORKER_DOCKER_IMAGE, worker.getDockerImage());
+    yamlConfigs.put(CliConstants.WORKER_LAUNCH_CMD, worker.getLaunchCmd());
+  }
+
+  private void initScheduling(Map<String, String> yamlConfigValues,
+      Scheduling scheduling) {
+    if (scheduling == null) {
+      return;
+    }
+    yamlConfigValues.put(CliConstants.QUEUE, scheduling.getQueue());
+  }
+
+  private void initSecurity(Map<String, String> yamlConfigValues,
+      Security security) {
+    if (security == null) {
+      return;
+    }
+    yamlConfigValues.put(CliConstants.KEYTAB, security.getKeytab());
+    yamlConfigValues.put(CliConstants.PRINCIPAL, security.getPrincipal());
+    yamlConfigValues.put(CliConstants.DISTRIBUTE_KEYTAB,
+        String.valueOf(security.isDistributeKeytab()));
+  }
+
+  private void initTensorBoard(Map<String, String> yamlConfigValues,
+      TensorBoard tensorBoard) {
+    if (tensorBoard == null) {
+      return;
+    }
+    yamlConfigValues.put(CliConstants.TENSORBOARD, Boolean.TRUE.toString());
+    yamlConfigValues.put(CliConstants.TENSORBOARD_DOCKER_IMAGE,
+        tensorBoard.getDockerImage());
+    yamlConfigValues.put(CliConstants.TENSORBOARD_RESOURCES,
+        tensorBoard.getResources());
+  }
+
+  private List<String> convertToEnvsList(Map<String, String> envs) {
+    if (envs == null) {
+      return Collections.emptyList();
+    }
+    return envs.entrySet().stream()
+        .map(e -> String.format("%s=%s", e.getKey(), e.getValue()))
+        .collect(Collectors.toList());
+  }
+
+  public static ParametersHolder createWithCmdLine(CommandLine cli) {
+    return new ParametersHolder(cli, null);
+  }
+
+  public static ParametersHolder createWithCmdLineAndYaml(CommandLine cli,
+      YamlConfigFile yamlConfig) {
+    return new ParametersHolder(cli, yamlConfig);
+  }
+
+  /**
+   * Gets the option value, either from the CLI arguments or YAML config,
+   * if present.
+   * @param option Name of the config.
+   * @return The value of the config
+   */
+  String getOptionValue(String option) throws YarnException {
+    ensureConfigIsDefinedOnce(option, true);
+    if (onlyDefinedWithCliArgs.contains(option) ||
+        parsedCommandLine.hasOption(option)) {
+      return getValueFromCLI(option);
+    }
+    return getValueFromYaml(option);
+  }
+
+  /**
+   * Gets the option values, either from the CLI arguments or YAML config,
+   * if present.
+   * @param option Name of the config.
+   * @return The values of the config
+   */
+  List<String> getOptionValues(String option) throws YarnException {
+    ensureConfigIsDefinedOnce(option, false);
+    if (onlyDefinedWithCliArgs.contains(option) ||
+        parsedCommandLine.hasOption(option)) {
+      return getValuesFromCLI(option);
+    }
+    return getValuesFromYaml(option);
+  }
+
+  private void ensureConfigIsDefinedOnce(String option, boolean stringValue)
+      throws YarnException {
+    boolean definedWithYaml;
+    if (stringValue) {
+      definedWithYaml = yamlStringConfigs.containsKey(option);
+    } else {
+      definedWithYaml = yamlListConfigs.containsKey(option);
+    }
+
+    if (parsedCommandLine.hasOption(option) && definedWithYaml) {
+      throw new YarnException("Config '%s' is defined both with YAML config" +
+          " and with CLI argument, please only use either way!");
+    }
+  }
+
+  private String getValueFromCLI(String option) {
+    String value = parsedCommandLine.getOptionValue(option);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found config value {} for key {} " +
+          "from CLI configuration.", value, option);
+    }
+    return value;
+  }
+
+  private List<String> getValuesFromCLI(String option) {
+    String[] optionValues = parsedCommandLine.getOptionValues(option);
+    if (optionValues != null) {
+      List<String> values = Arrays.asList(optionValues);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found config values {} for key {} " +
+            "from CLI configuration.", values, option);
+      }
+      return values;
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("No config values found for key {} " +
+            "from CLI configuration.", option);
+      }
+      return Lists.newArrayList();
+    }
+  }
+
+  private String getValueFromYaml(String option) {
+    String value = yamlStringConfigs.get(option);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found config value {} for key {} " +
+          "from YAML configuration.", value, option);
+    }
+    return value;
+  }
+
+  private List<String> getValuesFromYaml(String option) {
+    List<String> values = yamlListConfigs.get(option);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found config values {} for key {} " +
+          "from YAML configuration.", values, option);
+    }
+    return values;
+  }
+
+  /**
+   * Returns the boolean value of option.
+   * First, we check if the CLI value is defined for the option.
+   * If not, then we check the YAML value.
+   * @param option name of the option
+   * @return true, if the option is found in the CLI args or in the YAML config,
+   * false otherwise.
+   */
+  boolean hasOption(String option) {
+    if (onlyDefinedWithCliArgs.contains(option)) {
+      boolean value = parsedCommandLine.hasOption(option);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found boolean config with value {} for key {} " +
+            "from CLI configuration.", value, option);
+      }
+      return value;
+    }
+    if (parsedCommandLine.hasOption(option)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found boolean config value for key {} " +
+            "from CLI configuration.", option);
+      }
+      return true;
+    }
+    return getBooleanValueFromYaml(option);
+  }
+
+  private boolean getBooleanValueFromYaml(String option) {
+    String stringValue = yamlStringConfigs.get(option);
+    boolean result = stringValue != null
+        && Boolean.valueOf(stringValue).equals(Boolean.TRUE);
+    LOG.debug("Found config value {} for key {} " +
+        "from YAML configuration.", result, option);
+    return result;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunJobParameters.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunJobParameters.java
index 9a01dad..4792144 100644
--- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunJobParameters.java
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunJobParameters.java
@@ -14,8 +14,8 @@
 
 package org.apache.hadoop.yarn.submarine.client.cli.param;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.CaseFormat;
 import org.apache.commons.cli.ParseException;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -23,7 +23,10 @@ import org.apache.hadoop.yarn.submarine.client.cli.CliConstants;
 import org.apache.hadoop.yarn.submarine.client.cli.CliUtils;
 import org.apache.hadoop.yarn.submarine.common.ClientContext;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
+import org.yaml.snakeyaml.introspector.Property;
+import org.yaml.snakeyaml.introspector.PropertyUtils;
 
+import java.beans.IntrospectionException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -58,28 +61,31 @@ public class RunJobParameters extends RunParameters {
   private boolean distributeKeytab = false;
 
   @Override
-  public void updateParametersByParsedCommandline(CommandLine parsedCommandLine,
-      Options options, ClientContext clientContext)
+  public void updateParameters(ParametersHolder parametersHolder,
+      ClientContext clientContext)
       throws ParseException, IOException, YarnException {
 
-    String input = parsedCommandLine.getOptionValue(CliConstants.INPUT_PATH);
-    String jobDir = parsedCommandLine.getOptionValue(CliConstants.CHECKPOINT_PATH);
+    String input = parametersHolder.getOptionValue(CliConstants.INPUT_PATH);
+    String jobDir = parametersHolder.getOptionValue(
+        CliConstants.CHECKPOINT_PATH);
     int nWorkers = 1;
-    if (parsedCommandLine.getOptionValue(CliConstants.N_WORKERS) != null) {
+    if (parametersHolder.getOptionValue(CliConstants.N_WORKERS) != null) {
       nWorkers = Integer.parseInt(
-          parsedCommandLine.getOptionValue(CliConstants.N_WORKERS));
+          parametersHolder.getOptionValue(CliConstants.N_WORKERS));
       // Only check null value.
       // Training job shouldn't ignore INPUT_PATH option
-      // But if nWorkers is 0, INPUT_PATH can be ignored because user can only run Tensorboard
+      // But if nWorkers is 0, INPUT_PATH can be ignored because
+      // user can only run Tensorboard
       if (null == input && 0 != nWorkers) {
-        throw new ParseException("\"--" + CliConstants.INPUT_PATH + "\" is absent");
+        throw new ParseException("\"--" + CliConstants.INPUT_PATH +
+            "\" is absent");
       }
     }
 
     int nPS = 0;
-    if (parsedCommandLine.getOptionValue(CliConstants.N_PS) != null) {
+    if (parametersHolder.getOptionValue(CliConstants.N_PS) != null) {
       nPS = Integer.parseInt(
-          parsedCommandLine.getOptionValue(CliConstants.N_PS));
+          parametersHolder.getOptionValue(CliConstants.N_PS));
     }
 
     // Check #workers and #ps.
@@ -91,15 +97,15 @@ public class RunJobParameters extends RunParameters {
           + "please double check.");
     }
 
-    String kerberosKeytab = parsedCommandLine.getOptionValue(
+    String kerberosKeytab = parametersHolder.getOptionValue(
         CliConstants.KEYTAB);
-    String kerberosPrincipal = parsedCommandLine.getOptionValue(
+    String kerberosPrincipal = parametersHolder.getOptionValue(
         CliConstants.PRINCIPAL);
     CliUtils.doLoginIfSecure(kerberosKeytab, kerberosPrincipal);
 
     workerResource = null;
     if (nWorkers > 0) {
-      String workerResourceStr = parsedCommandLine.getOptionValue(
+      String workerResourceStr = parametersHolder.getOptionValue(
           CliConstants.WORKER_RES);
       if (workerResourceStr == null) {
         throw new ParseException(
@@ -112,7 +118,8 @@ public class RunJobParameters extends RunParameters {
 
     Resource psResource = null;
     if (nPS > 0) {
-      String psResourceStr = parsedCommandLine.getOptionValue(CliConstants.PS_RES);
+      String psResourceStr = parametersHolder.getOptionValue(
+          CliConstants.PS_RES);
       if (psResourceStr == null) {
         throw new ParseException("--" + CliConstants.PS_RES + " is absent.");
       }
@@ -121,9 +128,9 @@ public class RunJobParameters extends RunParameters {
     }
 
     boolean tensorboard = false;
-    if (parsedCommandLine.hasOption(CliConstants.TENSORBOARD)) {
+    if (parametersHolder.hasOption(CliConstants.TENSORBOARD)) {
       tensorboard = true;
-      String tensorboardResourceStr = parsedCommandLine.getOptionValue(
+      String tensorboardResourceStr = parametersHolder.getOptionValue(
           CliConstants.TENSORBOARD_RESOURCES);
       if (tensorboardResourceStr == null || tensorboardResourceStr.isEmpty()) {
         tensorboardResourceStr = CliConstants.TENSORBOARD_DEFAULT_RESOURCES;
@@ -131,17 +138,17 @@ public class RunJobParameters extends RunParameters {
       tensorboardResource = ResourceUtils.createResourceFromString(
           tensorboardResourceStr,
           clientContext.getOrCreateYarnClient().getResourceTypeInfo());
-      tensorboardDockerImage = parsedCommandLine.getOptionValue(
+      tensorboardDockerImage = parametersHolder.getOptionValue(
           CliConstants.TENSORBOARD_DOCKER_IMAGE);
       this.setTensorboardResource(tensorboardResource);
     }
 
-    if (parsedCommandLine.hasOption(CliConstants.WAIT_JOB_FINISH)) {
+    if (parametersHolder.hasOption(CliConstants.WAIT_JOB_FINISH)) {
       this.waitJobFinish = true;
     }
 
     // Quicklinks
-    String[] quicklinkStrs = parsedCommandLine.getOptionValues(
+    List<String> quicklinkStrs = parametersHolder.getOptionValues(
         CliConstants.QUICKLINK);
     if (quicklinkStrs != null) {
       for (String ql : quicklinkStrs) {
@@ -151,18 +158,18 @@ public class RunJobParameters extends RunParameters {
       }
     }
 
-    psDockerImage = parsedCommandLine.getOptionValue(
+    psDockerImage = parametersHolder.getOptionValue(
         CliConstants.PS_DOCKER_IMAGE);
-    workerDockerImage = parsedCommandLine.getOptionValue(
+    workerDockerImage = parametersHolder.getOptionValue(
         CliConstants.WORKER_DOCKER_IMAGE);
 
-    String workerLaunchCmd = parsedCommandLine.getOptionValue(
+    String workerLaunchCmd = parametersHolder.getOptionValue(
         CliConstants.WORKER_LAUNCH_CMD);
-    String psLaunchCommand = parsedCommandLine.getOptionValue(
+    String psLaunchCommand = parametersHolder.getOptionValue(
         CliConstants.PS_LAUNCH_CMD);
 
     // Localizations
-    String[] localizationsStr = parsedCommandLine.getOptionValues(
+    List<String> localizationsStr = parametersHolder.getOptionValues(
         CliConstants.LOCALIZATION);
     if (null != localizationsStr) {
       for (String loc : localizationsStr) {
@@ -171,10 +178,11 @@ public class RunJobParameters extends RunParameters {
         localizations.add(localization);
       }
     }
-    boolean distributeKerberosKeytab = parsedCommandLine.hasOption(CliConstants
+    boolean distributeKerberosKeytab = parametersHolder.hasOption(CliConstants
         .DISTRIBUTE_KEYTAB);
 
-    this.setInputPath(input).setCheckpointPath(jobDir).setNumPS(nPS).setNumWorkers(nWorkers)
+    this.setInputPath(input).setCheckpointPath(jobDir)
+        .setNumPS(nPS).setNumWorkers(nWorkers)
         .setPSLaunchCmd(psLaunchCommand).setWorkerLaunchCmd(workerLaunchCmd)
         .setPsResource(psResource)
         .setTensorboardEnabled(tensorboard)
@@ -182,8 +190,7 @@ public class RunJobParameters extends RunParameters {
         .setPrincipal(kerberosPrincipal)
         .setDistributeKeytab(distributeKerberosKeytab);
 
-    super.updateParametersByParsedCommandline(parsedCommandLine,
-        options, clientContext);
+    super.updateParameters(parametersHolder, clientContext);
   }
 
   public String getInputPath() {
@@ -331,4 +338,20 @@ public class RunJobParameters extends RunParameters {
     this.distributeKeytab = distributeKerberosKeytab;
     return this;
   }
+
+  @VisibleForTesting
+  public static class UnderscoreConverterPropertyUtils extends PropertyUtils {
+    @Override
+    public Property getProperty(Class<? extends Object> type, String name)
+        throws IntrospectionException {
+      if (name.indexOf('_') > -1) {
+        name = convertName(name);
+      }
+      return super.getProperty(type, name);
+    }
+
+    private static String convertName(String name) {
+      return CaseFormat.UPPER_UNDERSCORE.to(CaseFormat.LOWER_CAMEL, name);
+    }
+  }
 }
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunParameters.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunParameters.java
index 28884d8..34fbab2 100644
--- a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunParameters.java
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/RunParameters.java
@@ -14,8 +14,6 @@
 
 package org.apache.hadoop.yarn.submarine.client.cli.param;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.submarine.client.cli.CliConstants;
@@ -35,33 +33,36 @@ public abstract class RunParameters extends BaseParameters {
   private String queue;
 
   @Override
-  public void updateParametersByParsedCommandline(CommandLine parsedCommandLine,
-      Options options, ClientContext clientContext) throws ParseException,
+  public void updateParameters(ParametersHolder parametersHolder,
+      ClientContext clientContext) throws ParseException,
       IOException, YarnException {
-    String savedModelPath = parsedCommandLine.getOptionValue(
+    String savedModelPath = parametersHolder.getOptionValue(
         CliConstants.SAVED_MODEL_PATH);
     this.setSavedModelPath(savedModelPath);
 
-    // Envars
-    List<String> envarsList = new ArrayList<>();
-    String[] envars = parsedCommandLine.getOptionValues(CliConstants.ENV);
-    if (envars != null) {
-      for (String envar : envars) {
-        envarsList.add(envar);
-      }
-    }
-    this.setEnvars(envarsList);
+    List<String> envVars = getEnvVars(parametersHolder);
+    this.setEnvars(envVars);
 
-    String queue = parsedCommandLine.getOptionValue(
+    String queue = parametersHolder.getOptionValue(
         CliConstants.QUEUE);
     this.setQueue(queue);
 
-    String dockerImage = parsedCommandLine.getOptionValue(
+    String dockerImage = parametersHolder.getOptionValue(
         CliConstants.DOCKER_IMAGE);
     this.setDockerImageName(dockerImage);
 
-    super.updateParametersByParsedCommandline(parsedCommandLine,
-        options, clientContext);
+    super.updateParameters(parametersHolder, clientContext);
+  }
+
+  private List<String> getEnvVars(ParametersHolder parametersHolder)
+      throws YarnException {
+    List<String> result = new ArrayList<>();
+    List<String> envVarsArray = parametersHolder.getOptionValues(
+        CliConstants.ENV);
+    if (envVarsArray != null) {
+      result.addAll(envVarsArray);
+    }
+    return result;
   }
 
   public String getQueue() {
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Configs.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Configs.java
new file mode 100644
index 0000000..4636c80
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Configs.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Class that holds values found in 'configs' section of YAML configuration.
+ */
+public class Configs {
+  private String dockerImage;
+  private String inputPath;
+  private String savedModelPath;
+  private String checkpointPath;
+  private List<String> quicklinks;
+  private String waitJobFinish;
+  private Map<String, String> envs;
+  private List<String> localizations;
+  private List<String> mounts;
+
+  public String getDockerImage() {
+    return dockerImage;
+  }
+
+  public void setDockerImage(String dockerImage) {
+    this.dockerImage = dockerImage;
+  }
+
+  public String getInputPath() {
+    return inputPath;
+  }
+
+  public void setInputPath(String inputPath) {
+    this.inputPath = inputPath;
+  }
+
+  public String getSavedModelPath() {
+    return savedModelPath;
+  }
+
+  public void setSavedModelPath(String savedModelPath) {
+    this.savedModelPath = savedModelPath;
+  }
+
+  public String getCheckpointPath() {
+    return checkpointPath;
+  }
+
+  public void setCheckpointPath(String checkpointPath) {
+    this.checkpointPath = checkpointPath;
+  }
+
+  public Map<String, String> getEnvs() {
+    return envs;
+  }
+
+  public void setEnvs(Map<String, String> envs) {
+    this.envs = envs;
+  }
+
+  public List<String> getLocalizations() {
+    return localizations;
+  }
+
+  public void setLocalizations(List<String> localizations) {
+    this.localizations = localizations;
+  }
+
+  public List<String> getMounts() {
+    return mounts;
+  }
+
+  public void setMounts(List<String> mounts) {
+    this.mounts = mounts;
+  }
+
+  public List<String> getQuicklinks() {
+    return quicklinks;
+  }
+
+  public void setQuicklinks(List<String> quicklinks) {
+    this.quicklinks = quicklinks;
+  }
+
+  public String getWaitJobFinish() {
+    return waitJobFinish;
+  }
+
+  public void setWaitJobFinish(String waitJobFinish) {
+    this.waitJobFinish = waitJobFinish;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/PsRole.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/PsRole.java
new file mode 100644
index 0000000..9009c7e
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/PsRole.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * Holds configuration values for PS (parameter server).
+ * 'ps' is a section underneath the 'roles' section of the YAML
+ * configuration file.
+ */
+public class PsRole extends Role {
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Role.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Role.java
new file mode 100644
index 0000000..172fe65
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Role.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Base class for Roles. 'roles' is a section of the YAML configuration file.
+ */
+public class Role {
+  private String resources;
+  private int replicas;
+  private String launchCmd;
+
+  //Optional parameters (Can override global config)
+  private String dockerImage;
+  private Map<String, String> envs;
+  private List<String> localizations;
+  private List<String> mounts;
+
+  public String getResources() {
+    return resources;
+  }
+
+  public void setResources(String resources) {
+    this.resources = resources;
+  }
+
+  public int getReplicas() {
+    return replicas;
+  }
+
+  public void setReplicas(int replicas) {
+    this.replicas = replicas;
+  }
+
+  public String getLaunchCmd() {
+    return launchCmd;
+  }
+
+  public void setLaunchCmd(String launchCmd) {
+    this.launchCmd = launchCmd;
+  }
+
+  public String getDockerImage() {
+    return dockerImage;
+  }
+
+  public void setDockerImage(String dockerImage) {
+    this.dockerImage = dockerImage;
+  }
+
+  public Map<String, String> getEnvs() {
+    return envs;
+  }
+
+  public void setEnvs(Map<String, String> envs) {
+    this.envs = envs;
+  }
+
+  public List<String> getLocalizations() {
+    return localizations;
+  }
+
+  public void setLocalizations(List<String> localizations) {
+    this.localizations = localizations;
+  }
+
+  public List<String> getMounts() {
+    return mounts;
+  }
+
+  public void setMounts(List<String> mounts) {
+    this.mounts = mounts;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Roles.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Roles.java
new file mode 100644
index 0000000..faa7900
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Roles.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * This class represents a section of the YAML configuration file.
+ */
+public class Roles {
+  private Role worker;
+  private Role ps;
+
+  public Role getWorker() {
+    return worker;
+  }
+
+  public void setWorker(Role worker) {
+    this.worker = worker;
+  }
+
+  public Role getPs() {
+    return ps;
+  }
+
+  public void setPs(Role ps) {
+    this.ps = ps;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Scheduling.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Scheduling.java
new file mode 100644
index 0000000..b304909
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Scheduling.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * Class that holds values found in 'scheduling' section of YAML configuration.
+ */
+public class Scheduling {
+  private String queue;
+
+  public String getQueue() {
+    return queue;
+  }
+
+  public void setQueue(String queue) {
+    this.queue = queue;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Security.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Security.java
new file mode 100644
index 0000000..b2465da
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Security.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * Class that holds values found in 'security' section of YAML configuration.
+ */
+public class Security {
+  private String keytab;
+  private String principal;
+  private boolean distributeKeytab;
+
+  public String getKeytab() {
+    return keytab;
+  }
+
+  public void setKeytab(String keytab) {
+    this.keytab = keytab;
+  }
+
+  public String getPrincipal() {
+    return principal;
+  }
+
+  public void setPrincipal(String principal) {
+    this.principal = principal;
+  }
+
+  public boolean isDistributeKeytab() {
+    return distributeKeytab;
+  }
+
+  public void setDistributeKeytab(boolean distributeKeytab) {
+    this.distributeKeytab = distributeKeytab;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Spec.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Spec.java
new file mode 100644
index 0000000..64587b3
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/Spec.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * Class that holds values found in 'spec' section of YAML configuration.
+ */
+public class Spec {
+  private String name;
+  private String jobType;
+
+  public String getJobType() {
+    return jobType;
+  }
+
+  public void setJobType(String jobtype) {
+    this.jobType = jobtype;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/TensorBoard.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/TensorBoard.java
new file mode 100644
index 0000000..ea028a8
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/TensorBoard.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * Class that holds values found in 'tensorboard' section of YAML configuration.
+ */
+public class TensorBoard {
+  private String dockerImage;
+  private String resources;
+
+  public String getDockerImage() {
+    return dockerImage;
+  }
+
+  public void setDockerImage(String dockerImage) {
+    this.dockerImage = dockerImage;
+  }
+
+  public String getResources() {
+    return resources;
+  }
+
+  public void setResources(String resources) {
+    this.resources = resources;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/WorkerRole.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/WorkerRole.java
new file mode 100644
index 0000000..8c50afc
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/WorkerRole.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * Holds configuration values for the worker role.
+ * 'worker' is a section underneath the 'roles' section of the YAML
+ * configuration file.
+ */
+public class WorkerRole extends Role {
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/YamlConfigFile.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/YamlConfigFile.java
new file mode 100644
index 0000000..82ec7ff
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/YamlConfigFile.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * Root class of YAML configuration.
+ */
+public class YamlConfigFile {
+  private Spec spec;
+  private Configs configs;
+  private Roles roles;
+  private Scheduling scheduling;
+  private Security security;
+  private TensorBoard tensorBoard;
+
+  public Spec getSpec() {
+    return spec;
+  }
+
+  public void setSpec(Spec spec) {
+    this.spec = spec;
+  }
+
+  public Configs getConfigs() {
+    return configs;
+  }
+
+  public void setConfigs(Configs configs) {
+    this.configs = configs;
+  }
+
+  public Roles getRoles() {
+    return roles;
+  }
+
+  public void setRoles(Roles roles) {
+    this.roles = roles;
+  }
+
+  public Scheduling getScheduling() {
+    return scheduling;
+  }
+
+  public void setScheduling(Scheduling scheduling) {
+    this.scheduling = scheduling;
+  }
+
+  public Security getSecurity() {
+    return security;
+  }
+
+  public void setSecurity(Security security) {
+    this.security = security;
+  }
+
+  public TensorBoard getTensorBoard() {
+    return tensorBoard;
+  }
+
+  public void setTensorBoard(TensorBoard tensorBoard) {
+    this.tensorBoard = tensorBoard;
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/YamlParseException.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/YamlParseException.java
new file mode 100644
index 0000000..6c5eedc
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/YamlParseException.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
+
+/**
+ * This exception is thrown if any issue arises while parsing the
+ * YAML configuration.
+ */
+public class YamlParseException extends RuntimeException {
+  public YamlParseException(String message) {
+    super(message);
+  }
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/package-info.java b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/package-info.java
new file mode 100644
index 0000000..26d44af
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/main/java/org/apache/hadoop/yarn/submarine/client/cli/param/yaml/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * This package contains value classes for the YAML parser.
+ */
+package org.apache.hadoop.yarn.submarine.client.cli.param.yaml;
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsing.java b/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsing.java
index 3c785a0..4ad0227 100644
--- a/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsing.java
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsing.java
@@ -36,11 +36,13 @@ import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.junit.Assert.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public class TestRunJobCliParsing {
+
   @Before
   public void before() {
     SubmarineLogs.verboseOff();
@@ -56,7 +58,7 @@ public class TestRunJobCliParsing {
     runJobCli.printUsages();
   }
 
-  private MockClientContext getMockClientContext()
+  static MockClientContext getMockClientContext()
       throws IOException, YarnException {
     MockClientContext mockClientContext = new MockClientContext();
     JobSubmitter mockJobSubmitter = mock(JobSubmitter.class);
@@ -92,21 +94,21 @@ public class TestRunJobCliParsing {
 
     RunJobParameters jobRunParameters = runJobCli.getRunJobParameters();
 
-    Assert.assertEquals(jobRunParameters.getInputPath(), "hdfs://input");
-    Assert.assertEquals(jobRunParameters.getCheckpointPath(), "hdfs://output");
-    Assert.assertEquals(jobRunParameters.getNumPS(), 2);
-    Assert.assertEquals(jobRunParameters.getPSLaunchCmd(), "python run-ps.py");
-    Assert.assertEquals(Resources.createResource(4096, 4),
+    assertEquals(jobRunParameters.getInputPath(), "hdfs://input");
+    assertEquals(jobRunParameters.getCheckpointPath(), "hdfs://output");
+    assertEquals(jobRunParameters.getNumPS(), 2);
+    assertEquals(jobRunParameters.getPSLaunchCmd(), "python run-ps.py");
+    assertEquals(Resources.createResource(4096, 4),
         jobRunParameters.getPsResource());
-    Assert.assertEquals(jobRunParameters.getWorkerLaunchCmd(),
+    assertEquals(jobRunParameters.getWorkerLaunchCmd(),
         "python run-job.py");
-    Assert.assertEquals(Resources.createResource(2048, 2),
+    assertEquals(Resources.createResource(2048, 2),
         jobRunParameters.getWorkerResource());
-    Assert.assertEquals(jobRunParameters.getDockerImageName(),
+    assertEquals(jobRunParameters.getDockerImageName(),
         "tf-docker:1.1.0");
-    Assert.assertEquals(jobRunParameters.getKeytab(),
+    assertEquals(jobRunParameters.getKeytab(),
         "/keytab/path");
-    Assert.assertEquals(jobRunParameters.getPrincipal(),
+    assertEquals(jobRunParameters.getPrincipal(),
         "user/_HOST@domain.com");
     Assert.assertTrue(jobRunParameters.isDistributeKeytab());
     Assert.assertTrue(SubmarineLogs.isVerbose());
@@ -126,12 +128,12 @@ public class TestRunJobCliParsing {
 
     RunJobParameters jobRunParameters = runJobCli.getRunJobParameters();
 
-    Assert.assertEquals(jobRunParameters.getInputPath(), "hdfs://input");
-    Assert.assertEquals(jobRunParameters.getCheckpointPath(), "hdfs://output");
-    Assert.assertEquals(jobRunParameters.getNumWorkers(), 1);
-    Assert.assertEquals(jobRunParameters.getWorkerLaunchCmd(),
+    assertEquals(jobRunParameters.getInputPath(), "hdfs://input");
+    assertEquals(jobRunParameters.getCheckpointPath(), "hdfs://output");
+    assertEquals(jobRunParameters.getNumWorkers(), 1);
+    assertEquals(jobRunParameters.getWorkerLaunchCmd(),
         "python run-job.py");
-    Assert.assertEquals(Resources.createResource(4096, 2),
+    assertEquals(Resources.createResource(4096, 2),
         jobRunParameters.getWorkerResource());
     Assert.assertTrue(SubmarineLogs.isVerbose());
     Assert.assertTrue(jobRunParameters.isWaitJobFinish());
@@ -153,7 +155,7 @@ public class TestRunJobCliParsing {
       actualMessage = e.getMessage();
       e.printStackTrace();
     }
-    Assert.assertEquals(expectedErrorMessage, actualMessage);
+    assertEquals(expectedErrorMessage, actualMessage);
   }
 
   /**
@@ -182,19 +184,23 @@ public class TestRunJobCliParsing {
 
     runJobCli.run(
         new String[] { "--name", "my-job", "--docker_image", "tf-docker:1.1.0",
-            "--input_path", "hdfs://input", "--checkpoint_path", "hdfs://output",
+            "--input_path", "hdfs://input", "--checkpoint_path",
+            "hdfs://output",
             "--num_workers", "3", "--num_ps", "2", "--worker_launch_cmd",
-            "python run-job.py --input=%input_path% --model_dir=%checkpoint_path% --export_dir=%saved_model_path%/savedmodel",
+            "python run-job.py --input=%input_path% " +
+                "--model_dir=%checkpoint_path% " +
+                "--export_dir=%saved_model_path%/savedmodel",
             "--worker_resources", "memory=2048,vcores=2", "--ps_resources",
             "memory=4096,vcores=4", "--tensorboard", "true", "--ps_launch_cmd",
-            "python run-ps.py --input=%input_path% --model_dir=%checkpoint_path%/model",
+            "python run-ps.py --input=%input_path% " +
+                "--model_dir=%checkpoint_path%/model",
             "--verbose" });
 
-    Assert.assertEquals(
+    assertEquals(
         "python run-job.py --input=hdfs://input --model_dir=hdfs://output "
             + "--export_dir=hdfs://output/savedmodel",
         runJobCli.getRunJobParameters().getWorkerLaunchCmd());
-    Assert.assertEquals(
+    assertEquals(
         "python run-ps.py --input=hdfs://input --model_dir=hdfs://output/model",
         runJobCli.getRunJobParameters().getPSLaunchCmd());
   }
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsingYaml.java b/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsingYaml.java
new file mode 100644
index 0000000..c8c03b5
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsingYaml.java
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.submarine.client.cli;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.api.records.ResourceTypeInfo;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.resourcetypes.ResourceTypesTestHelper;
+import org.apache.hadoop.yarn.submarine.client.cli.param.RunJobParameters;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.YamlParseException;
+import org.apache.hadoop.yarn.submarine.common.conf.SubmarineLogs;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.yarn.submarine.client.cli.TestRunJobCliParsing.getMockClientContext;
+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.assertTrue;
+
+/**
+ * Test class that verifies the correctness of YAML configuration parsing.
+ */
+public class TestRunJobCliParsingYaml {
+  private static final String OVERRIDDEN_PREFIX = "overridden_";
+  private static final String DIR_NAME = "runjobcliparsing";
+  private File yamlConfig;
+
+  @Before
+  public void before() {
+    SubmarineLogs.verboseOff();
+  }
+
+  @After
+  public void after() {
+    YamlConfigTestUtils.deleteFile(yamlConfig);
+  }
+
+  @BeforeClass
+  public static void configureResourceTypes() {
+    List<ResourceTypeInfo> resTypes = new ArrayList<>(
+        ResourceUtils.getResourcesTypeInfo());
+    resTypes.add(ResourceTypeInfo.newInstance(ResourceInformation.GPU_URI, ""));
+    ResourceUtils.reinitializeResources(resTypes);
+  }
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  private void verifyBasicConfigValues(RunJobParameters jobRunParameters) {
+    verifyBasicConfigValues(jobRunParameters,
+        ImmutableList.of("env1=env1Value", "env2=env2Value"));
+  }
+
+  private void verifyBasicConfigValues(RunJobParameters jobRunParameters,
+      List<String> expectedEnvs) {
+    assertEquals("testInputPath", jobRunParameters.getInputPath());
+    assertEquals("testCheckpointPath", jobRunParameters.getCheckpointPath());
+    assertEquals("testDockerImage", jobRunParameters.getDockerImageName());
+
+    assertNotNull(jobRunParameters.getLocalizations());
+    assertEquals(2, jobRunParameters.getLocalizations().size());
+
+    assertNotNull(jobRunParameters.getQuicklinks());
+    assertEquals(2, jobRunParameters.getQuicklinks().size());
+
+    assertTrue(SubmarineLogs.isVerbose());
+    assertTrue(jobRunParameters.isWaitJobFinish());
+
+    for (String env : expectedEnvs) {
+      assertTrue(String.format(
+          "%s should be in env list of jobRunParameters!", env),
+          jobRunParameters.getEnvars().contains(env));
+    }
+  }
+
+  private void verifyPsValues(RunJobParameters jobRunParameters,
+      String prefix) {
+    assertEquals(4, jobRunParameters.getNumPS());
+    assertEquals(prefix + "testLaunchCmdPs", jobRunParameters.getPSLaunchCmd());
+    assertEquals(prefix + "testDockerImagePs",
+        jobRunParameters.getPsDockerImage());
+    assertEquals(ResourceTypesTestHelper.newResource(20500L, 34,
+        ImmutableMap.<String, String> builder()
+            .put(ResourceInformation.GPU_URI, "4").build()),
+        jobRunParameters.getPsResource());
+  }
+
+  private void verifyWorkerValues(RunJobParameters jobRunParameters,
+      String prefix) {
+    assertEquals(3, jobRunParameters.getNumWorkers());
+    assertEquals(prefix + "testLaunchCmdWorker",
+        jobRunParameters.getWorkerLaunchCmd());
+    assertEquals(prefix + "testDockerImageWorker",
+        jobRunParameters.getWorkerDockerImage());
+    assertEquals(ResourceTypesTestHelper.newResource(20480L, 32,
+        ImmutableMap.<String, String> builder()
+            .put(ResourceInformation.GPU_URI, "2").build()),
+        jobRunParameters.getWorkerResource());
+  }
+
+  private void verifySecurityValues(RunJobParameters jobRunParameters) {
+    assertEquals("keytabPath", jobRunParameters.getKeytab());
+    assertEquals("testPrincipal", jobRunParameters.getPrincipal());
+    assertTrue(jobRunParameters.isDistributeKeytab());
+  }
+
+  private void verifyTensorboardValues(RunJobParameters jobRunParameters) {
+    assertTrue(jobRunParameters.isTensorboardEnabled());
+    assertEquals("tensorboardDockerImage",
+        jobRunParameters.getTensorboardDockerImage());
+    assertEquals(ResourceTypesTestHelper.newResource(21000L, 37,
+        ImmutableMap.<String, String> builder()
+            .put(ResourceInformation.GPU_URI, "3").build()),
+        jobRunParameters.getTensorboardResource());
+  }
+
+  @Test
+  public void testValidYamlParsing() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+    Assert.assertFalse(SubmarineLogs.isVerbose());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/valid-config.yaml");
+    runJobCli.run(
+        new String[] {"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+
+    RunJobParameters jobRunParameters = runJobCli.getRunJobParameters();
+    verifyBasicConfigValues(jobRunParameters);
+    verifyPsValues(jobRunParameters, "");
+    verifyWorkerValues(jobRunParameters, "");
+    verifySecurityValues(jobRunParameters);
+    verifyTensorboardValues(jobRunParameters);
+  }
+
+  @Test
+  public void testYamlAndCliOptionIsDefinedIsInvalid() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+    Assert.assertFalse(SubmarineLogs.isVerbose());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/valid-config.yaml");
+    String[] args = new String[] {"--name", "my-job",
+        "--docker_image", "tf-docker:1.1.0",
+        "-f", yamlConfig.getAbsolutePath() };
+
+    exception.expect(YarnException.class);
+    exception.expectMessage("defined both with YAML config and with " +
+        "CLI argument");
+
+    runJobCli.run(args);
+  }
+
+  @Test
+  public void testYamlAndCliOptionIsDefinedIsInvalidWithListOption()
+      throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+    Assert.assertFalse(SubmarineLogs.isVerbose());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/valid-config.yaml");
+    String[] args = new String[] {"--name", "my-job",
+        "--quicklink", "AAA=http://master-0:8321",
+        "--quicklink", "BBB=http://worker-0:1234",
+        "-f", yamlConfig.getAbsolutePath()};
+
+    exception.expect(YarnException.class);
+    exception.expectMessage("defined both with YAML config and with " +
+        "CLI argument");
+
+    runJobCli.run(args);
+  }
+
+  @Test
+  public void testRoleOverrides() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+    Assert.assertFalse(SubmarineLogs.isVerbose());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/valid-config-with-overrides.yaml");
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+
+    RunJobParameters jobRunParameters = runJobCli.getRunJobParameters();
+    verifyBasicConfigValues(jobRunParameters);
+    verifyPsValues(jobRunParameters, OVERRIDDEN_PREFIX);
+    verifyWorkerValues(jobRunParameters, OVERRIDDEN_PREFIX);
+    verifySecurityValues(jobRunParameters);
+    verifyTensorboardValues(jobRunParameters);
+  }
+
+  @Test
+  public void testFalseValuesForBooleanFields() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+    Assert.assertFalse(SubmarineLogs.isVerbose());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/test-false-values.yaml");
+    runJobCli.run(
+        new String[] {"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+    RunJobParameters jobRunParameters = runJobCli.getRunJobParameters();
+
+    assertFalse(jobRunParameters.isDistributeKeytab());
+    assertFalse(jobRunParameters.isWaitJobFinish());
+    assertFalse(jobRunParameters.isTensorboardEnabled());
+  }
+
+  @Test
+  public void testWrongIndentation() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+    Assert.assertFalse(SubmarineLogs.isVerbose());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/wrong-indentation.yaml");
+
+    exception.expect(YamlParseException.class);
+    exception.expectMessage("Failed to parse YAML config, details:");
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+  }
+
+  @Test
+  public void testWrongFilename() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+    Assert.assertFalse(SubmarineLogs.isVerbose());
+
+    exception.expect(YamlParseException.class);
+    runJobCli.run(
+        new String[]{"-f", "not-existing", "--verbose"});
+  }
+
+  @Test
+  public void testEmptyFile() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+
+    yamlConfig = YamlConfigTestUtils.createEmptyTempFile();
+
+    exception.expect(YamlParseException.class);
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+  }
+
+  @Test
+  public void testNotExistingFile() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+
+    exception.expect(YamlParseException.class);
+    exception.expectMessage("file does not exist");
+    runJobCli.run(
+        new String[]{"-f", "blabla", "--verbose"});
+  }
+
+  @Test
+  public void testWrongPropertyName() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/wrong-property-name.yaml");
+
+    exception.expect(YamlParseException.class);
+    exception.expectMessage("Failed to parse YAML config, details:");
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+  }
+
+  @Test
+  public void testMissingConfigsSection() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/missing-configs.yaml");
+
+    exception.expect(YamlParseException.class);
+    exception.expectMessage("config section should be defined, " +
+        "but it cannot be found");
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+  }
+
+  @Test
+  public void testMissingSectionsShouldParsed() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/some-sections-missing.yaml");
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+  }
+
+  @Test
+  public void testMissingPrincipalUnderSecuritySection() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/security-principal-is-missing.yaml");
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+
+    RunJobParameters jobRunParameters = runJobCli.getRunJobParameters();
+    verifyBasicConfigValues(jobRunParameters);
+    verifyPsValues(jobRunParameters, "");
+    verifyWorkerValues(jobRunParameters, "");
+    verifyTensorboardValues(jobRunParameters);
+
+    //Verify security values
+    assertEquals("keytabPath", jobRunParameters.getKeytab());
+    assertNull("Principal should be null!", jobRunParameters.getPrincipal());
+    assertTrue(jobRunParameters.isDistributeKeytab());
+  }
+
+  @Test
+  public void testMissingTensorBoardDockerImage() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/tensorboard-dockerimage-is-missing.yaml");
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+
+    RunJobParameters jobRunParameters = runJobCli.getRunJobParameters();
+    verifyBasicConfigValues(jobRunParameters);
+    verifyPsValues(jobRunParameters, "");
+    verifyWorkerValues(jobRunParameters, "");
+    verifySecurityValues(jobRunParameters);
+
+    assertTrue(jobRunParameters.isTensorboardEnabled());
+    assertNull("tensorboardDockerImage should be null!",
+        jobRunParameters.getTensorboardDockerImage());
+    assertEquals(ResourceTypesTestHelper.newResource(21000L, 37,
+        ImmutableMap.<String, String> builder()
+            .put(ResourceInformation.GPU_URI, "3").build()),
+        jobRunParameters.getTensorboardResource());
+  }
+
+  @Test
+  public void testMissingEnvs() throws Exception {
+    RunJobCli runJobCli = new RunJobCli(getMockClientContext());
+
+    yamlConfig = YamlConfigTestUtils.createTempFileWithContents(
+        DIR_NAME + "/envs-are-missing.yaml");
+    runJobCli.run(
+        new String[]{"-f", yamlConfig.getAbsolutePath(), "--verbose"});
+
+    RunJobParameters jobRunParameters = runJobCli.getRunJobParameters();
+    verifyBasicConfigValues(jobRunParameters, ImmutableList.of());
+    verifyPsValues(jobRunParameters, "");
+    verifyWorkerValues(jobRunParameters, "");
+    verifySecurityValues(jobRunParameters);
+    verifyTensorboardValues(jobRunParameters);
+  }
+
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsingYamlStandalone.java b/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsingYamlStandalone.java
new file mode 100644
index 0000000..165d97f
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/TestRunJobCliParsingYamlStandalone.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.submarine.client.cli;
+
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Configs;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Role;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Roles;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Scheduling;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Security;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.Spec;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.TensorBoard;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.YamlConfigFile;
+import org.apache.hadoop.yarn.submarine.common.conf.SubmarineLogs;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.yarn.submarine.client.cli.YamlConfigTestUtils.readYamlConfigFile;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class that verifies the correctness of YAML configuration parsing.
+ * Please note that this class just tests YAML parsing,
+ * but only in an isolated fashion.
+ */
+public class TestRunJobCliParsingYamlStandalone {
+  private static final String OVERRIDDEN_PREFIX = "overridden_";
+  private static final String DIR_NAME = "runjobcliparsing";
+
+  @Before
+  public void before() {
+    SubmarineLogs.verboseOff();
+  }
+
+  private void verifyBasicConfigValues(YamlConfigFile yamlConfigFile) {
+    assertNotNull("Spec file should not be null!", yamlConfigFile);
+    Spec spec = yamlConfigFile.getSpec();
+    assertNotNull("Spec should not be null!", spec);
+
+    assertEquals("testJobName", spec.getName());
+    assertEquals("testJobType", spec.getJobType());
+
+    Configs configs = yamlConfigFile.getConfigs();
+    assertNotNull("Configs should not be null!", configs);
+
+    assertEquals("testInputPath", configs.getInputPath());
+    assertEquals("testCheckpointPath", configs.getCheckpointPath());
+    assertEquals("testSavedModelPath", configs.getSavedModelPath());
+    assertEquals("testDockerImage", configs.getDockerImage());
+
+    Map<String, String> envs = configs.getEnvs();
+    assertNotNull("Envs should not be null!", envs);
+    assertEquals(2, envs.size());
+    assertEquals("env1Value", envs.get("env1"));
+    assertEquals("env2Value", envs.get("env2"));
+
+    List<String> localizations = configs.getLocalizations();
+    assertNotNull("Localizations should not be null!", localizations);
+    assertEquals("Size of localizations must be 2!", 2, localizations.size());
+    assertEquals("hdfs://remote-file1:/local-filename1:rw",
+        localizations.get(0));
+    assertEquals("nfs://remote-file2:/local-filename2:rw",
+        localizations.get(1));
+
+    List<String> mounts = configs.getMounts();
+    assertNotNull("Mounts should not be null!", mounts);
+    assertEquals("Size of mounts must be 2!", 2, mounts.size());
+    assertEquals("/etc/passwd:/etc/passwd:rw", mounts.get(0));
+    assertEquals("/etc/hosts:/etc/hosts:rw", mounts.get(1));
+
+    assertTrue(
+        configs.getQuicklinks().contains("Notebook_UI=https://master-0:7070"));
+    assertTrue(
+        configs.getQuicklinks().contains("Notebook_UI2=https://master-0:7071"));
+    assertEquals("true", configs.getWaitJobFinish());
+  }
+
+  private void assertRoleConfigOverrides(Role role, String prefix,
+      String roleType) {
+    assertNotNull(roleType + " role should not be null!", role);
+
+    assertEquals(String.format("%stestDockerImage%s", prefix, roleType),
+        role.getDockerImage());
+
+    //envs, localizations and mounts for Roles
+    // are only present in valid-config-with-overrides.yaml
+    boolean validateAll = !prefix.equals("");
+    if (validateAll) {
+      Map<String, String> envs = role.getEnvs();
+      assertNotNull("Envs should not be null!", envs);
+      assertEquals(String.format("%senv1%s", prefix, roleType),
+          envs.get("env1"));
+      assertEquals(String.format("%senv2%s", prefix, roleType),
+          envs.get("env2"));
+    }
+
+    if (validateAll) {
+      List<String> localizations = role.getLocalizations();
+      assertNotNull("Localizations should not be null!", localizations);
+      assertEquals("Size of localizations must be 2!", 2, localizations.size());
+      assertEquals(String.format("hdfs://remote-file1:/%slocal" +
+          "-filename1%s:rw", prefix, roleType), localizations.get(0));
+      assertEquals(String.format("nfs://remote-file2:/%slocal" +
+          "-filename2%s:rw", prefix, roleType), localizations.get(1));
+    }
+
+    if (validateAll) {
+      List<String> mounts = role.getMounts();
+      assertNotNull("Mounts should not be null!", mounts);
+      assertEquals("Size of mounts must be 2!", 2, mounts.size());
+      assertEquals(String.format("/etc/passwd:/%s%s", prefix, roleType),
+          mounts.get(0));
+      assertEquals(String.format("/etc/hosts:/%s%s", prefix, roleType),
+          mounts.get(1));
+    }
+  }
+
+  private void assertWorkerValues(Role worker) {
+    assertEquals("testLaunchCmdWorker", worker.getLaunchCmd());
+    assertEquals("testDockerImageWorker", worker.getDockerImage());
+    assertEquals("memory=20480M,vcores=32,gpu=2", worker.getResources());
+    assertEquals(3, worker.getReplicas());
+  }
+
+  private void assertPsValues(Role ps) {
+    assertEquals("testLaunchCmdPs", ps.getLaunchCmd());
+    assertEquals("testDockerImagePs", ps.getDockerImage());
+    assertEquals("memory=20500M,vcores=34,gpu=4", ps.getResources());
+    assertEquals(4, ps.getReplicas());
+  }
+
+  private void verifySchedulingValues(YamlConfigFile yamlConfigFile) {
+    Scheduling scheduling = yamlConfigFile.getScheduling();
+    assertNotNull("Scheduling should not be null!", scheduling);
+    assertEquals("queue1", scheduling.getQueue());
+  }
+
+  private void verifySecurityValues(YamlConfigFile yamlConfigFile) {
+    Security security = yamlConfigFile.getSecurity();
+    assertNotNull("Security should not be null!", security);
+    assertEquals("keytabPath", security.getKeytab());
+    assertEquals("testPrincipal", security.getPrincipal());
+    assertTrue(security.isDistributeKeytab());
+  }
+
+  private void verifyTensorboardValues(YamlConfigFile yamlConfigFile) {
+    TensorBoard tensorBoard = yamlConfigFile.getTensorBoard();
+    assertNotNull("Tensorboard should not be null!", tensorBoard);
+    assertEquals("tensorboardDockerImage", tensorBoard.getDockerImage());
+    assertEquals("memory=21000M,vcores=37,gpu=3", tensorBoard.getResources());
+  }
+
+  @Test
+  public void testLaunchCommandYaml() {
+    YamlConfigFile yamlConfigFile = readYamlConfigFile(DIR_NAME +
+        "/valid-config.yaml");
+
+    verifyBasicConfigValues(yamlConfigFile);
+
+    Roles roles = yamlConfigFile.getRoles();
+    assertNotNull("Roles should not be null!", roles);
+    assertRoleConfigOverrides(roles.getWorker(), "", "Worker");
+    assertRoleConfigOverrides(roles.getPs(), "", "Ps");
+
+    assertWorkerValues(roles.getWorker());
+    assertPsValues(roles.getPs());
+
+    verifySchedulingValues(yamlConfigFile);
+    verifySecurityValues(yamlConfigFile);
+    verifyTensorboardValues(yamlConfigFile);
+  }
+
+  @Test
+  public void testOverrides() {
+    YamlConfigFile yamlConfigFile = readYamlConfigFile(DIR_NAME +
+        "/valid-config-with-overrides.yaml");
+
+    verifyBasicConfigValues(yamlConfigFile);
+
+    Roles roles = yamlConfigFile.getRoles();
+    assertNotNull("Roles should not be null!", roles);
+    assertRoleConfigOverrides(roles.getWorker(), OVERRIDDEN_PREFIX, "Worker");
+    assertRoleConfigOverrides(roles.getPs(), OVERRIDDEN_PREFIX, "Ps");
+  }
+
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/YamlConfigTestUtils.java b/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/YamlConfigTestUtils.java
new file mode 100644
index 0000000..91ef885
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/java/org/apache/hadoop/yarn/submarine/client/cli/YamlConfigTestUtils.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.submarine.client.cli;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.yarn.submarine.client.cli.param.RunJobParameters.UnderscoreConverterPropertyUtils;
+import org.apache.hadoop.yarn.submarine.client.cli.param.yaml.YamlConfigFile;
+import org.yaml.snakeyaml.Yaml;
+import org.yaml.snakeyaml.constructor.Constructor;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Test utility class for test code that deals with YAML configuration parsing.
+ */
+public final class YamlConfigTestUtils {
+
+  private YamlConfigTestUtils() {}
+
+  static void deleteFile(File file) {
+    if (file != null) {
+      file.delete();
+    }
+  }
+
+  static YamlConfigFile readYamlConfigFile(String filename) {
+    Constructor constructor = new Constructor(YamlConfigFile.class);
+    constructor.setPropertyUtils(new UnderscoreConverterPropertyUtils());
+    Yaml yaml = new Yaml(constructor);
+    InputStream inputStream = YamlConfigTestUtils.class
+        .getClassLoader()
+        .getResourceAsStream(filename);
+    return yaml.loadAs(inputStream, YamlConfigFile.class);
+  }
+
+  static File createTempFileWithContents(String filename) throws IOException {
+    InputStream inputStream = YamlConfigTestUtils.class
+        .getClassLoader()
+        .getResourceAsStream(filename);
+    File targetFile = File.createTempFile("test", ".yaml");
+    FileUtils.copyInputStreamToFile(inputStream, targetFile);
+    return targetFile;
+  }
+
+  static File createEmptyTempFile() throws IOException {
+    return File.createTempFile("test", ".yaml");
+  }
+
+}
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/envs-are-missing.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/envs-are-missing.yaml
new file mode 100644
index 0000000..187193b
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/envs-are-missing.yaml
@@ -0,0 +1,59 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+configs:
+  input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+  wait_job_finish: true
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+  quicklinks:
+  - Notebook_UI=https://master-0:7070
+  - Notebook_UI2=https://master-0:7071
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+    docker_image: testDockerImageWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
+    docker_image: testDockerImagePs
+
+security:
+  keytab: keytabPath
+  principal: testPrincipal
+  distribute_keytab: true
+
+tensorBoard:
+  resources: memory=21000M,vcores=37,gpu=3
+  docker_image: tensorboardDockerImage
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/missing-configs.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/missing-configs.yaml
new file mode 100644
index 0000000..db19dc2
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/missing-configs.yaml
@@ -0,0 +1,41 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
+
+security:
+  keytab: keytabPath
+  principal: testPrincipal
+  distribute_keytab: true
+
+tensorBoard:
+  resources: memory=21000M,vcores=37,gpu=3
+  docker_image: tensorboardDockerImage
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/security-principal-is-missing.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/security-principal-is-missing.yaml
new file mode 100644
index 0000000..39113b6
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/security-principal-is-missing.yaml
@@ -0,0 +1,61 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+configs:
+  input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+  wait_job_finish: true
+  envs:
+    env1: 'env1Value'
+    env2: 'env2Value'
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+  quicklinks:
+  - Notebook_UI=https://master-0:7070
+  - Notebook_UI2=https://master-0:7071
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+    docker_image: testDockerImageWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
+    docker_image: testDockerImagePs
+
+security:
+  keytab: keytabPath
+  distribute_keytab: true
+
+tensorBoard:
+  resources: memory=21000M,vcores=37,gpu=3
+  docker_image: tensorboardDockerImage
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/some-sections-missing.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/some-sections-missing.yaml
new file mode 100644
index 0000000..4fe0ca7
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/some-sections-missing.yaml
@@ -0,0 +1,48 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+configs:
+  input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+  quicklinks:
+  - Notebook_UI=https://master-0:7070
+  - Notebook_UI2=https://master-0:7071
+  wait_job_finish: true
+  envs:
+    env1: 'env1Value'
+    env2: 'env2Value'
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/tensorboard-dockerimage-is-missing.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/tensorboard-dockerimage-is-missing.yaml
new file mode 100644
index 0000000..7000e91
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/tensorboard-dockerimage-is-missing.yaml
@@ -0,0 +1,61 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+configs:
+  input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+  wait_job_finish: true
+  envs:
+    env1: 'env1Value'
+    env2: 'env2Value'
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+  quicklinks:
+  - Notebook_UI=https://master-0:7070
+  - Notebook_UI2=https://master-0:7071
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+    docker_image: testDockerImageWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
+    docker_image: testDockerImagePs
+
+security:
+  keytab: keytabPath
+  principal: testPrincipal
+  distribute_keytab: true
+
+tensorBoard:
+  resources: memory=21000M,vcores=37,gpu=3
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/test-false-values.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/test-false-values.yaml
new file mode 100644
index 0000000..564be54
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/test-false-values.yaml
@@ -0,0 +1,56 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+configs:
+  input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+  quicklinks:
+  - Notebook_UI=https://master-0:7070
+  - Notebook_UI2=https://master-0:7071
+  wait_job_finish: false
+  envs:
+    env1: 'env1Value'
+    env2: 'env2Value'
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
+
+security:
+  keytab: keytabPath
+  principal: testPrincipal
+  distribute_keytab: false
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/valid-config-with-overrides.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/valid-config-with-overrides.yaml
new file mode 100644
index 0000000..8873486
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/valid-config-with-overrides.yaml
@@ -0,0 +1,81 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+configs:
+  input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+  wait_job_finish: true
+  envs:
+    env1: 'env1Value'
+    env2: 'env2Value'
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+  quicklinks:
+  - Notebook_UI=https://master-0:7070
+  - Notebook_UI2=https://master-0:7071
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: overridden_testLaunchCmdWorker
+    docker_image: overridden_testDockerImageWorker
+    envs:
+      env1: 'overridden_env1Worker'
+      env2: 'overridden_env2Worker'
+    localizations:
+    - hdfs://remote-file1:/overridden_local-filename1Worker:rw
+    - nfs://remote-file2:/overridden_local-filename2Worker:rw
+    mounts:
+    - /etc/passwd:/overridden_Worker
+    - /etc/hosts:/overridden_Worker
+
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: overridden_testLaunchCmdPs
+    docker_image: overridden_testDockerImagePs
+    envs:
+      env1: 'overridden_env1Ps'
+      env2: 'overridden_env2Ps'
+    localizations:
+    - hdfs://remote-file1:/overridden_local-filename1Ps:rw
+    - nfs://remote-file2:/overridden_local-filename2Ps:rw
+    mounts:
+    - /etc/passwd:/overridden_Ps
+    - /etc/hosts:/overridden_Ps
+
+security:
+  keytab: keytabPath
+  principal: testPrincipal
+  distribute_keytab: true
+
+tensorBoard:
+  resources: memory=21000M,vcores=37,gpu=3
+  docker_image: tensorboardDockerImage
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/valid-config.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/valid-config.yaml
new file mode 100644
index 0000000..be67fbb
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/valid-config.yaml
@@ -0,0 +1,62 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+configs:
+  input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+  wait_job_finish: true
+  envs:
+    env1: 'env1Value'
+    env2: 'env2Value'
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+  quicklinks:
+  - Notebook_UI=https://master-0:7070
+  - Notebook_UI2=https://master-0:7071
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+    docker_image: testDockerImageWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
+    docker_image: testDockerImagePs
+
+security:
+  keytab: keytabPath
+  principal: testPrincipal
+  distribute_keytab: true
+
+tensorBoard:
+  resources: memory=21000M,vcores=37,gpu=3
+  docker_image: tensorboardDockerImage
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/wrong-indentation.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/wrong-indentation.yaml
new file mode 100644
index 0000000..66f0e16
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/wrong-indentation.yaml
@@ -0,0 +1,60 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+configs:
+      input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+    quicklinks:
+    - Notebook_UI=https://master-0:7070
+    - Notebook_UI2=https://master-0:7071
+  wait_job_finish: true
+  envs:
+    env1: 'env1Value'
+    env2: 'env2Value'
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
+
+security:
+  keytab: keytabPath
+  principal: testPrincipal
+  distribute_keytab: true
+
+tensorBoard:
+  resources: memory=21000M,vcores=37,gpu=3
+  docker_image: tensorboardDockerImage
\ No newline at end of file
diff --git a/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/wrong-property-name.yaml b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/wrong-property-name.yaml
new file mode 100644
index 0000000..41b5f2b
--- /dev/null
+++ b/hadoop-submarine/hadoop-submarine-core/src/test/resources/runjobcliparsing/wrong-property-name.yaml
@@ -0,0 +1,60 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+spec:
+  name: testJobName
+  job_type: testJobType
+
+CONFIGS:
+      input_path: testInputPath
+  checkpoint_path: testCheckpointPath
+  saved_model_path: testSavedModelPath
+  docker_image: testDockerImage
+    quicklinks:
+    - Notebook_UI=https://master-0:7070
+    - Notebook_UI2=https://master-0:7071
+  wait_job_finish: true
+  envs:
+    env1: 'env1Value'
+    env2: 'env2Value'
+  localizations:
+  - hdfs://remote-file1:/local-filename1:rw
+  - nfs://remote-file2:/local-filename2:rw
+  mounts:
+  - /etc/passwd:/etc/passwd:rw
+  - /etc/hosts:/etc/hosts:rw
+
+scheduling:
+  queue: queue1
+
+roles:
+  worker:
+    resources: memory=20480M,vcores=32,gpu=2
+    replicas: 3
+    launch_cmd: testLaunchCmdWorker
+  ps:
+    resources: memory=20500M,vcores=34,gpu=4
+    replicas: 4
+    launch_cmd: testLaunchCmdPs
+
+security:
+  keytab: keytabPath
+  principal: testPrincipal
+  distribute_keytab: true
+
+tensorBoard:
+  resources: memory=21000M,vcores=37,gpu=3
+  docker_image: tensorboardDockerImage
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org