[SPARK-24319][SPARK SUBMIT] Fix spark-submit execution where no main class is required.
authorGabor Somogyi <gabor.g.somogyi@gmail.com>
Thu, 14 Jun 2018 21:54:46 +0000 (14:54 -0700)
committerMarcelo Vanzin <vanzin@cloudera.com>
Thu, 14 Jun 2018 21:54:46 +0000 (14:54 -0700)
## What changes were proposed in this pull request?

With [PR 20925](https://github.com/apache/spark/pull/20925) now it's not possible to execute the following commands:
* run-example
* run-example --help
* run-example --version
* run-example --usage-error
* run-example --status ...
* run-example --kill ...

In this PR the execution will be allowed for the mentioned commands.

## How was this patch tested?

Existing unit tests extended + additional written.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #21450 from gaborgsomogyi/SPARK-24319.

launcher/src/main/java/org/apache/spark/launcher/Main.java
launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java

index 1e34bb8..d967aa3 100644 (file)
@@ -17,6 +17,7 @@
 
 package org.apache.spark.launcher;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -54,10 +55,12 @@ class Main {
     String className = args.remove(0);
 
     boolean printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND"));
-    AbstractCommandBuilder builder;
+    Map<String, String> env = new HashMap<>();
+    List<String> cmd;
     if (className.equals("org.apache.spark.deploy.SparkSubmit")) {
       try {
-        builder = new SparkSubmitCommandBuilder(args);
+        AbstractCommandBuilder builder = new SparkSubmitCommandBuilder(args);
+        cmd = buildCommand(builder, env, printLaunchCommand);
       } catch (IllegalArgumentException e) {
         printLaunchCommand = false;
         System.err.println("Error: " + e.getMessage());
@@ -76,17 +79,12 @@ class Main {
           help.add(parser.className);
         }
         help.add(parser.USAGE_ERROR);
-        builder = new SparkSubmitCommandBuilder(help);
+        AbstractCommandBuilder builder = new SparkSubmitCommandBuilder(help);
+        cmd = buildCommand(builder, env, printLaunchCommand);
       }
     } else {
-      builder = new SparkClassCommandBuilder(className, args);
-    }
-
-    Map<String, String> env = new HashMap<>();
-    List<String> cmd = builder.buildCommand(env);
-    if (printLaunchCommand) {
-      System.err.println("Spark Command: " + join(" ", cmd));
-      System.err.println("========================================");
+      AbstractCommandBuilder builder = new SparkClassCommandBuilder(className, args);
+      cmd = buildCommand(builder, env, printLaunchCommand);
     }
 
     if (isWindows()) {
@@ -102,6 +100,22 @@ class Main {
   }
 
   /**
+   * Prepare spark commands with the appropriate command builder.
+   * If printLaunchCommand is set then the commands will be printed to the stderr.
+   */
+  private static List<String> buildCommand(
+      AbstractCommandBuilder builder,
+      Map<String, String> env,
+      boolean printLaunchCommand) throws IOException, IllegalArgumentException {
+    List<String> cmd = builder.buildCommand(env);
+    if (printLaunchCommand) {
+      System.err.println("Spark Command: " + join(" ", cmd));
+      System.err.println("========================================");
+    }
+    return cmd;
+  }
+
+  /**
    * Prepare a command line for execution from a Windows batch script.
    *
    * The method quotes all arguments so that spaces are handled as expected. Quotes within arguments
index 5cb6457..cc65f78 100644 (file)
@@ -90,7 +90,8 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
 
   final List<String> userArgs;
   private final List<String> parsedArgs;
-  private final boolean requiresAppResource;
+  // Special command means no appResource and no mainClass required
+  private final boolean isSpecialCommand;
   private final boolean isExample;
 
   /**
@@ -105,7 +106,7 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
    * spark-submit argument list to be modified after creation.
    */
   SparkSubmitCommandBuilder() {
-    this.requiresAppResource = true;
+    this.isSpecialCommand = false;
     this.isExample = false;
     this.parsedArgs = new ArrayList<>();
     this.userArgs = new ArrayList<>();
@@ -138,25 +139,26 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
 
         case RUN_EXAMPLE:
           isExample = true;
+          appResource = SparkLauncher.NO_RESOURCE;
           submitArgs = args.subList(1, args.size());
       }
 
       this.isExample = isExample;
       OptionParser parser = new OptionParser(true);
       parser.parse(submitArgs);
-      this.requiresAppResource = parser.requiresAppResource;
+      this.isSpecialCommand = parser.isSpecialCommand;
     } else {
       this.isExample = isExample;
-      this.requiresAppResource = false;
+      this.isSpecialCommand = true;
     }
   }
 
   @Override
   public List<String> buildCommand(Map<String, String> env)
       throws IOException, IllegalArgumentException {
-    if (PYSPARK_SHELL.equals(appResource) && requiresAppResource) {
+    if (PYSPARK_SHELL.equals(appResource) && !isSpecialCommand) {
       return buildPySparkShellCommand(env);
-    } else if (SPARKR_SHELL.equals(appResource) && requiresAppResource) {
+    } else if (SPARKR_SHELL.equals(appResource) && !isSpecialCommand) {
       return buildSparkRCommand(env);
     } else {
       return buildSparkSubmitCommand(env);
@@ -166,18 +168,18 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
   List<String> buildSparkSubmitArgs() {
     List<String> args = new ArrayList<>();
     OptionParser parser = new OptionParser(false);
-    final boolean requiresAppResource;
+    final boolean isSpecialCommand;
 
     // If the user args array is not empty, we need to parse it to detect exactly what
     // the user is trying to run, so that checks below are correct.
     if (!userArgs.isEmpty()) {
       parser.parse(userArgs);
-      requiresAppResource = parser.requiresAppResource;
+      isSpecialCommand = parser.isSpecialCommand;
     } else {
-      requiresAppResource = this.requiresAppResource;
+      isSpecialCommand = this.isSpecialCommand;
     }
 
-    if (!allowsMixedArguments && requiresAppResource) {
+    if (!allowsMixedArguments && !isSpecialCommand) {
       checkArgument(appResource != null, "Missing application resource.");
     }
 
@@ -229,7 +231,7 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
       args.add(join(",", pyFiles));
     }
 
-    if (isExample) {
+    if (isExample && !isSpecialCommand) {
       checkArgument(mainClass != null, "Missing example class name.");
     }
 
@@ -421,7 +423,7 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
 
   private class OptionParser extends SparkSubmitOptionParser {
 
-    boolean requiresAppResource = true;
+    boolean isSpecialCommand = false;
     private final boolean errorOnUnknownArgs;
 
     OptionParser(boolean errorOnUnknownArgs) {
@@ -470,17 +472,14 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
           break;
         case KILL_SUBMISSION:
         case STATUS:
-          requiresAppResource = false;
+          isSpecialCommand = true;
           parsedArgs.add(opt);
           parsedArgs.add(value);
           break;
         case HELP:
         case USAGE_ERROR:
-          requiresAppResource = false;
-          parsedArgs.add(opt);
-          break;
         case VERSION:
-          requiresAppResource = false;
+          isSpecialCommand = true;
           parsedArgs.add(opt);
           break;
         default:
index 2e050f8..b343094 100644 (file)
@@ -18,6 +18,7 @@
 package org.apache.spark.launcher;
 
 import java.io.File;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -27,7 +28,10 @@ import java.util.regex.Pattern;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
 import static org.junit.Assert.*;
 
 public class SparkSubmitCommandBuilderSuite extends BaseSuite {
@@ -35,6 +39,9 @@ public class SparkSubmitCommandBuilderSuite extends BaseSuite {
   private static File dummyPropsFile;
   private static SparkSubmitOptionParser parser;
 
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
   @BeforeClass
   public static void setUp() throws Exception {
     dummyPropsFile = File.createTempFile("spark", "properties");
@@ -74,8 +81,11 @@ public class SparkSubmitCommandBuilderSuite extends BaseSuite {
 
   @Test
   public void testCliKillAndStatus() throws Exception {
-    testCLIOpts(parser.STATUS);
-    testCLIOpts(parser.KILL_SUBMISSION);
+    List<String> params = Arrays.asList("driver-20160531171222-0000");
+    testCLIOpts(null, parser.STATUS, params);
+    testCLIOpts(null, parser.KILL_SUBMISSION, params);
+    testCLIOpts(SparkSubmitCommandBuilder.RUN_EXAMPLE, parser.STATUS, params);
+    testCLIOpts(SparkSubmitCommandBuilder.RUN_EXAMPLE, parser.KILL_SUBMISSION, params);
   }
 
   @Test
@@ -190,6 +200,33 @@ public class SparkSubmitCommandBuilderSuite extends BaseSuite {
       env.get("SPARKR_SUBMIT_ARGS"));
   }
 
+  @Test(expected = IllegalArgumentException.class)
+  public void testExamplesRunnerNoArg() throws Exception {
+    List<String> sparkSubmitArgs = Arrays.asList(SparkSubmitCommandBuilder.RUN_EXAMPLE);
+    Map<String, String> env = new HashMap<>();
+    buildCommand(sparkSubmitArgs, env);
+  }
+
+  @Test
+  public void testExamplesRunnerNoMainClass() throws Exception {
+    testCLIOpts(SparkSubmitCommandBuilder.RUN_EXAMPLE, parser.HELP, null);
+    testCLIOpts(SparkSubmitCommandBuilder.RUN_EXAMPLE, parser.USAGE_ERROR, null);
+    testCLIOpts(SparkSubmitCommandBuilder.RUN_EXAMPLE, parser.VERSION, null);
+  }
+
+  @Test
+  public void testExamplesRunnerWithMasterNoMainClass() throws Exception {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage("Missing example class name.");
+
+    List<String> sparkSubmitArgs = Arrays.asList(
+      SparkSubmitCommandBuilder.RUN_EXAMPLE,
+      parser.MASTER + "=foo"
+    );
+    Map<String, String> env = new HashMap<>();
+    buildCommand(sparkSubmitArgs, env);
+  }
+
   @Test
   public void testExamplesRunner() throws Exception {
     List<String> sparkSubmitArgs = Arrays.asList(
@@ -344,10 +381,17 @@ public class SparkSubmitCommandBuilderSuite extends BaseSuite {
     return newCommandBuilder(args).buildCommand(env);
   }
 
-  private void testCLIOpts(String opt) throws Exception {
-    List<String> helpArgs = Arrays.asList(opt, "driver-20160531171222-0000");
+  private void testCLIOpts(String appResource, String opt, List<String> params) throws Exception {
+    List<String> args = new ArrayList<>();
+    if (appResource != null) {
+      args.add(appResource);
+    }
+    args.add(opt);
+    if (params != null) {
+      args.addAll(params);
+    }
     Map<String, String> env = new HashMap<>();
-    List<String> cmd = buildCommand(helpArgs, env);
+    List<String> cmd = buildCommand(args, env);
     assertTrue(opt + " should be contained in the final cmd.",
       cmd.contains(opt));
   }