Skip to content

Commit

Permalink
Reorganize the code so that only SparkLauncher is public.
Browse files Browse the repository at this point in the history
This gets rid of LauncherCommon and AbstractLauncher so that there's only
one public class (*) in the library. The other "launchers" were renamed
to match what they actually do (build commands), and extend the public
SparkLauncher (which actually can launch things).

Shell handling code was moved into the Main class, since that's the only
place that should care about shells.

CommandUtils is currently broken, since it still expects AbstractLauncher
to be around; that will be fixed separately. The (*) refers to
SparkSubmitOptionParser, which is still public, but will receive
a similar privatizing treatment soon.
  • Loading branch information
Marcelo Vanzin committed Feb 19, 2015
1 parent e50dc5e commit e4c80b6
Show file tree
Hide file tree
Showing 10 changed files with 546 additions and 591 deletions.
499 changes: 0 additions & 499 deletions launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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.spark.launcher;

import java.io.IOException;
import java.util.List;
import java.util.Map;

/**
* Internal interface that defines a command builder.
*/
interface CommandBuilder {

List<String> buildCommand(Map<String, String> env) throws IOException;

}
Original file line number Diff line number Diff line change
Expand Up @@ -23,32 +23,9 @@
import java.util.Map;

/**
* Configuration key definitions for Spark apps, and some helper methods.
* Helper methods for command builders.
*/
public class LauncherCommon {

/** The Spark master. */
public static final String SPARK_MASTER = "spark.master";

/** Configuration key for the driver memory. */
public static final String DRIVER_MEMORY = "spark.driver.memory";
/** Configuration key for the driver class path. */
public static final String DRIVER_EXTRA_CLASSPATH = "spark.driver.extraClassPath";
/** Configuration key for the driver VM options. */
public static final String DRIVER_EXTRA_JAVA_OPTIONS = "spark.driver.extraJavaOptions";
/** Configuration key for the driver native library path. */
public static final String DRIVER_EXTRA_LIBRARY_PATH = "spark.driver.extraLibraryPath";

/** Configuration key for the executor memory. */
public static final String EXECUTOR_MEMORY = "spark.executor.memory";
/** Configuration key for the executor class path. */
public static final String EXECUTOR_EXTRA_CLASSPATH = "spark.executor.extraClassPath";
/** Configuration key for the executor VM options. */
public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions";
/** Configuration key for the executor native library path. */
public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryOptions";
/** Configuration key for the number of executor CPU cores. */
public static final String EXECUTOR_CORES = "spark.executor.cores";
class CommandBuilderUtils {

/** Returns whether the given string is null or empty. */
static boolean isEmpty(String s) {
Expand Down Expand Up @@ -244,7 +221,4 @@ static void checkState(boolean check, String msg, Object... args) {
}
}

// To avoid subclassing outside this package.
LauncherCommon() { }

}
108 changes: 97 additions & 11 deletions launcher/src/main/java/org/apache/spark/launcher/Main.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,16 @@

import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

import static org.apache.spark.launcher.CommandBuilderUtils.*;

/**
* Command line interface for the Spark launcher. Used internally by Spark scripts.
*/
class Main extends LauncherCommon {
class Main {

/**
* Usage: Main [class] [class args]
Expand All @@ -51,36 +54,119 @@ public static void main(String[] argsArray) throws Exception {
String className = args.remove(0);

boolean printLaunchCommand;
AbstractLauncher<?> launcher;
CommandBuilder builder;
try {
if (className.equals("org.apache.spark.deploy.SparkSubmit")) {
launcher = new SparkSubmitCliLauncher(args);
builder = new SparkSubmitCommandBuilder(args);
} else {
launcher = new SparkClassLauncher(className, args);
builder = new SparkClassLauncher(className, args);
}
printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND"));
} catch (IllegalArgumentException e) {
launcher = new UsageLauncher();
builder = new UsageLauncher();
printLaunchCommand = false;
}

List<String> cmd = launcher.buildShellCommand();
Map<String, String> env = new HashMap<String, String>();
List<String> cmd = builder.buildCommand(env);
if (printLaunchCommand) {
System.err.println("Spark Command: " + join(" ", cmd));
System.err.println("========================================");
}

if (isWindows()) {
String cmdLine = join(" ", cmd);
System.out.println(cmdLine);
List<String> winCmd = prepareForWindows(cmd, env);
System.out.println(join(" ", cmd));
} else {
for (String c : cmd) {
List<String> bashCmd = prepareForBash(cmd, env);
for (String c : bashCmd) {
System.out.print(c);
System.out.print('\0');
}
}
}

/**
* Prepare a command line for execution from a Windows batch script.
*
* Two things need to be done:
*
* - If a custom library path is needed, extend PATH to add it. Based on:
* http://superuser.com/questions/223104/setting-environment-variable-for-just-one-command-in-windows-cmd-exe
*
* - Quote all arguments so that spaces are handled as expected. Quotes within arguments are
* "double quoted" (which is batch for escaping a quote). This page has more details about
* quoting and other batch script fun stuff: http://ss64.com/nt/syntax-esc.html
*
* The command is executed using "cmd /c" and formatted as single line, since that's the
* easiest way to consume this from a batch script (see spark-class2.cmd).
*/
private static List<String> prepareForWindows(List<String> cmd, Map<String, String> childEnv) {
StringBuilder cmdline = new StringBuilder("cmd /c \"");
for (Map.Entry<String, String> e : childEnv.entrySet()) {
if (cmdline.length() > 0) {
cmdline.append(" ");
}
cmdline.append(String.format("set %s=%s", e.getKey(), e.getValue()));
cmdline.append(" &&");
}
for (String arg : cmd) {
if (cmdline.length() > 0) {
cmdline.append(" ");
}
cmdline.append(quoteForBatchScript(arg));
}
cmdline.append("\"");
return Arrays.asList(cmdline.toString());
}

/**
* Prepare the command for execution from a bash script. The final command will have commands to
* set up any needed environment variables needed by the child process.
*/
private static List<String> prepareForBash(List<String> cmd, Map<String, String> childEnv) {
if (childEnv.isEmpty()) {
return cmd;
}

List<String> newCmd = new ArrayList<String>();
newCmd.add("env");

for (Map.Entry<String, String> e : childEnv.entrySet()) {
newCmd.add(String.format("%s=%s", e.getKey(), e.getValue()));
}
newCmd.addAll(cmd);
return newCmd;
}

/**
* Quote a command argument for a command to be run by a Windows batch script, if the argument
* needs quoting. Arguments only seem to need quotes in batch scripts if they have whitespace.
*/
private static String quoteForBatchScript(String arg) {
boolean needsQuotes = false;
for (int i = 0; i < arg.length(); i++) {
if (Character.isWhitespace(arg.codePointAt(i))) {
needsQuotes = true;
break;
}
}
if (!needsQuotes) {
return arg;
}
StringBuilder quoted = new StringBuilder();
quoted.append("\"");
for (int i = 0; i < arg.length(); i++) {
int cp = arg.codePointAt(i);
if (cp == '\"') {
quoted.append("\"");
}
quoted.appendCodePoint(cp);
}
quoted.append("\"");
return quoted.toString();
}

/**
* Internal launcher used when command line parsing fails. This will behave differently depending
* on the platform:
Expand All @@ -94,10 +180,10 @@ public static void main(String[] argsArray) throws Exception {
* should check for this variable and print its usage, since batch scripts don't really support
* the "export -f" functionality used in bash.
*/
private static class UsageLauncher extends AbstractLauncher<UsageLauncher> {
private static class UsageLauncher implements CommandBuilder {

@Override
List<String> buildLauncherCommand(Map<String, String> env) {
public List<String> buildCommand(Map<String, String> env) {
if (isWindows()) {
return Arrays.asList("set SPARK_LAUNCHER_USAGE_ERROR=1");
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,13 +24,15 @@
import java.util.Map;
import java.util.regex.Pattern;

import static org.apache.spark.launcher.CommandBuilderUtils.*;

/**
* Launcher for internal Spark classes.
* <p/>
* This class handles building the command to launch all internal Spark classes except for
* SparkSubmit (which is handled by the public {@link SparkLauncher} class.
*/
class SparkClassLauncher extends AbstractLauncher<SparkClassLauncher> {
class SparkClassLauncher extends SparkLauncher implements CommandBuilder {

private final String className;
private final List<String> classArgs;
Expand All @@ -41,7 +43,7 @@ class SparkClassLauncher extends AbstractLauncher<SparkClassLauncher> {
}

@Override
List<String> buildLauncherCommand(Map<String, String> env) throws IOException {
public List<String> buildCommand(Map<String, String> env) throws IOException {
List<String> javaOptsKeys = new ArrayList<String>();
String memKey = null;
String extraClassPath = null;
Expand Down Expand Up @@ -89,7 +91,7 @@ List<String> buildLauncherCommand(Map<String, String> env) throws IOException {
javaOptsKeys.add("SPARK_JAVA_OPTS");
} else {
// Any classes not explicitly listed above are submitted using SparkSubmit.
return buildSparkSubmitCommand(env);
return createSparkSubmitCommand(env);
}

List<String> cmd = buildJavaCommand(extraClassPath);
Expand All @@ -106,14 +108,14 @@ List<String> buildLauncherCommand(Map<String, String> env) throws IOException {
return cmd;
}

private List<String> buildSparkSubmitCommand(Map<String, String> env) throws IOException {
private List<String> createSparkSubmitCommand(Map<String, String> env) throws IOException {
List<String> sparkSubmitArgs = new ArrayList<String>(classArgs);
sparkSubmitArgs.add(SparkSubmitOptionParser.CLASS);
sparkSubmitArgs.add(className);

SparkSubmitCliLauncher launcher = new SparkSubmitCliLauncher(true, sparkSubmitArgs);
launcher.setAppResource("spark-internal");
return launcher.buildLauncherCommand(env);
SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(true, sparkSubmitArgs);
builder.setAppResource("spark-internal");
return builder.buildCommand(env);
}

}
Loading

0 comments on commit e4c80b6

Please sign in to comment.