From 6f70eeaf7fdef89d54b45ec38eb0f7220596e537 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 6 Jan 2015 13:25:34 -0800 Subject: [PATCH 01/51] [SPARK-4924] Add a library for launching Spark jobs programatically. This change encapsulates all the logic involved in launching a Spark job into a small Java library that can be easily embedded into other applications. Only the `SparkLauncher` class is supposed to be public in the new launcher lib, but to be able to use those classes elsewhere in Spark some of the visibility modifiers were relaxed. This allows us to automate some checks in unit tests, when before you just had a comment that was easily missed. A subsequent commit will change Spark core and all the shell scripts to use this library instead of custom code that needs to be replicated for different OSes and, sometimes, also in Spark code. --- launcher/pom.xml | 77 +++ .../spark/launcher/AbstractLauncher.java | 590 ++++++++++++++++++ .../apache/spark/launcher/LauncherCommon.java | 120 ++++ .../java/org/apache/spark/launcher/Main.java | 106 ++++ .../spark/launcher/PySparkLauncher.java | 90 +++ .../spark/launcher/SparkClassLauncher.java | 107 ++++ .../apache/spark/launcher/SparkLauncher.java | 371 +++++++++++ .../launcher/SparkSubmitCliLauncher.java | 196 ++++++ .../spark/launcher/AbstractLauncherSuite.java | 89 +++ .../spark/launcher/SparkLauncherSuite.java | 281 +++++++++ .../launcher/SparkSubmitCliLauncherSuite.java | 71 +++ launcher/src/test/resources/log4j.properties | 31 + make-distribution.sh | 1 + pom.xml | 1 + 14 files changed, 2131 insertions(+) create mode 100644 launcher/pom.xml create mode 100644 launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/Main.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java create mode 100644 launcher/src/test/java/org/apache/spark/launcher/AbstractLauncherSuite.java create mode 100644 launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java create mode 100644 launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java create mode 100644 launcher/src/test/resources/log4j.properties diff --git a/launcher/pom.xml b/launcher/pom.xml new file mode 100644 index 000000000000..270689681fc1 --- /dev/null +++ b/launcher/pom.xml @@ -0,0 +1,77 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.3.0-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-launcher_2.10 + jar + Spark Launcher Project + http://spark.apache.org/ + + launcher + + + + + + log4j + log4j + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.slf4j + slf4j-api + test + + + org.slf4j + slf4j-log4j12 + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + ${project.basedir}/.. + + + + + + diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java new file mode 100644 index 000000000000..7cafcf410b97 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -0,0 +1,590 @@ +/* + * 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.BufferedReader; +import java.io.File; +import java.io.FileFilter; +import java.io.FileInputStream; +import java.io.InputStreamReader; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.jar.JarEntry; +import java.util.jar.JarFile; +import java.util.regex.Pattern; + +/** + * Basic functionality for launchers. + */ +public abstract class AbstractLauncher extends LauncherCommon { + + protected static final String DEFAULT_MEM = "512m"; + + protected String javaHome; + protected String sparkHome; + protected String propertiesFile; + protected final Map conf = new HashMap(); + private final Map env; + + protected AbstractLauncher() { + this(null); + } + + protected AbstractLauncher(Map env) { + this.env = env; + } + + @SuppressWarnings("unchecked") + private final T THIS = (T) this; + + /** Set a custom JAVA_HOME for launching the Spark application. */ + public T setJavaHome(String path) { + checkNotNull(path, "path"); + this.javaHome = path; + return THIS; + } + + /** Set a custom Spark installation location for the application. */ + public T setSparkHome(String path) { + checkNotNull(path, "path"); + this.sparkHome = path; + return THIS; + } + + /** Set a custom properties file with Spark configuration for the application. */ + public T setPropertiesFile(String path) { + checkNotNull(path, "path"); + this.propertiesFile = path; + return THIS; + } + + /** Set a single configuration value for the application. */ + public T setConf(String key, String value) { + checkNotNull(key, "key"); + checkNotNull(value, "value"); + checkArgument(key.startsWith("spark."), "'key' must start with 'spark.'"); + conf.put(key, value); + return THIS; + } + + /** + * Launchers should implement this to create the command to be executed. + */ + protected abstract List buildLauncherCommand() throws IOException; + + protected Properties loadPropertiesFile() throws IOException { + Properties props = new Properties(); + File propsFile; + if (propertiesFile != null) { + propsFile = new File(propertiesFile); + checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile); + } else { + String confDir = getenv("SPARK_CONF_DIR"); + if (confDir == null) { + confDir = join(File.separator, getSparkHome(), "conf"); + } + propsFile = new File(confDir, "spark-defaults.conf"); + } + + if (propsFile.isFile()) { + FileInputStream fd = null; + try { + fd = new FileInputStream(propsFile); + props.load(new InputStreamReader(fd, "UTF-8")); + } finally { + if (fd != null) { + try { + fd.close(); + } catch (IOException e) { + // Ignore. + } + } + } + } + + return props; + } + + protected String getSparkHome() { + String path = first(sparkHome, getenv("SPARK_HOME")); + checkState(path != null, + "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); + return path; + } + + protected List createJavaCommand() throws IOException { + List cmd = new ArrayList(); + if (javaHome == null) { + cmd.add(join(File.separator, System.getProperty("java.home"), "..", "bin", "java")); + } else { + cmd.add(join(File.separator, javaHome, "bin", "java")); + } + + // Don't set MaxPermSize for Java 8 and later. + String[] version = System.getProperty("java.version").split("\\."); + if (Integer.parseInt(version[0]) == 1 && Integer.parseInt(version[1]) < 8) { + cmd.add("-XX:MaxPermSize=128m"); + } + + // Load extra JAVA_OPTS from conf/java-opts, if it exists. + File javaOpts = new File(join(File.separator, getSparkHome(), "conf", "java-opts")); + if (javaOpts.isFile()) { + BufferedReader br = new BufferedReader(new InputStreamReader( + new FileInputStream(javaOpts), "UTF-8")); + try { + String line; + while ((line = br.readLine()) != null) { + addOptionString(cmd, line); + } + } finally { + br.close(); + } + } + + return cmd; + } + + protected void addOptionString(List cmd, String options) { + if (!isEmpty(options)) { + for (String opt : parseOptionString(options)) { + cmd.add(opt); + } + } + } + + /** + * Builds the classpath for the application. Returns a list with one classpath entry per element; + * each entry is formatted in the way expected by java.net.URLClassLoader (more + * specifically, with trailing slashes for directories). + */ + protected List buildClassPath(String appClassPath) throws IOException { + String sparkHome = getSparkHome(); + String scala = getScalaVersion(); + + List cp = new ArrayList(); + addToClassPath(cp, getenv("SPARK_CLASSPATH")); + addToClassPath(cp, appClassPath); + + String confDir = getenv("SPARK_CONF_DIR"); + if (!isEmpty(confDir)) { + addToClassPath(cp, confDir); + } else { + addToClassPath(cp, join(File.separator, getSparkHome(), "conf")); + } + + boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); + boolean isTesting = !isEmpty(getenv("SPARK_TESTING")); + if (prependClasses || isTesting) { + System.err.println( + "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + + "assembly."); + List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", + "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", + "yarn", "launcher"); + if (prependClasses) { + for (String project : projects) { + addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project, + scala)); + } + } + if (isTesting) { + for (String project : projects) { + addToClassPath(cp, String.format("%s/%s/target/scala-%s/test-classes", sparkHome, + project, scala)); + } + } + addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); + } + + String assembly = findAssembly(scala); + addToClassPath(cp, assembly); + + // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus + // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt + // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built + // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark + // assembly is built for Hive, before actually populating the CLASSPATH with the jars. + // + // This block also serves as a check for SPARK-1703, when the assembly jar is built with + // Java 7 and ends up with too many files, causing issues with other JDK versions. + boolean needsDataNucleus = false; + JarFile assemblyJar = null; + try { + assemblyJar = new JarFile(assembly); + needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; + } catch (IOException ioe) { + if (ioe.getMessage().indexOf("invalid CEN header") > 0) { + System.err.println( + "Loading Spark jar with '$JAR_CMD' failed.\n" + + "This is likely because Spark was compiled with Java 7 and run\n" + + "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + + "or build Spark with Java 6."); + System.exit(1); + } else { + throw ioe; + } + } finally { + if (assemblyJar != null) { + try { + assemblyJar.close(); + } catch (IOException e) { + // Ignore. + } + } + } + + if (needsDataNucleus) { + System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " + + "in classpath."); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + } else { + libdir = new File(sparkHome, "lib_managed/jars"); + } + + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + for (File jar : libdir.listFiles()) { + if (jar.getName().startsWith("datanucleus-")) { + addToClassPath(cp, jar.getAbsolutePath()); + } + } + } + + addToClassPath(cp, getenv("HADOOP_CONF_DIR")); + addToClassPath(cp, getenv("YARN_CONF_DIR")); + return cp; + } + + private void addToClassPath(List cp, String entries) { + if (isEmpty(entries)) { + return; + } + String[] split = entries.split(Pattern.quote(File.pathSeparator)); + for (String entry : split) { + if (!isEmpty(entry)) { + if (new File(entry).isDirectory() && !entry.endsWith(File.separator)) { + entry += File.separator; + } + cp.add(entry); + } + } + } + + protected String getScalaVersion() { + String scala = getenv("SPARK_SCALA_VERSION"); + if (scala != null) { + return scala; + } + + String sparkHome = getSparkHome(); + File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); + File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); + if (scala210.isDirectory() && scala211.isDirectory()) { + checkState(false, + "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); + } else if (scala210.isDirectory()) { + return "2.10"; + } else { + checkState(scala211.isDirectory(), "Cannot find any assembly build directories."); + return "2.11"; + } + + throw new IllegalStateException("Should not reach here."); + } + + /** + * Which OS is running defines two things: + * - the name of the environment variable used to define the lookup path for native libs + * - how to execute the command in general. + * + * The name is easy: PATH on Win32, DYLD_LIBRARY_PATH on MacOS, LD_LIBRARY_PATH elsewhere. + * + * On Unix-like, we're assuming bash is available. So we print one argument per line to + * the output, and use bash's array handling to execute the right thing. + * + * For Win32, see {@link #prepareForWindows(List,String)}. + */ + protected List prepareForOs(List cmd, + String libPath, + Map env) { + if (isWindows()) { + return prepareForWindows(cmd, libPath, env); + } + + if (isEmpty(libPath) && env.isEmpty()) { + return cmd; + } + + List newCmd = new ArrayList(); + newCmd.add("env"); + + if (!isEmpty(libPath)) { + String envName = getLibPathEnvName(); + String currEnvValue = getenv(envName); + String newEnvValue = join(File.pathSeparator, currEnvValue, libPath); + newCmd.add(String.format("%s=%s", envName, newEnvValue)); + } + for (Map.Entry e : env.entrySet()) { + newCmd.add(String.format("%s=%s", e.getKey(), e.getValue())); + } + newCmd.addAll(cmd); + return newCmd; + } + + protected String shQuote(String s) { + StringBuilder quoted = new StringBuilder(); + boolean hasWhitespace = false; + for (int i = 0; i < s.length(); i++) { + if (Character.isWhitespace(s.codePointAt(i))) { + quoted.append('"'); + hasWhitespace = true; + break; + } + } + + for (int i = 0; i < s.length(); i++) { + int cp = s.codePointAt(i); + switch (cp) { + case '\'': + if (hasWhitespace) { + quoted.appendCodePoint(cp); + break; + } + case '"': + case '\\': + quoted.append('\\'); + // Fall through. + default: + if (Character.isWhitespace(cp)) { + hasWhitespace=true; + } + quoted.appendCodePoint(cp); + } + } + if (hasWhitespace) { + quoted.append('"'); + } + return quoted.toString(); + } + + // Visible for testing. + List parseOptionString(String s) { + List opts = new ArrayList(); + StringBuilder opt = new StringBuilder(); + boolean inOpt = false; + boolean inSingleQuote = false; + boolean inDoubleQuote = false; + boolean escapeNext = false; + boolean hasData = false; + + for (int i = 0; i < s.length(); i++) { + int c = s.codePointAt(i); + if (escapeNext) { + if (!inOpt) { + inOpt = true; + } + opt.appendCodePoint(c); + escapeNext = false; + } else if (inOpt) { + switch (c) { + case '\\': + if (inSingleQuote) { + opt.appendCodePoint(c); + } else { + escapeNext = true; + } + break; + case '\'': + if (inDoubleQuote) { + opt.appendCodePoint(c); + } else { + inSingleQuote = !inSingleQuote; + } + break; + case '"': + if (inSingleQuote) { + opt.appendCodePoint(c); + } else { + inDoubleQuote = !inDoubleQuote; + } + break; + default: + if (inSingleQuote || inDoubleQuote || !Character.isWhitespace(c)) { + opt.appendCodePoint(c); + } else { + finishOpt(opts, opt); + inOpt = false; + hasData = false; + } + } + } else { + switch (c) { + case '\'': + inSingleQuote = true; + inOpt = true; + hasData = true; + break; + case '"': + inDoubleQuote = true; + inOpt = true; + hasData = true; + break; + case '\\': + escapeNext = true; + break; + default: + if (!Character.isWhitespace(c)) { + inOpt = true; + opt.appendCodePoint(c); + } + } + } + } + + checkArgument(!inSingleQuote && !inDoubleQuote && !escapeNext, "Invalid option string: %s", s); + if (opt.length() > 0 || hasData) { + opts.add(opt.toString()); + } + return opts; + } + + private void finishOpt(List opts, StringBuilder opt) { + opts.add(opt.toString()); + opt.setLength(0); + } + + private String findAssembly(String scalaVersion) { + String sparkHome = getSparkHome(); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + } else { + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion)); + } + + final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); + FileFilter filter = new FileFilter() { + @Override + public boolean accept(File file) { + return file.isFile() && re.matcher(file.getName()).matches(); + } + }; + File[] assemblies = libdir.listFiles(filter); + checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); + checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); + return assemblies[0].getAbsolutePath(); + } + + private String getenv(String key) { + return (env != null) ? env.get(key) : System.getenv(key); + } + + /** + * Prepare a command line for execution on Windows. + * + * 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 + */ + private List prepareForWindows(List cmd, + String libPath, + Map env) { + StringBuilder cmdline = new StringBuilder("cmd /c \""); + if (libPath != null) { + cmdline.append("set PATH=%PATH%;").append(libPath).append(" &&"); + } + for (Map.Entry e : env.entrySet()) { + 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(quote(arg)); + } + cmdline.append("\""); + return Arrays.asList(cmdline.toString()); + } + + /** + * Quoting arguments that don't need quoting in Windows seems to cause weird issues. So only + * quote arguments when there is whitespace in them. + */ + private boolean needsQuoting(String arg) { + for (int i = 0; i < arg.length(); i++) { + if (Character.isWhitespace(arg.codePointAt(i))) { + return true; + } + } + return false; + } + + private String quote(String arg) { + if (!needsQuoting(arg)) { + 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(); + } + + // Visible for testing. + String getLibPathEnvName() { + if (isWindows()) { + return "PATH"; + } + + String os = System.getProperty("os.name"); + if (os.startsWith("Mac OS X")) { + return "DYLD_LIBRARY_PATH"; + } else { + return "LD_LIBRARY_PATH"; + } + } + + protected boolean isWindows() { + String os = System.getProperty("os.name"); + return os.startsWith("Windows"); + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java new file mode 100644 index 000000000000..004c43019592 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java @@ -0,0 +1,120 @@ +/* + * 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.util.Map; + +/** + * Configuration key definitions for Spark jobs, and some helper methods. + */ +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_CLASSPATH = "spark.driver.extraClassPath"; + /** Configuration key for the driver VM options. */ + public static final String DRIVER_JAVA_OPTIONS = "spark.driver.extraJavaOptions"; + /** Configuration key for the driver native library path. */ + public static final String DRIVER_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_CLASSPATH = "spark.executor.extraClassPath"; + /** Configuration key for the executor VM options. */ + public static final String EXECUTOR_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; + /** Configuration key for the executor native library path. */ + public static final String EXECUTOR_LIBRARY_PATH = "spark.executor.extraLibraryOptions"; + /** Configuration key for the number of executor CPU cores. */ + public static final String EXECUTOR_CORES = "spark.executor.cores"; + + protected static boolean isEmpty(String s) { + return s == null || s.isEmpty(); + } + + protected static String join(String sep, String... elements) { + StringBuilder sb = new StringBuilder(); + for (String e : elements) { + if (e != null) { + if (sb.length() > 0) { + sb.append(sep); + } + sb.append(e); + } + } + return sb.toString(); + } + + protected static String join(String sep, Iterable elements) { + StringBuilder sb = new StringBuilder(); + for (String e : elements) { + if (e != null) { + if (sb.length() > 0) { + sb.append(sep); + } + sb.append(e); + } + } + return sb.toString(); + } + + protected static String find(String key, Map... maps) { + for (Map map : maps) { + String value = (String) map.get(key); + if (!isEmpty(value)) { + return value; + } + } + return null; + } + + protected static String first(String... candidates) { + for (String s : candidates) { + if (!isEmpty(s)) { + return s; + } + } + return null; + } + + protected static void checkNotNull(Object o, String arg) { + if (o == null) { + throw new IllegalArgumentException(String.format("'%s' must not be null.", arg)); + } + } + + protected static void checkArgument(boolean check, String msg, Object... args) { + if (!check) { + throw new IllegalArgumentException(String.format(msg, args)); + } + } + + protected static void checkState(boolean check, String msg, Object... args) { + if (!check) { + throw new IllegalStateException(String.format(msg, args)); + } + } + + // To avoid subclassing outside this package. + LauncherCommon() { } + +} \ No newline at end of file diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java new file mode 100644 index 000000000000..497c738614b6 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -0,0 +1,106 @@ +/* + * 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.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Command line interface for the Spark launcher. Used internally by Spark scripts. + */ +public class Main extends LauncherCommon { + + /** + * Usage: Main [class] [class args] + *

+ * This CLI works in two different modes: + *

    + *
  • "spark-submit": if class is "org.apache.spark.deploy.SparkSubmit", the + * {@link SparkLauncher} class is used to launch a Spark application.
  • + *
  • "spark-class": if another class is provided, an internal Spark class is run.
  • + *
+ * + * The ultimate command will not be run in the same process. Instead, the command to be executed + * will be printed to stdout. On Unix systems, this will be one argument per line. On Windows + * systems, this will be a single line containing the command to be executed. + */ + public static void main(String[] argsArray) throws Exception { + checkArgument(argsArray.length > 0, "Not enough arguments: missing class name."); + + List args = new ArrayList(Arrays.asList(argsArray)); + String className = args.remove(0); + + boolean printLaunchCommand = false; + AbstractLauncher launcher; + try { + if (className.equals("org.apache.spark.deploy.SparkSubmit")) { + launcher = new SparkSubmitCliLauncher(args); + printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); + } else if (className.equals("pyspark")) { + launcher = new PySparkLauncher(args); + } else { + launcher = new SparkClassLauncher(className, args); + printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); + } + } catch (IllegalArgumentException e) { + launcher = new UsageLauncher(); + } + + List cmd = launcher.buildLauncherCommand(); + if (printLaunchCommand) { + System.err.println("Spark Command: " + join(" ", cmd)); + System.err.println("========================================"); + } + + for (String c : cmd) { + System.out.println(c); + } + } + + /** + * Internal launcher used when command line parsing fails. This will behave differently depending + * on the platform: + * + * - On Unix-like systems, it will print a call to the "usage" function with argument "1". The + * function is expected to print the command's usage and exit with the provided exit code. + * The script should use "export -f usage" after declaring a function called "usage", so that + * the function is available to downstream scripts. + * + * - On Windows it will set the variable "SPARK_LAUNCHER_USAGE_ERROR" to "1". The batch script + * 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 { + + @Override + protected List buildLauncherCommand() { + if (isWindows()) { + return Arrays.asList("set SPARK_LAUNCHER_USAGE_ERROR=1"); + } else { + return Arrays.asList("usage 1"); + } + } + + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java new file mode 100644 index 000000000000..6786f395990d --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java @@ -0,0 +1,90 @@ +/* + * 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.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Launcher for PySpark. + *

+ * Handles parsing command line options passed to the pyspark script. This allows + * sharing that logic with other launchers, to keep them in sync. + */ +class PySparkLauncher extends AbstractLauncher { + + private final List args; + + PySparkLauncher(List args) { + this.args = args; + } + + @Override + protected List buildLauncherCommand() throws IOException { + SparkSubmitCliLauncher launcher = new SparkSubmitCliLauncher(true, args); + + // For backwards compatibility, if a script is specified in + // the pyspark command line, then run it using spark-submit. + if (!launcher.getArgs().isEmpty() && launcher.getArgs().get(0).endsWith(".py")) { + System.err.println( + "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + + "Use ./bin/spark-submit "); + return launcher.buildLauncherCommand(); + } + + // When launching the pyspark shell, the spark-submit arguments should be stored in the + // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable + // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS. + checkArgument(launcher.getArgs().isEmpty(), + "pyspark does not support any application options."); + + Properties props = loadPropertiesFile(); + String libPath = find(DRIVER_LIBRARY_PATH, conf, props); + + StringBuilder submitArgs = new StringBuilder(); + for (String arg : launcher.getSparkArgs()) { + if (submitArgs.length() > 0) { + submitArgs.append(" "); + } + submitArgs.append(shQuote(arg)); + } + for (String arg : launcher.getDriverArgs()) { + if (submitArgs.length() > 0) { + submitArgs.append(" "); + } + submitArgs.append(shQuote(arg)); + } + + Map env = new HashMap(); + env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); + + List pyargs = new ArrayList(); + pyargs.add(System.getenv("PYSPARK_DRIVER_PYTHON")); + String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); + if (!isEmpty(pyOpts)) { + pyargs.addAll(parseOptionString(pyOpts)); + } + + return prepareForOs(pyargs, libPath, env); + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java new file mode 100644 index 000000000000..d8e9571488f5 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.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.spark.launcher; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.regex.Pattern; + +/** + * Launcher for internal Spark classes. + *

+ * 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 { + + private final String className; + private final List classArgs; + + SparkClassLauncher(String className, List classArgs) { + this.className = className; + this.classArgs = classArgs; + } + + @Override + protected List buildLauncherCommand() throws IOException { + List cmd = createJavaCommand(); + + List javaOptsKeys = new ArrayList(); + String memKey = null; + String extraClassPath = null; + + // Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + + // SPARK_DAEMON_MEMORY. + if (className.equals("org.apache.spark.deploy.master.Master")) { + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_MASTER_OPTS"); + memKey = "SPARK_DAEMON_MEMORY"; + } else if (className.equals("org.apache.spark.deploy.worker.Worker")) { + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_WORKER_OPTS"); + memKey = "SPARK_DAEMON_MEMORY"; + } else if (className.equals("org.apache.spark.deploy.history.HistoryServer")) { + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_HISTORY_OPTS"); + memKey = "SPARK_DAEMON_MEMORY"; + } else if (className.equals("org.apache.spark.executor.CoarseGrainedExecutorBackend")) { + javaOptsKeys.add("SPARK_JAVA_OPTS"); + javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); + memKey = "SPARK_EXECUTOR_MEMORY"; + } else if (className.equals("org.apache.spark.executor.MesosExecutorBackend")) { + javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); + memKey = "SPARK_EXECUTOR_MEMORY"; + } else if (className.startsWith("org.apache.spark.tools.")) { + String sparkHome = getSparkHome(); + File toolsDir = new File(join(sparkHome, "tools", "target", + "scala-" + getScalaVersion())); + checkState(toolsDir.isDirectory(), "Cannot find tools build directory."); + + Pattern re = Pattern.compile("spark-tools-.*\\.jar"); + for (File f : toolsDir.listFiles()) { + if (re.matcher(f.getName()).matches()) { + extraClassPath = f.getAbsolutePath(); + break; + } + } + + checkState(extraClassPath != null, + "Failed to find Spark Tools Jar in %s. You need to build Spark before running %s.", + toolsDir.getAbsolutePath(), className); + + javaOptsKeys.add("SPARK_JAVA_OPTS"); + } + + for (String key : javaOptsKeys) { + addOptionString(cmd, System.getenv(key)); + } + + String mem = first(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM); + cmd.add("-Xms" + mem); + cmd.add("-Xmx" + mem); + cmd.add("-cp"); + cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); + cmd.add(className); + cmd.addAll(classArgs); + return prepareForOs(cmd, null, Collections.emptyMap()); + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java new file mode 100644 index 000000000000..3b3405eed2ef --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -0,0 +1,371 @@ +/* + * 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.File; +import java.io.IOException; +import java.lang.reflect.Method; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Launcher for Spark applications. + *

+ * Use this class to start Spark applications programatically. The class uses a builder pattern + * to allow clients to configure the Spark application and launch it as a child process. + *

+ * There's also support for running the application on a separate thread, although that is to + * be considered experimental and avoided in production environments. + *

+ * Note that launching Spark applications using this class will not automatically load environment + * variables from the "spark-env.sh" or "spark-env.cmd" scripts in the configuration directory. + */ +public class SparkLauncher extends AbstractLauncher { + + private static final AtomicLong THREAD_ID = new AtomicLong(); + + protected boolean verbose; + protected String appName; + protected String master; + protected String deployMode; + protected String userClass; + protected String userResource; + protected final List sparkArgs; + protected final List userArgs; + protected final List jars; + protected final List files; + protected final List pyFiles; + + public SparkLauncher() { + this.sparkArgs = new ArrayList(); + this.userArgs = new ArrayList(); + this.jars = new ArrayList(); + this.files = new ArrayList(); + this.pyFiles = new ArrayList(); + } + + /** Set the application name. */ + public SparkLauncher setAppName(String appName) { + checkNotNull(appName, "appName"); + this.appName = appName; + return this; + } + + /** Set the Spark master for the application. */ + public SparkLauncher setMaster(String master) { + checkNotNull(master, "master"); + this.master = master; + return this; + } + + /** Set the deploy mode for the application. */ + public SparkLauncher setDeployMode(String mode) { + checkNotNull(mode, "mode"); + this.deployMode = mode; + return this; + } + + /** + * Set the main application resource. This should be the location of a jar file for Scala/Java + * applications, or a python script for PySpark applications. + */ + public SparkLauncher setAppResource(String path) { + checkNotNull(path, "path"); + this.userResource = path; + return this; + } + + /** Sets the application class name for Java/Scala applications. */ + public SparkLauncher setClass(String userClass) { + checkNotNull(userClass, "userClass"); + this.userClass = userClass; + return this; + } + + /** Adds command line arguments for the application. */ + public SparkLauncher addArgs(String... args) { + for (String arg : args) { + checkNotNull(arg, "arg"); + userArgs.add(arg); + } + return this; + } + + /** Adds a jar file to be submitted with the application. */ + public SparkLauncher addJar(String path) { + checkNotNull(path, "path"); + jars.add(path); + return this; + } + + /** Adds a file to be submitted with the application. */ + public SparkLauncher addFile(String path) { + checkNotNull(path, "path"); + files.add(path); + return this; + } + + /** Adds a a python file / zip / egg to be submitted with the application. */ + public SparkLauncher addPyFile(String path) { + checkNotNull(path, "path"); + pyFiles.add(path); + return this; + } + + /** Enables verbose reporting for SparkSubmit. */ + public SparkLauncher setVerbose(boolean verbose) { + this.verbose = verbose; + return this; + } + + /** + * Starts a new thread that will run the Spark application. + *

+ * The application will run on a separate thread and use a separate, isolated class loader. + * No classes or resources from the current thread's class loader will be visible to the app. + *

+ * This mode does not support certain configuration parameters, like configuring the amount of + * driver memory or custom driver command line options. If such configuration is detected, an + * exception will be thrown. + *

+ * This is extremely experimental and should not be used in production environments. + *

+ * NOTE: SparkSubmit uses system properties to propagate some configuration value to the app + * are run concurrently, they may affect each other's configurations. + *

+ * NOTE: for users running JDK versions older than 8, this option can add a lot of overhead + * to the VM's perm gen. + * + * @param exceptionHandler Optional handler for handling exceptions in the app thread. + * @param daemon Whether to start a daemon thread. + * @return A non-daemon thread that will run the application using SparkSubmit. The thread will + * already be started. + */ + public Thread start(Thread.UncaughtExceptionHandler handler, boolean daemon) throws IOException { + // Do some sanity checking that incompatible driver options are not used, because they + // cannot be set in this mode. + Properties props = loadPropertiesFile(); + String extraClassPath = null; + if (isRunningDriver(props)) { + checkState( + find(DRIVER_JAVA_OPTIONS, conf, props) == null, + "Cannot set driver VM options when running in-process."); + checkState( + find(DRIVER_LIBRARY_PATH, conf, props) == null, + "Cannot set native library path when running in-process."); + checkState( + find(DRIVER_MEMORY, conf, props) == null, + "Cannot set driver memory when running in-process."); + extraClassPath = find(DRIVER_CLASSPATH, conf, props); + } + + List cp = buildClassPath(extraClassPath); + URL[] cpUrls = new URL[cp.size()]; + int idx = 0; + for (String entry : cp) { + cpUrls[idx++] = new File(entry).toURI().toURL(); + } + + URLClassLoader cl = new URLClassLoader(cpUrls, null); + + Thread appThread; + try { + Class sparkSubmit = cl.loadClass("org.apache.spark.deploy.SparkSubmit"); + Method main = sparkSubmit.getDeclaredMethod("main", String[].class); + List args = buildSparkSubmitArgs(); + appThread = new Thread(new SparkSubmitRunner(main, args)); + } catch (ClassNotFoundException cnfe) { + throw new IOException(cnfe); + } catch (NoSuchMethodException nsme) { + throw new IOException(nsme); + } + + appThread.setName("SparkLauncher-Submit-" + THREAD_ID.incrementAndGet()); + appThread.setContextClassLoader(cl); + if (handler != null) { + appThread.setUncaughtExceptionHandler(handler); + } + appThread.setDaemon(daemon); + appThread.start(); + return appThread; + } + + /** + * Launches a sub-process that will start the configured Spark application. + * + * @return A process handle for the Spark app. + */ + public Process launch() throws IOException { + List cmd = buildLauncherCommand(); + return Runtime.getRuntime().exec(cmd.toArray(new String[cmd.size()])); + } + + SparkLauncher addSparkArgs(String... args) { + for (String arg : args) { + sparkArgs.add(arg); + } + return this; + } + + // Visible for testing. + List buildSparkSubmitArgs() { + List args = new ArrayList(); + + if (verbose) { + args.add("--verbose"); + } + + if (master != null) { + args.add("--master"); + args.add(master); + } + + if (deployMode != null) { + args.add("--deploy-mode"); + args.add(deployMode); + } + + if (appName != null) { + args.add("--name"); + args.add(appName); + } + + for (Map.Entry e : conf.entrySet()) { + args.add("--conf"); + args.add(String.format("%s=%s", e.getKey(), e.getValue())); + } + + if (propertiesFile != null) { + args.add("--properties-file"); + args.add(propertiesFile); + } + + if (!jars.isEmpty()) { + args.add("--jars"); + args.add(join(",", jars)); + } + + if (!files.isEmpty()) { + args.add("--files"); + args.add(join(",", files)); + } + + if (!pyFiles.isEmpty()) { + args.add("--py-files"); + args.add(join(",", pyFiles)); + } + + if (userClass != null) { + args.add("--class"); + args.add(userClass); + } + + args.addAll(sparkArgs); + if (userResource != null) { + args.add(userResource); + } + args.addAll(userArgs); + + return args; + } + + @Override + protected List buildLauncherCommand() throws IOException { + List cmd = createJavaCommand(); + addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); + addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); + + // Load the properties file and check whether spark-submit will be running the app's driver + // or just launching a cluster app. When running the driver, the JVM's argument will be + // modified to cover the driver's configuration. + Properties props = loadPropertiesFile(); + boolean isRunningDriver = isRunningDriver(props); + + String extraClassPath = isRunningDriver ? find(DRIVER_CLASSPATH, conf, props) : null; + cmd.add("-cp"); + cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); + + String libPath = null; + if (isRunningDriver) { + // Figuring out where the memory value come from is a little tricky due to precedence. + // Precedence is observed in the following order: + // - explicit configuration (setConf()), which also covers --driver-memory cli argument. + // - user properties, if properties file is explicitly set. + // - SPARK_MEM env variable + // - user properties, if using default file + // - default value (512m) + String userMemSetting; + String defaultMemFromProps = null; + if (propertiesFile != null) { + userMemSetting = find(DRIVER_MEMORY, conf, props); + } else { + userMemSetting = conf.get(DRIVER_MEMORY); + defaultMemFromProps = props.getProperty(DRIVER_MEMORY); + } + + String memory = first(userMemSetting, System.getenv("SPARK_MEM"), defaultMemFromProps, + DEFAULT_MEM); + cmd.add("-Xms" + memory); + cmd.add("-Xmx" + memory); + addOptionString(cmd, find(DRIVER_JAVA_OPTIONS, conf, props)); + libPath = find(DRIVER_LIBRARY_PATH, conf, props); + } + + cmd.add("org.apache.spark.deploy.SparkSubmit"); + cmd.addAll(buildSparkSubmitArgs()); + return prepareForOs(cmd, libPath, Collections.emptyMap()); + } + + private boolean isRunningDriver(Properties userProps) { + String userMaster = first(master, (String) userProps.get(SPARK_MASTER)); + return userMaster == null || + "client".equals(deployMode) || + "yarn-client".equals(userMaster) || + (deployMode == null && userMaster != null && !userMaster.startsWith("yarn-")); + } + + private static class SparkSubmitRunner implements Runnable { + + private final Method main; + private final Object args; + + SparkSubmitRunner(Method main, List args) { + this.main = main; + this.args = args.toArray(new String[args.size()]); + } + + @Override + public void run() { + try { + main.invoke(null, args); + } catch (RuntimeException re) { + throw re; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java new file mode 100644 index 000000000000..187bd36d2810 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java @@ -0,0 +1,196 @@ +/* + * 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.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Special launcher for handling a CLI invocation of SparkSubmit. + *

+ * This launcher extends SparkLauncher to add command line parsing compatible with + * SparkSubmit. It handles setting driver-side options and special parsing needed + * for the different shells. + *

+ * This class has also some special features to aid PySparkLauncher. + */ +public class SparkSubmitCliLauncher extends SparkLauncher { + + /** List of spark-submit arguments that take an argument. */ + public static final List SPARK_SUBMIT_OPTS = Arrays.asList( + "--archives", + "--class", + "--conf", + "--deploy-mode", + "--driver-class-path", + "--driver-cores", + "--driver-java-options", + "--driver-library-path", + "--driver-memory", + "--executor-cores", + "--executor-memory", + "--files", + "--jars", + "--master", + "--name", + "--num-executors", + "--properties-file", + "--py-files", + "--queue", + "--total-executor-cores"); + + /** List of spark-submit arguments that do not take an argument. */ + public static final List SPARK_SUBMIT_SWITCHES = Arrays.asList( + "--supervise", + "--verbose", + "-v"); + + /** + * This map must match the class names for available shells, since this modifies the way + * command line parsing works. This maps the shell class name to the resource to use when + * calling spark-submit. + */ + private static final Map shells = new HashMap(); + static { + shells.put("org.apache.spark.repl.Main", "spark-shell"); + shells.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver", "spark-internal"); + } + + private final List driverArgs; + private boolean isShell; + + SparkSubmitCliLauncher(List args) { + this(false, args); + } + + SparkSubmitCliLauncher(boolean isShell, List args) { + boolean sparkSubmitOptionsEnded = false; + this.driverArgs = new ArrayList(); + this.isShell = isShell; + + Pattern eqSeparatedOpt = Pattern.compile("(--[^=]+)=(.+)"); + + for (Iterator it = args.iterator(); it.hasNext(); ) { + String arg = it.next(); + + Matcher m = eqSeparatedOpt.matcher(arg); + if (m.matches()) { + parseOpt(m.group(1), m.group(2), it); + } else { + parseOpt(arg, it); + } + } + } + + private void parseOpt(String arg, Iterator tail) { + if (SPARK_SUBMIT_SWITCHES.contains(arg)) { + addSparkArgs(arg); + } if (!SPARK_SUBMIT_OPTS.contains(arg)) { + parseOpt(arg, null, tail); + } else { + parseOpt(arg, getArgValue(tail, arg), tail); + } + } + + private void parseOpt(String arg, String value, Iterator tail) { + if (!SPARK_SUBMIT_OPTS.contains(arg)) { + // When running a shell, add unrecognized parameters directly to the user arguments list. + // In normal mode, any unrecognized parameter triggers the end of command line parsing. + // The remaining params will be appended to the list of SparkSubmit arguments. + if (isShell) { + addArgs(arg); + } else { + addSparkArgs(arg); + while (tail.hasNext()) { + addSparkArgs(tail.next()); + } + } + } else if (arg.equals("--master")) { + setMaster(value); + driverArgs.add(arg); + driverArgs.add(value); + } else if (arg.equals("--deploy-mode")) { + setDeployMode(value); + driverArgs.add(arg); + driverArgs.add(value); + } else if (arg.equals("--properties-file")) { + setPropertiesFile(value); + driverArgs.add(arg); + driverArgs.add(value); + } else if (arg.equals("--driver-memory")) { + setConf(DRIVER_MEMORY, value); + driverArgs.add(arg); + driverArgs.add(value); + } else if (arg.equals("--driver-java-options")) { + setConf(DRIVER_JAVA_OPTIONS, value); + driverArgs.add(arg); + driverArgs.add(value); + } else if (arg.equals("--driver-library-path")) { + setConf(DRIVER_LIBRARY_PATH, value); + driverArgs.add(arg); + driverArgs.add(value); + } else if (arg.equals("--driver-class-path")) { + setConf(DRIVER_CLASSPATH, value); + driverArgs.add(arg); + driverArgs.add(value); + } else if (arg.equals("--class")) { + // The shell launchers require some special command line handling, since they allow + // mixing spark-submit arguments with arguments that should be propagated to the shell + // itself. Note that for this to work, the "--class" argument must come before any + // non-spark-submit arguments. + setClass(value); + if (shells.containsKey(value)) { + isShell = true; + setAppResource(shells.get(value)); + } + } else { + addSparkArgs(arg, value); + } + } + /** Visible for PySparkLauncher. */ + String getAppResource() { + return userResource; + } + + /** Visible for PySparkLauncher. */ + List getArgs() { + return userArgs; + } + + /** Visible for PySparkLauncher. */ + List getSparkArgs() { + return sparkArgs; + } + + /** Visible for PySparkLauncher. */ + List getDriverArgs() { + return driverArgs; + } + + private String getArgValue(Iterator it, String name) { + checkArgument(it.hasNext(), "Missing argument for '%s'.", name); + return it.next(); + } + +} diff --git a/launcher/src/test/java/org/apache/spark/launcher/AbstractLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/AbstractLauncherSuite.java new file mode 100644 index 000000000000..af18147e36cb --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/AbstractLauncherSuite.java @@ -0,0 +1,89 @@ +/* + * 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.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class AbstractLauncherSuite { + + private AbstractLauncher launcher = new AbstractLauncher() { + @Override + protected List buildLauncherCommand() { + throw new UnsupportedOperationException(); + } + }; + + @Test + public void testValidOptionStrings() { + testOpt("a b c d e", Arrays.asList("a", "b", "c", "d", "e")); + testOpt("a 'b c' \"d\" e", Arrays.asList("a", "b c", "d", "e")); + testOpt("a 'b\\\"c' \"'d'\" e", Arrays.asList("a", "b\\\"c", "'d'", "e")); + testOpt("a 'b\"c' \"\\\"d\\\"\" e", Arrays.asList("a", "b\"c", "\"d\"", "e")); + + // Following tests ported from UtilsSuite.scala. + testOpt("", new ArrayList()); + testOpt("a", Arrays.asList("a")); + testOpt("aaa", Arrays.asList("aaa")); + testOpt("a b c", Arrays.asList("a", "b", "c")); + testOpt(" a b\t c ", Arrays.asList("a", "b", "c")); + testOpt("a 'b c'", Arrays.asList("a", "b c")); + testOpt("a 'b c' d", Arrays.asList("a", "b c", "d")); + testOpt("'b c'", Arrays.asList("b c")); + testOpt("a \"b c\"", Arrays.asList("a", "b c")); + testOpt("a \"b c\" d", Arrays.asList("a", "b c", "d")); + testOpt("\"b c\"", Arrays.asList("b c")); + testOpt("a 'b\" c' \"d' e\"", Arrays.asList("a", "b\" c", "d' e")); + testOpt("a\t'b\nc'\nd", Arrays.asList("a", "b\nc", "d")); + testOpt("a \"b\\\\c\"", Arrays.asList("a", "b\\c")); + testOpt("a \"b\\\"c\"", Arrays.asList("a", "b\"c")); + testOpt("a 'b\\\"c'", Arrays.asList("a", "b\\\"c")); + testOpt("'a'b", Arrays.asList("ab")); + testOpt("'a''b'", Arrays.asList("ab")); + testOpt("\"a\"b", Arrays.asList("ab")); + testOpt("\"a\"\"b\"", Arrays.asList("ab")); + testOpt("''", Arrays.asList("")); + testOpt("\"\"", Arrays.asList("")); + } + + @Test + public void testInalidOptionStrings() { + testInvalidOpt("\\"); + testInvalidOpt("\"abcde"); + testInvalidOpt("'abcde"); + } + + private void testOpt(String opts, List expected) { + assertEquals(String.format("test string failed to parse: [[ %s ]]", opts), + expected, launcher.parseOptionString(opts)); + } + + private void testInvalidOpt(String opts) { + try { + launcher.parseOptionString(opts); + fail("Expected exception for invalid option string."); + } catch (IllegalArgumentException e) { + // pass. + } + } + +} diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java new file mode 100644 index 000000000000..df19d1cdfda2 --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -0,0 +1,281 @@ +/* + * 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.BufferedReader; +import java.io.File; +import java.io.InputStreamReader; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static org.junit.Assert.*; + +/** + * These tests require the Spark assembly to be built before they can be run. + * + * TODO: these tests do not work on Win32. + */ +public class SparkLauncherSuite { + + private static final Logger LOG = LoggerFactory.getLogger(SparkLauncherSuite.class); + + @Test + public void testDriverCmdBuilder() throws Exception { + testCmdBuilder(true); + } + + @Test + public void testClusterCmdBuilder() throws Exception { + testCmdBuilder(false); + } + + @Test + public void testChildProcLauncher() throws Exception { + SparkLauncher launcher = new SparkLauncher() + .setSparkHome(System.getProperty("spark.test.home")) + .setMaster("local") + .setAppResource("spark-internal") + .setConf(SparkLauncher.DRIVER_JAVA_OPTIONS, "-Dfoo=bar -Dtest.name=-testChildProcLauncher") + .setConf(SparkLauncher.DRIVER_CLASSPATH, System.getProperty("java.class.path")) + .setClass(SparkLauncherTestApp.class.getName()) + .addArgs("proc"); + + printArgs(launcher.buildLauncherCommand()); + + final Process app = launcher.launch(); + Thread stderr = new Thread(new Runnable() { + @Override + public void run() { + try { + BufferedReader in = new BufferedReader( + new InputStreamReader(app.getErrorStream(), "UTF-8")); + String line; + while ((line = in.readLine()) != null) { + LOG.warn(line); + } + } catch (Exception e) { + e.printStackTrace(); + } + } + }); + stderr.setDaemon(true); + stderr.setName("stderr"); + stderr.start(); + + assertEquals(0, app.waitFor()); + } + + @Test + public void testThreadAppLauncher() throws Exception { + // Do this to avoid overwriting the main test log file. + System.setProperty("test.name", "-testThreadAppLauncher"); + + SparkLauncher launcher = new SparkLauncher() + .setSparkHome(System.getProperty("spark.test.home")) + .setMaster("local") + .setAppResource("spark-internal") + .setConf(SparkLauncher.DRIVER_CLASSPATH, System.getProperty("java.class.path")) + .setClass(SparkLauncherTestApp.class.getName()) + .addArgs("thread"); + + printArgs(launcher.buildLauncherCommand()); + + Thread app = launcher.start(new Thread.UncaughtExceptionHandler() { + @Override + public void uncaughtException(Thread t, Throwable e) { + String msg = "Uncaught exception in app."; + LOG.error(msg, e); + fail(msg); + } + }, true); + app.join(); + } + + @Test + public void testInProcessDriverArgValidator() throws Exception { + testInvalidDriverConf(SparkLauncher.DRIVER_JAVA_OPTIONS); + testInvalidDriverConf(SparkLauncher.DRIVER_MEMORY); + testInvalidDriverConf(SparkLauncher.DRIVER_LIBRARY_PATH); + } + + private void testCmdBuilder(boolean isDriver) throws Exception { + String deployMode = isDriver ? "client" : "cluster"; + + SparkLauncher launcher = new SparkLauncher() + .setSparkHome(System.getProperty("spark.test.home")) + .setMaster("yarn") + .setDeployMode(deployMode) + .setAppResource("/foo") + .setAppName("MyApp") + .setClass("my.Class") + .addArgs("foo", "bar") + .setConf(SparkLauncher.DRIVER_MEMORY, "1g") + .setConf(SparkLauncher.DRIVER_CLASSPATH, "/driver") + .setConf(SparkLauncher.DRIVER_JAVA_OPTIONS, "-Ddriver") + .setConf(SparkLauncher.DRIVER_LIBRARY_PATH, "/native") + .setConf("spark.foo", "foo"); + + List cmd = launcher.buildLauncherCommand(); + + // Checks below are different for driver and non-driver mode. + + if (isDriver) { + assertTrue("Driver -Xms should be configured.", contains("-Xms1g", cmd)); + assertTrue("Driver -Xmx should be configured.", contains("-Xmx1g", cmd)); + } else { + boolean found = false; + for (String arg : cmd) { + if (arg.startsWith("-Xms") || arg.startsWith("-Xmx")) { + found = true; + break; + } + } + assertFalse("Memory arguments should not be set.", found); + } + + String[] cp = findArgValue(cmd, "-cp").split(Pattern.quote(File.pathSeparator)); + if (isDriver) { + assertTrue("Driver classpath should contain provided entry.", contains("/driver", cp)); + } else { + assertFalse("Driver classpath should not be in command.", contains("/driver", cp)); + } + + String[] libPath = null; + String envName = launcher.getLibPathEnvName() + "="; + for (String arg : cmd) { + if (arg.startsWith(envName)) { + libPath = arg.substring(envName.length()).split(Pattern.quote(File.pathSeparator)); + break; + } + } + if (isDriver) { + assertNotNull("Native library path should be set.", libPath); + assertTrue("Native library path should contain provided entry.", contains("/native", libPath)); + } else { + assertNull("Native library should not be set.", libPath); + } + + // Checks below are the same for both driver and non-driver mode. + + assertEquals("yarn", findArgValue(cmd, "--master")); + assertEquals(deployMode, findArgValue(cmd, "--deploy-mode")); + assertEquals("my.Class", findArgValue(cmd, "--class")); + assertEquals("MyApp", findArgValue(cmd, "--name")); + + boolean appArgsOk = false; + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals("/foo")) { + assertEquals("foo", cmd.get(i + 1)); + assertEquals("bar", cmd.get(i + 2)); + assertEquals(cmd.size(), i + 3); + appArgsOk = true; + break; + } + } + assertTrue("App resource and args should be added to command.", appArgsOk); + + Map conf = parseConf(cmd); + assertEquals("foo", conf.get("spark.foo")); + } + + private void testInvalidDriverConf(String key) throws Exception { + try { + new SparkLauncher() + .setSparkHome(System.getProperty("spark.test.home")) + .setAppResource("spark-internal") + .setClass(SparkLauncherTestApp.class.getName()) + .addArgs("thread") + .setConf(key, "foo") + .start(null, true); + fail("Should have failed to start app."); + } catch (IllegalStateException e) { + assertTrue("Correct exception should be thrown.", + e.getMessage().indexOf("running in-process") > 0); + } + } + + private String findArgValue(List cmd, String name) { + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals(name)) { + return cmd.get(i + 1); + } + } + fail(String.format("arg '%s' not found", name)); + return null; + } + + private boolean contains(String needle, String[] haystack) { + for (String entry : haystack) { + if (entry.equals(needle)) { + return true; + } + } + return false; + } + + private boolean contains(String needle, Iterable haystack) { + for (String entry : haystack) { + if (entry.equals(needle)) { + return true; + } + } + return false; + } + + private Map parseConf(List cmd) { + Map conf = new HashMap(); + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals("--conf")) { + String[] val = cmd.get(i + 1).split("=", 2); + conf.put(val[0], val[1]); + i += 1; + } + } + return conf; + } + + private void printArgs(List cmd) { + StringBuilder cmdLine = new StringBuilder(); + for (String arg : cmd) { + if (cmdLine.length() > 0) { + cmdLine.append(" "); + } + cmdLine.append("[").append(arg).append("]"); + } + LOG.info("Launching SparkSubmit with args: {}", cmdLine.toString()); + } + + public static class SparkLauncherTestApp { + + public static void main(String[] args) { + if (args[0].equals("proc")) { + assertEquals("bar", System.getProperty("foo")); + } else { + assertEquals("thread", args[0]); + } + assertEquals("local", System.getProperty(SparkLauncher.SPARK_MASTER)); + } + + } + +} diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java new file mode 100644 index 000000000000..cc95f5c83bae --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java @@ -0,0 +1,71 @@ +/* + * 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.util.Arrays; +import java.util.List; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class SparkSubmitCliLauncherSuite { + + @Test + public void testShellCliParser() throws Exception { + List sparkSubmitArgs = Arrays.asList( + "--class", + "org.apache.spark.repl.Main", + "--master", + "foo", + "--app-arg", + "bar", + "--app-switch", + "--files", + "baz", + "--name", + "appName"); + + List args = new SparkSubmitCliLauncher(sparkSubmitArgs).buildSparkSubmitArgs(); + List expected = Arrays.asList("spark-shell", "--app-arg", "bar", "--app-switch"); + assertEquals(expected, args.subList(args.size() - expected.size(), args.size())); + } + + @Test + public void testAlternateSyntaxParsing() throws Exception { + List sparkSubmitArgs = Arrays.asList( + "--class=org.my.Class", + "--master=foo", + "--deploy-mode=bar"); + + List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs).buildSparkSubmitArgs(); + assertEquals("org.my.Class", findArgValue(cmd, "--class")); + assertEquals("foo", findArgValue(cmd, "--master")); + assertEquals("bar", findArgValue(cmd, "--deploy-mode")); + } + + private String findArgValue(List cmd, String name) { + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals(name)) { + return cmd.get(i + 1); + } + } + fail(String.format("arg '%s' not found", name)); + return null; + } + +} diff --git a/launcher/src/test/resources/log4j.properties b/launcher/src/test/resources/log4j.properties new file mode 100644 index 000000000000..00c20ad69cd4 --- /dev/null +++ b/launcher/src/test/resources/log4j.properties @@ -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. +# + +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false + +# Some tests will set "test.name" to avoid overwriting the main log file. +log4j.appender.file.file=target/unit-tests${test.name}.log + +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/make-distribution.sh b/make-distribution.sh index 45c99e42e5a5..d05f45e090d5 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -181,6 +181,7 @@ echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DI # Copy jars cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +cp "$FWDIR"/launcher/target/scala*/spark-launcher*.jar "$DISTDIR/lib/" # This will fail if the -Pyarn profile is not provided # In this case, silence the error and ignore the return code of this command cp "$FWDIR"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || : diff --git a/pom.xml b/pom.xml index 05f59a9b4140..343760a6a8e2 100644 --- a/pom.xml +++ b/pom.xml @@ -105,6 +105,7 @@ external/zeromq examples repl + launcher From 27be98a4fe1cbaa227f3694caeff476779089b4f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 6 Jan 2015 13:27:11 -0800 Subject: [PATCH 02/51] Modify Spark to use launcher lib. Change the existing scripts under bin/ to use the launcher library, to avoid code duplication and reduce the amount of coupling between scripts and Spark code. Also change some Spark core code to use the library instead of relying on scripts (either by calling them or with comments saying they should be kept in sync). While the library is now included in the assembly (by means of the spark-core dependency), it's still packaged directly into the final lib/ directory, because loading a small jar is much faster than the huge assembly jar, and that makes the start up time of Spark jobs much better. --- bin/compute-classpath.cmd | 117 ------------ bin/compute-classpath.sh | 149 --------------- bin/load-spark-env.sh | 8 +- bin/pyspark | 53 +----- bin/pyspark2.cmd | 56 ++---- bin/run-example | 2 +- bin/spark-class | 167 +++-------------- bin/spark-class2.cmd | 140 +++------------ bin/spark-shell | 18 +- bin/spark-shell2.cmd | 27 ++- bin/spark-sql | 15 +- bin/spark-submit | 49 +---- bin/spark-submit2.cmd | 49 +---- bin/utils.sh | 59 ------ bin/windows-utils.cmd | 59 ------ core/pom.xml | 5 + .../SparkSubmitDriverBootstrapper.scala | 170 ------------------ .../spark/deploy/worker/CommandUtils.scala | 54 +++--- project/SparkBuild.scala | 4 +- python/pyspark/java_gateway.py | 1 - 20 files changed, 145 insertions(+), 1057 deletions(-) delete mode 100644 bin/compute-classpath.cmd delete mode 100755 bin/compute-classpath.sh delete mode 100755 bin/utils.sh delete mode 100644 bin/windows-utils.cmd delete mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd deleted file mode 100644 index a4c099fb45b1..000000000000 --- a/bin/compute-classpath.cmd +++ /dev/null @@ -1,117 +0,0 @@ -@echo off - -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" -rem script and the ExecutorRunner in standalone cluster mode. - -rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting -rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we -rem need to set it here because we use !datanucleus_jars! below. -if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion -setlocal enabledelayedexpansion -:skip_delayed_expansion - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" - -rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% - -if not "x%SPARK_CONF_DIR%"=="x" ( - set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% -) else ( - set CLASSPATH=%CLASSPATH%;%FWDIR%conf -) - -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) else ( - for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) - -set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% - -rem When Hive support is needed, Datanucleus jars must be included on the classpath. -rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. -rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is -rem built with Hive, so look for them there. -if exist "%FWDIR%RELEASE" ( - set datanucleus_dir=%FWDIR%lib -) else ( - set datanucleus_dir=%FWDIR%lib_managed\jars -) -set "datanucleus_jars=" -for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( - set datanucleus_jars=!datanucleus_jars!;%%d -) -set CLASSPATH=%CLASSPATH%;%datanucleus_jars% - -set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes - -set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes - -if "x%SPARK_TESTING%"=="x1" ( - rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH - rem so that local compilation takes precedence over assembled jar - set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% -) - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - -rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh deleted file mode 100755 index a31ea73d3ce1..000000000000 --- a/bin/compute-classpath.sh +++ /dev/null @@ -1,149 +0,0 @@ -#!/usr/bin/env bash - -# -# 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 script computes Spark's classpath and prints it to stdout; it's used by both the "run" -# script and the ExecutorRunner in standalone cluster mode. - -# Figure out where Spark is installed -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" - -. "$FWDIR"/bin/load-spark-env.sh - -if [ -n "$SPARK_CLASSPATH" ]; then - CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" -else - CLASSPATH="$SPARK_SUBMIT_CLASSPATH" -fi - -# Build up classpath -if [ -n "$SPARK_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$SPARK_CONF_DIR" -else - CLASSPATH="$CLASSPATH:$FWDIR/conf" -fi - -ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SPARK_SCALA_VERSION" - -if [ -n "$JAVA_HOME" ]; then - JAR_CMD="$JAVA_HOME/bin/jar" -else - JAR_CMD="jar" -fi - -# A developer option to prepend more recently compiled Spark classes -if [ -n "$SPARK_PREPEND_CLASSES" ]; then - echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\ - "classes ahead of assembly." >&2 - CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*" - CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/tools/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SPARK_SCALA_VERSION/classes" -fi - -# Use spark-assembly jar from either RELEASE or assembly directory -if [ -f "$FWDIR/RELEASE" ]; then - assembly_folder="$FWDIR"/lib -else - assembly_folder="$ASSEMBLY_DIR" -fi - -num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar$" | wc -l)" -if [ "$num_jars" -eq "0" ]; then - echo "Failed to find Spark assembly in $assembly_folder" - echo "You need to build Spark before running this program." - exit 1 -fi -if [ "$num_jars" -gt "1" ]; then - jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar$") - echo "Found multiple Spark assembly jars in $assembly_folder:" - echo "$jars_list" - echo "Please remove all but one jar." - exit 1 -fi - -ASSEMBLY_JAR="$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null)" - -# Verify that versions of java used to build the jars and run Spark are compatible -jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) -if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then - echo "Loading Spark jar with '$JAR_CMD' failed. " 1>&2 - echo "This is likely because Spark was compiled with Java 7 and run " 1>&2 - echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " 1>&2 - echo "or build Spark with Java 6." 1>&2 - exit 1 -fi - -CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" - -# When Hive support is needed, Datanucleus jars must be included on the classpath. -# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. -# Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is -# built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark -# assembly is built for Hive, before actually populating the CLASSPATH with the jars. -# Note that this check order is faster (by up to half a second) in the case where Hive is not used. -if [ -f "$FWDIR/RELEASE" ]; then - datanucleus_dir="$FWDIR"/lib -else - datanucleus_dir="$FWDIR"/lib_managed/jars -fi - -datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar$")" -datanucleus_jars="$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g)" - -if [ -n "$datanucleus_jars" ]; then - hive_files=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null) - if [ -n "$hive_files" ]; then - echo "Spark assembly has been built with Hive, including Datanucleus jars on classpath" 1>&2 - CLASSPATH="$CLASSPATH:$datanucleus_jars" - fi -fi - -# Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1 -if [[ $SPARK_TESTING == 1 ]]; then - CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SPARK_SCALA_VERSION/test-classes" -fi - -# Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail ! -# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -# the configurtion files. -if [ -n "$HADOOP_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" -fi -if [ -n "$YARN_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" -fi - -echo "$CLASSPATH" diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 356b3d49b2ff..2d7070c25d32 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -41,9 +41,9 @@ fi if [ -z "$SPARK_SCALA_VERSION" ]; then - ASSEMBLY_DIR2="$FWDIR/assembly/target/scala-2.11" - ASSEMBLY_DIR1="$FWDIR/assembly/target/scala-2.10" - + ASSEMBLY_DIR2="$SPARK_HOME/assembly/target/scala-2.11" + ASSEMBLY_DIR1="$SPARK_HOME/assembly/target/scala-2.10" + if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2 echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION=2.11 in spark-env.sh.' 1>&2 @@ -54,5 +54,5 @@ if [ -z "$SPARK_SCALA_VERSION" ]; then export SPARK_SCALA_VERSION="2.11" else export SPARK_SCALA_VERSION="2.10" - fi + fi fi diff --git a/bin/pyspark b/bin/pyspark index 0b4f695dd06d..00cca0b2b3a2 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -18,36 +18,21 @@ # # Figure out where Spark is installed -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" +SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -# Export this as SPARK_HOME -export SPARK_HOME="$FWDIR" - -source "$FWDIR/bin/utils.sh" - -source "$FWDIR"/bin/load-spark-env.sh +source "$SPARK_HOME"/bin/load-spark-env.sh function usage() { echo "Usage: ./bin/pyspark [options]" 1>&2 - "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$SPARK_HOME"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 } +export -f usage if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage fi -# Exit if the user hasn't compiled Spark -if [ ! -f "$FWDIR/RELEASE" ]; then - # Exit if the user hasn't compiled Spark - ls "$FWDIR"/assembly/target/scala-$SPARK_SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null - if [[ $? != 0 ]]; then - echo "Failed to find Spark assembly in $FWDIR/assembly/target" 1>&2 - echo "You need to build Spark before running this program" 1>&2 - exit 1 - fi -fi - # In Spark <= 1.1, setting IPYTHON=1 would cause the driver to be launched using the `ipython` # executable, while the worker would still be launched using PYSPARK_PYTHON. # @@ -95,21 +80,7 @@ export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" -export PYTHONSTARTUP="$FWDIR/python/pyspark/shell.py" - -# Build up arguments list manually to preserve quotes and backslashes. -# We export Spark submit arguments as an environment variable because shell.py must run as a -# PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks. -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" -PYSPARK_SUBMIT_ARGS="" -whitespace="[[:space:]]" -for i in "${SUBMISSION_OPTS[@]}"; do - if [[ $i =~ \" ]]; then i=$(echo $i | sed 's/\"/\\\"/g'); fi - if [[ $i =~ $whitespace ]]; then i=\"$i\"; fi - PYSPARK_SUBMIT_ARGS="$PYSPARK_SUBMIT_ARGS $i" -done -export PYSPARK_SUBMIT_ARGS +export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" # For pyspark tests if [[ -n "$SPARK_TESTING" ]]; then @@ -123,14 +94,6 @@ if [[ -n "$SPARK_TESTING" ]]; then exit fi -# If a python file is provided, directly run spark-submit. -if [[ "$1" =~ \.py$ ]]; then - echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 - echo -e "Use ./bin/spark-submit \n" 1>&2 - primary="$1" - shift - gatherSparkSubmitOpts "$@" - exec "$FWDIR"/bin/spark-submit "${SUBMISSION_OPTS[@]}" "$primary" "${APPLICATION_OPTS[@]}" -else - exec "$PYSPARK_DRIVER_PYTHON" $PYSPARK_DRIVER_PYTHON_OPTS -fi +export PYSPARK_DRIVER_PYTHON +export PYSPARK_DRIVER_PYTHON_OPTS +exec $SPARK_HOME/bin/spark-class pyspark "$@" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index a542ec80b49d..b4e6b0a43c47 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -17,59 +17,23 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -set SCALA_VERSION=2.10 - rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Export this as SPARK_HOME -set SPARK_HOME=%FWDIR% - -rem Test whether the user has built Spark -if exist "%FWDIR%RELEASE" goto skip_build_test -set FOUND_JAR=0 -for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set FOUND_JAR=1 -) -if [%FOUND_JAR%] == [0] ( - echo Failed to find Spark assembly JAR. - echo You need to build Spark before running this program. - goto exit -) -:skip_build_test +set SPARK_HOME=%~dp0.. rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" +if exist "%SPARK_HOME%\conf\spark-env.cmd" call "%SPARK_HOME%\conf\spark-env.cmd" rem Figure out which Python to use. -if [%PYSPARK_PYTHON%] == [] set PYSPARK_PYTHON=python +if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( + set PYSPARK_DRIVER_PYTHON=python + if not [%PYSPARK_PYTHON%] == [] set PYSPARK_DRIVER_PYTHON=%PYSPARK_PYTHON% +) -set PYTHONPATH=%FWDIR%python;%PYTHONPATH% -set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% -set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py -set PYSPARK_SUBMIT_ARGS=%* +set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH% - -rem Check whether the argument is a file -for /f %%i in ('echo %1^| findstr /R "\.py"') do ( - set PYTHON_FILE=%%i -) - -if [%PYTHON_FILE%] == [] ( - if [%IPYTHON%] == [1] ( - ipython %IPYTHON_OPTS% - ) else ( - %PYSPARK_PYTHON% - ) -) else ( - echo. - echo WARNING: Running python applications through ./bin/pyspark.cmd is deprecated as of Spark 1.0. - echo Use ./bin/spark-submit ^ - echo. - "%FWDIR%\bin\spark-submit.cmd" %PYSPARK_SUBMIT_ARGS% -) - -:exit +call %SPARK_HOME%\bin\spark-class2.cmd pyspark %* diff --git a/bin/run-example b/bin/run-example index 3d932509426f..bab2823ca644 100755 --- a/bin/run-example +++ b/bin/run-example @@ -52,7 +52,7 @@ if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" fi -"$FWDIR"/bin/spark-submit \ +exec "$FWDIR"/bin/spark-submit \ --master $EXAMPLE_MASTER \ --class $EXAMPLE_CLASS \ "$SPARK_EXAMPLES_JAR" \ diff --git a/bin/spark-class b/bin/spark-class index 0d58d95c1aee..5553153c3cdb 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -16,86 +16,18 @@ # See the License for the specific language governing permissions and # limitations under the License. # - -# NOTE: Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! - -cygwin=false -case "`uname`" in - CYGWIN*) cygwin=true;; -esac +set -e # Figure out where Spark is installed -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -# Export this as SPARK_HOME -export SPARK_HOME="$FWDIR" - -. "$FWDIR"/bin/load-spark-env.sh +. "$SPARK_HOME"/bin/load-spark-env.sh if [ -z "$1" ]; then echo "Usage: spark-class []" 1>&2 exit 1 fi -if [ -n "$SPARK_MEM" ]; then - echo -e "Warning: SPARK_MEM is deprecated, please use a more specific config option" 1>&2 - echo -e "(e.g., spark.executor.memory or spark.driver.memory)." 1>&2 -fi - -# Use SPARK_MEM or 512m as the default memory, to be overridden by specific options -DEFAULT_MEM=${SPARK_MEM:-512m} - -SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" - -# Add java opts and memory settings for master, worker, history server, executors, and repl. -case "$1" in - # Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. - 'org.apache.spark.deploy.master.Master') - OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" - OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} - ;; - 'org.apache.spark.deploy.worker.Worker') - OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" - OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} - ;; - 'org.apache.spark.deploy.history.HistoryServer') - OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_HISTORY_OPTS" - OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} - ;; - - # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. - 'org.apache.spark.executor.CoarseGrainedExecutorBackend') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} - ;; - 'org.apache.spark.executor.MesosExecutorBackend') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} - ;; - - # Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + - # SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. - 'org.apache.spark.deploy.SparkSubmit') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS" - OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} - if [ -n "$SPARK_SUBMIT_LIBRARY_PATH" ]; then - if [[ $OSTYPE == darwin* ]]; then - export DYLD_LIBRARY_PATH="$SPARK_SUBMIT_LIBRARY_PATH:$DYLD_LIBRARY_PATH" - else - export LD_LIBRARY_PATH="$SPARK_SUBMIT_LIBRARY_PATH:$LD_LIBRARY_PATH" - fi - fi - if [ -n "$SPARK_SUBMIT_DRIVER_MEMORY" ]; then - OUR_JAVA_MEM="$SPARK_SUBMIT_DRIVER_MEMORY" - fi - ;; - - *) - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" - OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} - ;; -esac - # Find the java binary if [ -n "${JAVA_HOME}" ]; then RUNNER="${JAVA_HOME}/bin/java" @@ -107,83 +39,42 @@ else exit 1 fi fi -JAVA_VERSION=$("$RUNNER" -version 2>&1 | grep 'version' | sed 's/.* version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') -# Set JAVA_OPTS to be able to load native libraries and to set heap size -if [ "$JAVA_VERSION" -ge 18 ]; then - JAVA_OPTS="$OUR_JAVA_OPTS" +SPARK_LAUNCHER_CP= +if [ -f $SPARK_HOME/RELEASE ]; then + LAUNCHER_DIR="$SPARK_HOME/lib" else - JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" -fi -JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" - -# Load extra JAVA_OPTS from conf/java-opts, if it exists -if [ -e "$FWDIR/conf/java-opts" ] ; then - JAVA_OPTS="$JAVA_OPTS `cat "$FWDIR"/conf/java-opts`" -fi - -# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! - -TOOLS_DIR="$FWDIR"/tools -SPARK_TOOLS_JAR="" -if [ -e "$TOOLS_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-tools*[0-9Tg].jar ]; then - # Use the JAR from the SBT build - export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-tools*[0-9Tg].jar`" -fi -if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then - # Use the JAR from the Maven build - # TODO: this also needs to become an assembly! - export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`" + LAUNCHER_DIR="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION" + if [ -n "$SPARK_PREPEND_CLASSES" ]; then + SPARK_LAUNCHER_CP="$LAUNCHER_DIR/classes:" + fi fi -# Compute classpath using external script -classpath_output=$("$FWDIR"/bin/compute-classpath.sh) -if [[ "$?" != "0" ]]; then - echo "$classpath_output" +num_jars="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" | wc -l)" +if [ "$num_jars" -eq "0" ]; then + echo "Failed to find Spark launcher in $LAUNCHER_DIR." 1>&2 + echo "You need to build Spark before running this program." 1>&2 exit 1 -else - CLASSPATH="$classpath_output" fi -if [[ "$1" =~ org.apache.spark.tools.* ]]; then - if test -z "$SPARK_TOOLS_JAR"; then - echo "Failed to find Spark Tools Jar in $FWDIR/tools/target/scala-$SPARK_SCALA_VERSION/" 1>&2 - echo "You need to build Spark before running $1." 1>&2 - exit 1 - fi - CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" +LAUNCHER_JARS="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$")" +if [ "$num_jars" -gt "1" ]; then + echo "Found multiple Spark launcher jars in $LAUNCHER_DIR:" 1>&2 + echo "$LAUNCHER_JARS" 1>&2 + echo "Please remove all but one jar." 1>&2 + exit 1 fi -if $cygwin; then - CLASSPATH="`cygpath -wp "$CLASSPATH"`" - if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then - export SPARK_TOOLS_JAR="`cygpath -w "$SPARK_TOOLS_JAR"`" - fi -fi -export CLASSPATH +SPARK_LAUNCHER_CP="${SPARK_LAUNCHER_CP}${LAUNCHER_DIR}/${LAUNCHER_JARS}" -# In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. -# Here we must parse the properties file for relevant "spark.driver.*" configs before launching -# the driver JVM itself. Instead of handling this complexity in Bash, we launch a separate JVM -# to prepare the launch environment of this driver JVM. +# The launcher library will print one argument per line of its output. The next line sets +# the new line char as the only delimiter used when parsing that output into an array. +IFS=" +" +CMD=($($RUNNER -cp $SPARK_LAUNCHER_CP org.apache.spark.launcher.Main "$@")) -if [ -n "$SPARK_SUBMIT_BOOTSTRAP_DRIVER" ]; then - # This is used only if the properties file actually contains these special configs - # Export the environment variables needed by SparkSubmitDriverBootstrapper - export RUNNER - export CLASSPATH - export JAVA_OPTS - export OUR_JAVA_MEM - export SPARK_CLASS=1 - shift # Ignore main class (org.apache.spark.deploy.SparkSubmit) and use our own - exec "$RUNNER" org.apache.spark.deploy.SparkSubmitDriverBootstrapper "$@" +if [ "${CMD[0]}" = "usage" ]; then + "${CMD[@]}" else - # Note: The format of this command is closely echoed in SparkSubmitDriverBootstrapper.scala - if [ -n "$SPARK_PRINT_LAUNCH_COMMAND" ]; then - echo -n "Spark Command: " 1>&2 - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" 1>&2 - echo -e "========================================\n" 1>&2 - fi - exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" + exec "${CMD[@]}" fi - diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index da46543647ef..eff0bb17afd0 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -17,135 +17,53 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! - -setlocal enabledelayedexpansion - -set SCALA_VERSION=2.10 - rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Export this as SPARK_HOME -set SPARK_HOME=%FWDIR% +set SPARK_HOME=%~dp0.. rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" +if exist "%SPARK_HOME%\conf\spark-env.cmd" call "%SPARK_HOME%\conf\spark-env.cmd" rem Test that an argument was given -if not "x%1"=="x" goto arg_given +if "x%1"=="x" ( echo Usage: spark-class ^ [^] - goto exit -:arg_given - -if not "x%SPARK_MEM%"=="x" ( - echo Warning: SPARK_MEM is deprecated, please use a more specific config option - echo e.g., spark.executor.memory or spark.driver.memory. + exit /b 1 ) -rem Use SPARK_MEM or 512m as the default memory, to be overridden by specific options -set OUR_JAVA_MEM=%SPARK_MEM% -if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m - -set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true - -rem Add java opts and memory settings for master, worker, history server, executors, and repl. -rem Master, Worker and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. -if "%1"=="org.apache.spark.deploy.master.Master" ( - set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% - if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% -) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( - set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% - if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% -) else if "%1"=="org.apache.spark.deploy.history.HistoryServer" ( - set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_HISTORY_OPTS% - if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% - -rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. -) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% - if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% -) else if "%1"=="org.apache.spark.executor.MesosExecutorBackend" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% - if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% +set LAUNCHER_JAR=0 +if exist "%SPARK_HOME%\RELEASE" goto find_release_launcher -rem Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + -rem SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. -rem The repl also uses SPARK_REPL_OPTS. -) else if "%1"=="org.apache.spark.deploy.SparkSubmit" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_SUBMIT_OPTS% %SPARK_REPL_OPTS% - if not "x%SPARK_SUBMIT_LIBRARY_PATH%"=="x" ( - set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_SUBMIT_LIBRARY_PATH% - ) else if not "x%SPARK_LIBRARY_PATH%"=="x" ( - set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_LIBRARY_PATH% - ) - if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% - if not "x%SPARK_SUBMIT_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_SUBMIT_DRIVER_MEMORY% -) else ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% - if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% +rem Look for the Spark launcher in both Scala build directories. The launcher doesn't use +rem Scala so it doesn't really matter which one is picked up. +for %%d in (%SPARK_HOME%\launcher\target\scala-2.10\spark-launcher*.jar) do ( + set LAUNCHER_JAR=%%d + set SPARK_SCALA_VERSION=2.10 ) - -rem Set JAVA_OPTS to be able to load native libraries and to set heap size -for /f "tokens=3" %%i in ('java -version 2^>^&1 ^| find "version"') do set jversion=%%i -for /f "tokens=1 delims=_" %%i in ("%jversion:~1,-1%") do set jversion=%%i -if "%jversion%" geq "1.8.0" ( - set JAVA_OPTS=%OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% -) else ( - set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% -) -rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! - -rem Test whether the user has built Spark -if exist "%FWDIR%RELEASE" goto skip_build_test -set FOUND_JAR=0 -for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set FOUND_JAR=1 +for %%d in (%SPARK_HOME%\launcher\target\scala-2.11\spark-launcher*.jar) do ( + set LAUNCHER_JAR=%%d + set SPARK_SCALA_VERSION=2.11 ) -if "%FOUND_JAR%"=="0" ( - echo Failed to find Spark assembly JAR. - echo You need to build Spark before running this program. - goto exit +if not "x%SPARK_PREPEND_CLASSES"=="x" ( + set LAUNCHER_JAR=%SPARK_HOME%\launcher\target\scala-%SPARK_SCALA_VERSION%\classes;%LAUNCHER_JAR% ) -:skip_build_test +goto check_launcher -set TOOLS_DIR=%FWDIR%tools -set SPARK_TOOLS_JAR= -for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar") do ( - set SPARK_TOOLS_JAR=%%d +:find_release_launcher +for %%d in (%SPARK_HOME%\lib\spark-launcher*.jar) do ( + set LAUNCHER_JAR=%%d ) -rem Compute classpath using external script -set DONT_PRINT_CLASSPATH=1 -call "%FWDIR%bin\compute-classpath.cmd" -set DONT_PRINT_CLASSPATH=0 -set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR% +:check_launcher +if "%LAUNCHER_JAR%"=="0" ( + echo Failed to find Spark launcher JAR. + echo You need to build Spark before running this program. + exit /b 1 +) rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java -rem In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. -rem Here we must parse the properties file for relevant "spark.driver.*" configs before launching -rem the driver JVM itself. Instead of handling this complexity here, we launch a separate JVM -rem to prepare the launch environment of this driver JVM. - -rem In this case, leave out the main class (org.apache.spark.deploy.SparkSubmit) and use our own. -rem Leaving out the first argument is surprisingly difficult to do in Windows. Note that this must -rem be done here because the Windows "shift" command does not work in a conditional block. -set BOOTSTRAP_ARGS= -shift -:start_parse -if "%~1" == "" goto end_parse -set BOOTSTRAP_ARGS=%BOOTSTRAP_ARGS% %~1 -shift -goto start_parse -:end_parse - -if not [%SPARK_SUBMIT_BOOTSTRAP_DRIVER%] == [] ( - set SPARK_CLASS=1 - "%RUNNER%" org.apache.spark.deploy.SparkSubmitDriverBootstrapper %BOOTSTRAP_ARGS% -) else ( - "%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCHER_JAR% org.apache.spark.launcher.Main %*"') do ( + set SPARK_CMD=%%i ) -:exit +%SPARK_CMD% diff --git a/bin/spark-shell b/bin/spark-shell index cca5aa067612..57672b102e9e 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -28,23 +28,19 @@ esac # Enter posix mode for bash set -o posix -## Global script variables -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" +export FWDIR="$(cd "`dirname "$0"`"/..; pwd)" -function usage() { +usage() { echo "Usage: ./bin/spark-shell [options]" "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 + exit $1 } +export -f usage if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - usage + usage 0 fi -source "$FWDIR"/bin/utils.sh -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" - # SPARK-4161: scala does not assume use of the java classpath, # so we need to add the "-Dscala.usejavacp=true" flag mnually. We # do this specifically for the Spark shell because the scala REPL @@ -61,11 +57,11 @@ function main() { # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "$@" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "$@" fi } diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index 1d1a40da315e..aadb2d00d741 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -25,17 +25,28 @@ if %ERRORLEVEL% equ 0 ( exit /b 0 ) -call %SPARK_HOME%\bin\windows-utils.cmd %* -if %ERRORLEVEL% equ 1 ( +rem SPARK-4161: scala does not assume use of the java classpath, +rem so we need to add the "-Dscala.usejavacp=true" flag mnually. We +rem do this specifically for the Spark shell because the scala REPL +rem has its own class loader, and any additional classpath specified +rem through spark.driver.extraClassPath is not automatically propagated. +if "x%SPARK_SUBMIT_OPTS%"=="x" ( + set SPARK_SUBMIT_OPTS=-Dscala.usejavacp=true + goto run_shell +) +set SPARK_SUBMIT_OPTS="%SPARK_SUBMIT_OPTS% -Dscala.usejavacp=true" + +:run_shell +call %SPARK_HOME%\bin\spark-submit2.cmd --class org.apache.spark.repl.Main %* +set SPARK_ERROR_LEVEL=%ERRORLEVEL% +echo AFTER SPARK SUBMIT: %SPARK_ERROR_LEVEL% - %SPARK_LAUNCHER_USAGE_ERROR% +if "%SPARK_LAUNCHER_USAGE_ERROR%"=="1" ( call :usage exit /b 1 ) - -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %SUBMISSION_OPTS% spark-shell %APPLICATION_OPTS% - -exit /b 0 +exit /b %SPARK_ERROR_LEVEL% :usage echo "Usage: .\bin\spark-shell.cmd [options]" >&2 -%SPARK_HOME%\bin\spark-submit --help 2>&1 | findstr /V "Usage" 1>&2 -exit /b 0 +call %SPARK_HOME%\bin\spark-submit2.cmd --help 2>&1 | findstr /V "Usage" 1>&2 +goto :eof diff --git a/bin/spark-sql b/bin/spark-sql index 3b6cc420fea8..70f3d3d96343 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -25,10 +25,10 @@ set -o posix # NOTE: This exact class name is matched downstream by SparkSubmit. # Any changes need to be reflected there. -CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +export CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" # Figure out where Spark is installed -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" +export FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage { echo "Usage: ./bin/spark-sql [options] [cli option]" @@ -43,15 +43,12 @@ function usage { echo echo "CLI options:" "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + exit $1 } +export -f usage if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - usage - exit 0 + usage 0 fi -source "$FWDIR"/bin/utils.sh -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" - -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exec "$FWDIR"/bin/spark-submit --class $CLASS "$@" diff --git a/bin/spark-submit b/bin/spark-submit index f92d90c3a66b..dc1773148c33 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -17,50 +17,5 @@ # limitations under the License. # -# NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! - -export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -ORIG_ARGS=("$@") - -# Set COLUMNS for progress bar -export COLUMNS=`tput cols` - -while (($#)); do - if [ "$1" = "--deploy-mode" ]; then - SPARK_SUBMIT_DEPLOY_MODE=$2 - elif [ "$1" = "--properties-file" ]; then - SPARK_SUBMIT_PROPERTIES_FILE=$2 - elif [ "$1" = "--driver-memory" ]; then - export SPARK_SUBMIT_DRIVER_MEMORY=$2 - elif [ "$1" = "--driver-library-path" ]; then - export SPARK_SUBMIT_LIBRARY_PATH=$2 - elif [ "$1" = "--driver-class-path" ]; then - export SPARK_SUBMIT_CLASSPATH=$2 - elif [ "$1" = "--driver-java-options" ]; then - export SPARK_SUBMIT_OPTS=$2 - fi - shift -done - -DEFAULT_PROPERTIES_FILE="$SPARK_HOME/conf/spark-defaults.conf" -export SPARK_SUBMIT_DEPLOY_MODE=${SPARK_SUBMIT_DEPLOY_MODE:-"client"} -export SPARK_SUBMIT_PROPERTIES_FILE=${SPARK_SUBMIT_PROPERTIES_FILE:-"$DEFAULT_PROPERTIES_FILE"} - -# For client mode, the driver will be launched in the same JVM that launches -# SparkSubmit, so we may need to read the properties file for any extra class -# paths, library paths, java options and memory early on. Otherwise, it will -# be too late by the time the driver JVM has started. - -if [[ "$SPARK_SUBMIT_DEPLOY_MODE" == "client" && -f "$SPARK_SUBMIT_PROPERTIES_FILE" ]]; then - # Parse the properties file only if the special configs exist - contains_special_configs=$( - grep -e "spark.driver.extra*\|spark.driver.memory" "$SPARK_SUBMIT_PROPERTIES_FILE" | \ - grep -v "^[[:space:]]*#" - ) - if [ -n "$contains_special_configs" ]; then - export SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 - fi -fi - -exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" - +SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "$@" diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index cf6046d1547a..f8c4a627b53c 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -17,52 +17,5 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! - set SPARK_HOME=%~dp0.. -set ORIG_ARGS=%* - -rem Reset the values of all variables used -set SPARK_SUBMIT_DEPLOY_MODE=client -set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_HOME%\conf\spark-defaults.conf -set SPARK_SUBMIT_DRIVER_MEMORY= -set SPARK_SUBMIT_LIBRARY_PATH= -set SPARK_SUBMIT_CLASSPATH= -set SPARK_SUBMIT_OPTS= -set SPARK_SUBMIT_BOOTSTRAP_DRIVER= - -:loop -if [%1] == [] goto continue - if [%1] == [--deploy-mode] ( - set SPARK_SUBMIT_DEPLOY_MODE=%2 - ) else if [%1] == [--properties-file] ( - set SPARK_SUBMIT_PROPERTIES_FILE=%2 - ) else if [%1] == [--driver-memory] ( - set SPARK_SUBMIT_DRIVER_MEMORY=%2 - ) else if [%1] == [--driver-library-path] ( - set SPARK_SUBMIT_LIBRARY_PATH=%2 - ) else if [%1] == [--driver-class-path] ( - set SPARK_SUBMIT_CLASSPATH=%2 - ) else if [%1] == [--driver-java-options] ( - set SPARK_SUBMIT_OPTS=%2 - ) - shift -goto loop -:continue - -rem For client mode, the driver will be launched in the same JVM that launches -rem SparkSubmit, so we may need to read the properties file for any extra class -rem paths, library paths, java options and memory early on. Otherwise, it will -rem be too late by the time the driver JVM has started. - -if [%SPARK_SUBMIT_DEPLOY_MODE%] == [client] ( - if exist %SPARK_SUBMIT_PROPERTIES_FILE% ( - rem Parse the properties file only if the special configs exist - for /f %%i in ('findstr /r /c:"^[\t ]*spark.driver.memory" /c:"^[\t ]*spark.driver.extra" ^ - %SPARK_SUBMIT_PROPERTIES_FILE%') do ( - set SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 - ) - ) -) - -cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% +call %SPARK_HOME%\bin\spark-class2.cmd org.apache.spark.deploy.SparkSubmit %* diff --git a/bin/utils.sh b/bin/utils.sh deleted file mode 100755 index 22ea2b9a6d58..000000000000 --- a/bin/utils.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# Gather all spark-submit options into SUBMISSION_OPTS -function gatherSparkSubmitOpts() { - - if [ -z "$SUBMIT_USAGE_FUNCTION" ]; then - echo "Function for printing usage of $0 is not set." 1>&2 - echo "Please set usage function to shell variable 'SUBMIT_USAGE_FUNCTION' in $0" 1>&2 - exit 1 - fi - - # NOTE: If you add or remove spark-sumbmit options, - # modify NOT ONLY this script but also SparkSubmitArgument.scala - SUBMISSION_OPTS=() - APPLICATION_OPTS=() - while (($#)); do - case "$1" in - --master | --deploy-mode | --class | --name | --jars | --py-files | --files | \ - --conf | --properties-file | --driver-memory | --driver-java-options | \ - --driver-library-path | --driver-class-path | --executor-memory | --driver-cores | \ - --total-executor-cores | --executor-cores | --queue | --num-executors | --archives) - if [[ $# -lt 2 ]]; then - "$SUBMIT_USAGE_FUNCTION" - exit 1; - fi - SUBMISSION_OPTS+=("$1"); shift - SUBMISSION_OPTS+=("$1"); shift - ;; - - --verbose | -v | --supervise) - SUBMISSION_OPTS+=("$1"); shift - ;; - - *) - APPLICATION_OPTS+=("$1"); shift - ;; - esac - done - - export SUBMISSION_OPTS - export APPLICATION_OPTS -} diff --git a/bin/windows-utils.cmd b/bin/windows-utils.cmd deleted file mode 100644 index 1082a952dac9..000000000000 --- a/bin/windows-utils.cmd +++ /dev/null @@ -1,59 +0,0 @@ -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -rem Gather all spark-submit options into SUBMISSION_OPTS - -set SUBMISSION_OPTS= -set APPLICATION_OPTS= - -rem NOTE: If you add or remove spark-sumbmit options, -rem modify NOT ONLY this script but also SparkSubmitArgument.scala - -:OptsLoop -if "x%1"=="x" ( - goto :OptsLoopEnd -) - -SET opts="\<--master\> \<--deploy-mode\> \<--class\> \<--name\> \<--jars\> \<--py-files\> \<--files\>" -SET opts="%opts:~1,-1% \<--conf\> \<--properties-file\> \<--driver-memory\> \<--driver-java-options\>" -SET opts="%opts:~1,-1% \<--driver-library-path\> \<--driver-class-path\> \<--executor-memory\>" -SET opts="%opts:~1,-1% \<--driver-cores\> \<--total-executor-cores\> \<--executor-cores\> \<--queue\>" -SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\>" - -echo %1 | findstr %opts% >nul -if %ERRORLEVEL% equ 0 ( - if "x%2"=="x" ( - echo "%1" requires an argument. >&2 - exit /b 1 - ) - set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 %2 - shift - shift - goto :OptsLoop -) -echo %1 | findstr "\<--verbose\> \<-v\> \<--supervise\>" >nul -if %ERRORLEVEL% equ 0 ( - set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 - shift - goto :OptsLoop -) -set APPLICATION_OPTS=%APPLICATION_OPTS% %1 -shift -goto :OptsLoop - -:OptsLoopEnd -exit /b 0 diff --git a/core/pom.xml b/core/pom.xml index c5c41b2b5de4..7ac70ed5952b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -72,6 +72,11 @@ + + org.apache.spark + spark-launcher_${scala.binary.version} + ${project.version} + org.apache.spark spark-network-common_${scala.binary.version} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala deleted file mode 100644 index 2eab9981845e..000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ /dev/null @@ -1,170 +0,0 @@ -/* - * 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.deploy - -import java.io.File - -import scala.collection.JavaConversions._ - -import org.apache.spark.util.{RedirectThread, Utils} - -/** - * Launch an application through Spark submit in client mode with the appropriate classpath, - * library paths, java options and memory. These properties of the JVM must be set before the - * driver JVM is launched. The sole purpose of this class is to avoid handling the complexity - * of parsing the properties file for such relevant configs in Bash. - * - * Usage: org.apache.spark.deploy.SparkSubmitDriverBootstrapper - */ -private[spark] object SparkSubmitDriverBootstrapper { - - // Note: This class depends on the behavior of `bin/spark-class` and `bin/spark-submit`. - // Any changes made there must be reflected in this file. - - def main(args: Array[String]): Unit = { - - // This should be called only from `bin/spark-class` - if (!sys.env.contains("SPARK_CLASS")) { - System.err.println("SparkSubmitDriverBootstrapper must be called from `bin/spark-class`!") - System.exit(1) - } - - val submitArgs = args - val runner = sys.env("RUNNER") - val classpath = sys.env("CLASSPATH") - val javaOpts = sys.env("JAVA_OPTS") - val defaultDriverMemory = sys.env("OUR_JAVA_MEM") - - // Spark submit specific environment variables - val deployMode = sys.env("SPARK_SUBMIT_DEPLOY_MODE") - val propertiesFile = sys.env("SPARK_SUBMIT_PROPERTIES_FILE") - val bootstrapDriver = sys.env("SPARK_SUBMIT_BOOTSTRAP_DRIVER") - val submitDriverMemory = sys.env.get("SPARK_SUBMIT_DRIVER_MEMORY") - val submitLibraryPath = sys.env.get("SPARK_SUBMIT_LIBRARY_PATH") - val submitClasspath = sys.env.get("SPARK_SUBMIT_CLASSPATH") - val submitJavaOpts = sys.env.get("SPARK_SUBMIT_OPTS") - - assume(runner != null, "RUNNER must be set") - assume(classpath != null, "CLASSPATH must be set") - assume(javaOpts != null, "JAVA_OPTS must be set") - assume(defaultDriverMemory != null, "OUR_JAVA_MEM must be set") - assume(deployMode == "client", "SPARK_SUBMIT_DEPLOY_MODE must be \"client\"!") - assume(propertiesFile != null, "SPARK_SUBMIT_PROPERTIES_FILE must be set") - assume(bootstrapDriver != null, "SPARK_SUBMIT_BOOTSTRAP_DRIVER must be set") - - // Parse the properties file for the equivalent spark.driver.* configs - val properties = Utils.getPropertiesFromFile(propertiesFile) - val confDriverMemory = properties.get("spark.driver.memory") - val confLibraryPath = properties.get("spark.driver.extraLibraryPath") - val confClasspath = properties.get("spark.driver.extraClassPath") - val confJavaOpts = properties.get("spark.driver.extraJavaOptions") - - // Favor Spark submit arguments over the equivalent configs in the properties file. - // Note that we do not actually use the Spark submit values for library path, classpath, - // and Java opts here, because we have already captured them in Bash. - - val newDriverMemory = submitDriverMemory - .orElse(confDriverMemory) - .getOrElse(defaultDriverMemory) - - val newClasspath = - if (submitClasspath.isDefined) { - classpath - } else { - classpath + confClasspath.map(sys.props("path.separator") + _).getOrElse("") - } - - val newJavaOpts = - if (submitJavaOpts.isDefined) { - // SPARK_SUBMIT_OPTS is already captured in JAVA_OPTS - javaOpts - } else { - javaOpts + confJavaOpts.map(" " + _).getOrElse("") - } - - val filteredJavaOpts = Utils.splitCommandString(newJavaOpts) - .filterNot(_.startsWith("-Xms")) - .filterNot(_.startsWith("-Xmx")) - - // Build up command - val command: Seq[String] = - Seq(runner) ++ - Seq("-cp", newClasspath) ++ - filteredJavaOpts ++ - Seq(s"-Xms$newDriverMemory", s"-Xmx$newDriverMemory") ++ - Seq("org.apache.spark.deploy.SparkSubmit") ++ - submitArgs - - // Print the launch command. This follows closely the format used in `bin/spark-class`. - if (sys.env.contains("SPARK_PRINT_LAUNCH_COMMAND")) { - System.err.print("Spark Command: ") - System.err.println(command.mkString(" ")) - System.err.println("========================================\n") - } - - // Start the driver JVM - val filteredCommand = command.filter(_.nonEmpty) - val builder = new ProcessBuilder(filteredCommand) - val env = builder.environment() - - if (submitLibraryPath.isEmpty && confLibraryPath.nonEmpty) { - val libraryPaths = confLibraryPath ++ sys.env.get(Utils.libraryPathEnvName) - env.put(Utils.libraryPathEnvName, libraryPaths.mkString(sys.props("path.separator"))) - } - - val process = builder.start() - - // If we kill an app while it's running, its sub-process should be killed too. - Runtime.getRuntime().addShutdownHook(new Thread() { - override def run() = { - if (process != null) { - process.destroy() - process.waitFor() - } - } - }) - - // Redirect stdout and stderr from the child JVM - val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout") - val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr") - stdoutThread.start() - stderrThread.start() - - // Redirect stdin to child JVM only if we're not running Windows. This is because the - // subprocess there already reads directly from our stdin, so we should avoid spawning a - // thread that contends with the subprocess in reading from System.in. - val isWindows = Utils.isWindows - val isSubprocess = sys.env.contains("IS_SUBPROCESS") - if (!isWindows) { - val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin", - propagateEof = true) - stdinThread.start() - // Spark submit (JVM) may run as a subprocess, and so this JVM should terminate on - // broken pipe, signaling that the parent process has exited. This is the case if the - // application is launched directly from python, as in the PySpark shell. In Windows, - // the termination logic is handled in java_gateway.py - if (isSubprocess) { - stdinThread.join() - process.destroy() - } - } - val returnCode = process.waitFor() - sys.exit(returnCode) - } - -} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 28e9662db5da..4450ff4e8b48 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -19,11 +19,14 @@ package org.apache.spark.deploy.worker import java.io.{File, FileOutputStream, InputStream, IOException} import java.lang.System._ +import java.util.{ArrayList, List => JList} +import scala.collection.JavaConversions._ import scala.collection.Map import org.apache.spark.Logging import org.apache.spark.deploy.Command +import org.apache.spark.launcher.AbstractLauncher import org.apache.spark.util.Utils /** @@ -54,12 +57,10 @@ object CommandUtils extends Logging { } private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val runner = sys.env.get("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") - // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows - Seq(runner) ++ buildJavaOpts(command, memory, sparkHome) ++ Seq(command.mainClass) ++ - command.arguments + val cmd = new CommandLauncher(sparkHome, memory, command.environment).buildLauncherCommand() + cmd.toSeq ++ Seq(command.mainClass) ++ command.arguments } /** @@ -92,34 +93,6 @@ object CommandUtils extends Logging { command.javaOpts) } - /** - * Attention: this must always be aligned with the environment variables in the run scripts and - * the way the JAVA_OPTS are assembled there. - */ - private def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") - - // Exists for backwards compatibility with older Spark versions - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString) - .getOrElse(Nil) - if (workerLocalOpts.length > 0) { - logWarning("SPARK_JAVA_OPTS was set on the worker. It is deprecated in Spark 1.0.") - logWarning("Set SPARK_LOCAL_DIRS for node-specific storage locations.") - } - - // Figure out our classpath with the external compute-classpath script - val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" - val classPath = Utils.executeAndGetOutput( - Seq(sparkHome + "/bin/compute-classpath" + ext), - extraEnvironment = command.environment) - val userClassPath = command.classPathEntries ++ Seq(classPath) - - val javaVersion = System.getProperty("java.version") - val permGenOpt = if (!javaVersion.startsWith("1.8")) Some("-XX:MaxPermSize=128m") else None - Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - permGenOpt ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts - } - /** Spawn a thread that will redirect a given stream to a file */ def redirectStream(in: InputStream, file: File) { val out = new FileOutputStream(file, true) @@ -137,3 +110,20 @@ object CommandUtils extends Logging { }.start() } } + +private class CommandLauncher(sparkHome: String, memory: Int, env: Map[String, String]) + extends AbstractLauncher[CommandLauncher](env) { + + setSparkHome(sparkHome) + + override def buildLauncherCommand(): JList[String] = { + val cmd = createJavaCommand() + cmd.add("-cp") + cmd.add(buildClassPath(null).mkString(File.pathSeparator)) + cmd.add(s"-Xms${memory}M") + cmd.add(s"-Xmx${memory}M") + addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) + cmd + } + +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 46a54c681840..b0f561e267a5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -34,11 +34,11 @@ object BuildCommons { val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, sql, networkCommon, networkShuffle, streaming, streamingFlumeSink, streamingFlume, streamingKafka, - streamingMqtt, streamingTwitter, streamingZeromq) = + streamingMqtt, streamingTwitter, streamingZeromq, launcher) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", "sql", "network-common", "network-shuffle", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", - "streaming-zeromq").map(ProjectRef(buildLocation, _)) + "streaming-zeromq", "launcher").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl", diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index a975dc19cb78..a7d2e689a95b 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -46,7 +46,6 @@ def launch_gateway(): def preexec_func(): signal.signal(signal.SIGINT, signal.SIG_IGN) env = dict(os.environ) - env["IS_SUBPROCESS"] = "1" # tell JVM to exit after python exits proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func, env=env) else: # preexec_fn not supported on Windows From 25c5ae6e0cfe084f96664405331e43a177123b9d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 24 Dec 2014 11:07:27 -0800 Subject: [PATCH 03/51] Centralize SparkSubmit command line parsing. Use a common base class to parse SparkSubmit command line arguments. This forces anyone who wants to add new arguments to modify the shared parser, updating all code that needs to know about SparkSubmit options in the process. Also create some constants to avoid copy & pasting strings around to actually process the options. --- .../spark/deploy/SparkSubmitArguments.scala | 141 ++++++------- launcher/pom.xml | 5 + .../launcher/SparkSubmitCliLauncher.java | 179 +++++++--------- .../launcher/SparkSubmitOptionParser.java | 197 ++++++++++++++++++ .../SparkSubmitOptionParserSuite.java | 135 ++++++++++++ 5 files changed, 473 insertions(+), 184 deletions(-) create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java create mode 100644 launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 1faabe91f49a..386ecba2222e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -18,17 +18,21 @@ package org.apache.spark.deploy import java.net.URI +import java.util.{List => JList} import java.util.jar.JarFile +import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} +import org.apache.spark.launcher.SparkSubmitOptionParser import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. * The env argument is used for testing. */ -private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) { +private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) + extends SparkSubmitOptionParser { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -73,7 +77,12 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } // Set parameters from command line arguments - parseOpts(args.toList) + try { + parse(args.toList) + } catch { + case e: IllegalArgumentException => + SparkSubmit.printErrorAndExit(e.getMessage()) + } // Populate `sparkProperties` map from properties file mergeDefaultSparkProperties() // Use `sparkProperties` map along with env vars to fill in any missing parameters @@ -224,138 +233,116 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St """.stripMargin } - /** - * Fill in values by parsing user options. - * NOTE: Any changes here must be reflected in YarnClientSchedulerBackend. - */ - private def parseOpts(opts: Seq[String]): Unit = { - val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r - - // Delineates parsing of Spark options from parsing of user options. - parse(opts) - - /** - * NOTE: If you add or remove spark-submit options, - * modify NOT ONLY this file but also utils.sh - */ - def parse(opts: Seq[String]): Unit = opts match { - case ("--name") :: value :: tail => + import SparkSubmitOptionParser._ + + /** Fill in values by parsing user options. */ + override protected def handle(opt: String, value: String): Boolean = { + opt match { + case NAME => name = value - parse(tail) - case ("--master") :: value :: tail => + case MASTER => master = value - parse(tail) - case ("--class") :: value :: tail => + case CLASS => mainClass = value - parse(tail) - case ("--deploy-mode") :: value :: tail => + case DEPLOY_MODE => if (value != "client" && value != "cluster") { SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") } deployMode = value - parse(tail) - case ("--num-executors") :: value :: tail => + case NUM_EXECUTORS => numExecutors = value - parse(tail) - case ("--total-executor-cores") :: value :: tail => + case TOTAL_EXECUTOR_CORES => totalExecutorCores = value - parse(tail) - case ("--executor-cores") :: value :: tail => + case EXECUTOR_CORES => executorCores = value - parse(tail) - case ("--executor-memory") :: value :: tail => + case EXECUTOR_MEMORY => executorMemory = value - parse(tail) - case ("--driver-memory") :: value :: tail => + case DRIVER_MEMORY => driverMemory = value - parse(tail) - case ("--driver-cores") :: value :: tail => + case DRIVER_CORES => driverCores = value - parse(tail) - case ("--driver-class-path") :: value :: tail => + case DRIVER_CLASS_PATH => driverExtraClassPath = value - parse(tail) - case ("--driver-java-options") :: value :: tail => + case DRIVER_JAVA_OPTIONS => driverExtraJavaOptions = value - parse(tail) - case ("--driver-library-path") :: value :: tail => + case DRIVER_LIBRARY_PATH => driverExtraLibraryPath = value - parse(tail) - case ("--properties-file") :: value :: tail => + case PROPERTIES_FILE => propertiesFile = value - parse(tail) - case ("--supervise") :: tail => + case SUPERVISE => supervise = true - parse(tail) - case ("--queue") :: value :: tail => + case QUEUE => queue = value - parse(tail) - case ("--files") :: value :: tail => + case FILES => files = Utils.resolveURIs(value) - parse(tail) - case ("--py-files") :: value :: tail => + case PY_FILES => pyFiles = Utils.resolveURIs(value) - parse(tail) - case ("--archives") :: value :: tail => + case ARCHIVES => archives = Utils.resolveURIs(value) - parse(tail) - case ("--jars") :: value :: tail => + case JARS => jars = Utils.resolveURIs(value) - parse(tail) - case ("--conf" | "-c") :: value :: tail => + case CONF => value.split("=", 2).toSeq match { case Seq(k, v) => sparkProperties(k) = v case _ => SparkSubmit.printErrorAndExit(s"Spark config without '=': $value") } - parse(tail) - case ("--help" | "-h") :: tail => + case HELP => printUsageAndExit(0) - case ("--verbose" | "-v") :: tail => + case VERBOSE => verbose = true - parse(tail) - case EQ_SEPARATED_OPT(opt, value) :: tail => - parse(opt :: value :: tail) + case _ => + throw new IllegalArgumentException(s"Unexpected argument '$opt'.") + } + true + } - case value :: tail if value.startsWith("-") => - SparkSubmit.printErrorAndExit(s"Unrecognized option '$value'.") + /** + * The first unrecognized option is treated as the "primary resource". Everything else is + * treated as application arguments. + */ + override protected def handleUnknown(opt: String): Boolean = { + if (opt.startsWith("-")) { + SparkSubmit.printErrorAndExit(s"Unrecognized option '$opt'.") + } - case value :: tail => - primaryResource = - if (!SparkSubmit.isShell(value) && !SparkSubmit.isInternal(value)) { - Utils.resolveURI(value).toString - } else { - value - } - isPython = SparkSubmit.isPython(value) - childArgs ++= tail + primaryResource = + if (!SparkSubmit.isShell(opt) && !SparkSubmit.isInternal(opt)) { + Utils.resolveURI(opt).toString + } else { + opt + } + isPython = SparkSubmit.isPython(opt) + false + } - case Nil => - } + override protected def handleExtraArgs(extra: JList[String]): Unit = { + childArgs ++= extra } + private def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = { val outStream = SparkSubmit.printStream if (unknownParam != null) { diff --git a/launcher/pom.xml b/launcher/pom.xml index 270689681fc1..47138c135eb1 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -42,6 +42,11 @@ log4j test + + org.mockito + mockito-all + test + org.scalatest scalatest_${scala.binary.version} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java index 187bd36d2810..46d4c726e217 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java @@ -37,35 +37,6 @@ */ public class SparkSubmitCliLauncher extends SparkLauncher { - /** List of spark-submit arguments that take an argument. */ - public static final List SPARK_SUBMIT_OPTS = Arrays.asList( - "--archives", - "--class", - "--conf", - "--deploy-mode", - "--driver-class-path", - "--driver-cores", - "--driver-java-options", - "--driver-library-path", - "--driver-memory", - "--executor-cores", - "--executor-memory", - "--files", - "--jars", - "--master", - "--name", - "--num-executors", - "--properties-file", - "--py-files", - "--queue", - "--total-executor-cores"); - - /** List of spark-submit arguments that do not take an argument. */ - public static final List SPARK_SUBMIT_SWITCHES = Arrays.asList( - "--supervise", - "--verbose", - "-v"); - /** * This map must match the class names for available shells, since this modifies the way * command line parsing works. This maps the shell class name to the resource to use when @@ -88,86 +59,9 @@ public class SparkSubmitCliLauncher extends SparkLauncher { boolean sparkSubmitOptionsEnded = false; this.driverArgs = new ArrayList(); this.isShell = isShell; - - Pattern eqSeparatedOpt = Pattern.compile("(--[^=]+)=(.+)"); - - for (Iterator it = args.iterator(); it.hasNext(); ) { - String arg = it.next(); - - Matcher m = eqSeparatedOpt.matcher(arg); - if (m.matches()) { - parseOpt(m.group(1), m.group(2), it); - } else { - parseOpt(arg, it); - } - } - } - - private void parseOpt(String arg, Iterator tail) { - if (SPARK_SUBMIT_SWITCHES.contains(arg)) { - addSparkArgs(arg); - } if (!SPARK_SUBMIT_OPTS.contains(arg)) { - parseOpt(arg, null, tail); - } else { - parseOpt(arg, getArgValue(tail, arg), tail); - } + new OptionParser().parse(args); } - private void parseOpt(String arg, String value, Iterator tail) { - if (!SPARK_SUBMIT_OPTS.contains(arg)) { - // When running a shell, add unrecognized parameters directly to the user arguments list. - // In normal mode, any unrecognized parameter triggers the end of command line parsing. - // The remaining params will be appended to the list of SparkSubmit arguments. - if (isShell) { - addArgs(arg); - } else { - addSparkArgs(arg); - while (tail.hasNext()) { - addSparkArgs(tail.next()); - } - } - } else if (arg.equals("--master")) { - setMaster(value); - driverArgs.add(arg); - driverArgs.add(value); - } else if (arg.equals("--deploy-mode")) { - setDeployMode(value); - driverArgs.add(arg); - driverArgs.add(value); - } else if (arg.equals("--properties-file")) { - setPropertiesFile(value); - driverArgs.add(arg); - driverArgs.add(value); - } else if (arg.equals("--driver-memory")) { - setConf(DRIVER_MEMORY, value); - driverArgs.add(arg); - driverArgs.add(value); - } else if (arg.equals("--driver-java-options")) { - setConf(DRIVER_JAVA_OPTIONS, value); - driverArgs.add(arg); - driverArgs.add(value); - } else if (arg.equals("--driver-library-path")) { - setConf(DRIVER_LIBRARY_PATH, value); - driverArgs.add(arg); - driverArgs.add(value); - } else if (arg.equals("--driver-class-path")) { - setConf(DRIVER_CLASSPATH, value); - driverArgs.add(arg); - driverArgs.add(value); - } else if (arg.equals("--class")) { - // The shell launchers require some special command line handling, since they allow - // mixing spark-submit arguments with arguments that should be propagated to the shell - // itself. Note that for this to work, the "--class" argument must come before any - // non-spark-submit arguments. - setClass(value); - if (shells.containsKey(value)) { - isShell = true; - setAppResource(shells.get(value)); - } - } else { - addSparkArgs(arg, value); - } - } /** Visible for PySparkLauncher. */ String getAppResource() { return userResource; @@ -193,4 +87,75 @@ private String getArgValue(Iterator it, String name) { return it.next(); } + private class OptionParser extends SparkSubmitOptionParser { + + @Override + protected boolean handle(String opt, String value) { + if (opt.equals(MASTER)) { + setMaster(value); + driverArgs.add(opt); + driverArgs.add(value); + } else if (opt.equals(DEPLOY_MODE)) { + setDeployMode(value); + driverArgs.add(opt); + driverArgs.add(value); + } else if (opt.equals(PROPERTIES_FILE)) { + setPropertiesFile(value); + driverArgs.add(opt); + driverArgs.add(value); + } else if (opt.equals(DRIVER_MEMORY)) { + setConf(LauncherCommon.DRIVER_MEMORY, value); + driverArgs.add(opt); + driverArgs.add(value); + } else if (opt.equals(DRIVER_JAVA_OPTIONS)) { + setConf(LauncherCommon.DRIVER_JAVA_OPTIONS, value); + driverArgs.add(opt); + driverArgs.add(value); + } else if (opt.equals(DRIVER_LIBRARY_PATH)) { + setConf(LauncherCommon.DRIVER_LIBRARY_PATH, value); + driverArgs.add(opt); + driverArgs.add(value); + } else if (opt.equals(DRIVER_CLASS_PATH)) { + setConf(LauncherCommon.DRIVER_CLASSPATH, value); + driverArgs.add(opt); + driverArgs.add(value); + } else if (opt.equals(CLASS)) { + // The shell launchers require some special command line handling, since they allow + // mixing spark-submit arguments with arguments that should be propagated to the shell + // itself. Note that for this to work, the "--class" argument must come before any + // non-spark-submit arguments. + setClass(value); + if (shells.containsKey(value)) { + isShell = true; + setAppResource(shells.get(value)); + } + } else { + addSparkArgs(opt, value); + } + return true; + } + + @Override + protected boolean handleUnknown(String opt) { + // When running a shell, add unrecognized parameters directly to the user arguments list. + // In normal mode, any unrecognized parameter triggers the end of command line parsing. + // The remaining params will be appended to the list of SparkSubmit arguments. + if (isShell) { + addArgs(opt); + return true; + } else { + addSparkArgs(opt); + return false; + } + } + + @Override + protected void handleExtraArgs(List extra) { + for (String arg : extra) { + addSparkArgs(arg); + } + } + + } + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java new file mode 100644 index 000000000000..eb322ae7c066 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -0,0 +1,197 @@ +/* + * 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.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Parser for spark-submit command line options. + *

+ * This class, although public, is not designed to be used outside of Spark. + *

+ * This class encapsulates the parsing code for spark-submit command line options, so that there + * is a single list of options that needs to be maintained (well, sort of, but it makes it harder + * to break things). + */ +public abstract class SparkSubmitOptionParser { + + // The following constants define the "main" name for the available options. They're defined + // to avoid copy & paste of the raw strings where they're needed. + protected static final String ARCHIVES = "--archives"; + protected static final String CLASS = "--class"; + protected static final String CONF = "--conf"; + protected static final String DEPLOY_MODE = "--deploy-mode"; + protected static final String DRIVER_CLASS_PATH = "--driver-class-path"; + protected static final String DRIVER_CORES = "--driver-cores"; + protected static final String DRIVER_JAVA_OPTIONS = "--driver-java-options"; + protected static final String DRIVER_LIBRARY_PATH = "--driver-library -path"; + protected static final String DRIVER_MEMORY = "--driver-memory"; + protected static final String EXECUTOR_CORES = "--executor-cores"; + protected static final String EXECUTOR_MEMORY = "--executor-memory"; + protected static final String FILES = "--files"; + protected static final String JARS = "--jars"; + protected static final String MASTER = "--master"; + protected static final String NAME = "--name"; + protected static final String NUM_EXECUTORS = "--num-executors"; + protected static final String PROPERTIES_FILE = "--properties-file"; + protected static final String PY_FILES = "--py-files"; + protected static final String QUEUE = "--queue"; + protected static final String TOTAL_EXECUTOR_CORES = "--total-executor-cores"; + + protected static final String HELP = "--help"; + protected static final String SUPERVISE = "--supervise"; + protected static final String VERBOSE = "--verbose"; + + /** + * This is the canonical list of spark-submit options. Each entry in the array contains the + * different aliases for the same option; the first element of each entry is the "official" + * name of the option, passed to {@link #handle(String, String)}. + *

+ * Options not listed here nor in the "switch" list below will result in a call to + * {@link $#handleUnknown(String)}. + */ + private final String[][] opts = { + { ARCHIVES }, + { CLASS }, + { CONF, "-c" }, + { DEPLOY_MODE }, + { DRIVER_CLASS_PATH }, + { DRIVER_CORES }, + { DRIVER_JAVA_OPTIONS }, + { DRIVER_LIBRARY_PATH }, + { DRIVER_MEMORY }, + { EXECUTOR_CORES }, + { EXECUTOR_MEMORY }, + { FILES }, + { JARS }, + { MASTER }, + { NAME }, + { NUM_EXECUTORS }, + { PROPERTIES_FILE }, + { PY_FILES }, + { QUEUE }, + { TOTAL_EXECUTOR_CORES }, + }; + + /** + * List of switches (command line options that do not take parameters) recognized by spark-submit. + */ + private final String[][] switches = { + { HELP, "-h" }, + { SUPERVISE }, + { VERBOSE, "-v" }, + }; + + /** + * Parse a list of spark-submit command line options. + *

+ * See SparkSubmitArguments.scala for a more formal description of available options. + * + * @throws IllegalArgumentException If an error is found during parsing. + */ + protected final void parse(List args) { + Pattern eqSeparatedOpt = Pattern.compile("(--[^=]+)=(.+)"); + + int idx = 0; + for (idx = 0; idx < args.size(); idx++) { + String arg = args.get(idx); + String value = null; + + Matcher m = eqSeparatedOpt.matcher(arg); + if (m.matches()) { + arg = m.group(1); + value = m.group(2); + } + + // Look for options with a value. + String name = findCliOption(arg, opts); + if (name != null) { + if (value == null) { + if (idx == args.size() - 1) { + throw new IllegalArgumentException( + String.format("Missing argument for option '%s'.", arg)); + } + idx++; + value = args.get(idx); + } + if (!handle(name, value)) { + break; + } + continue; + } + + // Look for a switch. + name = findCliOption(arg, switches); + if (name != null) { + if (!handle(name, null)) { + break; + } + continue; + } + + if (!handleUnknown(arg)) { + break; + } + } + + if (idx < args.size()) { + idx++; + } + handleExtraArgs(args.subList(idx, args.size())); + } + + /** + * Callback for when an option with an argument is parsed. + * + * @param opt The long name of the cli option (might differ from actual command line). + * @param value The value. This will be null if the option does not take a value. + * @return Whether to continue parsing the argument list. + */ + protected abstract boolean handle(String opt, String value); + + /** + * Callback for when an unrecognized option is parsed. + * + * @param opt Unrecognized option from the command line. + * @return Whether to continue parsing the argument list. + */ + protected abstract boolean handleUnknown(String opt); + + /** + * Callback for remaining command line arguments after either {@link #handle(String, String)} or + * {@link #handleUnknown(String)} return "false". This will be called at the end of parsing even + * when there are no remaining arguments. + * + * @param extra List of remaining arguments. + */ + protected abstract void handleExtraArgs(List extra); + + private String findCliOption(String name, String[][] available) { + for (String[] candidates : available) { + for (String candidate : candidates) { + if (candidate.equals(name)) { + return candidates[0]; + } + } + } + return null; + } + +} diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java new file mode 100644 index 000000000000..7c6551831182 --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java @@ -0,0 +1,135 @@ +/* + * 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.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import static org.apache.spark.launcher.SparkSubmitOptionParser.*; + +public class SparkSubmitOptionParserSuite { + + private SparkSubmitOptionParser parser; + + @Before + public void setUp() { + parser = spy(new DummyParser()); + } + + @Test + public void testAllOptions() { + List args = Arrays.asList( + ARCHIVES, ARCHIVES, + CLASS, CLASS, + CONF, CONF, + DEPLOY_MODE, DEPLOY_MODE, + DRIVER_CLASS_PATH, DRIVER_CLASS_PATH, + DRIVER_CORES, DRIVER_CORES, + DRIVER_JAVA_OPTIONS, DRIVER_JAVA_OPTIONS, + DRIVER_LIBRARY_PATH, DRIVER_LIBRARY_PATH, + DRIVER_MEMORY, DRIVER_MEMORY, + EXECUTOR_CORES, EXECUTOR_CORES, + EXECUTOR_MEMORY, EXECUTOR_MEMORY, + FILES, FILES, + JARS, JARS, + MASTER, MASTER, + NAME, NAME, + NUM_EXECUTORS, NUM_EXECUTORS, + PROPERTIES_FILE, PROPERTIES_FILE, + PY_FILES, PY_FILES, + QUEUE, QUEUE, + TOTAL_EXECUTOR_CORES, TOTAL_EXECUTOR_CORES, + HELP, + SUPERVISE, + VERBOSE); + + parser.parse(args); + verify(parser).handle(eq(ARCHIVES), eq(ARCHIVES)); + verify(parser).handle(eq(CLASS), eq(CLASS)); + verify(parser).handle(eq(CONF), eq(CONF)); + verify(parser).handle(eq(DEPLOY_MODE), eq(DEPLOY_MODE)); + verify(parser).handle(eq(DRIVER_CLASS_PATH), eq(DRIVER_CLASS_PATH)); + verify(parser).handle(eq(DRIVER_CORES), eq(DRIVER_CORES)); + verify(parser).handle(eq(DRIVER_JAVA_OPTIONS), eq(DRIVER_JAVA_OPTIONS)); + verify(parser).handle(eq(DRIVER_LIBRARY_PATH), eq(DRIVER_LIBRARY_PATH)); + verify(parser).handle(eq(DRIVER_MEMORY), eq(DRIVER_MEMORY)); + verify(parser).handle(eq(EXECUTOR_CORES), eq(EXECUTOR_CORES)); + verify(parser).handle(eq(EXECUTOR_MEMORY), eq(EXECUTOR_MEMORY)); + verify(parser).handle(eq(FILES), eq(FILES)); + verify(parser).handle(eq(JARS), eq(JARS)); + verify(parser).handle(eq(MASTER), eq(MASTER)); + verify(parser).handle(eq(NAME), eq(NAME)); + verify(parser).handle(eq(NUM_EXECUTORS), eq(NUM_EXECUTORS)); + verify(parser).handle(eq(PROPERTIES_FILE), eq(PROPERTIES_FILE)); + verify(parser).handle(eq(PY_FILES), eq(PY_FILES)); + verify(parser).handle(eq(QUEUE), eq(QUEUE)); + verify(parser).handle(eq(TOTAL_EXECUTOR_CORES), eq(TOTAL_EXECUTOR_CORES)); + verify(parser).handle(eq(HELP), same((String) null)); + verify(parser).handle(eq(SUPERVISE), same((String) null)); + verify(parser).handle(eq(VERBOSE), same((String) null)); + verify(parser).handleExtraArgs(eq(Collections.emptyList())); + } + + @Test + public void testExtraOptions() { + List args = Arrays.asList(MASTER, MASTER, "foo", "bar"); + parser.parse(args); + verify(parser).handle(eq(MASTER), eq(MASTER)); + verify(parser).handleUnknown(eq("foo")); + verify(parser).handleExtraArgs(eq(Arrays.asList("bar"))); + } + + @Test(expected=IllegalArgumentException.class) + public void testMissingArg() { + parser.parse(Arrays.asList(MASTER)); + } + + @Test + public void testEqualSeparatedOption() { + List args = Arrays.asList(MASTER + "=" + MASTER); + parser.parse(args); + verify(parser).handle(eq(MASTER), eq(MASTER)); + verify(parser).handleExtraArgs(eq(Collections.emptyList())); + } + + private static class DummyParser extends SparkSubmitOptionParser { + + @Override + protected boolean handle(String opt, String value) { + return true; + } + + @Override + protected boolean handleUnknown(String opt) { + return false; + } + + @Override + protected void handleExtraArgs(List extra) { + + } + + } + +} From 1b3f6e938ff90e1f0c3fc9e8402f47144834d456 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 5 Jan 2015 10:52:15 -0800 Subject: [PATCH 04/51] Call SparkSubmit from spark-class launcher for unknown classes. For new-style launchers, do the launching using SparkSubmit; hopefully this will be the preferred method of launching new daemons (if any). Currently it handles the thrift server daemon. --- .../apache/spark/launcher/AbstractLauncher.java | 8 ++++---- .../spark/launcher/SparkClassLauncher.java | 16 ++++++++++++++-- .../spark/launcher/SparkSubmitCliLauncher.java | 12 ++++++------ sbin/spark-daemon.sh | 11 ++--------- sbin/start-thriftserver.sh | 2 +- 5 files changed, 27 insertions(+), 22 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index 7cafcf410b97..06c118159a77 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -194,15 +194,15 @@ protected List buildClassPath(String appClassPath) throws IOException { } boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); - boolean isTesting = !isEmpty(getenv("SPARK_TESTING")); + boolean isTesting = "1".equals(getenv("SPARK_TESTING")); if (prependClasses || isTesting) { - System.err.println( - "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + - "assembly."); List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", "yarn", "launcher"); if (prependClasses) { + System.err.println( + "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + + "assembly."); for (String project : projects) { addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project, scala)); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java index d8e9571488f5..ed51c5ee8fba 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java @@ -42,8 +42,6 @@ class SparkClassLauncher extends AbstractLauncher { @Override protected List buildLauncherCommand() throws IOException { - List cmd = createJavaCommand(); - List javaOptsKeys = new ArrayList(); String memKey = null; String extraClassPath = null; @@ -88,8 +86,12 @@ protected List buildLauncherCommand() throws IOException { toolsDir.getAbsolutePath(), className); javaOptsKeys.add("SPARK_JAVA_OPTS"); + } else { + // Any classes not explicitly listed above are submitted using SparkSubmit. + return buildSparkSubmitCommand(); } + List cmd = createJavaCommand(); for (String key : javaOptsKeys) { addOptionString(cmd, System.getenv(key)); } @@ -104,4 +106,14 @@ protected List buildLauncherCommand() throws IOException { return prepareForOs(cmd, null, Collections.emptyMap()); } + private List buildSparkSubmitCommand() throws IOException { + List sparkSubmitArgs = new ArrayList(classArgs); + sparkSubmitArgs.add(SparkSubmitOptionParser.CLASS); + sparkSubmitArgs.add(className); + + SparkSubmitCliLauncher launcher = new SparkSubmitCliLauncher(true, sparkSubmitArgs); + launcher.setAppResource("spark-internal"); + return launcher.buildLauncherCommand(); + } + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java index 46d4c726e217..b2d8077d90c8 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java @@ -49,16 +49,16 @@ public class SparkSubmitCliLauncher extends SparkLauncher { } private final List driverArgs; - private boolean isShell; + private boolean hasMixedArguments; SparkSubmitCliLauncher(List args) { this(false, args); } - SparkSubmitCliLauncher(boolean isShell, List args) { + SparkSubmitCliLauncher(boolean hasMixedArguments, List args) { boolean sparkSubmitOptionsEnded = false; this.driverArgs = new ArrayList(); - this.isShell = isShell; + this.hasMixedArguments = hasMixedArguments; new OptionParser().parse(args); } @@ -126,7 +126,7 @@ protected boolean handle(String opt, String value) { // non-spark-submit arguments. setClass(value); if (shells.containsKey(value)) { - isShell = true; + hasMixedArguments = true; setAppResource(shells.get(value)); } } else { @@ -137,10 +137,10 @@ protected boolean handle(String opt, String value) { @Override protected boolean handleUnknown(String opt) { - // When running a shell, add unrecognized parameters directly to the user arguments list. + // When mixing arguments, add unrecognized parameters directly to the user arguments list. // In normal mode, any unrecognized parameter triggers the end of command line parsing. // The remaining params will be appended to the list of SparkSubmit arguments. - if (isShell) { + if (hasMixedArguments) { addArgs(opt); return true; } else { diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 89608bc41b71..51a4262ac141 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -124,7 +124,7 @@ fi case $option in - (start|spark-submit) + (start) mkdir -p "$SPARK_PID_DIR" @@ -142,14 +142,7 @@ case $option in spark_rotate_log "$log" echo starting $command, logging to $log - if [ $option == spark-submit ]; then - source "$SPARK_HOME"/bin/utils.sh - gatherSparkSubmitOpts "$@" - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-submit --class $command \ - "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" >> "$log" 2>&1 < /dev/null & - else - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & - fi + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & newpid=$! echo $newpid > $pid sleep 2 diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 50e8e06418b0..09017839b81f 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -50,4 +50,4 @@ fi export SUBMIT_USAGE_FUNCTION=usage -exec "$FWDIR"/sbin/spark-daemon.sh spark-submit $CLASS 1 "$@" +exec "$FWDIR"/sbin/spark-daemon.sh start $CLASS 1 "$@" From 7a01e4adaeec0e2d8a27c3231bec90a2a09b1cbb Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 6 Jan 2015 13:16:10 -0800 Subject: [PATCH 05/51] Fix pyspark on Yarn. pyspark (at least) relies on SPARK_HOME (the env variable) to be set for things to work properly. The launcher wasn't making sure that variable was set in all cases, so do that. Also, separately, the Yarn backend didn't seem to propagate that variable to the AM for some reason, so do that too. (Not sure how things worked previously...) Extra: add ".pyo" files to .gitignore (these are generated by `python -O`). --- .gitignore | 1 + bin/pyspark | 2 +- bin/spark-shell | 2 +- bin/spark-shell2.cmd | 2 +- .../apache/spark/launcher/AbstractLauncher.java | 15 ++++++++++++--- .../org/apache/spark/deploy/yarn/ClientBase.scala | 7 +++++++ 6 files changed, 23 insertions(+), 6 deletions(-) diff --git a/.gitignore b/.gitignore index 9757054a50f9..d162fa9cca99 100644 --- a/.gitignore +++ b/.gitignore @@ -6,6 +6,7 @@ *.iml *.iws *.pyc +*.pyo .idea/ .idea_modules/ build/*.jar diff --git a/bin/pyspark b/bin/pyspark index 00cca0b2b3a2..ccf4356d2f87 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -96,4 +96,4 @@ fi export PYSPARK_DRIVER_PYTHON export PYSPARK_DRIVER_PYTHON_OPTS -exec $SPARK_HOME/bin/spark-class pyspark "$@" +exec "$SPARK_HOME"/bin/spark-class pyspark "$@" diff --git a/bin/spark-shell b/bin/spark-shell index 57672b102e9e..80cab8ad1d3c 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -42,7 +42,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then fi # SPARK-4161: scala does not assume use of the java classpath, -# so we need to add the "-Dscala.usejavacp=true" flag mnually. We +# so we need to add the "-Dscala.usejavacp=true" flag manually. We # do this specifically for the Spark shell because the scala REPL # has its own class loader, and any additional classpath specified # through spark.driver.extraClassPath is not automatically propagated. diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index aadb2d00d741..2f791fa9cd98 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -26,7 +26,7 @@ if %ERRORLEVEL% equ 0 ( ) rem SPARK-4161: scala does not assume use of the java classpath, -rem so we need to add the "-Dscala.usejavacp=true" flag mnually. We +rem so we need to add the "-Dscala.usejavacp=true" flag manually. We rem do this specifically for the Spark shell because the scala REPL rem has its own class loader, and any additional classpath specified rem through spark.driver.extraClassPath is not automatically propagated. diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index 06c118159a77..8f7f35561eed 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -331,11 +331,20 @@ protected String getScalaVersion() { protected List prepareForOs(List cmd, String libPath, Map env) { + + // If SPARK_HOME does not come from the environment, explicitly set it + // in the child's environment. + Map childEnv = env; + if (System.getenv("SPARK_HOME") == null && !env.containsKey("SPARK_HOME")) { + childEnv = new HashMap(env); + childEnv.put("SPARK_HOME", sparkHome); + } + if (isWindows()) { - return prepareForWindows(cmd, libPath, env); + return prepareForWindows(cmd, libPath, childEnv); } - if (isEmpty(libPath) && env.isEmpty()) { + if (isEmpty(libPath) && childEnv.isEmpty()) { return cmd; } @@ -348,7 +357,7 @@ protected List prepareForOs(List cmd, String newEnvValue = join(File.pathSeparator, currEnvValue, libPath); newCmd.add(String.format("%s=%s", envName, newEnvValue)); } - for (Map.Entry e : env.entrySet()) { + for (Map.Entry e : childEnv.entrySet()) { newCmd.add(String.format("%s=%s", e.getKey(), e.getValue())); } newCmd.addAll(cmd); diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index eb97a7b3c59a..212071d76cac 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -242,6 +242,13 @@ private[spark] trait ClientBase extends Logging { env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() + // Propagate SPARK_HOME to the containers. This is needed for pyspark to + // work, since the executor's PYTHONPATH is built based on the location + // of SPARK_HOME. + sparkConf.getOption("spark.home").orElse(sys.env.get("SPARK_HOME")).foreach { path => + env("SPARK_HOME") = path + } + // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) From 4d511e76ca15702bb0903fec6f677a11b25b809f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 6 Jan 2015 13:53:22 -0800 Subject: [PATCH 06/51] Fix tools search code. --- .../java/org/apache/spark/launcher/SparkClassLauncher.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java index ed51c5ee8fba..2e42dbf5bda0 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java @@ -69,11 +69,11 @@ protected List buildLauncherCommand() throws IOException { memKey = "SPARK_EXECUTOR_MEMORY"; } else if (className.startsWith("org.apache.spark.tools.")) { String sparkHome = getSparkHome(); - File toolsDir = new File(join(sparkHome, "tools", "target", + File toolsDir = new File(join(File.separator, sparkHome, "tools", "target", "scala-" + getScalaVersion())); checkState(toolsDir.isDirectory(), "Cannot find tools build directory."); - Pattern re = Pattern.compile("spark-tools-.*\\.jar"); + Pattern re = Pattern.compile("spark-tools_.*\\.jar"); for (File f : toolsDir.listFiles()) { if (re.matcher(f.getName()).matches()) { extraClassPath = f.getAbsolutePath(); From 656374e4c1a2599c1f10c35d2f7e7e08521ee390 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 6 Jan 2015 13:57:23 -0800 Subject: [PATCH 07/51] Mima fixes. --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b0f561e267a5..5ee6e00d370a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -165,8 +165,9 @@ object SparkBuild extends PomBuild { (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) // TODO: Add Sql to mima checks + // TODO: remove launcher from this list after 1.3. allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - networkCommon, networkShuffle, networkYarn).contains(x)).foreach { + networkCommon, networkShuffle, networkYarn, launcher).contains(x)).foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) } From a7936ef9f7b068ea2e5d2b12babc32fe9eeb08d0 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 6 Jan 2015 20:52:50 -0800 Subject: [PATCH 08/51] Fix pyspark tests. --- bin/pyspark | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/pyspark b/bin/pyspark index ccf4356d2f87..6baece5bfcfb 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -18,7 +18,7 @@ # # Figure out where Spark is installed -SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" source "$SPARK_HOME"/bin/load-spark-env.sh From 2f4e8b456ed9af78c69cf551ebed8f0015f828d6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 9 Jan 2015 10:11:07 -0800 Subject: [PATCH 09/51] Changes needed to make this work with SPARK-4048. Also some minor tweaks for the maven build. --- launcher/pom.xml | 19 +++++++++++++++++++ .../spark/launcher/AbstractLauncher.java | 3 ++- pom.xml | 8 +++++++- 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/launcher/pom.xml b/launcher/pom.xml index 47138c135eb1..92213671bc16 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -42,6 +42,11 @@ log4j test + + junit + junit + test + org.mockito mockito-all @@ -62,12 +67,26 @@ slf4j-log4j12 test + + + + org.apache.hadoop + hadoop-client + test + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + org.apache.maven.plugins + maven-jar-plugin + + target/scala-${scala.binary.version} + + org.scalatest scalatest-maven-plugin diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index 8f7f35561eed..264ec8243189 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -275,6 +275,7 @@ protected List buildClassPath(String appClassPath) throws IOException { addToClassPath(cp, getenv("HADOOP_CONF_DIR")); addToClassPath(cp, getenv("YARN_CONF_DIR")); + addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH")); return cp; } @@ -511,7 +512,7 @@ public boolean accept(File file) { } private String getenv(String key) { - return (env != null) ? env.get(key) : System.getenv(key); + return first(env != null ? env.get(key) : null, System.getenv(key)); } /** diff --git a/pom.xml b/pom.xml index dcf1e35c9f61..0fe53f3bdfde 100644 --- a/pom.xml +++ b/pom.xml @@ -1122,13 +1122,19 @@ ${project.build.directory}/surefire-reports -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m + + + ${test_classpath} + true ${session.executionRootDirectory} 1 false false - ${test_classpath} true From f26556b498cdae3fa23ea5837d673b4f5cb98c58 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 9 Jan 2015 16:30:55 -0800 Subject: [PATCH 10/51] Fix a thread-safety issue in "local" mode. The issue is that SparkConf is not thread-safe; so it was possible for the executor thread to try to read the configuration while the context thread was modifying it. In my tests this caused the executor to consistently miss the "spark.driver.port" config and fail tests. Long term, it would probably be better to investigate using a concurrent map implementation in SparkConf (instead of a HashMap). --- .../scala/org/apache/spark/executor/Executor.scala | 11 +++++++++-- .../apache/spark/scheduler/local/LocalBackend.scala | 8 +++++--- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 0f99cd9f3b08..80c9de843ef3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -43,9 +43,18 @@ private[spark] class Executor( executorId: String, slaveHostname: String, env: SparkEnv, + conf: SparkConf, isLocal: Boolean = false) extends Logging { + + def this(executorId: String, + slaveHostname: String, + env: SparkEnv, + isLocal: Boolean = false) = { + this(executorId, slaveHostname, env, env.conf, isLocal) + } + // Application dependencies (added through SparkContext) that we've fetched so far on this node. // Each map holds the master's timestamp for the version of that file or JAR we got. private val currentFiles: HashMap[String, Long] = new HashMap[String, Long]() @@ -53,8 +62,6 @@ private[spark] class Executor( private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) - private val conf = env.conf - @volatile private var isStopped = false // No ip or host:port - just hostname diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index b3bd3110ac80..d21e2daed118 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import akka.actor.{Actor, ActorRef, Props} -import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} @@ -43,6 +43,7 @@ private case class StopExecutor() private[spark] class LocalActor( scheduler: TaskSchedulerImpl, executorBackend: LocalBackend, + conf: SparkConf, private val totalCores: Int) extends Actor with ActorLogReceive with Logging { @@ -52,7 +53,7 @@ private[spark] class LocalActor( private val localExecutorHostname = "localhost" private val executor = new Executor( - localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true) + localExecutorId, localExecutorHostname, SparkEnv.get, conf, isLocal = true) override def receiveWithLogging = { case ReviveOffers => @@ -90,11 +91,12 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: extends SchedulerBackend with ExecutorBackend { private val appId = "local-" + System.currentTimeMillis + private val conf = SparkEnv.get.conf.clone() var localActor: ActorRef = null override def start() { localActor = SparkEnv.get.actorSystem.actorOf( - Props(new LocalActor(scheduler, this, totalCores)), + Props(new LocalActor(scheduler, this, conf, totalCores)), "LocalBackendActor") } From c617539320b001d53fab74cb8bf04675768fa37f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 13 Jan 2015 15:45:04 -0800 Subject: [PATCH 11/51] Review feedback round 1. --- bin/pyspark2.cmd | 1 - .../spark/deploy/worker/CommandUtils.scala | 2 +- .../spark/launcher/AbstractLauncher.java | 223 ++++-------------- .../apache/spark/launcher/LauncherCommon.java | 134 ++++++++++- .../java/org/apache/spark/launcher/Main.java | 10 +- .../spark/launcher/PySparkLauncher.java | 28 ++- .../spark/launcher/SparkClassLauncher.java | 7 +- .../apache/spark/launcher/SparkLauncher.java | 102 ++++---- .../launcher/SparkSubmitCliLauncher.java | 46 ++-- .../launcher/SparkSubmitOptionParser.java | 13 +- ...herSuite.java => LauncherCommonSuite.java} | 16 +- .../spark/launcher/SparkLauncherSuite.java | 32 +-- 12 files changed, 305 insertions(+), 309 deletions(-) rename launcher/src/test/java/org/apache/spark/launcher/{AbstractLauncherSuite.java => LauncherCommonSuite.java} (88%) diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index b4e6b0a43c47..ff5862d049f9 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -35,5 +35,4 @@ set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py -echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH% call %SPARK_HOME%\bin\spark-class2.cmd pyspark %* diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 4450ff4e8b48..d1947523ad3c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -117,7 +117,7 @@ private class CommandLauncher(sparkHome: String, memory: Int, env: Map[String, S setSparkHome(sparkHome) override def buildLauncherCommand(): JList[String] = { - val cmd = createJavaCommand() + val cmd = buildJavaCommand() cmd.add("-cp") cmd.add(buildClassPath(null).mkString(File.pathSeparator)) cmd.add(s"-Xms${memory}M") diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index 264ec8243189..995db4b726f4 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -25,12 +25,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Enumeration; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.jar.JarEntry; import java.util.jar.JarFile; import java.util.regex.Pattern; @@ -39,6 +38,7 @@ */ public abstract class AbstractLauncher extends LauncherCommon { + private static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; protected static final String DEFAULT_MEM = "512m"; protected String javaHome; @@ -93,6 +93,11 @@ public T setConf(String key, String value) { */ protected abstract List buildLauncherCommand() throws IOException; + /** + * Loads the configuration file for the application, if it exists. This is either the + * user-specified properties file, or the spark-defaults.conf file under the Spark configuration + * directory. + */ protected Properties loadPropertiesFile() throws IOException { Properties props = new Properties(); File propsFile; @@ -100,11 +105,7 @@ protected Properties loadPropertiesFile() throws IOException { propsFile = new File(propertiesFile); checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile); } else { - String confDir = getenv("SPARK_CONF_DIR"); - if (confDir == null) { - confDir = join(File.separator, getSparkHome(), "conf"); - } - propsFile = new File(confDir, "spark-defaults.conf"); + propsFile = new File(getConfDir(), DEFAULT_PROPERTIES_FILE); } if (propsFile.isFile()) { @@ -127,16 +128,16 @@ protected Properties loadPropertiesFile() throws IOException { } protected String getSparkHome() { - String path = first(sparkHome, getenv("SPARK_HOME")); + String path = firstNonEmpty(sparkHome, getenv("SPARK_HOME")); checkState(path != null, - "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); + "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); return path; } - protected List createJavaCommand() throws IOException { + protected List buildJavaCommand() throws IOException { List cmd = new ArrayList(); if (javaHome == null) { - cmd.add(join(File.separator, System.getProperty("java.home"), "..", "bin", "java")); + cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); } else { cmd.add(join(File.separator, javaHome, "bin", "java")); } @@ -186,12 +187,7 @@ protected List buildClassPath(String appClassPath) throws IOException { addToClassPath(cp, getenv("SPARK_CLASSPATH")); addToClassPath(cp, appClassPath); - String confDir = getenv("SPARK_CONF_DIR"); - if (!isEmpty(confDir)) { - addToClassPath(cp, confDir); - } else { - addToClassPath(cp, join(File.separator, getSparkHome(), "conf")); - } + addToClassPath(cp, getConfDir()); boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); boolean isTesting = "1".equals(getenv("SPARK_TESTING")); @@ -236,7 +232,7 @@ protected List buildClassPath(String appClassPath) throws IOException { } catch (IOException ioe) { if (ioe.getMessage().indexOf("invalid CEN header") > 0) { System.err.println( - "Loading Spark jar with '$JAR_CMD' failed.\n" + + "Loading Spark jar failed.\n" + "This is likely because Spark was compiled with Java 7 and run\n" + "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + "or build Spark with Java 6."); @@ -279,6 +275,12 @@ protected List buildClassPath(String appClassPath) throws IOException { return cp; } + /** + * Adds entries to the classpath. + * + * @param cp List where to appended the new classpath entries. + * @param entries New classpath entries (separated by File.pathSeparator). + */ private void addToClassPath(List cp, String entries) { if (isEmpty(entries)) { return; @@ -317,7 +319,14 @@ protected String getScalaVersion() { throw new IllegalStateException("Should not reach here."); } + protected List prepareForOs(List cmd, String libPath) { + return prepareForOs(cmd, libPath, Collections.emptyMap()); + } + /** + * Prepare the command for execution under the current OS, setting the passed environment + * variables. + * * Which OS is running defines two things: * - the name of the environment variable used to define the lookup path for native libs * - how to execute the command in general. @@ -329,7 +338,8 @@ protected String getScalaVersion() { * * For Win32, see {@link #prepareForWindows(List,String)}. */ - protected List prepareForOs(List cmd, + protected List prepareForOs( + List cmd, String libPath, Map env) { @@ -365,128 +375,6 @@ protected List prepareForOs(List cmd, return newCmd; } - protected String shQuote(String s) { - StringBuilder quoted = new StringBuilder(); - boolean hasWhitespace = false; - for (int i = 0; i < s.length(); i++) { - if (Character.isWhitespace(s.codePointAt(i))) { - quoted.append('"'); - hasWhitespace = true; - break; - } - } - - for (int i = 0; i < s.length(); i++) { - int cp = s.codePointAt(i); - switch (cp) { - case '\'': - if (hasWhitespace) { - quoted.appendCodePoint(cp); - break; - } - case '"': - case '\\': - quoted.append('\\'); - // Fall through. - default: - if (Character.isWhitespace(cp)) { - hasWhitespace=true; - } - quoted.appendCodePoint(cp); - } - } - if (hasWhitespace) { - quoted.append('"'); - } - return quoted.toString(); - } - - // Visible for testing. - List parseOptionString(String s) { - List opts = new ArrayList(); - StringBuilder opt = new StringBuilder(); - boolean inOpt = false; - boolean inSingleQuote = false; - boolean inDoubleQuote = false; - boolean escapeNext = false; - boolean hasData = false; - - for (int i = 0; i < s.length(); i++) { - int c = s.codePointAt(i); - if (escapeNext) { - if (!inOpt) { - inOpt = true; - } - opt.appendCodePoint(c); - escapeNext = false; - } else if (inOpt) { - switch (c) { - case '\\': - if (inSingleQuote) { - opt.appendCodePoint(c); - } else { - escapeNext = true; - } - break; - case '\'': - if (inDoubleQuote) { - opt.appendCodePoint(c); - } else { - inSingleQuote = !inSingleQuote; - } - break; - case '"': - if (inSingleQuote) { - opt.appendCodePoint(c); - } else { - inDoubleQuote = !inDoubleQuote; - } - break; - default: - if (inSingleQuote || inDoubleQuote || !Character.isWhitespace(c)) { - opt.appendCodePoint(c); - } else { - finishOpt(opts, opt); - inOpt = false; - hasData = false; - } - } - } else { - switch (c) { - case '\'': - inSingleQuote = true; - inOpt = true; - hasData = true; - break; - case '"': - inDoubleQuote = true; - inOpt = true; - hasData = true; - break; - case '\\': - escapeNext = true; - break; - default: - if (!Character.isWhitespace(c)) { - inOpt = true; - opt.appendCodePoint(c); - } - } - } - } - - checkArgument(!inSingleQuote && !inDoubleQuote && !escapeNext, "Invalid option string: %s", s); - if (opt.length() > 0 || hasData) { - opts.add(opt.toString()); - } - return opts; - } - - private void finishOpt(List opts, StringBuilder opt) { - opts.add(opt.toString()); - opt.setLength(0); - } - private String findAssembly(String scalaVersion) { String sparkHome = getSparkHome(); File libdir; @@ -512,7 +400,12 @@ public boolean accept(File file) { } private String getenv(String key) { - return first(env != null ? env.get(key) : null, System.getenv(key)); + return firstNonEmpty(env != null ? env.get(key) : null, System.getenv(key)); + } + + private String getConfDir() { + String confDir = getenv("SPARK_CONF_DIR"); + return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); } /** @@ -526,8 +419,12 @@ private String getenv(String key) { * - 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 in a single line, since that's the + * easiest way to consume this from a batch script (see spark-class2.cmd). */ - private List prepareForWindows(List cmd, + private List prepareForWindows( + List cmd, String libPath, Map env) { StringBuilder cmdline = new StringBuilder("cmd /c \""); @@ -535,6 +432,9 @@ private List prepareForWindows(List cmd, cmdline.append("set PATH=%PATH%;").append(libPath).append(" &&"); } for (Map.Entry e : env.entrySet()) { + if (cmdline.length() > 0) { + cmdline.append(" "); + } cmdline.append(String.format("set %s=%s", e.getKey(), e.getValue())); cmdline.append(" &&"); } @@ -542,27 +442,25 @@ private List prepareForWindows(List cmd, if (cmdline.length() > 0) { cmdline.append(" "); } - cmdline.append(quote(arg)); + cmdline.append(quoteForBatchScript(arg)); } cmdline.append("\""); return Arrays.asList(cmdline.toString()); } /** - * Quoting arguments that don't need quoting in Windows seems to cause weird issues. So only - * quote arguments when there is whitespace in them. + * 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 boolean needsQuoting(String arg) { + private String quoteForBatchScript(String arg) { + boolean needsQuotes = false; for (int i = 0; i < arg.length(); i++) { if (Character.isWhitespace(arg.codePointAt(i))) { - return true; + needsQuotes = true; + break; } } - return false; - } - - private String quote(String arg) { - if (!needsQuoting(arg)) { + if (!needsQuotes) { return arg; } StringBuilder quoted = new StringBuilder(); @@ -578,23 +476,4 @@ private String quote(String arg) { return quoted.toString(); } - // Visible for testing. - String getLibPathEnvName() { - if (isWindows()) { - return "PATH"; - } - - String os = System.getProperty("os.name"); - if (os.startsWith("Mac OS X")) { - return "DYLD_LIBRARY_PATH"; - } else { - return "LD_LIBRARY_PATH"; - } - } - - protected boolean isWindows() { - String os = System.getProperty("os.name"); - return os.startsWith("Windows"); - } - } diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java index 004c43019592..a76bc6ebb5fc 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java @@ -17,6 +17,8 @@ package org.apache.spark.launcher; +import java.util.ArrayList; +import java.util.List; import java.util.Map; /** @@ -30,27 +32,29 @@ public class LauncherCommon { /** 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_CLASSPATH = "spark.driver.extraClassPath"; + public static final String DRIVER_EXTRA_CLASSPATH = "spark.driver.extraClassPath"; /** Configuration key for the driver VM options. */ - public static final String DRIVER_JAVA_OPTIONS = "spark.driver.extraJavaOptions"; + public static final String DRIVER_EXTRA_JAVA_OPTIONS = "spark.driver.extraJavaOptions"; /** Configuration key for the driver native library path. */ - public static final String DRIVER_LIBRARY_PATH = "spark.driver.extraLibraryPath"; + 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_CLASSPATH = "spark.executor.extraClassPath"; + public static final String EXECUTOR_EXTRA_CLASSPATH = "spark.executor.extraClassPath"; /** Configuration key for the executor VM options. */ - public static final String EXECUTOR_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; + public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; /** Configuration key for the executor native library path. */ - public static final String EXECUTOR_LIBRARY_PATH = "spark.executor.extraLibraryOptions"; + 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"; + /** Returns whether the given string is null or empty. */ protected static boolean isEmpty(String s) { return s == null || s.isEmpty(); } + /** Joins a list of strings using the given separator. */ protected static String join(String sep, String... elements) { StringBuilder sb = new StringBuilder(); for (String e : elements) { @@ -64,6 +68,7 @@ protected static String join(String sep, String... elements) { return sb.toString(); } + /** Joins a list of strings using the given separator. */ protected static String join(String sep, Iterable elements) { StringBuilder sb = new StringBuilder(); for (String e : elements) { @@ -77,6 +82,7 @@ protected static String join(String sep, Iterable elements) { return sb.toString(); } + /** Returns the first value mapped to the given key in the given maps. */ protected static String find(String key, Map... maps) { for (Map map : maps) { String value = (String) map.get(key); @@ -87,7 +93,8 @@ protected static String find(String key, Map... maps) { return null; } - protected static String first(String... candidates) { + /** Returns the first non-empty, non-null string in the given list. */ + protected static String firstNonEmpty(String... candidates) { for (String s : candidates) { if (!isEmpty(s)) { return s; @@ -96,18 +103,129 @@ protected static String first(String... candidates) { return null; } + /** Returns the name of the env variable that holds the native library path. */ + protected static String getLibPathEnvName() { + if (isWindows()) { + return "PATH"; + } + + String os = System.getProperty("os.name"); + if (os.startsWith("Mac OS X")) { + return "DYLD_LIBRARY_PATH"; + } else { + return "LD_LIBRARY_PATH"; + } + } + + /** Returns whether the OS is Windows. */ + protected static boolean isWindows() { + String os = System.getProperty("os.name"); + return os.startsWith("Windows"); + } + + /** + * Parse a string as if it were a list of arguments, in the way that a shell would. + * This tries to follow the way bash parses strings. For example: + * + * Input: "\"ab cd\" efgh 'i \" j'" + * Output: [ "ab cd", "efgh", "i \" j" ] + */ + protected static List parseOptionString(String s) { + List opts = new ArrayList(); + StringBuilder opt = new StringBuilder(); + boolean inOpt = false; + boolean inSingleQuote = false; + boolean inDoubleQuote = false; + boolean escapeNext = false; + boolean hasData = false; + + for (int i = 0; i < s.length(); i++) { + int c = s.codePointAt(i); + if (escapeNext) { + opt.appendCodePoint(c); + escapeNext = false; + } else if (inOpt) { + switch (c) { + case '\\': + if (inSingleQuote) { + opt.appendCodePoint(c); + } else { + escapeNext = true; + } + break; + case '\'': + if (inDoubleQuote) { + opt.appendCodePoint(c); + } else { + inSingleQuote = !inSingleQuote; + } + break; + case '"': + if (inSingleQuote) { + opt.appendCodePoint(c); + } else { + inDoubleQuote = !inDoubleQuote; + } + break; + default: + if (inSingleQuote || inDoubleQuote || !Character.isWhitespace(c)) { + opt.appendCodePoint(c); + } else { + opts.add(opt.toString()); + opt.setLength(0); + inOpt = false; + hasData = false; + } + } + } else { + switch (c) { + case '\'': + inSingleQuote = true; + inOpt = true; + hasData = true; + break; + case '"': + inDoubleQuote = true; + inOpt = true; + hasData = true; + break; + case '\\': + escapeNext = true; + inOpt = true; + hasData = true; + break; + default: + if (!Character.isWhitespace(c)) { + inOpt = true; + hasData = true; + opt.appendCodePoint(c); + } + } + } + } + + checkArgument(!inSingleQuote && !inDoubleQuote && !escapeNext, "Invalid option string: %s", s); + if (hasData) { + opts.add(opt.toString()); + } + return opts; + } + + /** Throws IllegalArgumentException if the given object is null. */ protected static void checkNotNull(Object o, String arg) { if (o == null) { throw new IllegalArgumentException(String.format("'%s' must not be null.", arg)); } } + /** Throws IllegalArgumentException with the given message if the check is false. */ protected static void checkArgument(boolean check, String msg, Object... args) { if (!check) { throw new IllegalArgumentException(String.format(msg, args)); } } + /** Throws IllegalStateException with the given message if the check is false. */ protected static void checkState(boolean check, String msg, Object... args) { if (!check) { throw new IllegalStateException(String.format(msg, args)); @@ -117,4 +235,4 @@ protected static void checkState(boolean check, String msg, Object... args) { // To avoid subclassing outside this package. LauncherCommon() { } -} \ No newline at end of file +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index 497c738614b6..ce1cdb3d3601 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -17,13 +17,9 @@ package org.apache.spark.launcher; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; -import java.util.Iterator; import java.util.List; -import java.util.Map; /** * Command line interface for the Spark launcher. Used internally by Spark scripts. @@ -50,20 +46,20 @@ public static void main(String[] argsArray) throws Exception { List args = new ArrayList(Arrays.asList(argsArray)); String className = args.remove(0); - boolean printLaunchCommand = false; + boolean printLaunchCommand; AbstractLauncher launcher; try { if (className.equals("org.apache.spark.deploy.SparkSubmit")) { launcher = new SparkSubmitCliLauncher(args); - printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); } else if (className.equals("pyspark")) { launcher = new PySparkLauncher(args); } else { launcher = new SparkClassLauncher(className, args); - printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); } + printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); } catch (IllegalArgumentException e) { launcher = new UsageLauncher(); + printLaunchCommand = false; } List cmd = launcher.buildLauncherCommand(); diff --git a/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java index 6786f395990d..2caa4fb2fb26 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java @@ -44,7 +44,7 @@ protected List buildLauncherCommand() throws IOException { // For backwards compatibility, if a script is specified in // the pyspark command line, then run it using spark-submit. - if (!launcher.getArgs().isEmpty() && launcher.getArgs().get(0).endsWith(".py")) { + if (!launcher.getAppArgs().isEmpty() && launcher.getAppArgs().get(0).endsWith(".py")) { System.err.println( "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + "Use ./bin/spark-submit "); @@ -54,24 +54,24 @@ protected List buildLauncherCommand() throws IOException { // When launching the pyspark shell, the spark-submit arguments should be stored in the // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS. - checkArgument(launcher.getArgs().isEmpty(), - "pyspark does not support any application options."); + checkArgument(launcher.getAppArgs().isEmpty(), + "pyspark does not support any application options."); Properties props = loadPropertiesFile(); - String libPath = find(DRIVER_LIBRARY_PATH, conf, props); + String libPath = find(DRIVER_EXTRA_LIBRARY_PATH, conf, props); StringBuilder submitArgs = new StringBuilder(); for (String arg : launcher.getSparkArgs()) { if (submitArgs.length() > 0) { submitArgs.append(" "); } - submitArgs.append(shQuote(arg)); + submitArgs.append(quote(arg)); } for (String arg : launcher.getDriverArgs()) { if (submitArgs.length() > 0) { submitArgs.append(" "); } - submitArgs.append(shQuote(arg)); + submitArgs.append(quote(arg)); } Map env = new HashMap(); @@ -87,4 +87,20 @@ protected List buildLauncherCommand() throws IOException { return prepareForOs(pyargs, libPath, env); } + /** + * Quotes a string so that it can be used in a command string and be parsed back into a single + * argument by python's "shlex.split()" function. + */ + private String quote(String s) { + StringBuilder quoted = new StringBuilder().append('"'); + for (int i = 0; i < s.length(); i++) { + int cp = s.codePointAt(i); + if (cp == '"' || cp == '\\') { + quoted.appendCodePoint('\\'); + } + quoted.appendCodePoint(cp); + } + return quoted.append('"').toString(); + } + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java index 2e42dbf5bda0..e7e137b90c94 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java @@ -20,7 +20,6 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.regex.Pattern; @@ -91,19 +90,19 @@ protected List buildLauncherCommand() throws IOException { return buildSparkSubmitCommand(); } - List cmd = createJavaCommand(); + List cmd = buildJavaCommand(); for (String key : javaOptsKeys) { addOptionString(cmd, System.getenv(key)); } - String mem = first(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM); + String mem = firstNonEmpty(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM); cmd.add("-Xms" + mem); cmd.add("-Xmx" + mem); cmd.add("-cp"); cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); cmd.add(className); cmd.addAll(classArgs); - return prepareForOs(cmd, null, Collections.emptyMap()); + return prepareForOs(cmd, null); } private List buildSparkSubmitCommand() throws IOException { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 3b3405eed2ef..f41c007d54af 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -23,7 +23,6 @@ import java.net.URL; import java.net.URLClassLoader; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Properties; @@ -32,7 +31,7 @@ /** * Launcher for Spark applications. *

- * Use this class to start Spark applications programatically. The class uses a builder pattern + * Use this class to start Spark applications programmatically. The class uses a builder pattern * to allow clients to configure the Spark application and launch it as a child process. *

* There's also support for running the application on a separate thread, although that is to @@ -49,17 +48,17 @@ public class SparkLauncher extends AbstractLauncher { protected String appName; protected String master; protected String deployMode; - protected String userClass; - protected String userResource; + protected String mainClass; + protected String appResource; protected final List sparkArgs; - protected final List userArgs; + protected final List appArgs; protected final List jars; protected final List files; protected final List pyFiles; public SparkLauncher() { this.sparkArgs = new ArrayList(); - this.userArgs = new ArrayList(); + this.appArgs = new ArrayList(); this.jars = new ArrayList(); this.files = new ArrayList(); this.pyFiles = new ArrayList(); @@ -90,46 +89,46 @@ public SparkLauncher setDeployMode(String mode) { * Set the main application resource. This should be the location of a jar file for Scala/Java * applications, or a python script for PySpark applications. */ - public SparkLauncher setAppResource(String path) { - checkNotNull(path, "path"); - this.userResource = path; + public SparkLauncher setAppResource(String resource) { + checkNotNull(resource, "resource"); + this.appResource = resource; return this; } /** Sets the application class name for Java/Scala applications. */ - public SparkLauncher setClass(String userClass) { - checkNotNull(userClass, "userClass"); - this.userClass = userClass; + public SparkLauncher setMainClass(String mainClass) { + checkNotNull(mainClass, "mainClass"); + this.mainClass = mainClass; return this; } /** Adds command line arguments for the application. */ - public SparkLauncher addArgs(String... args) { + public SparkLauncher addAppArgs(String... args) { for (String arg : args) { checkNotNull(arg, "arg"); - userArgs.add(arg); + appArgs.add(arg); } return this; } /** Adds a jar file to be submitted with the application. */ - public SparkLauncher addJar(String path) { - checkNotNull(path, "path"); - jars.add(path); + public SparkLauncher addJar(String jar) { + checkNotNull(jar, "jar"); + jars.add(jar); return this; } /** Adds a file to be submitted with the application. */ - public SparkLauncher addFile(String path) { - checkNotNull(path, "path"); - files.add(path); + public SparkLauncher addFile(String file) { + checkNotNull(file, "file"); + files.add(file); return this; } - /** Adds a a python file / zip / egg to be submitted with the application. */ - public SparkLauncher addPyFile(String path) { - checkNotNull(path, "path"); - pyFiles.add(path); + /** Adds a python file / zip / egg to be submitted with the application. */ + public SparkLauncher addPyFile(String file) { + checkNotNull(file, "file"); + pyFiles.add(file); return this; } @@ -167,17 +166,17 @@ public Thread start(Thread.UncaughtExceptionHandler handler, boolean daemon) thr // cannot be set in this mode. Properties props = loadPropertiesFile(); String extraClassPath = null; - if (isRunningDriver(props)) { + if (isClientMode(props)) { checkState( - find(DRIVER_JAVA_OPTIONS, conf, props) == null, + find(DRIVER_EXTRA_JAVA_OPTIONS, conf, props) == null, "Cannot set driver VM options when running in-process."); checkState( - find(DRIVER_LIBRARY_PATH, conf, props) == null, + find(DRIVER_EXTRA_LIBRARY_PATH, conf, props) == null, "Cannot set native library path when running in-process."); checkState( find(DRIVER_MEMORY, conf, props) == null, "Cannot set driver memory when running in-process."); - extraClassPath = find(DRIVER_CLASSPATH, conf, props); + extraClassPath = find(DRIVER_EXTRA_CLASSPATH, conf, props); } List cp = buildClassPath(extraClassPath); @@ -276,23 +275,23 @@ List buildSparkSubmitArgs() { args.add(join(",", pyFiles)); } - if (userClass != null) { + if (mainClass != null) { args.add("--class"); - args.add(userClass); + args.add(mainClass); } args.addAll(sparkArgs); - if (userResource != null) { - args.add(userResource); + if (appResource != null) { + args.add(appResource); } - args.addAll(userArgs); + args.addAll(appArgs); return args; } @Override protected List buildLauncherCommand() throws IOException { - List cmd = createJavaCommand(); + List cmd = buildJavaCommand(); addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); @@ -300,49 +299,40 @@ protected List buildLauncherCommand() throws IOException { // or just launching a cluster app. When running the driver, the JVM's argument will be // modified to cover the driver's configuration. Properties props = loadPropertiesFile(); - boolean isRunningDriver = isRunningDriver(props); + boolean isClientMode = isClientMode(props); - String extraClassPath = isRunningDriver ? find(DRIVER_CLASSPATH, conf, props) : null; + String extraClassPath = isClientMode ? find(DRIVER_EXTRA_CLASSPATH, conf, props) : null; cmd.add("-cp"); cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); String libPath = null; - if (isRunningDriver) { + if (isClientMode) { // Figuring out where the memory value come from is a little tricky due to precedence. // Precedence is observed in the following order: // - explicit configuration (setConf()), which also covers --driver-memory cli argument. - // - user properties, if properties file is explicitly set. + // - properties file. + // - SPARK_DRIVER_MEMORY env variable // - SPARK_MEM env variable - // - user properties, if using default file // - default value (512m) - String userMemSetting; - String defaultMemFromProps = null; - if (propertiesFile != null) { - userMemSetting = find(DRIVER_MEMORY, conf, props); - } else { - userMemSetting = conf.get(DRIVER_MEMORY); - defaultMemFromProps = props.getProperty(DRIVER_MEMORY); - } - - String memory = first(userMemSetting, System.getenv("SPARK_MEM"), defaultMemFromProps, - DEFAULT_MEM); + String memory = firstNonEmpty(find(DRIVER_MEMORY, conf, props), + System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); cmd.add("-Xms" + memory); cmd.add("-Xmx" + memory); - addOptionString(cmd, find(DRIVER_JAVA_OPTIONS, conf, props)); - libPath = find(DRIVER_LIBRARY_PATH, conf, props); + addOptionString(cmd, find(DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); + libPath = find(DRIVER_EXTRA_LIBRARY_PATH, conf, props); } cmd.add("org.apache.spark.deploy.SparkSubmit"); cmd.addAll(buildSparkSubmitArgs()); - return prepareForOs(cmd, libPath, Collections.emptyMap()); + return prepareForOs(cmd, libPath); } - private boolean isRunningDriver(Properties userProps) { - String userMaster = first(master, (String) userProps.get(SPARK_MASTER)); + private boolean isClientMode(Properties userProps) { + String userMaster = firstNonEmpty(master, (String) userProps.get(SPARK_MASTER)); return userMaster == null || "client".equals(deployMode) || "yarn-client".equals(userMaster) || - (deployMode == null && userMaster != null && !userMaster.startsWith("yarn-")); + (deployMode == null && !userMaster.startsWith("yarn-")); } private static class SparkSubmitRunner implements Runnable { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java index b2d8077d90c8..cf5b8df44f7e 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java @@ -31,21 +31,22 @@ *

* This launcher extends SparkLauncher to add command line parsing compatible with * SparkSubmit. It handles setting driver-side options and special parsing needed - * for the different shells. + * for the different specialClasses. *

* This class has also some special features to aid PySparkLauncher. */ public class SparkSubmitCliLauncher extends SparkLauncher { /** - * This map must match the class names for available shells, since this modifies the way - * command line parsing works. This maps the shell class name to the resource to use when - * calling spark-submit. + * This map must match the class names for available special classes, since this modifies the way + * command line parsing works. This maps the class name to the resource to use when calling + * spark-submit. */ - private static final Map shells = new HashMap(); + private static final Map specialClasses = new HashMap(); static { - shells.put("org.apache.spark.repl.Main", "spark-shell"); - shells.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver", "spark-internal"); + specialClasses.put("org.apache.spark.repl.Main", "spark-shell"); + specialClasses.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver", + "spark-internal"); } private final List driverArgs; @@ -56,28 +57,27 @@ public class SparkSubmitCliLauncher extends SparkLauncher { } SparkSubmitCliLauncher(boolean hasMixedArguments, List args) { - boolean sparkSubmitOptionsEnded = false; this.driverArgs = new ArrayList(); this.hasMixedArguments = hasMixedArguments; new OptionParser().parse(args); } - /** Visible for PySparkLauncher. */ + // Visible for PySparkLauncher. String getAppResource() { - return userResource; + return appResource; } - /** Visible for PySparkLauncher. */ - List getArgs() { - return userArgs; + // Visible for PySparkLauncher. + List getAppArgs() { + return appArgs; } - /** Visible for PySparkLauncher. */ + // Visible for PySparkLauncher. List getSparkArgs() { return sparkArgs; } - /** Visible for PySparkLauncher. */ + // Visible for PySparkLauncher. List getDriverArgs() { return driverArgs; } @@ -108,26 +108,26 @@ protected boolean handle(String opt, String value) { driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_JAVA_OPTIONS)) { - setConf(LauncherCommon.DRIVER_JAVA_OPTIONS, value); + setConf(LauncherCommon.DRIVER_EXTRA_JAVA_OPTIONS, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_LIBRARY_PATH)) { - setConf(LauncherCommon.DRIVER_LIBRARY_PATH, value); + setConf(LauncherCommon.DRIVER_EXTRA_LIBRARY_PATH, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_CLASS_PATH)) { - setConf(LauncherCommon.DRIVER_CLASSPATH, value); + setConf(LauncherCommon.DRIVER_EXTRA_CLASSPATH, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(CLASS)) { - // The shell launchers require some special command line handling, since they allow + // The special classes require some special command line handling, since they allow // mixing spark-submit arguments with arguments that should be propagated to the shell // itself. Note that for this to work, the "--class" argument must come before any // non-spark-submit arguments. - setClass(value); - if (shells.containsKey(value)) { + setMainClass(value); + if (specialClasses.containsKey(value)) { hasMixedArguments = true; - setAppResource(shells.get(value)); + setAppResource(specialClasses.get(value)); } } else { addSparkArgs(opt, value); @@ -141,7 +141,7 @@ protected boolean handleUnknown(String opt) { // In normal mode, any unrecognized parameter triggers the end of command line parsing. // The remaining params will be appended to the list of SparkSubmit arguments. if (hasMixedArguments) { - addArgs(opt); + addAppArgs(opt); return true; } else { addSparkArgs(opt); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index eb322ae7c066..5e744ccb0ee5 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -34,31 +34,34 @@ public abstract class SparkSubmitOptionParser { // The following constants define the "main" name for the available options. They're defined // to avoid copy & paste of the raw strings where they're needed. - protected static final String ARCHIVES = "--archives"; protected static final String CLASS = "--class"; protected static final String CONF = "--conf"; protected static final String DEPLOY_MODE = "--deploy-mode"; protected static final String DRIVER_CLASS_PATH = "--driver-class-path"; protected static final String DRIVER_CORES = "--driver-cores"; protected static final String DRIVER_JAVA_OPTIONS = "--driver-java-options"; - protected static final String DRIVER_LIBRARY_PATH = "--driver-library -path"; + protected static final String DRIVER_LIBRARY_PATH = "--driver-library-path"; protected static final String DRIVER_MEMORY = "--driver-memory"; - protected static final String EXECUTOR_CORES = "--executor-cores"; protected static final String EXECUTOR_MEMORY = "--executor-memory"; protected static final String FILES = "--files"; protected static final String JARS = "--jars"; protected static final String MASTER = "--master"; protected static final String NAME = "--name"; - protected static final String NUM_EXECUTORS = "--num-executors"; protected static final String PROPERTIES_FILE = "--properties-file"; protected static final String PY_FILES = "--py-files"; - protected static final String QUEUE = "--queue"; protected static final String TOTAL_EXECUTOR_CORES = "--total-executor-cores"; + // Options that do not take arguments. protected static final String HELP = "--help"; protected static final String SUPERVISE = "--supervise"; protected static final String VERBOSE = "--verbose"; + // YARN-only options. + protected static final String ARCHIVES = "--archives"; + protected static final String EXECUTOR_CORES = "--executor-cores"; + protected static final String QUEUE = "--queue"; + protected static final String NUM_EXECUTORS = "--num-executors"; + /** * This is the canonical list of spark-submit options. Each entry in the array contains the * different aliases for the same option; the first element of each entry is the "official" diff --git a/launcher/src/test/java/org/apache/spark/launcher/AbstractLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/LauncherCommonSuite.java similarity index 88% rename from launcher/src/test/java/org/apache/spark/launcher/AbstractLauncherSuite.java rename to launcher/src/test/java/org/apache/spark/launcher/LauncherCommonSuite.java index af18147e36cb..49b088f0e6ae 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/AbstractLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/LauncherCommonSuite.java @@ -24,14 +24,9 @@ import org.junit.Test; import static org.junit.Assert.*; -public class AbstractLauncherSuite { +import static org.apache.spark.launcher.LauncherCommon.*; - private AbstractLauncher launcher = new AbstractLauncher() { - @Override - protected List buildLauncherCommand() { - throw new UnsupportedOperationException(); - } - }; +public class LauncherCommonSuite { @Test public void testValidOptionStrings() { @@ -39,6 +34,7 @@ public void testValidOptionStrings() { testOpt("a 'b c' \"d\" e", Arrays.asList("a", "b c", "d", "e")); testOpt("a 'b\\\"c' \"'d'\" e", Arrays.asList("a", "b\\\"c", "'d'", "e")); testOpt("a 'b\"c' \"\\\"d\\\"\" e", Arrays.asList("a", "b\"c", "\"d\"", "e")); + testOpt(" a b c \\\\ ", Arrays.asList("a", "b", "c", "\\")); // Following tests ported from UtilsSuite.scala. testOpt("", new ArrayList()); @@ -66,7 +62,7 @@ public void testValidOptionStrings() { } @Test - public void testInalidOptionStrings() { + public void testInvalidOptionStrings() { testInvalidOpt("\\"); testInvalidOpt("\"abcde"); testInvalidOpt("'abcde"); @@ -74,12 +70,12 @@ public void testInalidOptionStrings() { private void testOpt(String opts, List expected) { assertEquals(String.format("test string failed to parse: [[ %s ]]", opts), - expected, launcher.parseOptionString(opts)); + expected, parseOptionString(opts)); } private void testInvalidOpt(String opts) { try { - launcher.parseOptionString(opts); + parseOptionString(opts); fail("Expected exception for invalid option string."); } catch (IllegalArgumentException e) { // pass. diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index df19d1cdfda2..c3e08bc95d48 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -55,10 +55,10 @@ public void testChildProcLauncher() throws Exception { .setSparkHome(System.getProperty("spark.test.home")) .setMaster("local") .setAppResource("spark-internal") - .setConf(SparkLauncher.DRIVER_JAVA_OPTIONS, "-Dfoo=bar -Dtest.name=-testChildProcLauncher") - .setConf(SparkLauncher.DRIVER_CLASSPATH, System.getProperty("java.class.path")) - .setClass(SparkLauncherTestApp.class.getName()) - .addArgs("proc"); + .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Dfoo=bar -Dtest.name=-testChildProcLauncher") + .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) + .setMainClass(SparkLauncherTestApp.class.getName()) + .addAppArgs("proc"); printArgs(launcher.buildLauncherCommand()); @@ -94,9 +94,9 @@ public void testThreadAppLauncher() throws Exception { .setSparkHome(System.getProperty("spark.test.home")) .setMaster("local") .setAppResource("spark-internal") - .setConf(SparkLauncher.DRIVER_CLASSPATH, System.getProperty("java.class.path")) - .setClass(SparkLauncherTestApp.class.getName()) - .addArgs("thread"); + .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) + .setMainClass(SparkLauncherTestApp.class.getName()) + .addAppArgs("thread"); printArgs(launcher.buildLauncherCommand()); @@ -113,9 +113,9 @@ public void uncaughtException(Thread t, Throwable e) { @Test public void testInProcessDriverArgValidator() throws Exception { - testInvalidDriverConf(SparkLauncher.DRIVER_JAVA_OPTIONS); + testInvalidDriverConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS); testInvalidDriverConf(SparkLauncher.DRIVER_MEMORY); - testInvalidDriverConf(SparkLauncher.DRIVER_LIBRARY_PATH); + testInvalidDriverConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH); } private void testCmdBuilder(boolean isDriver) throws Exception { @@ -127,12 +127,12 @@ private void testCmdBuilder(boolean isDriver) throws Exception { .setDeployMode(deployMode) .setAppResource("/foo") .setAppName("MyApp") - .setClass("my.Class") - .addArgs("foo", "bar") + .setMainClass("my.Class") + .addAppArgs("foo", "bar") .setConf(SparkLauncher.DRIVER_MEMORY, "1g") - .setConf(SparkLauncher.DRIVER_CLASSPATH, "/driver") - .setConf(SparkLauncher.DRIVER_JAVA_OPTIONS, "-Ddriver") - .setConf(SparkLauncher.DRIVER_LIBRARY_PATH, "/native") + .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver") + .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver") + .setConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native") .setConf("spark.foo", "foo"); List cmd = launcher.buildLauncherCommand(); @@ -203,8 +203,8 @@ private void testInvalidDriverConf(String key) throws Exception { new SparkLauncher() .setSparkHome(System.getProperty("spark.test.home")) .setAppResource("spark-internal") - .setClass(SparkLauncherTestApp.class.getName()) - .addArgs("thread") + .setMainClass(SparkLauncherTestApp.class.getName()) + .addAppArgs("thread") .setConf(key, "foo") .start(null, true); fail("Should have failed to start app."); From e946a998b5f8e747adf5905fe4e91908042b629e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 13 Jan 2015 16:52:18 -0800 Subject: [PATCH 12/51] Merge PySparkLauncher into SparkSubmitCliLauncher. This allows cleaning up the code slightly. --- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- .../java/org/apache/spark/launcher/Main.java | 2 - .../spark/launcher/PySparkLauncher.java | 106 ----------------- .../launcher/SparkSubmitCliLauncher.java | 108 +++++++++++++++--- .../launcher/SparkSubmitCliLauncherSuite.java | 31 +++++ 6 files changed, 122 insertions(+), 129 deletions(-) delete mode 100644 launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java diff --git a/bin/pyspark b/bin/pyspark index 6baece5bfcfb..be88f7a6e743 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -96,4 +96,4 @@ fi export PYSPARK_DRIVER_PYTHON export PYSPARK_DRIVER_PYTHON_OPTS -exec "$SPARK_HOME"/bin/spark-class pyspark "$@" +exec "$SPARK_HOME"/bin/spark-submit pyspark-shell-main "$@" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index ff5862d049f9..4f5eb5e20614 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -35,4 +35,4 @@ set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py -call %SPARK_HOME%\bin\spark-class2.cmd pyspark %* +call %SPARK_HOME%\bin\spark-submit2.cmd pyspark-shell-main %* diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index ce1cdb3d3601..5f87795a64d4 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -51,8 +51,6 @@ public static void main(String[] argsArray) throws Exception { try { if (className.equals("org.apache.spark.deploy.SparkSubmit")) { launcher = new SparkSubmitCliLauncher(args); - } else if (className.equals("pyspark")) { - launcher = new PySparkLauncher(args); } else { launcher = new SparkClassLauncher(className, args); } diff --git a/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java deleted file mode 100644 index 2caa4fb2fb26..000000000000 --- a/launcher/src/main/java/org/apache/spark/launcher/PySparkLauncher.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -/** - * Launcher for PySpark. - *

- * Handles parsing command line options passed to the pyspark script. This allows - * sharing that logic with other launchers, to keep them in sync. - */ -class PySparkLauncher extends AbstractLauncher { - - private final List args; - - PySparkLauncher(List args) { - this.args = args; - } - - @Override - protected List buildLauncherCommand() throws IOException { - SparkSubmitCliLauncher launcher = new SparkSubmitCliLauncher(true, args); - - // For backwards compatibility, if a script is specified in - // the pyspark command line, then run it using spark-submit. - if (!launcher.getAppArgs().isEmpty() && launcher.getAppArgs().get(0).endsWith(".py")) { - System.err.println( - "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + - "Use ./bin/spark-submit "); - return launcher.buildLauncherCommand(); - } - - // When launching the pyspark shell, the spark-submit arguments should be stored in the - // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable - // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS. - checkArgument(launcher.getAppArgs().isEmpty(), - "pyspark does not support any application options."); - - Properties props = loadPropertiesFile(); - String libPath = find(DRIVER_EXTRA_LIBRARY_PATH, conf, props); - - StringBuilder submitArgs = new StringBuilder(); - for (String arg : launcher.getSparkArgs()) { - if (submitArgs.length() > 0) { - submitArgs.append(" "); - } - submitArgs.append(quote(arg)); - } - for (String arg : launcher.getDriverArgs()) { - if (submitArgs.length() > 0) { - submitArgs.append(" "); - } - submitArgs.append(quote(arg)); - } - - Map env = new HashMap(); - env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); - - List pyargs = new ArrayList(); - pyargs.add(System.getenv("PYSPARK_DRIVER_PYTHON")); - String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); - if (!isEmpty(pyOpts)) { - pyargs.addAll(parseOptionString(pyOpts)); - } - - return prepareForOs(pyargs, libPath, env); - } - - /** - * Quotes a string so that it can be used in a command string and be parsed back into a single - * argument by python's "shlex.split()" function. - */ - private String quote(String s) { - StringBuilder quoted = new StringBuilder().append('"'); - for (int i = 0; i < s.length(); i++) { - int cp = s.codePointAt(i); - if (cp == '"' || cp == '\\') { - quoted.appendCodePoint('\\'); - } - quoted.appendCodePoint(cp); - } - return quoted.append('"').toString(); - } - -} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java index cf5b8df44f7e..01f9fe4a578a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java @@ -17,12 +17,14 @@ package org.apache.spark.launcher; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -37,6 +39,15 @@ */ public class SparkSubmitCliLauncher extends SparkLauncher { + /** + * Name of the app resource used to identify the PySpark shell. The command line parser expects + * the resource name to be the very first argument to spark-submit in this case. + * + * NOTE: this cannot be "pyspark-shell" since that identifies the PySpark shell to SparkSubmit + * (see java_gateway.py), and can cause this code to enter into an infinite loop. + */ + static final String PYSPARK_SHELL = "pyspark-shell-main"; + /** * This map must match the class names for available special classes, since this modifies the way * command line parsing works. This maps the class name to the resource to use when calling @@ -58,33 +69,89 @@ public class SparkSubmitCliLauncher extends SparkLauncher { SparkSubmitCliLauncher(boolean hasMixedArguments, List args) { this.driverArgs = new ArrayList(); - this.hasMixedArguments = hasMixedArguments; - new OptionParser().parse(args); - } - // Visible for PySparkLauncher. - String getAppResource() { - return appResource; - } + List submitArgs = args; + if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { + this.hasMixedArguments = true; + setAppResource(PYSPARK_SHELL); + submitArgs = args.subList(1, args.size()); + } else { + this.hasMixedArguments = hasMixedArguments; + } - // Visible for PySparkLauncher. - List getAppArgs() { - return appArgs; + new OptionParser().parse(submitArgs); } - // Visible for PySparkLauncher. - List getSparkArgs() { - return sparkArgs; + @Override + protected List buildLauncherCommand() throws IOException { + if (PYSPARK_SHELL.equals(appResource)) { + return buildPySparkShellCommand(); + } else { + return super.buildLauncherCommand(); + } } - // Visible for PySparkLauncher. - List getDriverArgs() { - return driverArgs; + private List buildPySparkShellCommand() throws IOException { + // For backwards compatibility, if a script is specified in + // the pyspark command line, then run it using spark-submit. + if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".py")) { + System.err.println( + "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + + "Use ./bin/spark-submit "); + setAppResource(appArgs.get(0)); + appArgs.remove(0); + return buildLauncherCommand(); + } + + // When launching the pyspark shell, the spark-submit arguments should be stored in the + // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable + // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS. + checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); + + Properties props = loadPropertiesFile(); + String libPath = find(DRIVER_EXTRA_LIBRARY_PATH, conf, props); + + StringBuilder submitArgs = new StringBuilder(); + for (String arg : sparkArgs) { + if (submitArgs.length() > 0) { + submitArgs.append(" "); + } + submitArgs.append(quote(arg)); + } + for (String arg : driverArgs) { + if (submitArgs.length() > 0) { + submitArgs.append(" "); + } + submitArgs.append(quote(arg)); + } + + Map env = new HashMap(); + env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); + + List pyargs = new ArrayList(); + pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); + String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); + if (!isEmpty(pyOpts)) { + pyargs.addAll(parseOptionString(pyOpts)); + } + + return prepareForOs(pyargs, libPath, env); } - private String getArgValue(Iterator it, String name) { - checkArgument(it.hasNext(), "Missing argument for '%s'.", name); - return it.next(); + /** + * Quotes a string so that it can be used in a command string and be parsed back into a single + * argument by python's "shlex.split()" function. + */ + private String quote(String s) { + StringBuilder quoted = new StringBuilder().append('"'); + for (int i = 0; i < s.length(); i++) { + int cp = s.codePointAt(i); + if (cp == '"' || cp == '\\') { + quoted.appendCodePoint('\\'); + } + quoted.appendCodePoint(cp); + } + return quoted.append('"').toString(); } private class OptionParser extends SparkSubmitOptionParser { @@ -129,6 +196,9 @@ protected boolean handle(String opt, String value) { hasMixedArguments = true; setAppResource(specialClasses.get(value)); } + } else if (opt.equals(PYSPARK_SHELL)) { + hasMixedArguments = true; + setAppResource(opt); } else { addSparkArgs(opt, value); } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java index cc95f5c83bae..4a5c92235605 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java @@ -58,6 +58,37 @@ public void testAlternateSyntaxParsing() throws Exception { assertEquals("bar", findArgValue(cmd, "--deploy-mode")); } + @Test + public void testPySparkLauncher() throws Exception { + List sparkSubmitArgs = Arrays.asList( + SparkSubmitCliLauncher.PYSPARK_SHELL, + "--master=foo", + "--deploy-mode=bar"); + + List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs) + .setSparkHome(System.getProperty("spark.test.home")) + .buildLauncherCommand(); + assertEquals("python", cmd.get(cmd.size() - 1)); + assertTrue(cmd.contains("PYSPARK_SUBMIT_ARGS=\"--master\" \"foo\" \"--deploy-mode\" \"bar\"")); + } + + @Test + public void testPySparkFallback() throws Exception { + List sparkSubmitArgs = Arrays.asList( + "--master=foo", + "--deploy-mode=bar", + "script.py", + "arg1"); + + List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs) + .setSparkHome(System.getProperty("spark.test.home")) + .buildLauncherCommand(); + assertEquals("foo", findArgValue(cmd, "--master")); + assertEquals("bar", findArgValue(cmd, "--deploy-mode")); + assertEquals("script.py", cmd.get(cmd.size() - 2)); + assertEquals("arg1", cmd.get(cmd.size() - 1)); + } + private String findArgValue(List cmd, String name) { for (int i = 0; i < cmd.size(); i++) { if (cmd.get(i).equals(name)) { From 8ac4e923d64fe36b018310cdced8e0c323a1a8a8 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 13 Jan 2015 17:22:37 -0800 Subject: [PATCH 13/51] Minor test cleanup. --- .../spark/launcher/SparkLauncherSuite.java | 53 +++++++++++-------- 1 file changed, 31 insertions(+), 22 deletions(-) diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index c3e08bc95d48..ecbcc7b8d8d1 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -19,6 +19,7 @@ import java.io.BufferedReader; import java.io.File; +import java.io.InputStream; import java.io.InputStreamReader; import java.util.HashMap; import java.util.List; @@ -59,29 +60,10 @@ public void testChildProcLauncher() throws Exception { .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) .setMainClass(SparkLauncherTestApp.class.getName()) .addAppArgs("proc"); - printArgs(launcher.buildLauncherCommand()); - final Process app = launcher.launch(); - Thread stderr = new Thread(new Runnable() { - @Override - public void run() { - try { - BufferedReader in = new BufferedReader( - new InputStreamReader(app.getErrorStream(), "UTF-8")); - String line; - while ((line = in.readLine()) != null) { - LOG.warn(line); - } - } catch (Exception e) { - e.printStackTrace(); - } - } - }); - stderr.setDaemon(true); - stderr.setName("stderr"); - stderr.start(); - + new Redirector("stdout", app.getInputStream()).start(); + new Redirector("stderr", app.getErrorStream()).start(); assertEquals(0, app.waitFor()); } @@ -267,9 +249,11 @@ private void printArgs(List cmd) { public static class SparkLauncherTestApp { - public static void main(String[] args) { + public static void main(String[] args) throws Exception { if (args[0].equals("proc")) { assertEquals("bar", System.getProperty("foo")); + } else if (args[0].equals("arg")) { + assertEquals("newline=", args[1]); } else { assertEquals("thread", args[0]); } @@ -278,4 +262,29 @@ public static void main(String[] args) { } + private static class Redirector extends Thread { + + private final InputStream in; + + Redirector(String name, InputStream in) { + this.in = in; + setName(name); + setDaemon(true); + } + + @Override + public void run() { + try { + BufferedReader reader = new BufferedReader(new InputStreamReader(in, "UTF-8")); + String line; + while ((line = reader.readLine()) != null) { + LOG.warn(line); + } + } catch (Exception e) { + LOG.error("Error reading process output.", e); + } + } + + } + } From 525ef5bc239550fd503ff6a954419bb3e6b82813 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 13 Jan 2015 21:39:28 -0800 Subject: [PATCH 14/51] Rework Unix spark-class to handle argument with newlines. --- bin/spark-class | 11 ++++++----- .../spark/launcher/AbstractLauncher.java | 4 ++-- .../java/org/apache/spark/launcher/Main.java | 19 ++++++++++++++----- 3 files changed, 22 insertions(+), 12 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 5553153c3cdb..5d298ac740fe 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -67,11 +67,12 @@ fi SPARK_LAUNCHER_CP="${SPARK_LAUNCHER_CP}${LAUNCHER_DIR}/${LAUNCHER_JARS}" -# The launcher library will print one argument per line of its output. The next line sets -# the new line char as the only delimiter used when parsing that output into an array. -IFS=" -" -CMD=($($RUNNER -cp $SPARK_LAUNCHER_CP org.apache.spark.launcher.Main "$@")) +# The launcher library will print arguments separated by a NULL character. Read that in a while +# loop, populating an array that will be used to exec the final command. +CMD=() +while IFS= read -d '' -r ARG; do + CMD+=("$ARG") +done < <($RUNNER -cp "$SPARK_LAUNCHER_CP" org.apache.spark.launcher.Main "$@") if [ "${CMD[0]}" = "usage" ]; then "${CMD[@]}" diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index 995db4b726f4..25cd5f07b31e 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -427,7 +427,7 @@ private List prepareForWindows( List cmd, String libPath, Map env) { - StringBuilder cmdline = new StringBuilder("cmd /c \""); + StringBuilder cmdline = new StringBuilder("\""); if (libPath != null) { cmdline.append("set PATH=%PATH%;").append(libPath).append(" &&"); } @@ -445,7 +445,7 @@ private List prepareForWindows( cmdline.append(quoteForBatchScript(arg)); } cmdline.append("\""); - return Arrays.asList(cmdline.toString()); + return Arrays.asList("cmd", "/c", cmdline.toString()); } /** diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index 5f87795a64d4..83c9099af3cb 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -36,9 +36,12 @@ public class Main extends LauncherCommon { *

  • "spark-class": if another class is provided, an internal Spark class is run.
  • * * - * The ultimate command will not be run in the same process. Instead, the command to be executed - * will be printed to stdout. On Unix systems, this will be one argument per line. On Windows - * systems, this will be a single line containing the command to be executed. + * This class works in tandem with the "bin/spark-class" script on Unix-like systems, and + * "bin/spark-class2.cmd" batch script on Windows to execute the final command. + *

    + * On Unix-like systems, the output is a list of command arguments, separated by the NULL + * character. On Windows, the output is single command line suitable for direct execution + * form the script. */ public static void main(String[] argsArray) throws Exception { checkArgument(argsArray.length > 0, "Not enough arguments: missing class name."); @@ -66,8 +69,14 @@ public static void main(String[] argsArray) throws Exception { System.err.println("========================================"); } - for (String c : cmd) { - System.out.println(c); + if (isWindows()) { + String cmdLine = join(" ", cmd); + System.out.println(cmdLine); + } else { + for (String c : cmd) { + System.out.print(c); + System.out.print('\0'); + } } } From e584fc31c8c9e73282acdf2073d1d1fbbd06ec3a Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Jan 2015 10:52:10 -0800 Subject: [PATCH 15/51] Rework command building a little bit. Separate "build the command array" from "preparing command for execution under a shell". The first one is more suitable for usage with Runtime.exec() and ProcessBuilder, while the latter is targeted at servicing the shell scripts shipped with Spark. --- .../spark/deploy/worker/CommandUtils.scala | 7 +- .../spark/launcher/AbstractLauncher.java | 136 +++++++----------- .../apache/spark/launcher/LauncherCommon.java | 12 ++ .../java/org/apache/spark/launcher/Main.java | 5 +- .../spark/launcher/SparkClassLauncher.java | 11 +- .../apache/spark/launcher/SparkLauncher.java | 17 ++- .../launcher/SparkSubmitCliLauncher.java | 15 +- .../spark/launcher/SparkLauncherSuite.java | 12 +- .../launcher/SparkSubmitCliLauncherSuite.java | 4 +- 9 files changed, 106 insertions(+), 113 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index d1947523ad3c..6b30deced499 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.worker import java.io.{File, FileOutputStream, InputStream, IOException} import java.lang.System._ -import java.util.{ArrayList, List => JList} +import java.util.{ArrayList, List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.collection.Map @@ -59,7 +59,8 @@ object CommandUtils extends Logging { private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows - val cmd = new CommandLauncher(sparkHome, memory, command.environment).buildLauncherCommand() + val cmd = new CommandLauncher(sparkHome, memory, command.environment) + .buildLauncherCommand(command.environment) cmd.toSeq ++ Seq(command.mainClass) ++ command.arguments } @@ -116,7 +117,7 @@ private class CommandLauncher(sparkHome: String, memory: Int, env: Map[String, S setSparkHome(sparkHome) - override def buildLauncherCommand(): JList[String] = { + override def buildLauncherCommand(env: JMap[String, String]): JList[String] = { val cmd = buildJavaCommand() cmd.add("-cp") cmd.add(buildClassPath(null).mkString(File.pathSeparator)) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index 25cd5f07b31e..e13fa67f9143 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -38,37 +38,39 @@ */ public abstract class AbstractLauncher extends LauncherCommon { + private static final String ENV_SPARK_HOME = "SPARK_HOME"; private static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; protected static final String DEFAULT_MEM = "512m"; protected String javaHome; protected String sparkHome; protected String propertiesFile; - protected final Map conf = new HashMap(); - private final Map env; + protected final Map conf; + protected final Map launcherEnv; protected AbstractLauncher() { - this(null); + this(Collections.emptyMap()); } protected AbstractLauncher(Map env) { - this.env = env; + this.conf = new HashMap(); + this.launcherEnv = new HashMap(env); } @SuppressWarnings("unchecked") private final T THIS = (T) this; /** Set a custom JAVA_HOME for launching the Spark application. */ - public T setJavaHome(String path) { - checkNotNull(path, "path"); - this.javaHome = path; + public T setJavaHome(String javaHome) { + checkNotNull(javaHome, "javaHome"); + this.javaHome = javaHome; return THIS; } /** Set a custom Spark installation location for the application. */ - public T setSparkHome(String path) { - checkNotNull(path, "path"); - this.sparkHome = path; + public T setSparkHome(String sparkHome) { + checkNotNull(sparkHome, "sparkHome"); + launcherEnv.put(ENV_SPARK_HOME, sparkHome); return THIS; } @@ -89,9 +91,22 @@ public T setConf(String key, String value) { } /** - * Launchers should implement this to create the command to be executed. + * Launchers should implement this to create the command to be executed. This method should + * also update the environment map with any environment variables needed by the child process. + * + * @param env Map containing environment variables to set for the Spark job. */ - protected abstract List buildLauncherCommand() throws IOException; + protected abstract List buildLauncherCommand(Map env) throws IOException; + + /** + * Prepares the launcher command for execution from a shell script. This is used by the `Main` + * class to service the scripts shipped with the Spark distribution. + */ + List buildShellCommand() throws IOException { + Map childEnv = new HashMap(launcherEnv); + List cmd = buildLauncherCommand(childEnv); + return isWindows() ? prepareForWindows(cmd, childEnv) : prepareForBash(cmd, childEnv); + } /** * Loads the configuration file for the application, if it exists. This is either the @@ -128,7 +143,7 @@ protected Properties loadPropertiesFile() throws IOException { } protected String getSparkHome() { - String path = firstNonEmpty(sparkHome, getenv("SPARK_HOME")); + String path = getenv(ENV_SPARK_HOME); checkState(path != null, "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); return path; @@ -319,62 +334,6 @@ protected String getScalaVersion() { throw new IllegalStateException("Should not reach here."); } - protected List prepareForOs(List cmd, String libPath) { - return prepareForOs(cmd, libPath, Collections.emptyMap()); - } - - /** - * Prepare the command for execution under the current OS, setting the passed environment - * variables. - * - * Which OS is running defines two things: - * - the name of the environment variable used to define the lookup path for native libs - * - how to execute the command in general. - * - * The name is easy: PATH on Win32, DYLD_LIBRARY_PATH on MacOS, LD_LIBRARY_PATH elsewhere. - * - * On Unix-like, we're assuming bash is available. So we print one argument per line to - * the output, and use bash's array handling to execute the right thing. - * - * For Win32, see {@link #prepareForWindows(List,String)}. - */ - protected List prepareForOs( - List cmd, - String libPath, - Map env) { - - // If SPARK_HOME does not come from the environment, explicitly set it - // in the child's environment. - Map childEnv = env; - if (System.getenv("SPARK_HOME") == null && !env.containsKey("SPARK_HOME")) { - childEnv = new HashMap(env); - childEnv.put("SPARK_HOME", sparkHome); - } - - if (isWindows()) { - return prepareForWindows(cmd, libPath, childEnv); - } - - if (isEmpty(libPath) && childEnv.isEmpty()) { - return cmd; - } - - List newCmd = new ArrayList(); - newCmd.add("env"); - - if (!isEmpty(libPath)) { - String envName = getLibPathEnvName(); - String currEnvValue = getenv(envName); - String newEnvValue = join(File.pathSeparator, currEnvValue, libPath); - newCmd.add(String.format("%s=%s", envName, newEnvValue)); - } - for (Map.Entry e : childEnv.entrySet()) { - newCmd.add(String.format("%s=%s", e.getKey(), e.getValue())); - } - newCmd.addAll(cmd); - return newCmd; - } - private String findAssembly(String scalaVersion) { String sparkHome = getSparkHome(); File libdir; @@ -400,7 +359,7 @@ public boolean accept(File file) { } private String getenv(String key) { - return firstNonEmpty(env != null ? env.get(key) : null, System.getenv(key)); + return firstNonEmpty(launcherEnv.get(key), System.getenv(key)); } private String getConfDir() { @@ -409,7 +368,26 @@ private String getConfDir() { } /** - * Prepare a command line for execution on Windows. + * 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 List prepareForBash(List cmd, Map childEnv) { + if (childEnv.isEmpty()) { + return cmd; + } + + List newCmd = new ArrayList(); + newCmd.add("env"); + + for (Map.Entry e : childEnv.entrySet()) { + newCmd.add(String.format("%s=%s", e.getKey(), e.getValue())); + } + newCmd.addAll(cmd); + return newCmd; + } + + /** + * Prepare a command line for execution from a Windows batch script. * * Two things need to be done: * @@ -420,18 +398,12 @@ private String getConfDir() { * "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 in a single line, since that's the + * 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 List prepareForWindows( - List cmd, - String libPath, - Map env) { - StringBuilder cmdline = new StringBuilder("\""); - if (libPath != null) { - cmdline.append("set PATH=%PATH%;").append(libPath).append(" &&"); - } - for (Map.Entry e : env.entrySet()) { + private List prepareForWindows(List cmd, Map childEnv) { + StringBuilder cmdline = new StringBuilder("cmd /c \""); + for (Map.Entry e : childEnv.entrySet()) { if (cmdline.length() > 0) { cmdline.append(" "); } @@ -445,7 +417,7 @@ private List prepareForWindows( cmdline.append(quoteForBatchScript(arg)); } cmdline.append("\""); - return Arrays.asList("cmd", "/c", cmdline.toString()); + return Arrays.asList(cmdline.toString()); } /** diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java index a76bc6ebb5fc..0d429914975d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java @@ -17,6 +17,7 @@ package org.apache.spark.launcher; +import java.io.File; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -123,6 +124,17 @@ protected static boolean isWindows() { return os.startsWith("Windows"); } + /** + * Updates the user environment to contain the merged value of "envKey" after appending + * the given path list. + */ + protected void mergeEnvPathList(Map userEnv, String envKey, String pathList) { + if (!isEmpty(pathList)) { + String current = firstNonEmpty(userEnv.get(envKey), System.getenv(envKey)); + userEnv.put(envKey, join(File.pathSeparator, current, pathList)); + } + } + /** * Parse a string as if it were a list of arguments, in the way that a shell would. * This tries to follow the way bash parses strings. For example: diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index 83c9099af3cb..a1ac51c7a8c9 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; /** * Command line interface for the Spark launcher. Used internally by Spark scripts. @@ -63,7 +64,7 @@ public static void main(String[] argsArray) throws Exception { printLaunchCommand = false; } - List cmd = launcher.buildLauncherCommand(); + List cmd = launcher.buildShellCommand(); if (printLaunchCommand) { System.err.println("Spark Command: " + join(" ", cmd)); System.err.println("========================================"); @@ -96,7 +97,7 @@ public static void main(String[] argsArray) throws Exception { private static class UsageLauncher extends AbstractLauncher { @Override - protected List buildLauncherCommand() { + protected List buildLauncherCommand(Map env) { if (isWindows()) { return Arrays.asList("set SPARK_LAUNCHER_USAGE_ERROR=1"); } else { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java index e7e137b90c94..3ebac1994bf4 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.regex.Pattern; /** @@ -40,7 +41,7 @@ class SparkClassLauncher extends AbstractLauncher { } @Override - protected List buildLauncherCommand() throws IOException { + protected List buildLauncherCommand(Map env) throws IOException { List javaOptsKeys = new ArrayList(); String memKey = null; String extraClassPath = null; @@ -87,7 +88,7 @@ protected List buildLauncherCommand() throws IOException { javaOptsKeys.add("SPARK_JAVA_OPTS"); } else { // Any classes not explicitly listed above are submitted using SparkSubmit. - return buildSparkSubmitCommand(); + return buildSparkSubmitCommand(env); } List cmd = buildJavaCommand(); @@ -102,17 +103,17 @@ protected List buildLauncherCommand() throws IOException { cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); cmd.add(className); cmd.addAll(classArgs); - return prepareForOs(cmd, null); + return cmd; } - private List buildSparkSubmitCommand() throws IOException { + private List buildSparkSubmitCommand(Map env) throws IOException { List sparkSubmitArgs = new ArrayList(classArgs); sparkSubmitArgs.add(SparkSubmitOptionParser.CLASS); sparkSubmitArgs.add(className); SparkSubmitCliLauncher launcher = new SparkSubmitCliLauncher(true, sparkSubmitArgs); launcher.setAppResource("spark-internal"); - return launcher.buildLauncherCommand(); + return launcher.buildLauncherCommand(env); } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index f41c007d54af..43c832697844 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -23,6 +23,7 @@ import java.net.URL; import java.net.URLClassLoader; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -216,8 +217,13 @@ public Thread start(Thread.UncaughtExceptionHandler handler, boolean daemon) thr * @return A process handle for the Spark app. */ public Process launch() throws IOException { - List cmd = buildLauncherCommand(); - return Runtime.getRuntime().exec(cmd.toArray(new String[cmd.size()])); + Map childEnv = new HashMap(launcherEnv); + List cmd = buildLauncherCommand(childEnv); + ProcessBuilder pb = new ProcessBuilder(cmd.toArray(new String[cmd.size()])); + for (Map.Entry e : childEnv.entrySet()) { + pb.environment().put(e.getKey(), e.getValue()); + } + return pb.start(); } SparkLauncher addSparkArgs(String... args) { @@ -290,7 +296,7 @@ List buildSparkSubmitArgs() { } @Override - protected List buildLauncherCommand() throws IOException { + protected List buildLauncherCommand(Map env) throws IOException { List cmd = buildJavaCommand(); addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); @@ -305,7 +311,6 @@ protected List buildLauncherCommand() throws IOException { cmd.add("-cp"); cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); - String libPath = null; if (isClientMode) { // Figuring out where the memory value come from is a little tricky due to precedence. // Precedence is observed in the following order: @@ -319,12 +324,12 @@ protected List buildLauncherCommand() throws IOException { cmd.add("-Xms" + memory); cmd.add("-Xmx" + memory); addOptionString(cmd, find(DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); - libPath = find(DRIVER_EXTRA_LIBRARY_PATH, conf, props); + mergeEnvPathList(env, getLibPathEnvName(), find(DRIVER_EXTRA_LIBRARY_PATH, conf, props)); } cmd.add("org.apache.spark.deploy.SparkSubmit"); cmd.addAll(buildSparkSubmitArgs()); - return prepareForOs(cmd, libPath); + return cmd; } private boolean isClientMode(Properties userProps) { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java index 01f9fe4a578a..d7f216cf89da 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java @@ -83,15 +83,15 @@ public class SparkSubmitCliLauncher extends SparkLauncher { } @Override - protected List buildLauncherCommand() throws IOException { + protected List buildLauncherCommand(Map env) throws IOException { if (PYSPARK_SHELL.equals(appResource)) { - return buildPySparkShellCommand(); + return buildPySparkShellCommand(env); } else { - return super.buildLauncherCommand(); + return super.buildLauncherCommand(env); } } - private List buildPySparkShellCommand() throws IOException { + private List buildPySparkShellCommand(Map env) throws IOException { // For backwards compatibility, if a script is specified in // the pyspark command line, then run it using spark-submit. if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".py")) { @@ -100,7 +100,7 @@ private List buildPySparkShellCommand() throws IOException { "Use ./bin/spark-submit "); setAppResource(appArgs.get(0)); appArgs.remove(0); - return buildLauncherCommand(); + return buildLauncherCommand(env); } // When launching the pyspark shell, the spark-submit arguments should be stored in the @@ -109,7 +109,7 @@ private List buildPySparkShellCommand() throws IOException { checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); Properties props = loadPropertiesFile(); - String libPath = find(DRIVER_EXTRA_LIBRARY_PATH, conf, props); + mergeEnvPathList(env, getLibPathEnvName(), find(DRIVER_EXTRA_LIBRARY_PATH, conf, props)); StringBuilder submitArgs = new StringBuilder(); for (String arg : sparkArgs) { @@ -125,7 +125,6 @@ private List buildPySparkShellCommand() throws IOException { submitArgs.append(quote(arg)); } - Map env = new HashMap(); env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); List pyargs = new ArrayList(); @@ -135,7 +134,7 @@ private List buildPySparkShellCommand() throws IOException { pyargs.addAll(parseOptionString(pyOpts)); } - return prepareForOs(pyargs, libPath, env); + return pyargs; } /** diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index ecbcc7b8d8d1..b43bcc580724 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -56,11 +56,12 @@ public void testChildProcLauncher() throws Exception { .setSparkHome(System.getProperty("spark.test.home")) .setMaster("local") .setAppResource("spark-internal") - .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Dfoo=bar -Dtest.name=-testChildProcLauncher") + .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, + "-Dfoo=bar -Dtest.name=-testChildProcLauncher") .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) .setMainClass(SparkLauncherTestApp.class.getName()) .addAppArgs("proc"); - printArgs(launcher.buildLauncherCommand()); + printArgs(launcher.buildShellCommand()); final Process app = launcher.launch(); new Redirector("stdout", app.getInputStream()).start(); new Redirector("stderr", app.getErrorStream()).start(); @@ -80,7 +81,7 @@ public void testThreadAppLauncher() throws Exception { .setMainClass(SparkLauncherTestApp.class.getName()) .addAppArgs("thread"); - printArgs(launcher.buildLauncherCommand()); + printArgs(launcher.buildShellCommand()); Thread app = launcher.start(new Thread.UncaughtExceptionHandler() { @Override @@ -117,7 +118,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { .setConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native") .setConf("spark.foo", "foo"); - List cmd = launcher.buildLauncherCommand(); + List cmd = launcher.buildShellCommand(); // Checks below are different for driver and non-driver mode. @@ -152,7 +153,8 @@ private void testCmdBuilder(boolean isDriver) throws Exception { } if (isDriver) { assertNotNull("Native library path should be set.", libPath); - assertTrue("Native library path should contain provided entry.", contains("/native", libPath)); + assertTrue("Native library path should contain provided entry.", + contains("/native", libPath)); } else { assertNull("Native library should not be set.", libPath); } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java index 4a5c92235605..2018720884db 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java @@ -67,7 +67,7 @@ public void testPySparkLauncher() throws Exception { List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs) .setSparkHome(System.getProperty("spark.test.home")) - .buildLauncherCommand(); + .buildShellCommand(); assertEquals("python", cmd.get(cmd.size() - 1)); assertTrue(cmd.contains("PYSPARK_SUBMIT_ARGS=\"--master\" \"foo\" \"--deploy-mode\" \"bar\"")); } @@ -82,7 +82,7 @@ public void testPySparkFallback() throws Exception { List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs) .setSparkHome(System.getProperty("spark.test.home")) - .buildLauncherCommand(); + .buildShellCommand(); assertEquals("foo", findArgValue(cmd, "--master")); assertEquals("bar", findArgValue(cmd, "--deploy-mode")); assertEquals("script.py", cmd.get(cmd.size() - 2)); From aae58977878799ccede6587f0e47bbffb5ec3505 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Jan 2015 11:43:11 -0800 Subject: [PATCH 16/51] Use launcher classes instead of jars in non-release mode. This makes it easier for people who are used to building spark with "sbt assembly" instead of "sbt package" to use the new launcher without having to modify their workflow. --- bin/spark-class | 43 ++++++++++++++++++++++++------------------- bin/spark-class2.cmd | 31 +++++++++++++++---------------- launcher/pom.xml | 18 ------------------ pom.xml | 2 +- 4 files changed, 40 insertions(+), 54 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 5d298ac740fe..4aec4660ed64 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -40,33 +40,38 @@ else fi fi +# Look for the launcher. In non-release mode, add the compiled classes directly to the classpath +# instead of looking for a jar file, since it's very common for people using sbt to use the +# "assembly" target instead of "package". SPARK_LAUNCHER_CP= if [ -f $SPARK_HOME/RELEASE ]; then LAUNCHER_DIR="$SPARK_HOME/lib" -else - LAUNCHER_DIR="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION" - if [ -n "$SPARK_PREPEND_CLASSES" ]; then - SPARK_LAUNCHER_CP="$LAUNCHER_DIR/classes:" + num_jars="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" | wc -l)" + if [ "$num_jars" -eq "0" -a -z "$SPARK_LAUNCHER_CP" ]; then + echo "Failed to find Spark launcher in $LAUNCHER_DIR." 1>&2 + echo "You need to build Spark before running this program." 1>&2 + exit 1 fi -fi -num_jars="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" | wc -l)" -if [ "$num_jars" -eq "0" ]; then - echo "Failed to find Spark launcher in $LAUNCHER_DIR." 1>&2 - echo "You need to build Spark before running this program." 1>&2 - exit 1 -fi + LAUNCHER_JARS="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" || true)" + if [ "$num_jars" -gt "1" ]; then + echo "Found multiple Spark launcher jars in $LAUNCHER_DIR:" 1>&2 + echo "$LAUNCHER_JARS" 1>&2 + echo "Please remove all but one jar." 1>&2 + exit 1 + fi -LAUNCHER_JARS="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$")" -if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark launcher jars in $LAUNCHER_DIR:" 1>&2 - echo "$LAUNCHER_JARS" 1>&2 - echo "Please remove all but one jar." 1>&2 - exit 1 + SPARK_LAUNCHER_CP="${LAUNCHER_DIR}/${LAUNCHER_JARS}" +else + LAUNCHER_DIR="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION" + if [ ! -d "$LAUNCHER_DIR/classes" ]; then + echo "Failed to find Spark launcher classes in $LAUNCHER_DIR." 1>&2 + echo "You need to build Spark before running this program." 1>&2 + exit 1 + fi + SPARK_LAUNCHER_CP="$LAUNCHER_DIR/classes" fi -SPARK_LAUNCHER_CP="${SPARK_LAUNCHER_CP}${LAUNCHER_DIR}/${LAUNCHER_JARS}" - # The launcher library will print arguments separated by a NULL character. Read that in a while # loop, populating an array that will be used to exec the final command. CMD=() diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index eff0bb17afd0..99632e634a61 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -29,31 +29,30 @@ if "x%1"=="x" ( exit /b 1 ) -set LAUNCHER_JAR=0 -if exist "%SPARK_HOME%\RELEASE" goto find_release_launcher +set LAUNCHER_CP=0 +if exist %SPARK_HOME%\RELEASE goto find_release_launcher -rem Look for the Spark launcher in both Scala build directories. The launcher doesn't use -rem Scala so it doesn't really matter which one is picked up. -for %%d in (%SPARK_HOME%\launcher\target\scala-2.10\spark-launcher*.jar) do ( - set LAUNCHER_JAR=%%d - set SPARK_SCALA_VERSION=2.10 +rem Look for the Spark launcher in both Scala build directories. The launcher doesn't use Scala so +rem it doesn't really matter which one is picked up. Add the compiled classes directly to the +rem classpath instead of looking for a jar file, since it's very common for people using sbt to use +rem the "assembly" target instead of "package". +set LAUNCHER_CLASSES=%SPARK_HOME%\launcher\target\scala-2.10\classes +if exist %LAUNCHER_CLASSES% ( + set LAUNCHER_CP=%LAUNCHER_CLASSES% ) -for %%d in (%SPARK_HOME%\launcher\target\scala-2.11\spark-launcher*.jar) do ( - set LAUNCHER_JAR=%%d - set SPARK_SCALA_VERSION=2.11 -) -if not "x%SPARK_PREPEND_CLASSES"=="x" ( - set LAUNCHER_JAR=%SPARK_HOME%\launcher\target\scala-%SPARK_SCALA_VERSION%\classes;%LAUNCHER_JAR% +set LAUNCHER_CLASSES=%SPARK_HOME%\launcher\target\scala-2.11\classes +if exist %LAUNCHER_CLASSES% ( + set LAUNCHER_CP=%LAUNCHER_CLASSES% ) goto check_launcher :find_release_launcher for %%d in (%SPARK_HOME%\lib\spark-launcher*.jar) do ( - set LAUNCHER_JAR=%%d + set LAUNCHER_CP=%%d ) :check_launcher -if "%LAUNCHER_JAR%"=="0" ( +if "%LAUNCHER_CP%"=="0" ( echo Failed to find Spark launcher JAR. echo You need to build Spark before running this program. exit /b 1 @@ -63,7 +62,7 @@ rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java -for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCHER_JAR% org.apache.spark.launcher.Main %*"') do ( +for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCHER_CP% org.apache.spark.launcher.Main %*"') do ( set SPARK_CMD=%%i ) %SPARK_CMD% diff --git a/launcher/pom.xml b/launcher/pom.xml index 92213671bc16..2202de4a84a9 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -79,23 +79,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-jar-plugin - - target/scala-${scala.binary.version} - - - - org.scalatest - scalatest-maven-plugin - - - ${project.basedir}/.. - - - - diff --git a/pom.xml b/pom.xml index cd5e59108b6e..993ded783576 100644 --- a/pom.xml +++ b/pom.xml @@ -1131,7 +1131,7 @@ true - ${session.executionRootDirectory} + ${spark.test.home} 1 false false From 61919df21853eba479ddb591fb89dcecfd341988 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Jan 2015 11:45:03 -0800 Subject: [PATCH 17/51] Clean leftover debug statement. --- bin/spark-shell2.cmd | 1 - 1 file changed, 1 deletion(-) diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index 2f791fa9cd98..d59bff71f02c 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -39,7 +39,6 @@ set SPARK_SUBMIT_OPTS="%SPARK_SUBMIT_OPTS% -Dscala.usejavacp=true" :run_shell call %SPARK_HOME%\bin\spark-submit2.cmd --class org.apache.spark.repl.Main %* set SPARK_ERROR_LEVEL=%ERRORLEVEL% -echo AFTER SPARK SUBMIT: %SPARK_ERROR_LEVEL% - %SPARK_LAUNCHER_USAGE_ERROR% if "%SPARK_LAUNCHER_USAGE_ERROR%"=="1" ( call :usage exit /b 1 From 7ed8859a1926788fae5b0bff47a4dac4d22d8cff Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Jan 2015 14:39:49 -0800 Subject: [PATCH 18/51] Some more feedback. --- .../org/apache/spark/deploy/worker/CommandUtils.scala | 8 ++++---- .../java/org/apache/spark/launcher/AbstractLauncher.java | 4 +++- .../java/org/apache/spark/launcher/LauncherCommon.java | 4 ++-- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 6b30deced499..5e417d352a9d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.worker import java.io.{File, FileOutputStream, InputStream, IOException} import java.lang.System._ -import java.util.{ArrayList, List => JList, Map => JMap} +import java.util.{List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.collection.Map @@ -112,7 +112,7 @@ object CommandUtils extends Logging { } } -private class CommandLauncher(sparkHome: String, memory: Int, env: Map[String, String]) +private class CommandLauncher(sparkHome: String, memoryMb: Int, env: Map[String, String]) extends AbstractLauncher[CommandLauncher](env) { setSparkHome(sparkHome) @@ -121,8 +121,8 @@ private class CommandLauncher(sparkHome: String, memory: Int, env: Map[String, S val cmd = buildJavaCommand() cmd.add("-cp") cmd.add(buildClassPath(null).mkString(File.pathSeparator)) - cmd.add(s"-Xms${memory}M") - cmd.add(s"-Xmx${memory}M") + cmd.add(s"-Xms${memoryMb}M") + cmd.add(s"-Xmx${memoryMb}M") addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) cmd } diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index e13fa67f9143..28824b1d599e 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -34,7 +34,9 @@ import java.util.regex.Pattern; /** - * Basic functionality for launchers. + * Basic functionality for launchers - objects that encapsulate knowledge about how to build the + * commands to run a Spark application or service. This class is not meant to be extended by user + * code. */ public abstract class AbstractLauncher extends LauncherCommon { diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java index 0d429914975d..4f50bac7fbe6 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java @@ -23,7 +23,7 @@ import java.util.Map; /** - * Configuration key definitions for Spark jobs, and some helper methods. + * Configuration key definitions for Spark apps, and some helper methods. */ public class LauncherCommon { @@ -128,7 +128,7 @@ protected static boolean isWindows() { * Updates the user environment to contain the merged value of "envKey" after appending * the given path list. */ - protected void mergeEnvPathList(Map userEnv, String envKey, String pathList) { + protected static void mergeEnvPathList(Map userEnv, String envKey, String pathList) { if (!isEmpty(pathList)) { String current = firstNonEmpty(userEnv.get(envKey), System.getenv(envKey)); userEnv.put(envKey, join(File.pathSeparator, current, pathList)); From f7cacffa9f648e04ce6c51e8e6c499e4cf3e18c8 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Jan 2015 16:13:02 -0800 Subject: [PATCH 19/51] Remove "launch Spark in new thread" feature. --- .../apache/spark/launcher/SparkLauncher.java | 101 ------------------ .../spark/launcher/SparkLauncherSuite.java | 59 +--------- 2 files changed, 3 insertions(+), 157 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 43c832697844..e919a95f9e3a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -19,15 +19,11 @@ import java.io.File; import java.io.IOException; -import java.lang.reflect.Method; -import java.net.URL; -import java.net.URLClassLoader; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.concurrent.atomic.AtomicLong; /** * Launcher for Spark applications. @@ -43,8 +39,6 @@ */ public class SparkLauncher extends AbstractLauncher { - private static final AtomicLong THREAD_ID = new AtomicLong(); - protected boolean verbose; protected String appName; protected String master; @@ -139,78 +133,6 @@ public SparkLauncher setVerbose(boolean verbose) { return this; } - /** - * Starts a new thread that will run the Spark application. - *

    - * The application will run on a separate thread and use a separate, isolated class loader. - * No classes or resources from the current thread's class loader will be visible to the app. - *

    - * This mode does not support certain configuration parameters, like configuring the amount of - * driver memory or custom driver command line options. If such configuration is detected, an - * exception will be thrown. - *

    - * This is extremely experimental and should not be used in production environments. - *

    - * NOTE: SparkSubmit uses system properties to propagate some configuration value to the app - * are run concurrently, they may affect each other's configurations. - *

    - * NOTE: for users running JDK versions older than 8, this option can add a lot of overhead - * to the VM's perm gen. - * - * @param exceptionHandler Optional handler for handling exceptions in the app thread. - * @param daemon Whether to start a daemon thread. - * @return A non-daemon thread that will run the application using SparkSubmit. The thread will - * already be started. - */ - public Thread start(Thread.UncaughtExceptionHandler handler, boolean daemon) throws IOException { - // Do some sanity checking that incompatible driver options are not used, because they - // cannot be set in this mode. - Properties props = loadPropertiesFile(); - String extraClassPath = null; - if (isClientMode(props)) { - checkState( - find(DRIVER_EXTRA_JAVA_OPTIONS, conf, props) == null, - "Cannot set driver VM options when running in-process."); - checkState( - find(DRIVER_EXTRA_LIBRARY_PATH, conf, props) == null, - "Cannot set native library path when running in-process."); - checkState( - find(DRIVER_MEMORY, conf, props) == null, - "Cannot set driver memory when running in-process."); - extraClassPath = find(DRIVER_EXTRA_CLASSPATH, conf, props); - } - - List cp = buildClassPath(extraClassPath); - URL[] cpUrls = new URL[cp.size()]; - int idx = 0; - for (String entry : cp) { - cpUrls[idx++] = new File(entry).toURI().toURL(); - } - - URLClassLoader cl = new URLClassLoader(cpUrls, null); - - Thread appThread; - try { - Class sparkSubmit = cl.loadClass("org.apache.spark.deploy.SparkSubmit"); - Method main = sparkSubmit.getDeclaredMethod("main", String[].class); - List args = buildSparkSubmitArgs(); - appThread = new Thread(new SparkSubmitRunner(main, args)); - } catch (ClassNotFoundException cnfe) { - throw new IOException(cnfe); - } catch (NoSuchMethodException nsme) { - throw new IOException(nsme); - } - - appThread.setName("SparkLauncher-Submit-" + THREAD_ID.incrementAndGet()); - appThread.setContextClassLoader(cl); - if (handler != null) { - appThread.setUncaughtExceptionHandler(handler); - } - appThread.setDaemon(daemon); - appThread.start(); - return appThread; - } - /** * Launches a sub-process that will start the configured Spark application. * @@ -340,27 +262,4 @@ private boolean isClientMode(Properties userProps) { (deployMode == null && !userMaster.startsWith("yarn-")); } - private static class SparkSubmitRunner implements Runnable { - - private final Method main; - private final Object args; - - SparkSubmitRunner(Method main, List args) { - this.main = main; - this.args = args.toArray(new String[args.size()]); - } - - @Override - public void run() { - try { - main.invoke(null, args); - } catch (RuntimeException re) { - throw re; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - } - } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index b43bcc580724..fce2003069c0 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -68,39 +68,6 @@ public void testChildProcLauncher() throws Exception { assertEquals(0, app.waitFor()); } - @Test - public void testThreadAppLauncher() throws Exception { - // Do this to avoid overwriting the main test log file. - System.setProperty("test.name", "-testThreadAppLauncher"); - - SparkLauncher launcher = new SparkLauncher() - .setSparkHome(System.getProperty("spark.test.home")) - .setMaster("local") - .setAppResource("spark-internal") - .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) - .setMainClass(SparkLauncherTestApp.class.getName()) - .addAppArgs("thread"); - - printArgs(launcher.buildShellCommand()); - - Thread app = launcher.start(new Thread.UncaughtExceptionHandler() { - @Override - public void uncaughtException(Thread t, Throwable e) { - String msg = "Uncaught exception in app."; - LOG.error(msg, e); - fail(msg); - } - }, true); - app.join(); - } - - @Test - public void testInProcessDriverArgValidator() throws Exception { - testInvalidDriverConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS); - testInvalidDriverConf(SparkLauncher.DRIVER_MEMORY); - testInvalidDriverConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH); - } - private void testCmdBuilder(boolean isDriver) throws Exception { String deployMode = isDriver ? "client" : "cluster"; @@ -182,22 +149,6 @@ private void testCmdBuilder(boolean isDriver) throws Exception { assertEquals("foo", conf.get("spark.foo")); } - private void testInvalidDriverConf(String key) throws Exception { - try { - new SparkLauncher() - .setSparkHome(System.getProperty("spark.test.home")) - .setAppResource("spark-internal") - .setMainClass(SparkLauncherTestApp.class.getName()) - .addAppArgs("thread") - .setConf(key, "foo") - .start(null, true); - fail("Should have failed to start app."); - } catch (IllegalStateException e) { - assertTrue("Correct exception should be thrown.", - e.getMessage().indexOf("running in-process") > 0); - } - } - private String findArgValue(List cmd, String name) { for (int i = 0; i < cmd.size(); i++) { if (cmd.get(i).equals(name)) { @@ -252,13 +203,9 @@ private void printArgs(List cmd) { public static class SparkLauncherTestApp { public static void main(String[] args) throws Exception { - if (args[0].equals("proc")) { - assertEquals("bar", System.getProperty("foo")); - } else if (args[0].equals("arg")) { - assertEquals("newline=", args[1]); - } else { - assertEquals("thread", args[0]); - } + assertEquals(1, args.length); + assertEquals("proc", args[0]); + assertEquals("bar", System.getProperty("foo")); assertEquals("local", System.getProperty(SparkLauncher.SPARK_MASTER)); } From 44cd5f7611251725e00224bcde059635c912b543 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Jan 2015 16:44:20 -0800 Subject: [PATCH 20/51] Add package-info.java, clean up javadocs. Mostly play around with visibility modifiers so that we remove most of the internal APIs from the generated javadoc. The remaining leakages are because of CommandUtils.scala, which is in a different package and thus needs some methods to be protected instead of package-private... --- .../spark/deploy/worker/CommandUtils.scala | 9 ++-- .../spark/launcher/AbstractLauncher.java | 34 ++++++++------ .../apache/spark/launcher/LauncherCommon.java | 24 +++++----- .../java/org/apache/spark/launcher/Main.java | 4 +- .../spark/launcher/SparkClassLauncher.java | 6 +-- .../apache/spark/launcher/SparkLauncher.java | 38 +++++++--------- .../launcher/SparkSubmitCliLauncher.java | 4 +- .../apache/spark/launcher/package-info.java | 45 +++++++++++++++++++ 8 files changed, 103 insertions(+), 61 deletions(-) create mode 100644 launcher/src/main/java/org/apache/spark/launcher/package-info.java diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 5e417d352a9d..b0724b28510f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -59,8 +59,7 @@ object CommandUtils extends Logging { private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows - val cmd = new CommandLauncher(sparkHome, memory, command.environment) - .buildLauncherCommand(command.environment) + val cmd = new CommandLauncher(sparkHome, memory, command.environment).buildCommand() cmd.toSeq ++ Seq(command.mainClass) ++ command.arguments } @@ -117,10 +116,8 @@ private class CommandLauncher(sparkHome: String, memoryMb: Int, env: Map[String, setSparkHome(sparkHome) - override def buildLauncherCommand(env: JMap[String, String]): JList[String] = { - val cmd = buildJavaCommand() - cmd.add("-cp") - cmd.add(buildClassPath(null).mkString(File.pathSeparator)) + def buildCommand(): JList[String] = { + val cmd = buildJavaCommand(null) cmd.add(s"-Xms${memoryMb}M") cmd.add(s"-Xmx${memoryMb}M") addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index 28824b1d599e..61d408f41d25 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -42,15 +42,15 @@ public abstract class AbstractLauncher extends Launc private static final String ENV_SPARK_HOME = "SPARK_HOME"; private static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; - protected static final String DEFAULT_MEM = "512m"; + static final String DEFAULT_MEM = "512m"; - protected String javaHome; - protected String sparkHome; - protected String propertiesFile; - protected final Map conf; - protected final Map launcherEnv; + String javaHome; + String sparkHome; + String propertiesFile; + final Map conf; + final Map launcherEnv; - protected AbstractLauncher() { + AbstractLauncher() { this(Collections.emptyMap()); } @@ -95,10 +95,16 @@ public T setConf(String key, String value) { /** * Launchers should implement this to create the command to be executed. This method should * also update the environment map with any environment variables needed by the child process. + *

    + * Note that this method is a no-op in the base class, even though subclasses in this package + * really must implement it. This approach was taken to allow this method to be package private + * while still allowing CommandUtils.scala to extend this class for its use. * * @param env Map containing environment variables to set for the Spark job. */ - protected abstract List buildLauncherCommand(Map env) throws IOException; + List buildLauncherCommand(Map env) throws IOException { + throw new UnsupportedOperationException("Subclasses must implement this method."); + } /** * Prepares the launcher command for execution from a shell script. This is used by the `Main` @@ -115,7 +121,7 @@ List buildShellCommand() throws IOException { * user-specified properties file, or the spark-defaults.conf file under the Spark configuration * directory. */ - protected Properties loadPropertiesFile() throws IOException { + Properties loadPropertiesFile() throws IOException { Properties props = new Properties(); File propsFile; if (propertiesFile != null) { @@ -144,14 +150,14 @@ protected Properties loadPropertiesFile() throws IOException { return props; } - protected String getSparkHome() { + String getSparkHome() { String path = getenv(ENV_SPARK_HOME); checkState(path != null, "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); return path; } - protected List buildJavaCommand() throws IOException { + protected List buildJavaCommand(String extraClassPath) throws IOException { List cmd = new ArrayList(); if (javaHome == null) { cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); @@ -180,6 +186,8 @@ protected List buildJavaCommand() throws IOException { } } + cmd.add("-cp"); + cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); return cmd; } @@ -196,7 +204,7 @@ protected void addOptionString(List cmd, String options) { * each entry is formatted in the way expected by java.net.URLClassLoader (more * specifically, with trailing slashes for directories). */ - protected List buildClassPath(String appClassPath) throws IOException { + List buildClassPath(String appClassPath) throws IOException { String sparkHome = getSparkHome(); String scala = getScalaVersion(); @@ -313,7 +321,7 @@ private void addToClassPath(List cp, String entries) { } } - protected String getScalaVersion() { + String getScalaVersion() { String scala = getenv("SPARK_SCALA_VERSION"); if (scala != null) { return scala; diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java index 4f50bac7fbe6..aa9447841f89 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java @@ -51,12 +51,12 @@ public class LauncherCommon { public static final String EXECUTOR_CORES = "spark.executor.cores"; /** Returns whether the given string is null or empty. */ - protected static boolean isEmpty(String s) { + static boolean isEmpty(String s) { return s == null || s.isEmpty(); } /** Joins a list of strings using the given separator. */ - protected static String join(String sep, String... elements) { + static String join(String sep, String... elements) { StringBuilder sb = new StringBuilder(); for (String e : elements) { if (e != null) { @@ -70,7 +70,7 @@ protected static String join(String sep, String... elements) { } /** Joins a list of strings using the given separator. */ - protected static String join(String sep, Iterable elements) { + static String join(String sep, Iterable elements) { StringBuilder sb = new StringBuilder(); for (String e : elements) { if (e != null) { @@ -84,7 +84,7 @@ protected static String join(String sep, Iterable elements) { } /** Returns the first value mapped to the given key in the given maps. */ - protected static String find(String key, Map... maps) { + static String find(String key, Map... maps) { for (Map map : maps) { String value = (String) map.get(key); if (!isEmpty(value)) { @@ -95,7 +95,7 @@ protected static String find(String key, Map... maps) { } /** Returns the first non-empty, non-null string in the given list. */ - protected static String firstNonEmpty(String... candidates) { + static String firstNonEmpty(String... candidates) { for (String s : candidates) { if (!isEmpty(s)) { return s; @@ -105,7 +105,7 @@ protected static String firstNonEmpty(String... candidates) { } /** Returns the name of the env variable that holds the native library path. */ - protected static String getLibPathEnvName() { + static String getLibPathEnvName() { if (isWindows()) { return "PATH"; } @@ -119,7 +119,7 @@ protected static String getLibPathEnvName() { } /** Returns whether the OS is Windows. */ - protected static boolean isWindows() { + static boolean isWindows() { String os = System.getProperty("os.name"); return os.startsWith("Windows"); } @@ -128,7 +128,7 @@ protected static boolean isWindows() { * Updates the user environment to contain the merged value of "envKey" after appending * the given path list. */ - protected static void mergeEnvPathList(Map userEnv, String envKey, String pathList) { + static void mergeEnvPathList(Map userEnv, String envKey, String pathList) { if (!isEmpty(pathList)) { String current = firstNonEmpty(userEnv.get(envKey), System.getenv(envKey)); userEnv.put(envKey, join(File.pathSeparator, current, pathList)); @@ -142,7 +142,7 @@ protected static void mergeEnvPathList(Map userEnv, String envKe * Input: "\"ab cd\" efgh 'i \" j'" * Output: [ "ab cd", "efgh", "i \" j" ] */ - protected static List parseOptionString(String s) { + static List parseOptionString(String s) { List opts = new ArrayList(); StringBuilder opt = new StringBuilder(); boolean inOpt = false; @@ -224,21 +224,21 @@ protected static List parseOptionString(String s) { } /** Throws IllegalArgumentException if the given object is null. */ - protected static void checkNotNull(Object o, String arg) { + static void checkNotNull(Object o, String arg) { if (o == null) { throw new IllegalArgumentException(String.format("'%s' must not be null.", arg)); } } /** Throws IllegalArgumentException with the given message if the check is false. */ - protected static void checkArgument(boolean check, String msg, Object... args) { + static void checkArgument(boolean check, String msg, Object... args) { if (!check) { throw new IllegalArgumentException(String.format(msg, args)); } } /** Throws IllegalStateException with the given message if the check is false. */ - protected static void checkState(boolean check, String msg, Object... args) { + static void checkState(boolean check, String msg, Object... args) { if (!check) { throw new IllegalStateException(String.format(msg, args)); } diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index a1ac51c7a8c9..15094e9fffcd 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -25,7 +25,7 @@ /** * Command line interface for the Spark launcher. Used internally by Spark scripts. */ -public class Main extends LauncherCommon { +class Main extends LauncherCommon { /** * Usage: Main [class] [class args] @@ -97,7 +97,7 @@ public static void main(String[] argsArray) throws Exception { private static class UsageLauncher extends AbstractLauncher { @Override - protected List buildLauncherCommand(Map env) { + List buildLauncherCommand(Map env) { if (isWindows()) { return Arrays.asList("set SPARK_LAUNCHER_USAGE_ERROR=1"); } else { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java index 3ebac1994bf4..4a0fc32d028a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java @@ -41,7 +41,7 @@ class SparkClassLauncher extends AbstractLauncher { } @Override - protected List buildLauncherCommand(Map env) throws IOException { + List buildLauncherCommand(Map env) throws IOException { List javaOptsKeys = new ArrayList(); String memKey = null; String extraClassPath = null; @@ -91,7 +91,7 @@ protected List buildLauncherCommand(Map env) throws IOEx return buildSparkSubmitCommand(env); } - List cmd = buildJavaCommand(); + List cmd = buildJavaCommand(extraClassPath); for (String key : javaOptsKeys) { addOptionString(cmd, System.getenv(key)); } @@ -99,8 +99,6 @@ protected List buildLauncherCommand(Map env) throws IOEx String mem = firstNonEmpty(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM); cmd.add("-Xms" + mem); cmd.add("-Xmx" + mem); - cmd.add("-cp"); - cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); cmd.add(className); cmd.addAll(classArgs); return cmd; diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index e919a95f9e3a..03e803d7b4f5 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -31,25 +31,22 @@ * Use this class to start Spark applications programmatically. The class uses a builder pattern * to allow clients to configure the Spark application and launch it as a child process. *

    - * There's also support for running the application on a separate thread, although that is to - * be considered experimental and avoided in production environments. - *

    * Note that launching Spark applications using this class will not automatically load environment * variables from the "spark-env.sh" or "spark-env.cmd" scripts in the configuration directory. */ public class SparkLauncher extends AbstractLauncher { - protected boolean verbose; - protected String appName; - protected String master; - protected String deployMode; - protected String mainClass; - protected String appResource; - protected final List sparkArgs; - protected final List appArgs; - protected final List jars; - protected final List files; - protected final List pyFiles; + boolean verbose; + String appName; + String master; + String deployMode; + String mainClass; + String appResource; + final List sparkArgs; + final List appArgs; + final List jars; + final List files; + final List pyFiles; public SparkLauncher() { this.sparkArgs = new ArrayList(); @@ -218,20 +215,17 @@ List buildSparkSubmitArgs() { } @Override - protected List buildLauncherCommand(Map env) throws IOException { - List cmd = buildJavaCommand(); - addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); - addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); - + List buildLauncherCommand(Map env) throws IOException { // Load the properties file and check whether spark-submit will be running the app's driver // or just launching a cluster app. When running the driver, the JVM's argument will be // modified to cover the driver's configuration. Properties props = loadPropertiesFile(); boolean isClientMode = isClientMode(props); - String extraClassPath = isClientMode ? find(DRIVER_EXTRA_CLASSPATH, conf, props) : null; - cmd.add("-cp"); - cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); + + List cmd = buildJavaCommand(extraClassPath); + addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); + addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); if (isClientMode) { // Figuring out where the memory value come from is a little tricky due to precedence. diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java index d7f216cf89da..d4e74351d71c 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java @@ -37,7 +37,7 @@ *

    * This class has also some special features to aid PySparkLauncher. */ -public class SparkSubmitCliLauncher extends SparkLauncher { +class SparkSubmitCliLauncher extends SparkLauncher { /** * Name of the app resource used to identify the PySpark shell. The command line parser expects @@ -83,7 +83,7 @@ public class SparkSubmitCliLauncher extends SparkLauncher { } @Override - protected List buildLauncherCommand(Map env) throws IOException { + List buildLauncherCommand(Map env) throws IOException { if (PYSPARK_SHELL.equals(appResource)) { return buildPySparkShellCommand(env); } else { diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java new file mode 100644 index 000000000000..96f33ecafedd --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java @@ -0,0 +1,45 @@ +/* + * 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. + */ + +/** + * Library for launching Spark applications. + *

    + * This library allows applications to launch Spark programmatically. There's only one entry + * point to the library - the {@link org.apache.spark.launcher.SparkLauncher} class. + *

    + * To launch a Spark application, just instantiate a {@link org.apache.spark.launcher.SparkLauncher} + * and configure the application to run. For example: + * + *

    + * {@code
    + *   import org.apache.spark.launcher.SparkLauncher;
    + *
    + *   public class MyLauncher {
    + *     public static void main(String[] args) throws Exception {
    + *       Process spark = new SparkLauncher()
    + *         .setAppResource("/my/app.jar")
    + *         .setMainClass("my.spark.app.Main")
    + *         .setMaster("local")
    + *         .setConf(SparkLauncher.DRIVER_MEMORY, "2g")
    + *         .launch();
    + *       spark.waitFor();
    + *     }
    + *   }
    + * }
    + * 
    + */ +package org.apache.spark.launcher; \ No newline at end of file From eae4d8e19a9904116b206a66c424dcca5ea75543 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 20 Jan 2015 15:18:00 -0800 Subject: [PATCH 21/51] Fix new unit tests on Windows. --- .../spark/launcher/SparkLauncherSuite.java | 16 ++++------------ .../launcher/SparkSubmitCliLauncherSuite.java | 11 ++++++++--- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index fce2003069c0..151b135dd73a 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -33,8 +33,6 @@ /** * These tests require the Spark assembly to be built before they can be run. - * - * TODO: these tests do not work on Win32. */ public class SparkLauncherSuite { @@ -85,7 +83,8 @@ private void testCmdBuilder(boolean isDriver) throws Exception { .setConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native") .setConf("spark.foo", "foo"); - List cmd = launcher.buildShellCommand(); + Map env = new HashMap(); + List cmd = launcher.buildLauncherCommand(env); // Checks below are different for driver and non-driver mode. @@ -110,18 +109,11 @@ private void testCmdBuilder(boolean isDriver) throws Exception { assertFalse("Driver classpath should not be in command.", contains("/driver", cp)); } - String[] libPath = null; - String envName = launcher.getLibPathEnvName() + "="; - for (String arg : cmd) { - if (arg.startsWith(envName)) { - libPath = arg.substring(envName.length()).split(Pattern.quote(File.pathSeparator)); - break; - } - } + String libPath = env.get(launcher.getLibPathEnvName()); if (isDriver) { assertNotNull("Native library path should be set.", libPath); assertTrue("Native library path should contain provided entry.", - contains("/native", libPath)); + contains("/native", libPath.split(Pattern.quote(File.pathSeparator)))); } else { assertNull("Native library should not be set.", libPath); } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java index 2018720884db..50f2d20e728e 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java @@ -18,7 +18,9 @@ package org.apache.spark.launcher; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.junit.Test; import static org.junit.Assert.*; @@ -65,11 +67,12 @@ public void testPySparkLauncher() throws Exception { "--master=foo", "--deploy-mode=bar"); + Map env = new HashMap(); List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs) .setSparkHome(System.getProperty("spark.test.home")) - .buildShellCommand(); + .buildLauncherCommand(env); assertEquals("python", cmd.get(cmd.size() - 1)); - assertTrue(cmd.contains("PYSPARK_SUBMIT_ARGS=\"--master\" \"foo\" \"--deploy-mode\" \"bar\"")); + assertEquals("\"--master\" \"foo\" \"--deploy-mode\" \"bar\"", env.get("PYSPARK_SUBMIT_ARGS")); } @Test @@ -80,9 +83,11 @@ public void testPySparkFallback() throws Exception { "script.py", "arg1"); + Map env = new HashMap(); List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs) .setSparkHome(System.getProperty("spark.test.home")) - .buildShellCommand(); + .buildLauncherCommand(env); + assertEquals("foo", findArgValue(cmd, "--master")); assertEquals("bar", findArgValue(cmd, "--deploy-mode")); assertEquals("script.py", cmd.get(cmd.size() - 2)); From 7cff919ce01cea56f9cf8d5201adb145dbc6821f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 21 Jan 2015 12:00:41 -0800 Subject: [PATCH 22/51] Javadoc updates. --- .../spark/launcher/AbstractLauncher.java | 31 +++++++-- .../apache/spark/launcher/SparkLauncher.java | 66 ++++++++++++++++--- 2 files changed, 83 insertions(+), 14 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index 61d408f41d25..caef63b6dd27 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -62,28 +62,49 @@ protected AbstractLauncher(Map env) { @SuppressWarnings("unchecked") private final T THIS = (T) this; - /** Set a custom JAVA_HOME for launching the Spark application. */ + /** + * Set a custom JAVA_HOME for launching the Spark application. + * + * @param javaHome Path to the JAVA_HOME to use. + * @return This launcher. + */ public T setJavaHome(String javaHome) { checkNotNull(javaHome, "javaHome"); this.javaHome = javaHome; return THIS; } - /** Set a custom Spark installation location for the application. */ + /** + * Set a custom Spark installation location for the application. + * + * @param sparkHome Path to the Spark installation to use. + * @return This launcher. + */ public T setSparkHome(String sparkHome) { checkNotNull(sparkHome, "sparkHome"); launcherEnv.put(ENV_SPARK_HOME, sparkHome); return THIS; } - /** Set a custom properties file with Spark configuration for the application. */ + /** + * Set a custom properties file with Spark configuration for the application. + * + * @param path Path to custom properties file to use. + * @return This launcher. + */ public T setPropertiesFile(String path) { checkNotNull(path, "path"); this.propertiesFile = path; return THIS; } - /** Set a single configuration value for the application. */ + /** + * Set a single configuration value for the application. + * + * @param key Configuration key. + * @param value The value to use. + * @return This launcher. + */ public T setConf(String key, String value) { checkNotNull(key, "key"); checkNotNull(value, "value"); @@ -117,7 +138,7 @@ List buildShellCommand() throws IOException { } /** - * Loads the configuration file for the application, if it exists. This is either the + * Loads the configuration file for the application, if it exists. This is either the * user-specified properties file, or the spark-defaults.conf file under the Spark configuration * directory. */ diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 03e803d7b4f5..e9be7794b56a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -56,21 +56,36 @@ public SparkLauncher() { this.pyFiles = new ArrayList(); } - /** Set the application name. */ + /** + * Set the application name. + * + * @param appName Application name. + * @return This launcher. + */ public SparkLauncher setAppName(String appName) { checkNotNull(appName, "appName"); this.appName = appName; return this; } - /** Set the Spark master for the application. */ + /** + * Set the Spark master for the application. + * + * @param master Spark master. + * @return This launcher. + */ public SparkLauncher setMaster(String master) { checkNotNull(master, "master"); this.master = master; return this; } - /** Set the deploy mode for the application. */ + /** + * Set the deploy mode for the application. + * + * @param mode Deploy mode. + * @return This launcher. + */ public SparkLauncher setDeployMode(String mode) { checkNotNull(mode, "mode"); this.deployMode = mode; @@ -80,6 +95,9 @@ public SparkLauncher setDeployMode(String mode) { /** * Set the main application resource. This should be the location of a jar file for Scala/Java * applications, or a python script for PySpark applications. + * + * @param resource Path to the main application resource. + * @return This launcher. */ public SparkLauncher setAppResource(String resource) { checkNotNull(resource, "resource"); @@ -87,14 +105,24 @@ public SparkLauncher setAppResource(String resource) { return this; } - /** Sets the application class name for Java/Scala applications. */ + /** + * Sets the application class name for Java/Scala applications. + * + * @param mainClass Application's main class. + * @return This launcher. + */ public SparkLauncher setMainClass(String mainClass) { checkNotNull(mainClass, "mainClass"); this.mainClass = mainClass; return this; } - /** Adds command line arguments for the application. */ + /** + * Adds command line arguments for the application. + * + * @param args Arguments to pass to the application's main class. + * @return This launcher. + */ public SparkLauncher addAppArgs(String... args) { for (String arg : args) { checkNotNull(arg, "arg"); @@ -103,28 +131,48 @@ public SparkLauncher addAppArgs(String... args) { return this; } - /** Adds a jar file to be submitted with the application. */ + /** + * Adds a jar file to be submitted with the application. + * + * @param jar Path to the jar file. + * @return This launcher. + */ public SparkLauncher addJar(String jar) { checkNotNull(jar, "jar"); jars.add(jar); return this; } - /** Adds a file to be submitted with the application. */ + /** + * Adds a file to be submitted with the application. + * + * @param file Path to the file. + * @return This launcher. + */ public SparkLauncher addFile(String file) { checkNotNull(file, "file"); files.add(file); return this; } - /** Adds a python file / zip / egg to be submitted with the application. */ + /** + * Adds a python file / zip / egg to be submitted with the application. + * + * @param file Path to the file. + * @return This launcher. + */ public SparkLauncher addPyFile(String file) { checkNotNull(file, "file"); pyFiles.add(file); return this; } - /** Enables verbose reporting for SparkSubmit. */ + /** + * Enables verbose reporting for SparkSubmit. + * + * @param verbose Whether to enable verbose output. + * @return This launcher. + */ public SparkLauncher setVerbose(boolean verbose) { this.verbose = verbose; return this; From 23aa2a9c7a0e39987bc487c51e9ad70ecb972e8f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 23 Jan 2015 10:15:23 -0800 Subject: [PATCH 23/51] Read java-opts from conf dir, not spark home. --- .../main/java/org/apache/spark/launcher/AbstractLauncher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java index caef63b6dd27..81f8920ae4df 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java @@ -193,7 +193,7 @@ protected List buildJavaCommand(String extraClassPath) throws IOExceptio } // Load extra JAVA_OPTS from conf/java-opts, if it exists. - File javaOpts = new File(join(File.separator, getSparkHome(), "conf", "java-opts")); + File javaOpts = new File(join(File.separator, getConfDir(), "java-opts")); if (javaOpts.isFile()) { BufferedReader br = new BufferedReader(new InputStreamReader( new FileInputStream(javaOpts), "UTF-8")); From ad03c4859b4da74e95909ab966989d2ded8c8f72 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 26 Jan 2015 10:20:18 -0800 Subject: [PATCH 24/51] Revert "Fix a thread-safety issue in "local" mode." This reverts commit f26556b498cdae3fa23ea5837d673b4f5cb98c58. Conflicts: core/src/main/scala/org/apache/spark/executor/Executor.scala --- .../scala/org/apache/spark/executor/Executor.scala | 11 ++--------- .../apache/spark/scheduler/local/LocalBackend.scala | 8 +++----- 2 files changed, 5 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 1e19dfee87e9..d62461c40b88 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -43,18 +43,9 @@ private[spark] class Executor( executorId: String, executorHostname: String, env: SparkEnv, - conf: SparkConf, isLocal: Boolean = false) extends Logging { - - def this(executorId: String, - slaveHostname: String, - env: SparkEnv, - isLocal: Boolean = false) = { - this(executorId, slaveHostname, env, env.conf, isLocal) - } - logInfo(s"Starting executor ID $executorId on host $executorHostname") // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -64,6 +55,8 @@ private[spark] class Executor( private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) + private val conf = env.conf + @volatile private var isStopped = false // No ip or host:port - just hostname diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 80ce31d9134f..05b6fa54564b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import akka.actor.{Actor, ActorRef, Props} -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} @@ -43,7 +43,6 @@ private case class StopExecutor() private[spark] class LocalActor( scheduler: TaskSchedulerImpl, executorBackend: LocalBackend, - conf: SparkConf, private val totalCores: Int) extends Actor with ActorLogReceive with Logging { @@ -53,7 +52,7 @@ private[spark] class LocalActor( private val localExecutorHostname = "localhost" private val executor = new Executor( - localExecutorId, localExecutorHostname, SparkEnv.get, conf, isLocal = true) + localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true) override def receiveWithLogging = { case ReviveOffers => @@ -92,12 +91,11 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: extends SchedulerBackend with ExecutorBackend { private val appId = "local-" + System.currentTimeMillis - private val conf = SparkEnv.get.conf.clone() var localActor: ActorRef = null override def start() { localActor = SparkEnv.get.actorSystem.actorOf( - Props(new LocalActor(scheduler, this, conf, totalCores)), + Props(new LocalActor(scheduler, this, totalCores)), "LocalBackendActor") } From 46d46da00f48347e05984525839ab9c9b42c837c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 4 Feb 2015 13:22:17 -0800 Subject: [PATCH 25/51] Clean up a test and make it more future-proof. --- .../launcher/SparkSubmitOptionParser.java | 6 +- .../SparkSubmitOptionParserSuite.java | 75 ++++++------------- 2 files changed, 26 insertions(+), 55 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index d67359fce942..9b668b07bb43 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -71,8 +71,10 @@ public abstract class SparkSubmitOptionParser { *

    * Options not listed here nor in the "switch" list below will result in a call to * {@link $#handleUnknown(String)}. + *

    + * These two arrays are visible for tests. */ - private final String[][] opts = { + final String[][] opts = { { ARCHIVES }, { CLASS }, { CONF, "-c" }, @@ -100,7 +102,7 @@ public abstract class SparkSubmitOptionParser { /** * List of switches (command line options that do not take parameters) recognized by spark-submit. */ - private final String[][] switches = { + final String[][] switches = { { HELP, "-h" }, { SUPERVISE }, { VERBOSE, "-v" }, diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java index 4aa5a09d0307..02616d776001 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java @@ -39,60 +39,29 @@ public void setUp() { @Test public void testAllOptions() { - List args = Arrays.asList( - ARCHIVES, ARCHIVES, - CLASS, CLASS, - CONF, CONF, - DEPLOY_MODE, DEPLOY_MODE, - DRIVER_CLASS_PATH, DRIVER_CLASS_PATH, - DRIVER_CORES, DRIVER_CORES, - DRIVER_JAVA_OPTIONS, DRIVER_JAVA_OPTIONS, - DRIVER_LIBRARY_PATH, DRIVER_LIBRARY_PATH, - DRIVER_MEMORY, DRIVER_MEMORY, - EXECUTOR_CORES, EXECUTOR_CORES, - EXECUTOR_MEMORY, EXECUTOR_MEMORY, - FILES, FILES, - JARS, JARS, - MASTER, MASTER, - NAME, NAME, - NUM_EXECUTORS, NUM_EXECUTORS, - PACKAGES, PACKAGES, - PROPERTIES_FILE, PROPERTIES_FILE, - PY_FILES, PY_FILES, - QUEUE, QUEUE, - TOTAL_EXECUTOR_CORES, TOTAL_EXECUTOR_CORES, - REPOSITORIES, REPOSITORIES, - HELP, - SUPERVISE, - VERBOSE); + int count = 0; + for (String[] optNames : parser.opts) { + for (String optName : optNames) { + String value = optName + "-value"; + parser.parse(Arrays.asList(optName, value)); + count++; + verify(parser).handle(eq(optNames[0]), eq(value)); + verify(parser, times(count)).handle(anyString(), anyString()); + verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); + } + } - parser.parse(args); - verify(parser).handle(eq(ARCHIVES), eq(ARCHIVES)); - verify(parser).handle(eq(CLASS), eq(CLASS)); - verify(parser).handle(eq(CONF), eq(CONF)); - verify(parser).handle(eq(DEPLOY_MODE), eq(DEPLOY_MODE)); - verify(parser).handle(eq(DRIVER_CLASS_PATH), eq(DRIVER_CLASS_PATH)); - verify(parser).handle(eq(DRIVER_CORES), eq(DRIVER_CORES)); - verify(parser).handle(eq(DRIVER_JAVA_OPTIONS), eq(DRIVER_JAVA_OPTIONS)); - verify(parser).handle(eq(DRIVER_LIBRARY_PATH), eq(DRIVER_LIBRARY_PATH)); - verify(parser).handle(eq(DRIVER_MEMORY), eq(DRIVER_MEMORY)); - verify(parser).handle(eq(EXECUTOR_CORES), eq(EXECUTOR_CORES)); - verify(parser).handle(eq(EXECUTOR_MEMORY), eq(EXECUTOR_MEMORY)); - verify(parser).handle(eq(FILES), eq(FILES)); - verify(parser).handle(eq(JARS), eq(JARS)); - verify(parser).handle(eq(MASTER), eq(MASTER)); - verify(parser).handle(eq(NAME), eq(NAME)); - verify(parser).handle(eq(NUM_EXECUTORS), eq(NUM_EXECUTORS)); - verify(parser).handle(eq(PACKAGES), eq(PACKAGES)); - verify(parser).handle(eq(PROPERTIES_FILE), eq(PROPERTIES_FILE)); - verify(parser).handle(eq(PY_FILES), eq(PY_FILES)); - verify(parser).handle(eq(QUEUE), eq(QUEUE)); - verify(parser).handle(eq(REPOSITORIES), eq(REPOSITORIES)); - verify(parser).handle(eq(TOTAL_EXECUTOR_CORES), eq(TOTAL_EXECUTOR_CORES)); - verify(parser).handle(eq(HELP), same((String) null)); - verify(parser).handle(eq(SUPERVISE), same((String) null)); - verify(parser).handle(eq(VERBOSE), same((String) null)); - verify(parser).handleExtraArgs(eq(Collections.emptyList())); + for (String[] switchNames : parser.switches) { + int switchCount = 0; + for (String name : switchNames) { + parser.parse(Arrays.asList(name)); + count++; + switchCount++; + verify(parser, times(switchCount)).handle(eq(switchNames[0]), same((String) null)); + verify(parser, times(count)).handle(anyString(), any(String.class)); + verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); + } + } } @Test From de81da218b8650931f9f11da388583b8ba227da2 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 10 Feb 2015 15:57:43 -0800 Subject: [PATCH 26/51] Fix CommandUtils. --- .../org/apache/spark/deploy/worker/CommandUtils.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index b0724b28510f..54f1bd596627 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -59,7 +59,7 @@ object CommandUtils extends Logging { private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows - val cmd = new CommandLauncher(sparkHome, memory, command.environment).buildCommand() + val cmd = new CommandLauncher(sparkHome, memory, command).buildCommand() cmd.toSeq ++ Seq(command.mainClass) ++ command.arguments } @@ -111,15 +111,16 @@ object CommandUtils extends Logging { } } -private class CommandLauncher(sparkHome: String, memoryMb: Int, env: Map[String, String]) - extends AbstractLauncher[CommandLauncher](env) { +private class CommandLauncher(sparkHome: String, memoryMb: Int, command: Command) + extends AbstractLauncher[CommandLauncher](command.environment) { setSparkHome(sparkHome) def buildCommand(): JList[String] = { - val cmd = buildJavaCommand(null) + val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator)) cmd.add(s"-Xms${memoryMb}M") cmd.add(s"-Xmx${memoryMb}M") + command.javaOpts.foreach { cmd.add } addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) cmd } From e4c80b686cdd2aa21206319512203d779b799f5c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 19 Feb 2015 11:53:19 -0800 Subject: [PATCH 27/51] Reorganize the code so that only SparkLauncher is public. 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. --- .../spark/launcher/AbstractLauncher.java | 499 ------------------ .../apache/spark/launcher/CommandBuilder.java | 31 ++ ...erCommon.java => CommandBuilderUtils.java} | 30 +- .../java/org/apache/spark/launcher/Main.java | 108 +++- .../spark/launcher/SparkClassLauncher.java | 16 +- .../apache/spark/launcher/SparkLauncher.java | 389 +++++++++++++- ...er.java => SparkSubmitCommandBuilder.java} | 22 +- ...ite.java => CommandBuilderUtilsSuite.java} | 4 +- .../spark/launcher/SparkLauncherSuite.java | 16 +- ...va => SparkSubmitCommandBuilderSuite.java} | 22 +- 10 files changed, 546 insertions(+), 591 deletions(-) delete mode 100644 launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java rename launcher/src/main/java/org/apache/spark/launcher/{LauncherCommon.java => CommandBuilderUtils.java} (80%) rename launcher/src/main/java/org/apache/spark/launcher/{SparkSubmitCliLauncher.java => SparkSubmitCommandBuilder.java} (92%) rename launcher/src/test/java/org/apache/spark/launcher/{LauncherCommonSuite.java => CommandBuilderUtilsSuite.java} (96%) rename launcher/src/test/java/org/apache/spark/launcher/{SparkSubmitCliLauncherSuite.java => SparkSubmitCommandBuilderSuite.java} (81%) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java deleted file mode 100644 index b0713c8db5d0..000000000000 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java +++ /dev/null @@ -1,499 +0,0 @@ -/* - * 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.BufferedReader; -import java.io.File; -import java.io.FileFilter; -import java.io.FileInputStream; -import java.io.InputStreamReader; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.jar.JarFile; -import java.util.regex.Pattern; - -/** - * Basic functionality for launchers - objects that encapsulate knowledge about how to build the - * commands to run a Spark application or service. This class is not meant to be extended by user - * code. - */ -public abstract class AbstractLauncher extends LauncherCommon { - - private static final String ENV_SPARK_HOME = "SPARK_HOME"; - private static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; - static final String DEFAULT_MEM = "512m"; - - String javaHome; - String sparkHome; - String propertiesFile; - final Map conf; - final Map launcherEnv; - - AbstractLauncher() { - this(Collections.emptyMap()); - } - - protected AbstractLauncher(Map env) { - this.conf = new HashMap(); - this.launcherEnv = new HashMap(env); - } - - @SuppressWarnings("unchecked") - private final T THIS = (T) this; - - /** - * Set a custom JAVA_HOME for launching the Spark application. - * - * @param javaHome Path to the JAVA_HOME to use. - * @return This launcher. - */ - public T setJavaHome(String javaHome) { - checkNotNull(javaHome, "javaHome"); - this.javaHome = javaHome; - return THIS; - } - - /** - * Set a custom Spark installation location for the application. - * - * @param sparkHome Path to the Spark installation to use. - * @return This launcher. - */ - public T setSparkHome(String sparkHome) { - checkNotNull(sparkHome, "sparkHome"); - launcherEnv.put(ENV_SPARK_HOME, sparkHome); - return THIS; - } - - /** - * Set a custom properties file with Spark configuration for the application. - * - * @param path Path to custom properties file to use. - * @return This launcher. - */ - public T setPropertiesFile(String path) { - checkNotNull(path, "path"); - this.propertiesFile = path; - return THIS; - } - - /** - * Set a single configuration value for the application. - * - * @param key Configuration key. - * @param value The value to use. - * @return This launcher. - */ - public T setConf(String key, String value) { - checkNotNull(key, "key"); - checkNotNull(value, "value"); - checkArgument(key.startsWith("spark."), "'key' must start with 'spark.'"); - conf.put(key, value); - return THIS; - } - - /** - * Launchers should implement this to create the command to be executed. This method should - * also update the environment map with any environment variables needed by the child process. - *

    - * Note that this method is a no-op in the base class, even though subclasses in this package - * really must implement it. This approach was taken to allow this method to be package private - * while still allowing CommandUtils.scala to extend this class for its use. - * - * @param env Map containing environment variables to set for the Spark job. - */ - List buildLauncherCommand(Map env) throws IOException { - throw new UnsupportedOperationException("Subclasses must implement this method."); - } - - /** - * Prepares the launcher command for execution from a shell script. This is used by the `Main` - * class to service the scripts shipped with the Spark distribution. - */ - List buildShellCommand() throws IOException { - Map childEnv = new HashMap(launcherEnv); - List cmd = buildLauncherCommand(childEnv); - return isWindows() ? prepareForWindows(cmd, childEnv) : prepareForBash(cmd, childEnv); - } - - /** - * Loads the configuration file for the application, if it exists. This is either the - * user-specified properties file, or the spark-defaults.conf file under the Spark configuration - * directory. - */ - Properties loadPropertiesFile() throws IOException { - Properties props = new Properties(); - File propsFile; - if (propertiesFile != null) { - propsFile = new File(propertiesFile); - checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile); - } else { - propsFile = new File(getConfDir(), DEFAULT_PROPERTIES_FILE); - } - - if (propsFile.isFile()) { - FileInputStream fd = null; - try { - fd = new FileInputStream(propsFile); - props.load(new InputStreamReader(fd, "UTF-8")); - } finally { - if (fd != null) { - try { - fd.close(); - } catch (IOException e) { - // Ignore. - } - } - } - } - - return props; - } - - String getSparkHome() { - String path = getenv(ENV_SPARK_HOME); - checkState(path != null, - "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); - return path; - } - - protected List buildJavaCommand(String extraClassPath) throws IOException { - List cmd = new ArrayList(); - if (javaHome == null) { - cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); - } else { - cmd.add(join(File.separator, javaHome, "bin", "java")); - } - - // Load extra JAVA_OPTS from conf/java-opts, if it exists. - File javaOpts = new File(join(File.separator, getConfDir(), "java-opts")); - if (javaOpts.isFile()) { - BufferedReader br = new BufferedReader(new InputStreamReader( - new FileInputStream(javaOpts), "UTF-8")); - try { - String line; - while ((line = br.readLine()) != null) { - addOptionString(cmd, line); - } - } finally { - br.close(); - } - } - - cmd.add("-cp"); - cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); - return cmd; - } - - /** - * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't - * set it. - */ - protected void addPermGenSizeOpt(List cmd) { - // Don't set MaxPermSize for Java 8 and later. - String[] version = System.getProperty("java.version").split("\\."); - if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) { - return; - } - - for (String arg : cmd) { - if (arg.startsWith("-XX:MaxPermSize=")) { - return; - } - } - - cmd.add("-XX:MaxPermSize=128m"); - } - - protected void addOptionString(List cmd, String options) { - if (!isEmpty(options)) { - for (String opt : parseOptionString(options)) { - cmd.add(opt); - } - } - } - - /** - * Builds the classpath for the application. Returns a list with one classpath entry per element; - * each entry is formatted in the way expected by java.net.URLClassLoader (more - * specifically, with trailing slashes for directories). - */ - List buildClassPath(String appClassPath) throws IOException { - String sparkHome = getSparkHome(); - String scala = getScalaVersion(); - - List cp = new ArrayList(); - addToClassPath(cp, getenv("SPARK_CLASSPATH")); - addToClassPath(cp, appClassPath); - - addToClassPath(cp, getConfDir()); - - boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); - boolean isTesting = "1".equals(getenv("SPARK_TESTING")); - if (prependClasses || isTesting) { - List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", - "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", - "yarn", "launcher"); - if (prependClasses) { - System.err.println( - "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + - "assembly."); - for (String project : projects) { - addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project, - scala)); - } - } - if (isTesting) { - for (String project : projects) { - addToClassPath(cp, String.format("%s/%s/target/scala-%s/test-classes", sparkHome, - project, scala)); - } - } - - // Add this path to include jars that are shaded in the final deliverable created during - // the maven build. These jars are copied to this directory during the build. - addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); - } - - String assembly = findAssembly(scala); - addToClassPath(cp, assembly); - - // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus - // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt - // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built - // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark - // assembly is built for Hive, before actually populating the CLASSPATH with the jars. - // - // This block also serves as a check for SPARK-1703, when the assembly jar is built with - // Java 7 and ends up with too many files, causing issues with other JDK versions. - boolean needsDataNucleus = false; - JarFile assemblyJar = null; - try { - assemblyJar = new JarFile(assembly); - needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; - } catch (IOException ioe) { - if (ioe.getMessage().indexOf("invalid CEN header") > 0) { - System.err.println( - "Loading Spark jar failed.\n" + - "This is likely because Spark was compiled with Java 7 and run\n" + - "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + - "or build Spark with Java 6."); - System.exit(1); - } else { - throw ioe; - } - } finally { - if (assemblyJar != null) { - try { - assemblyJar.close(); - } catch (IOException e) { - // Ignore. - } - } - } - - if (needsDataNucleus) { - System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " + - "in classpath."); - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - } else { - libdir = new File(sparkHome, "lib_managed/jars"); - } - - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - for (File jar : libdir.listFiles()) { - if (jar.getName().startsWith("datanucleus-")) { - addToClassPath(cp, jar.getAbsolutePath()); - } - } - } - - addToClassPath(cp, getenv("HADOOP_CONF_DIR")); - addToClassPath(cp, getenv("YARN_CONF_DIR")); - addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH")); - return cp; - } - - /** - * Adds entries to the classpath. - * - * @param cp List where to appended the new classpath entries. - * @param entries New classpath entries (separated by File.pathSeparator). - */ - private void addToClassPath(List cp, String entries) { - if (isEmpty(entries)) { - return; - } - String[] split = entries.split(Pattern.quote(File.pathSeparator)); - for (String entry : split) { - if (!isEmpty(entry)) { - if (new File(entry).isDirectory() && !entry.endsWith(File.separator)) { - entry += File.separator; - } - cp.add(entry); - } - } - } - - String getScalaVersion() { - String scala = getenv("SPARK_SCALA_VERSION"); - if (scala != null) { - return scala; - } - - String sparkHome = getSparkHome(); - File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); - File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); - if (scala210.isDirectory() && scala211.isDirectory()) { - checkState(false, - "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + - "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); - } else if (scala210.isDirectory()) { - return "2.10"; - } else { - checkState(scala211.isDirectory(), "Cannot find any assembly build directories."); - return "2.11"; - } - - throw new IllegalStateException("Should not reach here."); - } - - private String findAssembly(String scalaVersion) { - String sparkHome = getSparkHome(); - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - } else { - libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion)); - } - - final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); - FileFilter filter = new FileFilter() { - @Override - public boolean accept(File file) { - return file.isFile() && re.matcher(file.getName()).matches(); - } - }; - File[] assemblies = libdir.listFiles(filter); - checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); - checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); - return assemblies[0].getAbsolutePath(); - } - - private String getenv(String key) { - return firstNonEmpty(launcherEnv.get(key), System.getenv(key)); - } - - private String getConfDir() { - String confDir = getenv("SPARK_CONF_DIR"); - return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); - } - - /** - * 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 List prepareForBash(List cmd, Map childEnv) { - if (childEnv.isEmpty()) { - return cmd; - } - - List newCmd = new ArrayList(); - newCmd.add("env"); - - for (Map.Entry e : childEnv.entrySet()) { - newCmd.add(String.format("%s=%s", e.getKey(), e.getValue())); - } - newCmd.addAll(cmd); - return newCmd; - } - - /** - * 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 List prepareForWindows(List cmd, Map childEnv) { - StringBuilder cmdline = new StringBuilder("cmd /c \""); - for (Map.Entry 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()); - } - - /** - * 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 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(); - } - -} diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java new file mode 100644 index 000000000000..548a7633e71d --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java @@ -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 buildCommand(Map env) throws IOException; + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java similarity index 80% rename from launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java rename to launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index aa9447841f89..fb1bba179237 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherCommon.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -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) { @@ -244,7 +221,4 @@ static void checkState(boolean check, String msg, Object... args) { } } - // To avoid subclassing outside this package. - LauncherCommon() { } - } diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index 15094e9fffcd..b256f17fd240 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -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] @@ -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 cmd = launcher.buildShellCommand(); + Map env = new HashMap(); + List 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 winCmd = prepareForWindows(cmd, env); + System.out.println(join(" ", cmd)); } else { - for (String c : cmd) { + List 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 prepareForWindows(List cmd, Map childEnv) { + StringBuilder cmdline = new StringBuilder("cmd /c \""); + for (Map.Entry 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 prepareForBash(List cmd, Map childEnv) { + if (childEnv.isEmpty()) { + return cmd; + } + + List newCmd = new ArrayList(); + newCmd.add("env"); + + for (Map.Entry 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: @@ -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 { + private static class UsageLauncher implements CommandBuilder { @Override - List buildLauncherCommand(Map env) { + public List buildCommand(Map env) { if (isWindows()) { return Arrays.asList("set SPARK_LAUNCHER_USAGE_ERROR=1"); } else { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java index 1d53c95939c0..aaf8458967da 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java @@ -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. *

    * 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 { +class SparkClassLauncher extends SparkLauncher implements CommandBuilder { private final String className; private final List classArgs; @@ -41,7 +43,7 @@ class SparkClassLauncher extends AbstractLauncher { } @Override - List buildLauncherCommand(Map env) throws IOException { + public List buildCommand(Map env) throws IOException { List javaOptsKeys = new ArrayList(); String memKey = null; String extraClassPath = null; @@ -89,7 +91,7 @@ List buildLauncherCommand(Map 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 cmd = buildJavaCommand(extraClassPath); @@ -106,14 +108,14 @@ List buildLauncherCommand(Map env) throws IOException { return cmd; } - private List buildSparkSubmitCommand(Map env) throws IOException { + private List createSparkSubmitCommand(Map env) throws IOException { List sparkSubmitArgs = new ArrayList(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); } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index e64961e4bebe..722c3c2ec5b8 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -17,13 +17,22 @@ package org.apache.spark.launcher; +import java.io.BufferedReader; import java.io.File; +import java.io.FileFilter; +import java.io.FileInputStream; +import java.io.InputStreamReader; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.jar.JarFile; +import java.util.regex.Pattern; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; /** * Launcher for Spark applications. @@ -34,26 +43,109 @@ * Note that launching Spark applications using this class will not automatically load environment * variables from the "spark-env.sh" or "spark-env.cmd" scripts in the configuration directory. */ -public class SparkLauncher extends AbstractLauncher { +public class SparkLauncher { + + /** 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"; + + private static final String ENV_SPARK_HOME = "SPARK_HOME"; + private static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; + static final String DEFAULT_MEM = "512m"; boolean verbose; String appName; - String master; + String appResource; String deployMode; + String javaHome; String mainClass; - String appResource; + String master; + String propertiesFile; final List sparkArgs; final List appArgs; final List jars; final List files; final List pyFiles; + final Map childEnv; + final Map conf; public SparkLauncher() { - this.sparkArgs = new ArrayList(); this.appArgs = new ArrayList(); - this.jars = new ArrayList(); + this.childEnv = new HashMap(); + this.conf = new HashMap(); this.files = new ArrayList(); + this.jars = new ArrayList(); this.pyFiles = new ArrayList(); + this.sparkArgs = new ArrayList(); + } + + /** + * Set a custom JAVA_HOME for launching the Spark application. + * + * @param javaHome Path to the JAVA_HOME to use. + * @return This launcher. + */ + public SparkLauncher setJavaHome(String javaHome) { + checkNotNull(javaHome, "javaHome"); + this.javaHome = javaHome; + return this; + } + + /** + * Set a custom Spark installation location for the application. + * + * @param sparkHome Path to the Spark installation to use. + * @return This launcher. + */ + public SparkLauncher setSparkHome(String sparkHome) { + checkNotNull(sparkHome, "sparkHome"); + childEnv.put(ENV_SPARK_HOME, sparkHome); + return this; + } + + /** + * Set a custom properties file with Spark configuration for the application. + * + * @param path Path to custom properties file to use. + * @return This launcher. + */ + public SparkLauncher setPropertiesFile(String path) { + checkNotNull(path, "path"); + return this; + } + + /** + * Set a single configuration value for the application. + * + * @param key Configuration key. + * @param value The value to use. + * @return This launcher. + */ + public SparkLauncher setConf(String key, String value) { + checkNotNull(key, "key"); + checkNotNull(value, "value"); + checkArgument(key.startsWith("spark."), "'key' must start with 'spark.'"); + conf.put(key, value); + return this; } /** @@ -184,15 +276,219 @@ public SparkLauncher setVerbose(boolean verbose) { * @return A process handle for the Spark app. */ public Process launch() throws IOException { - Map childEnv = new HashMap(launcherEnv); - List cmd = buildLauncherCommand(childEnv); + Map procEnv = new HashMap(childEnv); + List cmd = buildSparkSubmitCommand(procEnv); ProcessBuilder pb = new ProcessBuilder(cmd.toArray(new String[cmd.size()])); - for (Map.Entry e : childEnv.entrySet()) { + for (Map.Entry e : procEnv.entrySet()) { pb.environment().put(e.getKey(), e.getValue()); } return pb.start(); } + protected List buildJavaCommand(String extraClassPath) throws IOException { + List cmd = new ArrayList(); + if (javaHome == null) { + cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); + } else { + cmd.add(join(File.separator, javaHome, "bin", "java")); + } + + // Load extra JAVA_OPTS from conf/java-opts, if it exists. + File javaOpts = new File(join(File.separator, getConfDir(), "java-opts")); + if (javaOpts.isFile()) { + BufferedReader br = new BufferedReader(new InputStreamReader( + new FileInputStream(javaOpts), "UTF-8")); + try { + String line; + while ((line = br.readLine()) != null) { + addOptionString(cmd, line); + } + } finally { + br.close(); + } + } + + cmd.add("-cp"); + cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); + return cmd; + } + + /** + * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't + * set it. + */ + protected void addPermGenSizeOpt(List cmd) { + // Don't set MaxPermSize for Java 8 and later. + String[] version = System.getProperty("java.version").split("\\."); + if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) { + return; + } + + for (String arg : cmd) { + if (arg.startsWith("-XX:MaxPermSize=")) { + return; + } + } + + cmd.add("-XX:MaxPermSize=128m"); + } + + protected void addOptionString(List cmd, String options) { + if (!isEmpty(options)) { + for (String opt : parseOptionString(options)) { + cmd.add(opt); + } + } + } + + /** + * Builds the classpath for the application. Returns a list with one classpath entry per element; + * each entry is formatted in the way expected by java.net.URLClassLoader (more + * specifically, with trailing slashes for directories). + */ + List buildClassPath(String appClassPath) throws IOException { + String sparkHome = getSparkHome(); + String scala = getScalaVersion(); + + List cp = new ArrayList(); + addToClassPath(cp, getenv("SPARK_CLASSPATH")); + addToClassPath(cp, appClassPath); + + addToClassPath(cp, getConfDir()); + + boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); + boolean isTesting = "1".equals(getenv("SPARK_TESTING")); + if (prependClasses || isTesting) { + List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", + "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", + "yarn", "launcher"); + if (prependClasses) { + System.err.println( + "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + + "assembly."); + for (String project : projects) { + addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project, + scala)); + } + } + if (isTesting) { + for (String project : projects) { + addToClassPath(cp, String.format("%s/%s/target/scala-%s/test-classes", sparkHome, + project, scala)); + } + } + + // Add this path to include jars that are shaded in the final deliverable created during + // the maven build. These jars are copied to this directory during the build. + addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); + } + + String assembly = findAssembly(scala); + addToClassPath(cp, assembly); + + // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus + // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt + // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built + // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark + // assembly is built for Hive, before actually populating the CLASSPATH with the jars. + // + // This block also serves as a check for SPARK-1703, when the assembly jar is built with + // Java 7 and ends up with too many files, causing issues with other JDK versions. + boolean needsDataNucleus = false; + JarFile assemblyJar = null; + try { + assemblyJar = new JarFile(assembly); + needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; + } catch (IOException ioe) { + if (ioe.getMessage().indexOf("invalid CEN header") > 0) { + System.err.println( + "Loading Spark jar failed.\n" + + "This is likely because Spark was compiled with Java 7 and run\n" + + "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + + "or build Spark with Java 6."); + System.exit(1); + } else { + throw ioe; + } + } finally { + if (assemblyJar != null) { + try { + assemblyJar.close(); + } catch (IOException e) { + // Ignore. + } + } + } + + if (needsDataNucleus) { + System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " + + "in classpath."); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + } else { + libdir = new File(sparkHome, "lib_managed/jars"); + } + + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + for (File jar : libdir.listFiles()) { + if (jar.getName().startsWith("datanucleus-")) { + addToClassPath(cp, jar.getAbsolutePath()); + } + } + } + + addToClassPath(cp, getenv("HADOOP_CONF_DIR")); + addToClassPath(cp, getenv("YARN_CONF_DIR")); + addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH")); + return cp; + } + + /** + * Adds entries to the classpath. + * + * @param cp List where to appended the new classpath entries. + * @param entries New classpath entries (separated by File.pathSeparator). + */ + private void addToClassPath(List cp, String entries) { + if (isEmpty(entries)) { + return; + } + String[] split = entries.split(Pattern.quote(File.pathSeparator)); + for (String entry : split) { + if (!isEmpty(entry)) { + if (new File(entry).isDirectory() && !entry.endsWith(File.separator)) { + entry += File.separator; + } + cp.add(entry); + } + } + } + + String getScalaVersion() { + String scala = getenv("SPARK_SCALA_VERSION"); + if (scala != null) { + return scala; + } + + String sparkHome = getSparkHome(); + File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); + File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); + if (scala210.isDirectory() && scala211.isDirectory()) { + checkState(false, + "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); + } else if (scala210.isDirectory()) { + return "2.10"; + } else { + checkState(scala211.isDirectory(), "Cannot find any assembly build directories."); + return "2.11"; + } + + throw new IllegalStateException("Should not reach here."); + } + SparkLauncher addSparkArgs(String... args) { for (String arg : args) { sparkArgs.add(arg); @@ -262,8 +558,7 @@ List buildSparkSubmitArgs() { return args; } - @Override - List buildLauncherCommand(Map env) throws IOException { + List buildSparkSubmitCommand(Map env) throws IOException { // Load the properties file and check whether spark-submit will be running the app's driver // or just launching a cluster app. When running the driver, the JVM's argument will be // modified to cover the driver's configuration. @@ -297,6 +592,47 @@ List buildLauncherCommand(Map env) throws IOException { return cmd; } + String getSparkHome() { + String path = getenv(ENV_SPARK_HOME); + checkState(path != null, + "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); + return path; + } + + /** + * Loads the configuration file for the application, if it exists. This is either the + * user-specified properties file, or the spark-defaults.conf file under the Spark configuration + * directory. + */ + Properties loadPropertiesFile() throws IOException { + Properties props = new Properties(); + File propsFile; + if (propertiesFile != null) { + propsFile = new File(propertiesFile); + checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile); + } else { + propsFile = new File(getConfDir(), DEFAULT_PROPERTIES_FILE); + } + + if (propsFile.isFile()) { + FileInputStream fd = null; + try { + fd = new FileInputStream(propsFile); + props.load(new InputStreamReader(fd, "UTF-8")); + } finally { + if (fd != null) { + try { + fd.close(); + } catch (IOException e) { + // Ignore. + } + } + } + } + + return props; + } + private boolean isClientMode(Properties userProps) { String userMaster = firstNonEmpty(master, (String) userProps.get(SPARK_MASTER)); return userMaster == null || @@ -305,4 +641,37 @@ private boolean isClientMode(Properties userProps) { (deployMode == null && !userMaster.startsWith("yarn-")); } + private String findAssembly(String scalaVersion) { + String sparkHome = getSparkHome(); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + } else { + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion)); + } + + final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); + FileFilter filter = new FileFilter() { + @Override + public boolean accept(File file) { + return file.isFile() && re.matcher(file.getName()).matches(); + } + }; + File[] assemblies = libdir.listFiles(filter); + checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); + checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); + return assemblies[0].getAbsolutePath(); + } + + private String getenv(String key) { + return firstNonEmpty(childEnv.get(key), System.getenv(key)); + } + + private String getConfDir() { + String confDir = getenv("SPARK_CONF_DIR"); + return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); + } + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java similarity index 92% rename from launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java rename to launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index d4e74351d71c..79032a177ee3 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCliLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -28,6 +28,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static org.apache.spark.launcher.CommandBuilderUtils.*; + /** * Special launcher for handling a CLI invocation of SparkSubmit. *

    @@ -37,7 +39,7 @@ *

    * This class has also some special features to aid PySparkLauncher. */ -class SparkSubmitCliLauncher extends SparkLauncher { +class SparkSubmitCommandBuilder extends SparkLauncher implements CommandBuilder { /** * Name of the app resource used to identify the PySpark shell. The command line parser expects @@ -63,11 +65,11 @@ class SparkSubmitCliLauncher extends SparkLauncher { private final List driverArgs; private boolean hasMixedArguments; - SparkSubmitCliLauncher(List args) { + SparkSubmitCommandBuilder(List args) { this(false, args); } - SparkSubmitCliLauncher(boolean hasMixedArguments, List args) { + SparkSubmitCommandBuilder(boolean hasMixedArguments, List args) { this.driverArgs = new ArrayList(); List submitArgs = args; @@ -83,11 +85,11 @@ class SparkSubmitCliLauncher extends SparkLauncher { } @Override - List buildLauncherCommand(Map env) throws IOException { + public List buildCommand(Map env) throws IOException { if (PYSPARK_SHELL.equals(appResource)) { return buildPySparkShellCommand(env); } else { - return super.buildLauncherCommand(env); + return super.buildSparkSubmitCommand(env); } } @@ -100,7 +102,7 @@ private List buildPySparkShellCommand(Map env) throws IO "Use ./bin/spark-submit "); setAppResource(appArgs.get(0)); appArgs.remove(0); - return buildLauncherCommand(env); + return buildCommand(env); } // When launching the pyspark shell, the spark-submit arguments should be stored in the @@ -170,19 +172,19 @@ protected boolean handle(String opt, String value) { driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_MEMORY)) { - setConf(LauncherCommon.DRIVER_MEMORY, value); + setConf(DRIVER_MEMORY, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_JAVA_OPTIONS)) { - setConf(LauncherCommon.DRIVER_EXTRA_JAVA_OPTIONS, value); + setConf(DRIVER_EXTRA_JAVA_OPTIONS, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_LIBRARY_PATH)) { - setConf(LauncherCommon.DRIVER_EXTRA_LIBRARY_PATH, value); + setConf(DRIVER_EXTRA_LIBRARY_PATH, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_CLASS_PATH)) { - setConf(LauncherCommon.DRIVER_EXTRA_CLASSPATH, value); + setConf(DRIVER_EXTRA_CLASSPATH, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(CLASS)) { diff --git a/launcher/src/test/java/org/apache/spark/launcher/LauncherCommonSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java similarity index 96% rename from launcher/src/test/java/org/apache/spark/launcher/LauncherCommonSuite.java rename to launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java index 49b088f0e6ae..7854bd826239 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/LauncherCommonSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -24,9 +24,9 @@ import org.junit.Test; import static org.junit.Assert.*; -import static org.apache.spark.launcher.LauncherCommon.*; +import static org.apache.spark.launcher.CommandBuilderUtils.*; -public class LauncherCommonSuite { +public class CommandBuilderUtilsSuite { @Test public void testValidOptionStrings() { diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index 71410c518b05..095a12ccb181 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -59,7 +59,6 @@ public void testChildProcLauncher() throws Exception { .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) .setMainClass(SparkLauncherTestApp.class.getName()) .addAppArgs("proc"); - printArgs(launcher.buildShellCommand()); final Process app = launcher.launch(); new Redirector("stdout", app.getInputStream()).start(); new Redirector("stderr", app.getErrorStream()).start(); @@ -84,7 +83,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { .setConf("spark.foo", "foo"); Map env = new HashMap(); - List cmd = launcher.buildLauncherCommand(env); + List cmd = launcher.buildSparkSubmitCommand(env); // Checks below are different for driver and non-driver mode. @@ -119,7 +118,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { assertFalse("Driver classpath should not be in command.", contains("/driver", cp)); } - String libPath = env.get(launcher.getLibPathEnvName()); + String libPath = env.get(CommandBuilderUtils.getLibPathEnvName()); if (isDriver) { assertNotNull("Native library path should be set.", libPath); assertTrue("Native library path should contain provided entry.", @@ -191,17 +190,6 @@ private Map parseConf(List cmd) { return conf; } - private void printArgs(List cmd) { - StringBuilder cmdLine = new StringBuilder(); - for (String arg : cmd) { - if (cmdLine.length() > 0) { - cmdLine.append(" "); - } - cmdLine.append("[").append(arg).append("]"); - } - LOG.info("Launching SparkSubmit with args: {}", cmdLine.toString()); - } - public static class SparkLauncherTestApp { public static void main(String[] args) throws Exception { diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java similarity index 81% rename from launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java rename to launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 50f2d20e728e..6c1c12632466 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCliLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -25,7 +25,7 @@ import org.junit.Test; import static org.junit.Assert.*; -public class SparkSubmitCliLauncherSuite { +public class SparkSubmitCommandBuilderSuite { @Test public void testShellCliParser() throws Exception { @@ -42,7 +42,7 @@ public void testShellCliParser() throws Exception { "--name", "appName"); - List args = new SparkSubmitCliLauncher(sparkSubmitArgs).buildSparkSubmitArgs(); + List args = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); List expected = Arrays.asList("spark-shell", "--app-arg", "bar", "--app-switch"); assertEquals(expected, args.subList(args.size() - expected.size(), args.size())); } @@ -54,7 +54,7 @@ public void testAlternateSyntaxParsing() throws Exception { "--master=foo", "--deploy-mode=bar"); - List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs).buildSparkSubmitArgs(); + List cmd = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); assertEquals("org.my.Class", findArgValue(cmd, "--class")); assertEquals("foo", findArgValue(cmd, "--master")); assertEquals("bar", findArgValue(cmd, "--deploy-mode")); @@ -63,14 +63,12 @@ public void testAlternateSyntaxParsing() throws Exception { @Test public void testPySparkLauncher() throws Exception { List sparkSubmitArgs = Arrays.asList( - SparkSubmitCliLauncher.PYSPARK_SHELL, + SparkSubmitCommandBuilder.PYSPARK_SHELL, "--master=foo", "--deploy-mode=bar"); Map env = new HashMap(); - List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs) - .setSparkHome(System.getProperty("spark.test.home")) - .buildLauncherCommand(env); + List cmd = buildCommand(sparkSubmitArgs, env); assertEquals("python", cmd.get(cmd.size() - 1)); assertEquals("\"--master\" \"foo\" \"--deploy-mode\" \"bar\"", env.get("PYSPARK_SUBMIT_ARGS")); } @@ -84,9 +82,7 @@ public void testPySparkFallback() throws Exception { "arg1"); Map env = new HashMap(); - List cmd = new SparkSubmitCliLauncher(sparkSubmitArgs) - .setSparkHome(System.getProperty("spark.test.home")) - .buildLauncherCommand(env); + List cmd = buildCommand(sparkSubmitArgs, env); assertEquals("foo", findArgValue(cmd, "--master")); assertEquals("bar", findArgValue(cmd, "--deploy-mode")); @@ -104,4 +100,10 @@ private String findArgValue(List cmd, String name) { return null; } + private List buildCommand(List args, Map env) throws Exception { + SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); + builder.setSparkHome(System.getProperty("spark.test.home")); + return builder.buildCommand(env); + } + } From 9cd5b445f8ab045d1e7f17af0bf26c4b1e2db6a9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 19 Feb 2015 13:09:15 -0800 Subject: [PATCH 28/51] Make all non-public APIs package-private. public/protected types and methods that were exposed because they were used in other parts of Spark now are package-private, and some helper code is added in core/ to expose them to the rest of Spark where needed. Now only SparkLauncher is public, and it only exposes public and protected methods that are meant to be exposed to external users. --- .../spark/deploy/SparkSubmitArguments.scala | 6 +- .../spark/deploy/worker/CommandUtils.scala | 20 +----- .../launcher/SparkSubmitArgumentsParser.scala | 26 ++++++++ .../spark/launcher/WorkerCommandBuilder.scala | 46 ++++++++++++++ .../spark/launcher/SparkClassLauncher.java | 25 +++++++- .../apache/spark/launcher/SparkLauncher.java | 21 ++++--- .../launcher/SparkSubmitOptionParser.java | 63 ++++++++++--------- .../SparkSubmitOptionParserSuite.java | 10 +-- 8 files changed, 151 insertions(+), 66 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala create mode 100644 core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 822dcddd18e4..38b9477a603b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.deploy.SparkSubmitAction._ -import org.apache.spark.launcher.SparkSubmitOptionParser +import org.apache.spark.launcher.SparkSubmitArgumentsParser import org.apache.spark.util.Utils /** @@ -33,7 +33,7 @@ import org.apache.spark.util.Utils * The env argument is used for testing. */ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) - extends SparkSubmitOptionParser { + extends SparkSubmitArgumentsParser { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -286,8 +286,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St """.stripMargin } - import SparkSubmitOptionParser._ - /** Fill in values by parsing user options. */ override protected def handle(opt: String, value: String): Boolean = { opt match { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 54f1bd596627..ccf4289ce1e6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -26,7 +26,7 @@ import scala.collection.Map import org.apache.spark.Logging import org.apache.spark.deploy.Command -import org.apache.spark.launcher.AbstractLauncher +import org.apache.spark.launcher.WorkerCommandBuilder import org.apache.spark.util.Utils /** @@ -59,7 +59,7 @@ object CommandUtils extends Logging { private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows - val cmd = new CommandLauncher(sparkHome, memory, command).buildCommand() + val cmd = new WorkerCommandBuilder(sparkHome, memory, command).buildCommand() cmd.toSeq ++ Seq(command.mainClass) ++ command.arguments } @@ -110,19 +110,3 @@ object CommandUtils extends Logging { }.start() } } - -private class CommandLauncher(sparkHome: String, memoryMb: Int, command: Command) - extends AbstractLauncher[CommandLauncher](command.environment) { - - setSparkHome(sparkHome) - - def buildCommand(): JList[String] = { - val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator)) - cmd.add(s"-Xms${memoryMb}M") - cmd.add(s"-Xmx${memoryMb}M") - command.javaOpts.foreach { cmd.add } - addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) - cmd - } - -} diff --git a/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala b/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala new file mode 100644 index 000000000000..7db086c550a0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala @@ -0,0 +1,26 @@ +/* + * 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 + +/** + * This class makes SparkSubmitOptionParser visible for Spark code outside of the `launcher` + * package. + */ +private[spark] abstract class SparkSubmitArgumentsParser extends SparkSubmitOptionParser { + +} diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala new file mode 100644 index 000000000000..2b59eab5ed29 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala @@ -0,0 +1,46 @@ +/* + * 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.File +import java.util.{List => JList} + +import scala.collection.JavaConversions._ + +import org.apache.spark.deploy.Command + +/** + * This class is used by CommandUtils. It uses some package-private APIs in SparkLauncher and so + * needs to live in the same package as the rest of the library. + */ +private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, command: Command) + extends SparkLauncher(command.environment) { + + setSparkHome(sparkHome) + + def buildCommand(): JList[String] = { + val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator)) + cmd.add(s"-Xms${memoryMb}M") + cmd.add(s"-Xmx${memoryMb}M") + command.javaOpts.foreach { cmd.add } + addPermGenSizeOpt(cmd) + addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) + cmd + } + +} \ No newline at end of file diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java index aaf8458967da..c960a940fb36 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java @@ -110,7 +110,30 @@ public List buildCommand(Map env) throws IOException { private List createSparkSubmitCommand(Map env) throws IOException { List sparkSubmitArgs = new ArrayList(classArgs); - sparkSubmitArgs.add(SparkSubmitOptionParser.CLASS); + + // This is a workaround for the fact that the constants in SparkSubmitOptionParser are not + // static. The parser itself is never used, we just don't want to hardcode the value of that + // option here. + SparkSubmitOptionParser parser = new SparkSubmitOptionParser() { + + @Override + protected boolean handle(String opt, String value) { + throw new UnsupportedOperationException(); + } + + @Override + protected boolean handleUnknown(String opt) { + throw new UnsupportedOperationException(); + } + + @Override + protected void handleExtraArgs(List extra) { + throw new UnsupportedOperationException(); + } + + }; + + sparkSubmitArgs.add(parser.CLASS); sparkSubmitArgs.add(className); SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(true, sparkSubmitArgs); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 722c3c2ec5b8..396617ee6aab 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -89,8 +90,12 @@ public class SparkLauncher { final Map conf; public SparkLauncher() { + this(Collections.emptyMap()); + } + + public SparkLauncher(Map env) { this.appArgs = new ArrayList(); - this.childEnv = new HashMap(); + this.childEnv = new HashMap(env); this.conf = new HashMap(); this.files = new ArrayList(); this.jars = new ArrayList(); @@ -285,7 +290,7 @@ public Process launch() throws IOException { return pb.start(); } - protected List buildJavaCommand(String extraClassPath) throws IOException { + List buildJavaCommand(String extraClassPath) throws IOException { List cmd = new ArrayList(); if (javaHome == null) { cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); @@ -317,7 +322,7 @@ protected List buildJavaCommand(String extraClassPath) throws IOExceptio * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't * set it. */ - protected void addPermGenSizeOpt(List cmd) { + void addPermGenSizeOpt(List cmd) { // Don't set MaxPermSize for Java 8 and later. String[] version = System.getProperty("java.version").split("\\."); if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) { @@ -333,7 +338,7 @@ protected void addPermGenSizeOpt(List cmd) { cmd.add("-XX:MaxPermSize=128m"); } - protected void addOptionString(List cmd, String options) { + void addOptionString(List cmd, String options) { if (!isEmpty(options)) { for (String opt : parseOptionString(options)) { cmd.add(opt); @@ -633,6 +638,10 @@ Properties loadPropertiesFile() throws IOException { return props; } + String getenv(String key) { + return firstNonEmpty(childEnv.get(key), System.getenv(key)); + } + private boolean isClientMode(Properties userProps) { String userMaster = firstNonEmpty(master, (String) userProps.get(SPARK_MASTER)); return userMaster == null || @@ -665,10 +674,6 @@ public boolean accept(File file) { return assemblies[0].getAbsolutePath(); } - private String getenv(String key) { - return firstNonEmpty(childEnv.get(key), System.getenv(key)); - } - private String getConfDir() { String confDir = getenv("SPARK_CONF_DIR"); return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index d29051ebd741..80271bec9aa2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -30,45 +30,48 @@ * is a single list of options that needs to be maintained (well, sort of, but it makes it harder * to break things). */ -public abstract class SparkSubmitOptionParser { +abstract class SparkSubmitOptionParser { // The following constants define the "main" name for the available options. They're defined // to avoid copy & paste of the raw strings where they're needed. - protected static final String CLASS = "--class"; - protected static final String CONF = "--conf"; - protected static final String DEPLOY_MODE = "--deploy-mode"; - protected static final String DRIVER_CLASS_PATH = "--driver-class-path"; - protected static final String DRIVER_CORES = "--driver-cores"; - protected static final String DRIVER_JAVA_OPTIONS = "--driver-java-options"; - protected static final String DRIVER_LIBRARY_PATH = "--driver-library-path"; - protected static final String DRIVER_MEMORY = "--driver-memory"; - protected static final String EXECUTOR_MEMORY = "--executor-memory"; - protected static final String FILES = "--files"; - protected static final String JARS = "--jars"; - protected static final String KILL_SUBMISSION = "--kill"; - protected static final String MASTER = "--master"; - protected static final String NAME = "--name"; - protected static final String PACKAGES = "--packages"; - protected static final String PROPERTIES_FILE = "--properties-file"; - protected static final String PROXY_USER = "--proxy-user"; - protected static final String PY_FILES = "--py-files"; - protected static final String REPOSITORIES = "--repositories"; - protected static final String STATUS = "--status"; - protected static final String TOTAL_EXECUTOR_CORES = "--total-executor-cores"; + // + // The fields are not static so that they're exposed to Scala code that uses this class. See + // SparkSubmitArguments.scala. + protected final String CLASS = "--class"; + protected final String CONF = "--conf"; + protected final String DEPLOY_MODE = "--deploy-mode"; + protected final String DRIVER_CLASS_PATH = "--driver-class-path"; + protected final String DRIVER_CORES = "--driver-cores"; + protected final String DRIVER_JAVA_OPTIONS = "--driver-java-options"; + protected final String DRIVER_LIBRARY_PATH = "--driver-library-path"; + protected final String DRIVER_MEMORY = "--driver-memory"; + protected final String EXECUTOR_MEMORY = "--executor-memory"; + protected final String FILES = "--files"; + protected final String JARS = "--jars"; + protected final String KILL_SUBMISSION = "--kill"; + protected final String MASTER = "--master"; + protected final String NAME = "--name"; + protected final String PACKAGES = "--packages"; + protected final String PROPERTIES_FILE = "--properties-file"; + protected final String PROXY_USER = "--proxy-user"; + protected final String PY_FILES = "--py-files"; + protected final String REPOSITORIES = "--repositories"; + protected final String STATUS = "--status"; + protected final String TOTAL_EXECUTOR_CORES = "--total-executor-cores"; // Options that do not take arguments. - protected static final String HELP = "--help"; - protected static final String SUPERVISE = "--supervise"; - protected static final String VERBOSE = "--verbose"; - protected static final String VERSION = "--version"; + protected final String HELP = "--help"; + protected final String SUPERVISE = "--supervise"; + protected final String VERBOSE = "--verbose"; + protected final String VERSION = "--version"; // Standalone-only options. // YARN-only options. - protected static final String ARCHIVES = "--archives"; - protected static final String EXECUTOR_CORES = "--executor-cores"; - protected static final String QUEUE = "--queue"; - protected static final String NUM_EXECUTORS = "--num-executors"; + protected final String ARCHIVES = "--archives"; + protected final String EXECUTOR_CORES = "--executor-cores"; + protected final String QUEUE = "--queue"; + protected final String NUM_EXECUTORS = "--num-executors"; /** * This is the canonical list of spark-submit options. Each entry in the array contains the diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java index 02616d776001..f3d210991705 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java @@ -66,23 +66,23 @@ public void testAllOptions() { @Test public void testExtraOptions() { - List args = Arrays.asList(MASTER, MASTER, "foo", "bar"); + List args = Arrays.asList(parser.MASTER, parser.MASTER, "foo", "bar"); parser.parse(args); - verify(parser).handle(eq(MASTER), eq(MASTER)); + verify(parser).handle(eq(parser.MASTER), eq(parser.MASTER)); verify(parser).handleUnknown(eq("foo")); verify(parser).handleExtraArgs(eq(Arrays.asList("bar"))); } @Test(expected=IllegalArgumentException.class) public void testMissingArg() { - parser.parse(Arrays.asList(MASTER)); + parser.parse(Arrays.asList(parser.MASTER)); } @Test public void testEqualSeparatedOption() { - List args = Arrays.asList(MASTER + "=" + MASTER); + List args = Arrays.asList(parser.MASTER + "=" + parser.MASTER); parser.parse(args); - verify(parser).handle(eq(MASTER), eq(MASTER)); + verify(parser).handle(eq(parser.MASTER), eq(parser.MASTER)); verify(parser).handleExtraArgs(eq(Collections.emptyList())); } From 62978e463336f3354400bd0235d1af3d61e0c2ce Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 19 Feb 2015 14:15:40 -0800 Subject: [PATCH 29/51] Minor cleanup of Windows code path. --- .../java/org/apache/spark/launcher/Main.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index b256f17fd240..587640886e5b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -75,8 +75,7 @@ public static void main(String[] argsArray) throws Exception { } if (isWindows()) { - List winCmd = prepareForWindows(cmd, env); - System.out.println(join(" ", cmd)); + System.out.println(prepareForWindows(cmd, env)); } else { List bashCmd = prepareForBash(cmd, env); for (String c : bashCmd) { @@ -101,23 +100,18 @@ public static void main(String[] argsArray) throws Exception { * 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 prepareForWindows(List cmd, Map childEnv) { + private static String prepareForWindows(List cmd, Map childEnv) { StringBuilder cmdline = new StringBuilder("cmd /c \""); for (Map.Entry e : childEnv.entrySet()) { - if (cmdline.length() > 0) { - cmdline.append(" "); - } cmdline.append(String.format("set %s=%s", e.getKey(), e.getValue())); - cmdline.append(" &&"); + cmdline.append(" && "); } for (String arg : cmd) { - if (cmdline.length() > 0) { - cmdline.append(" "); - } cmdline.append(quoteForBatchScript(arg)); + cmdline.append(" "); } cmdline.append("\""); - return Arrays.asList(cmdline.toString()); + return cmdline.toString(); } /** From 72da7ecef7b9db368ee4feb577ca745cf5167e1f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 19 Feb 2015 14:20:06 -0800 Subject: [PATCH 30/51] Rename SparkClassLauncher. --- .../src/main/java/org/apache/spark/launcher/Main.java | 2 +- ...rkClassLauncher.java => SparkClassCommandBuilder.java} | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) rename launcher/src/main/java/org/apache/spark/launcher/{SparkClassLauncher.java => SparkClassCommandBuilder.java} (94%) diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index 587640886e5b..e2b23588bdf2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -59,7 +59,7 @@ public static void main(String[] argsArray) throws Exception { if (className.equals("org.apache.spark.deploy.SparkSubmit")) { builder = new SparkSubmitCommandBuilder(args); } else { - builder = new SparkClassLauncher(className, args); + builder = new SparkClassCommandBuilder(className, args); } printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); } catch (IllegalArgumentException e) { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java similarity index 94% rename from launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java rename to launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java index c960a940fb36..d34679959826 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -27,17 +27,17 @@ import static org.apache.spark.launcher.CommandBuilderUtils.*; /** - * Launcher for internal Spark classes. + * Command builder for internal Spark classes. *

    * This class handles building the command to launch all internal Spark classes except for - * SparkSubmit (which is handled by the public {@link SparkLauncher} class. + * SparkSubmit (which is handled by {@link SparkSubmitCommandBuilder} class. */ -class SparkClassLauncher extends SparkLauncher implements CommandBuilder { +class SparkClassCommandBuilder extends SparkLauncher implements CommandBuilder { private final String className; private final List classArgs; - SparkClassLauncher(String className, List classArgs) { + SparkClassCommandBuilder(String className, List classArgs) { this.className = className; this.classArgs = classArgs; } From 95ddfa846b5b46cd91150a5bb14be6f085a46b6d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 19 Feb 2015 14:30:29 -0800 Subject: [PATCH 31/51] Fix handling of --help for spark-class command builder. --- .../launcher/SparkClassCommandBuilder.java | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java index d34679959826..cbee74cf0417 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -109,35 +109,48 @@ public List buildCommand(Map env) throws IOException { } private List createSparkSubmitCommand(Map env) throws IOException { - List sparkSubmitArgs = new ArrayList(classArgs); + final List sparkSubmitArgs = new ArrayList(); + final List appArgs = new ArrayList(); - // This is a workaround for the fact that the constants in SparkSubmitOptionParser are not - // static. The parser itself is never used, we just don't want to hardcode the value of that - // option here. + // This parser exists for two reasons: + // - to expose the command line args constants, since they're not static + // - to special-case the HELP command line argument, and allow it to be propagated to + // the app being launched. SparkSubmitOptionParser parser = new SparkSubmitOptionParser() { @Override protected boolean handle(String opt, String value) { - throw new UnsupportedOperationException(); + if (opt.equals(HELP)) { + appArgs.add(opt); + } else { + sparkSubmitArgs.add(opt); + sparkSubmitArgs.add(value); + } + return true; } @Override protected boolean handleUnknown(String opt) { - throw new UnsupportedOperationException(); + appArgs.add(opt); + return true; } @Override protected void handleExtraArgs(List extra) { - throw new UnsupportedOperationException(); + appArgs.addAll(extra); } }; + parser.parse(classArgs); sparkSubmitArgs.add(parser.CLASS); sparkSubmitArgs.add(className); SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(true, sparkSubmitArgs); builder.setAppResource("spark-internal"); + for (String arg: appArgs) { + builder.addAppArgs(arg); + } return builder.buildCommand(env); } From 8ec0243105ead78995ec9cad9502ceac15721a8d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 19 Feb 2015 14:48:13 -0800 Subject: [PATCH 32/51] Add missing newline. --- .../scala/org/apache/spark/launcher/WorkerCommandBuilder.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala index 2b59eab5ed29..657ce3bb512f 100644 --- a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala +++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala @@ -43,4 +43,4 @@ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, comm cmd } -} \ No newline at end of file +} From bb67b93828e0f218afa9652fbceee9688fc3f992 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 19 Feb 2015 19:24:26 -0800 Subject: [PATCH 33/51] Remove unrelated Yarn change (that is also wrong). --- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 42de09b94519..46d9df93488c 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -314,13 +314,6 @@ private[spark] class Client( env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() - // Propagate SPARK_HOME to the containers. This is needed for pyspark to - // work, since the executor's PYTHONPATH is built based on the location - // of SPARK_HOME. - sparkConf.getOption("spark.home").orElse(sys.env.get("SPARK_HOME")).foreach { path => - env("SPARK_HOME") = path - } - // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) From 304333a37633c7789f5b43b7356bcc294e3e3286 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 20 Feb 2015 10:17:21 -0800 Subject: [PATCH 34/51] Fix propagation of properties file arg. Also add a test. And remove some hardcoded strings, using the constants defined in SparkSubmitOptionParser. --- .../launcher/SparkClassCommandBuilder.java | 6 ++-- .../apache/spark/launcher/SparkLauncher.java | 22 +++++++------ .../launcher/SparkSubmitOptionParser.java | 17 +++++++--- .../spark/launcher/SparkLauncherSuite.java | 31 ++++++++++++++----- 4 files changed, 50 insertions(+), 26 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java index cbee74cf0417..95c5df17508b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -112,10 +112,8 @@ private List createSparkSubmitCommand(Map env) throws IO final List sparkSubmitArgs = new ArrayList(); final List appArgs = new ArrayList(); - // This parser exists for two reasons: - // - to expose the command line args constants, since they're not static - // - to special-case the HELP command line argument, and allow it to be propagated to - // the app being launched. + // Parse the command line and special-case the HELP command line argument, allowing it to be + // propagated to the app being launched. SparkSubmitOptionParser parser = new SparkSubmitOptionParser() { @Override diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 396617ee6aab..e1bb66e02c9c 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -135,6 +135,7 @@ public SparkLauncher setSparkHome(String sparkHome) { */ public SparkLauncher setPropertiesFile(String path) { checkNotNull(path, "path"); + this.propertiesFile = path; return this; } @@ -504,53 +505,54 @@ SparkLauncher addSparkArgs(String... args) { // Visible for testing. List buildSparkSubmitArgs() { List args = new ArrayList(); + SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); if (verbose) { - args.add("--verbose"); + args.add(parser.VERBOSE); } if (master != null) { - args.add("--master"); + args.add(parser.MASTER); args.add(master); } if (deployMode != null) { - args.add("--deploy-mode"); + args.add(parser.DEPLOY_MODE); args.add(deployMode); } if (appName != null) { - args.add("--name"); + args.add(parser.NAME); args.add(appName); } for (Map.Entry e : conf.entrySet()) { - args.add("--conf"); + args.add(parser.CONF); args.add(String.format("%s=%s", e.getKey(), e.getValue())); } if (propertiesFile != null) { - args.add("--properties-file"); + args.add(parser.PROPERTIES_FILE); args.add(propertiesFile); } if (!jars.isEmpty()) { - args.add("--jars"); + args.add(parser.JARS); args.add(join(",", jars)); } if (!files.isEmpty()) { - args.add("--files"); + args.add(parser.FILES); args.add(join(",", files)); } if (!pyFiles.isEmpty()) { - args.add("--py-files"); + args.add(parser.PY_FILES); args.add(join(",", pyFiles)); } if (mainClass != null) { - args.add("--class"); + args.add(parser.CLASS); args.add(mainClass); } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 80271bec9aa2..7bb6e3f4fa5b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -30,13 +30,14 @@ * is a single list of options that needs to be maintained (well, sort of, but it makes it harder * to break things). */ -abstract class SparkSubmitOptionParser { +class SparkSubmitOptionParser { // The following constants define the "main" name for the available options. They're defined // to avoid copy & paste of the raw strings where they're needed. // // The fields are not static so that they're exposed to Scala code that uses this class. See - // SparkSubmitArguments.scala. + // SparkSubmitArguments.scala. That is also why this class is not abstract - to allow code to + // easily use these constants without having to create dummy implementations of this class. protected final String CLASS = "--class"; protected final String CONF = "--conf"; protected final String DEPLOY_MODE = "--deploy-mode"; @@ -186,7 +187,9 @@ protected final void parse(List args) { * @param value The value. This will be null if the option does not take a value. * @return Whether to continue parsing the argument list. */ - protected abstract boolean handle(String opt, String value); + protected boolean handle(String opt, String value) { + throw new UnsupportedOperationException(); + } /** * Callback for when an unrecognized option is parsed. @@ -194,7 +197,9 @@ protected final void parse(List args) { * @param opt Unrecognized option from the command line. * @return Whether to continue parsing the argument list. */ - protected abstract boolean handleUnknown(String opt); + protected boolean handleUnknown(String opt) { + throw new UnsupportedOperationException(); + } /** * Callback for remaining command line arguments after either {@link #handle(String, String)} or @@ -203,7 +208,9 @@ protected final void parse(List args) { * * @param extra List of remaining arguments. */ - protected abstract void handleExtraArgs(List extra); + protected void handleExtraArgs(List extra) { + throw new UnsupportedOperationException(); + } private String findCliOption(String name, String[][] available) { for (String[] candidates : available) { diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index 095a12ccb181..ead509504808 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -26,6 +26,8 @@ import java.util.Map; import java.util.regex.Pattern; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +40,18 @@ public class SparkLauncherSuite { private static final Logger LOG = LoggerFactory.getLogger(SparkLauncherSuite.class); + private static File dummyPropsFile; + + @BeforeClass + public static void setUp() throws Exception { + dummyPropsFile = File.createTempFile("spark", "properties"); + } + + @AfterClass + public static void cleanUp() throws Exception { + dummyPropsFile.delete(); + } + @Test public void testDriverCmdBuilder() throws Exception { testCmdBuilder(true); @@ -75,6 +89,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { .setAppResource("/foo") .setAppName("MyApp") .setMainClass("my.Class") + .setPropertiesFile(dummyPropsFile.getAbsolutePath()) .addAppArgs("foo", "bar") .setConf(SparkLauncher.DRIVER_MEMORY, "1g") .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver") @@ -128,11 +143,13 @@ private void testCmdBuilder(boolean isDriver) throws Exception { } // Checks below are the same for both driver and non-driver mode. + SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); - assertEquals("yarn", findArgValue(cmd, "--master")); - assertEquals(deployMode, findArgValue(cmd, "--deploy-mode")); - assertEquals("my.Class", findArgValue(cmd, "--class")); - assertEquals("MyApp", findArgValue(cmd, "--name")); + assertEquals(dummyPropsFile.getAbsolutePath(), findArgValue(cmd, parser.PROPERTIES_FILE)); + assertEquals("yarn", findArgValue(cmd, parser.MASTER)); + assertEquals(deployMode, findArgValue(cmd, parser.DEPLOY_MODE)); + assertEquals("my.Class", findArgValue(cmd, parser.CLASS)); + assertEquals("MyApp", findArgValue(cmd, parser.NAME)); boolean appArgsOk = false; for (int i = 0; i < cmd.size(); i++) { @@ -146,7 +163,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { } assertTrue("App resource and args should be added to command.", appArgsOk); - Map conf = parseConf(cmd); + Map conf = parseConf(cmd, parser); assertEquals("foo", conf.get("spark.foo")); } @@ -178,10 +195,10 @@ private boolean contains(String needle, Iterable haystack) { return false; } - private Map parseConf(List cmd) { + private Map parseConf(List cmd, SparkSubmitOptionParser parser) { Map conf = new HashMap(); for (int i = 0; i < cmd.size(); i++) { - if (cmd.get(i).equals("--conf")) { + if (cmd.get(i).equals(parser.CONF)) { String[] val = cmd.get(i + 1).split("=", 2); conf.put(val[0], val[1]); i += 1; From 28b1434721394a3cf1bae0e53caf7c69b9312fc3 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 23 Feb 2015 09:57:02 -0800 Subject: [PATCH 35/51] Add a comment. --- .../java/org/apache/spark/launcher/CommandBuilder.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java index 548a7633e71d..68838f21692d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java @@ -26,6 +26,14 @@ */ interface CommandBuilder { + /** + * Builds the command like to execute. + * + * @param env A map containing environment variables for the child process. It may already contain + * entries defined by the user (such as SPARK_HOME, or those defined by the + * SparkLauncher constructor that takes an environment), and may be modified to + * include other variables needed by the process to be executed. + */ List buildCommand(Map env) throws IOException; } From b4d6912ccad821e1dcb59a2597a42427a584a860 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 23 Feb 2015 13:23:39 -0800 Subject: [PATCH 36/51] Use spark-submit script in SparkLauncher. To reduce the surface for backwards compatibility, make SparkLauncher launch child processes through the spark-submit script. This way, the library doesn't need to know about the distribution layout of the Spark version being launched, as long as it has a "bin/spark-submit" script. Do some cleanup and fix a couple of issues that the change unearthed. --- .../java/org/apache/spark/launcher/Main.java | 13 +- .../apache/spark/launcher/SparkLauncher.java | 52 +---- .../launcher/SparkSubmitCommandBuilder.java | 73 ++++++- .../spark/launcher/SparkLauncherSuite.java | 160 +-------------- .../SparkSubmitCommandBuilderSuite.java | 188 +++++++++++++++++- 5 files changed, 256 insertions(+), 230 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index e2b23588bdf2..21f2f303d7df 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -88,16 +88,11 @@ public static void main(String[] argsArray) throws Exception { /** * Prepare a command line for execution from a Windows batch script. * - * Two things need to be done: + * The method quotes 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 * - * - 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 + * The command is executed using "cmd /c" and formatted in single line, since that's the * easiest way to consume this from a batch script (see spark-class2.cmd). */ private static String prepareForWindows(List cmd, Map childEnv) { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index e1bb66e02c9c..6ca63c7580a3 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -282,10 +282,12 @@ public SparkLauncher setVerbose(boolean verbose) { * @return A process handle for the Spark app. */ public Process launch() throws IOException { - Map procEnv = new HashMap(childEnv); - List cmd = buildSparkSubmitCommand(procEnv); + List cmd = new ArrayList(); + cmd.add(join(File.separator, getSparkHome(), "bin", "spark-submit")); + cmd.addAll(buildSparkSubmitArgs()); + ProcessBuilder pb = new ProcessBuilder(cmd.toArray(new String[cmd.size()])); - for (Map.Entry e : procEnv.entrySet()) { + for (Map.Entry e : childEnv.entrySet()) { pb.environment().put(e.getKey(), e.getValue()); } return pb.start(); @@ -406,7 +408,7 @@ List buildClassPath(String appClassPath) throws IOException { assemblyJar = new JarFile(assembly); needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; } catch (IOException ioe) { - if (ioe.getMessage().indexOf("invalid CEN header") > 0) { + if (ioe.getMessage().indexOf("invalid CEN header") >= 0) { System.err.println( "Loading Spark jar failed.\n" + "This is likely because Spark was compiled with Java 7 and run\n" + @@ -565,40 +567,6 @@ List buildSparkSubmitArgs() { return args; } - List buildSparkSubmitCommand(Map env) throws IOException { - // Load the properties file and check whether spark-submit will be running the app's driver - // or just launching a cluster app. When running the driver, the JVM's argument will be - // modified to cover the driver's configuration. - Properties props = loadPropertiesFile(); - boolean isClientMode = isClientMode(props); - String extraClassPath = isClientMode ? find(DRIVER_EXTRA_CLASSPATH, conf, props) : null; - - List cmd = buildJavaCommand(extraClassPath); - addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); - addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); - - if (isClientMode) { - // Figuring out where the memory value come from is a little tricky due to precedence. - // Precedence is observed in the following order: - // - explicit configuration (setConf()), which also covers --driver-memory cli argument. - // - properties file. - // - SPARK_DRIVER_MEMORY env variable - // - SPARK_MEM env variable - // - default value (512m) - String memory = firstNonEmpty(find(DRIVER_MEMORY, conf, props), - System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); - cmd.add("-Xms" + memory); - cmd.add("-Xmx" + memory); - addOptionString(cmd, find(DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); - mergeEnvPathList(env, getLibPathEnvName(), find(DRIVER_EXTRA_LIBRARY_PATH, conf, props)); - } - - addPermGenSizeOpt(cmd); - cmd.add("org.apache.spark.deploy.SparkSubmit"); - cmd.addAll(buildSparkSubmitArgs()); - return cmd; - } - String getSparkHome() { String path = getenv(ENV_SPARK_HOME); checkState(path != null, @@ -644,14 +612,6 @@ String getenv(String key) { return firstNonEmpty(childEnv.get(key), System.getenv(key)); } - private boolean isClientMode(Properties userProps) { - String userMaster = firstNonEmpty(master, (String) userProps.get(SPARK_MASTER)); - return userMaster == null || - "client".equals(deployMode) || - "yarn-client".equals(userMaster) || - (deployMode == null && !userMaster.startsWith("yarn-")); - } - private String findAssembly(String scalaVersion) { String sparkHome = getSparkHome(); File libdir; diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 79032a177ee3..098c246deabe 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -21,12 +21,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import static org.apache.spark.launcher.CommandBuilderUtils.*; @@ -89,10 +86,44 @@ public List buildCommand(Map env) throws IOException { if (PYSPARK_SHELL.equals(appResource)) { return buildPySparkShellCommand(env); } else { - return super.buildSparkSubmitCommand(env); + return buildSparkSubmitCommand(env); } } + private List buildSparkSubmitCommand(Map env) throws IOException { + // Load the properties file and check whether spark-submit will be running the app's driver + // or just launching a cluster app. When running the driver, the JVM's argument will be + // modified to cover the driver's configuration. + Properties props = loadPropertiesFile(); + boolean isClientMode = isClientMode(props); + String extraClassPath = isClientMode ? find(DRIVER_EXTRA_CLASSPATH, conf, props) : null; + + List cmd = buildJavaCommand(extraClassPath); + addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); + addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); + + if (isClientMode) { + // Figuring out where the memory value come from is a little tricky due to precedence. + // Precedence is observed in the following order: + // - explicit configuration (setConf()), which also covers --driver-memory cli argument. + // - properties file. + // - SPARK_DRIVER_MEMORY env variable + // - SPARK_MEM env variable + // - default value (512m) + String memory = firstNonEmpty(find(DRIVER_MEMORY, conf, props), + System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); + cmd.add("-Xms" + memory); + cmd.add("-Xmx" + memory); + addOptionString(cmd, find(DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); + mergeEnvPathList(env, getLibPathEnvName(), find(DRIVER_EXTRA_LIBRARY_PATH, conf, props)); + } + + addPermGenSizeOpt(cmd); + cmd.add("org.apache.spark.deploy.SparkSubmit"); + cmd.addAll(buildSparkSubmitArgs()); + return cmd; + } + private List buildPySparkShellCommand(Map env) throws IOException { // For backwards compatibility, if a script is specified in // the pyspark command line, then run it using spark-submit. @@ -139,6 +170,14 @@ private List buildPySparkShellCommand(Map env) throws IO return pyargs; } + private boolean isClientMode(Properties userProps) { + String userMaster = firstNonEmpty(master, (String) userProps.get(SPARK_MASTER)); + return userMaster == null || + "client".equals(deployMode) || + "yarn-client".equals(userMaster) || + (deployMode == null && !userMaster.startsWith("yarn-")); + } + /** * Quotes a string so that it can be used in a command string and be parsed back into a single * argument by python's "shlex.split()" function. @@ -172,19 +211,25 @@ protected boolean handle(String opt, String value) { driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_MEMORY)) { - setConf(DRIVER_MEMORY, value); + setConf(SparkLauncher.DRIVER_MEMORY, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_JAVA_OPTIONS)) { - setConf(DRIVER_EXTRA_JAVA_OPTIONS, value); + setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_LIBRARY_PATH)) { - setConf(DRIVER_EXTRA_LIBRARY_PATH, value); + setConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_CLASS_PATH)) { - setConf(DRIVER_EXTRA_CLASSPATH, value); + setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value); + driverArgs.add(opt); + driverArgs.add(value); + } else if (opt.equals(CONF)) { + String[] conf = value.split("=", 2); + checkArgument(conf.length == 2, "Invalid argument to %s: %s", CONF, value); + handleConf(conf[0], conf[1]); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(CLASS)) { @@ -227,6 +272,18 @@ protected void handleExtraArgs(List extra) { } } + private void handleConf(String key, String value) { + List driverJvmKeys = Arrays.asList( + SparkLauncher.DRIVER_EXTRA_CLASSPATH, + SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, + SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, + SparkLauncher.DRIVER_MEMORY); + + if (driverJvmKeys.contains(key)) { + setConf(key, value); + } + } + } } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index ead509504808..252d5abae1ca 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -18,16 +18,11 @@ package org.apache.spark.launcher; import java.io.BufferedReader; -import java.io.File; import java.io.InputStream; import java.io.InputStreamReader; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.regex.Pattern; -import org.junit.AfterClass; -import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,31 +35,12 @@ public class SparkLauncherSuite { private static final Logger LOG = LoggerFactory.getLogger(SparkLauncherSuite.class); - private static File dummyPropsFile; - - @BeforeClass - public static void setUp() throws Exception { - dummyPropsFile = File.createTempFile("spark", "properties"); - } - - @AfterClass - public static void cleanUp() throws Exception { - dummyPropsFile.delete(); - } - - @Test - public void testDriverCmdBuilder() throws Exception { - testCmdBuilder(true); - } - - @Test - public void testClusterCmdBuilder() throws Exception { - testCmdBuilder(false); - } - @Test public void testChildProcLauncher() throws Exception { - SparkLauncher launcher = new SparkLauncher() + Map env = new HashMap(); + env.put("SPARK_PRINT_LAUNCH_COMMAND", "1"); + + SparkLauncher launcher = new SparkLauncher(env) .setSparkHome(System.getProperty("spark.test.home")) .setMaster("local") .setAppResource("spark-internal") @@ -79,134 +55,6 @@ public void testChildProcLauncher() throws Exception { assertEquals(0, app.waitFor()); } - private void testCmdBuilder(boolean isDriver) throws Exception { - String deployMode = isDriver ? "client" : "cluster"; - - SparkLauncher launcher = new SparkLauncher() - .setSparkHome(System.getProperty("spark.test.home")) - .setMaster("yarn") - .setDeployMode(deployMode) - .setAppResource("/foo") - .setAppName("MyApp") - .setMainClass("my.Class") - .setPropertiesFile(dummyPropsFile.getAbsolutePath()) - .addAppArgs("foo", "bar") - .setConf(SparkLauncher.DRIVER_MEMORY, "1g") - .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver") - .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver -XX:MaxPermSize=256m") - .setConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native") - .setConf("spark.foo", "foo"); - - Map env = new HashMap(); - List cmd = launcher.buildSparkSubmitCommand(env); - - // Checks below are different for driver and non-driver mode. - - if (isDriver) { - assertTrue("Driver -Xms should be configured.", contains("-Xms1g", cmd)); - assertTrue("Driver -Xmx should be configured.", contains("-Xmx1g", cmd)); - } else { - boolean found = false; - for (String arg : cmd) { - if (arg.startsWith("-Xms") || arg.startsWith("-Xmx")) { - found = true; - break; - } - } - assertFalse("Memory arguments should not be set.", found); - } - - for (String arg : cmd) { - if (arg.startsWith("-XX:MaxPermSize=")) { - if (isDriver) { - assertEquals("-XX:MaxPermSize=256m", arg); - } else { - assertEquals("-XX:MaxPermSize=128m", arg); - } - } - } - - String[] cp = findArgValue(cmd, "-cp").split(Pattern.quote(File.pathSeparator)); - if (isDriver) { - assertTrue("Driver classpath should contain provided entry.", contains("/driver", cp)); - } else { - assertFalse("Driver classpath should not be in command.", contains("/driver", cp)); - } - - String libPath = env.get(CommandBuilderUtils.getLibPathEnvName()); - if (isDriver) { - assertNotNull("Native library path should be set.", libPath); - assertTrue("Native library path should contain provided entry.", - contains("/native", libPath.split(Pattern.quote(File.pathSeparator)))); - } else { - assertNull("Native library should not be set.", libPath); - } - - // Checks below are the same for both driver and non-driver mode. - SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); - - assertEquals(dummyPropsFile.getAbsolutePath(), findArgValue(cmd, parser.PROPERTIES_FILE)); - assertEquals("yarn", findArgValue(cmd, parser.MASTER)); - assertEquals(deployMode, findArgValue(cmd, parser.DEPLOY_MODE)); - assertEquals("my.Class", findArgValue(cmd, parser.CLASS)); - assertEquals("MyApp", findArgValue(cmd, parser.NAME)); - - boolean appArgsOk = false; - for (int i = 0; i < cmd.size(); i++) { - if (cmd.get(i).equals("/foo")) { - assertEquals("foo", cmd.get(i + 1)); - assertEquals("bar", cmd.get(i + 2)); - assertEquals(cmd.size(), i + 3); - appArgsOk = true; - break; - } - } - assertTrue("App resource and args should be added to command.", appArgsOk); - - Map conf = parseConf(cmd, parser); - assertEquals("foo", conf.get("spark.foo")); - } - - private String findArgValue(List cmd, String name) { - for (int i = 0; i < cmd.size(); i++) { - if (cmd.get(i).equals(name)) { - return cmd.get(i + 1); - } - } - fail(String.format("arg '%s' not found", name)); - return null; - } - - private boolean contains(String needle, String[] haystack) { - for (String entry : haystack) { - if (entry.equals(needle)) { - return true; - } - } - return false; - } - - private boolean contains(String needle, Iterable haystack) { - for (String entry : haystack) { - if (entry.equals(needle)) { - return true; - } - } - return false; - } - - private Map parseConf(List cmd, SparkSubmitOptionParser parser) { - Map conf = new HashMap(); - for (int i = 0; i < cmd.size(); i++) { - if (cmd.get(i).equals(parser.CONF)) { - String[] val = cmd.get(i + 1).split("=", 2); - conf.put(val[0], val[1]); - i += 1; - } - } - return conf; - } - public static class SparkLauncherTestApp { public static void main(String[] args) throws Exception { diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 6c1c12632466..bd713dde4e19 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -17,29 +17,81 @@ package org.apache.spark.launcher; +import java.io.File; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.regex.Pattern; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import static org.junit.Assert.*; public class SparkSubmitCommandBuilderSuite { + private static File dummyPropsFile; + private static SparkSubmitOptionParser parser; + + @BeforeClass + public static void setUp() throws Exception { + dummyPropsFile = File.createTempFile("spark", "properties"); + parser = new SparkSubmitOptionParser(); + } + + @AfterClass + public static void cleanUp() throws Exception { + dummyPropsFile.delete(); + } + + @Test + public void testDriverCmdBuilder() throws Exception { + testCmdBuilder(true); + } + + @Test + public void testClusterCmdBuilder() throws Exception { + testCmdBuilder(false); + } + + @Test + public void testCliParser() throws Exception { + List sparkSubmitArgs = Arrays.asList( + parser.MASTER, + "local", + parser.DRIVER_MEMORY, + "42g", + parser.DRIVER_CLASS_PATH, + "/driverCp", + parser.DRIVER_JAVA_OPTIONS, + "extraJavaOpt", + parser.CONF, + SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath"); + Map env = new HashMap(); + List cmd = buildCommand(sparkSubmitArgs, env); + + assertTrue(findInStringList(env.get(CommandBuilderUtils.getLibPathEnvName()), + File.pathSeparator, "/driverLibPath")); + assertTrue(findInStringList(findArgValue(cmd, "-cp"), File.pathSeparator, "/driverCp")); + assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms42g")); + assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx42g")); + } + @Test public void testShellCliParser() throws Exception { List sparkSubmitArgs = Arrays.asList( - "--class", + parser.CLASS, "org.apache.spark.repl.Main", - "--master", + parser.MASTER, "foo", "--app-arg", "bar", "--app-switch", - "--files", + parser.FILES, "baz", - "--name", + parser.NAME, "appName"); List args = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); @@ -50,14 +102,14 @@ public void testShellCliParser() throws Exception { @Test public void testAlternateSyntaxParsing() throws Exception { List sparkSubmitArgs = Arrays.asList( - "--class=org.my.Class", - "--master=foo", - "--deploy-mode=bar"); + parser.CLASS + "=org.my.Class", + parser.MASTER + "=foo", + parser.DEPLOY_MODE + "=bar"); List cmd = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); - assertEquals("org.my.Class", findArgValue(cmd, "--class")); - assertEquals("foo", findArgValue(cmd, "--master")); - assertEquals("bar", findArgValue(cmd, "--deploy-mode")); + assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS)); + assertEquals("foo", findArgValue(cmd, parser.MASTER)); + assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE)); } @Test @@ -70,7 +122,9 @@ public void testPySparkLauncher() throws Exception { Map env = new HashMap(); List cmd = buildCommand(sparkSubmitArgs, env); assertEquals("python", cmd.get(cmd.size() - 1)); - assertEquals("\"--master\" \"foo\" \"--deploy-mode\" \"bar\"", env.get("PYSPARK_SUBMIT_ARGS")); + assertEquals( + String.format("\"%s\" \"foo\" \"%s\" \"bar\"", parser.MASTER, parser.DEPLOY_MODE), + env.get("PYSPARK_SUBMIT_ARGS")); } @Test @@ -90,6 +144,114 @@ public void testPySparkFallback() throws Exception { assertEquals("arg1", cmd.get(cmd.size() - 1)); } + private void testCmdBuilder(boolean isDriver) throws Exception { + String deployMode = isDriver ? "client" : "cluster"; + + SparkSubmitCommandBuilder launcher = + new SparkSubmitCommandBuilder(Collections.emptyList()); + launcher.setSparkHome(System.getProperty("spark.test.home")) + .setMaster("yarn") + .setDeployMode(deployMode) + .setAppResource("/foo") + .setAppName("MyApp") + .setMainClass("my.Class") + .setPropertiesFile(dummyPropsFile.getAbsolutePath()) + .addAppArgs("foo", "bar") + .setConf(SparkLauncher.DRIVER_MEMORY, "1g") + .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver") + .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver -XX:MaxPermSize=256m") + .setConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native") + .setConf("spark.foo", "foo"); + + Map env = new HashMap(); + List cmd = launcher.buildCommand(env); + + // Checks below are different for driver and non-driver mode. + + if (isDriver) { + assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms1g")); + assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx1g")); + } else { + boolean found = false; + for (String arg : cmd) { + if (arg.startsWith("-Xms") || arg.startsWith("-Xmx")) { + found = true; + break; + } + } + assertFalse("Memory arguments should not be set.", found); + } + + for (String arg : cmd) { + if (arg.startsWith("-XX:MaxPermSize=")) { + if (isDriver) { + assertEquals("-XX:MaxPermSize=256m", arg); + } else { + assertEquals("-XX:MaxPermSize=128m", arg); + } + } + } + + String[] cp = findArgValue(cmd, "-cp").split(Pattern.quote(File.pathSeparator)); + if (isDriver) { + assertTrue("Driver classpath should contain provided entry.", contains("/driver", cp)); + } else { + assertFalse("Driver classpath should not be in command.", contains("/driver", cp)); + } + + String libPath = env.get(CommandBuilderUtils.getLibPathEnvName()); + if (isDriver) { + assertNotNull("Native library path should be set.", libPath); + assertTrue("Native library path should contain provided entry.", + contains("/native", libPath.split(Pattern.quote(File.pathSeparator)))); + } else { + assertNull("Native library should not be set.", libPath); + } + + // Checks below are the same for both driver and non-driver mode. + assertEquals(dummyPropsFile.getAbsolutePath(), findArgValue(cmd, parser.PROPERTIES_FILE)); + assertEquals("yarn", findArgValue(cmd, parser.MASTER)); + assertEquals(deployMode, findArgValue(cmd, parser.DEPLOY_MODE)); + assertEquals("my.Class", findArgValue(cmd, parser.CLASS)); + assertEquals("MyApp", findArgValue(cmd, parser.NAME)); + + boolean appArgsOk = false; + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals("/foo")) { + assertEquals("foo", cmd.get(i + 1)); + assertEquals("bar", cmd.get(i + 2)); + assertEquals(cmd.size(), i + 3); + appArgsOk = true; + break; + } + } + assertTrue("App resource and args should be added to command.", appArgsOk); + + Map conf = parseConf(cmd, parser); + assertEquals("foo", conf.get("spark.foo")); + } + + private boolean contains(String needle, String[] haystack) { + for (String entry : haystack) { + if (entry.equals(needle)) { + return true; + } + } + return false; + } + + private Map parseConf(List cmd, SparkSubmitOptionParser parser) { + Map conf = new HashMap(); + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals(parser.CONF)) { + String[] val = cmd.get(i + 1).split("=", 2); + conf.put(val[0], val[1]); + i += 1; + } + } + return conf; + } + private String findArgValue(List cmd, String name) { for (int i = 0; i < cmd.size(); i++) { if (cmd.get(i).equals(name)) { @@ -100,6 +262,10 @@ private String findArgValue(List cmd, String name) { return null; } + private boolean findInStringList(String list, String sep, String needle) { + return contains(needle, list.split(sep)); + } + private List buildCommand(List args, Map env) throws Exception { SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); builder.setSparkHome(System.getProperty("spark.test.home")); From 43008a750a10ca2ddb15dc24d287b489932f727b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 23 Feb 2015 14:01:17 -0800 Subject: [PATCH 37/51] Don't make builder extend SparkLauncher. Instead, make SparkLauncher a thin layer on top of SparkSubmitCommandBuilder, and make all the builder code internal to the library. --- .../spark/deploy/worker/CommandUtils.scala | 1 - .../spark/launcher/WorkerCommandBuilder.scala | 11 +- .../launcher/AbstractCommandBuilder.java | 367 +++++++++++++++ .../apache/spark/launcher/CommandBuilder.java | 39 -- .../spark/launcher/CommandBuilderUtils.java | 4 + .../java/org/apache/spark/launcher/Main.java | 4 +- .../launcher/SparkClassCommandBuilder.java | 6 +- .../apache/spark/launcher/SparkLauncher.java | 430 ++---------------- .../launcher/SparkSubmitCommandBuilder.java | 136 ++++-- .../SparkSubmitCommandBuilderSuite.java | 30 +- 10 files changed, 528 insertions(+), 500 deletions(-) create mode 100644 launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java delete mode 100644 launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index ccf4289ce1e6..83f78cf47306 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy.worker import java.io.{File, FileOutputStream, InputStream, IOException} import java.lang.System._ -import java.util.{List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.collection.Map diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala index 657ce3bb512f..4fe73cfc4fce 100644 --- a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala +++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala @@ -18,7 +18,7 @@ package org.apache.spark.launcher import java.io.File -import java.util.{List => JList} +import java.util.{HashMap => JHashMap, List => JList, Map => JMap} import scala.collection.JavaConversions._ @@ -29,11 +29,12 @@ import org.apache.spark.deploy.Command * needs to live in the same package as the rest of the library. */ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, command: Command) - extends SparkLauncher(command.environment) { + extends AbstractCommandBuilder { - setSparkHome(sparkHome) + childEnv.putAll(command.environment) + childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sparkHome) - def buildCommand(): JList[String] = { + override def buildCommand(env: JMap[String, String]): JList[String] = { val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator)) cmd.add(s"-Xms${memoryMb}M") cmd.add(s"-Xmx${memoryMb}M") @@ -43,4 +44,6 @@ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, comm cmd } + def buildCommand(): JList[String] = buildCommand(new JHashMap[String, String]()) + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java new file mode 100644 index 000000000000..2c3719690c8a --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -0,0 +1,367 @@ +/* + * 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.BufferedReader; +import java.io.File; +import java.io.FileFilter; +import java.io.FileInputStream; +import java.io.InputStreamReader; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.jar.JarFile; +import java.util.regex.Pattern; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; + +/** + * Abstract command builder that defines common functionality for all builders. + */ +abstract class AbstractCommandBuilder { + + boolean verbose; + String appName; + String appResource; + String deployMode; + String javaHome; + String mainClass; + String master; + String propertiesFile; + final List sparkArgs; + final List appArgs; + final List jars; + final List files; + final List pyFiles; + final Map childEnv; + final Map conf; + + public AbstractCommandBuilder() { + this(Collections.emptyMap()); + } + + public AbstractCommandBuilder(Map env) { + this.appArgs = new ArrayList(); + this.childEnv = new HashMap(env); + this.conf = new HashMap(); + this.files = new ArrayList(); + this.jars = new ArrayList(); + this.pyFiles = new ArrayList(); + this.sparkArgs = new ArrayList(); + } + + /** + * Builds the command like to execute. + * + * @param env A map containing environment variables for the child process. It may already contain + * entries defined by the user (such as SPARK_HOME, or those defined by the + * SparkLauncher constructor that takes an environment), and may be modified to + * include other variables needed by the process to be executed. + */ + abstract List buildCommand(Map env) throws IOException; + + List buildJavaCommand(String extraClassPath) throws IOException { + List cmd = new ArrayList(); + if (javaHome == null) { + cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); + } else { + cmd.add(join(File.separator, javaHome, "bin", "java")); + } + + // Load extra JAVA_OPTS from conf/java-opts, if it exists. + File javaOpts = new File(join(File.separator, getConfDir(), "java-opts")); + if (javaOpts.isFile()) { + BufferedReader br = new BufferedReader(new InputStreamReader( + new FileInputStream(javaOpts), "UTF-8")); + try { + String line; + while ((line = br.readLine()) != null) { + addOptionString(cmd, line); + } + } finally { + br.close(); + } + } + + cmd.add("-cp"); + cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); + return cmd; + } + + /** + * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't + * set it. + */ + void addPermGenSizeOpt(List cmd) { + // Don't set MaxPermSize for Java 8 and later. + String[] version = System.getProperty("java.version").split("\\."); + if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) { + return; + } + + for (String arg : cmd) { + if (arg.startsWith("-XX:MaxPermSize=")) { + return; + } + } + + cmd.add("-XX:MaxPermSize=128m"); + } + + void addOptionString(List cmd, String options) { + if (!isEmpty(options)) { + for (String opt : parseOptionString(options)) { + cmd.add(opt); + } + } + } + + /** + * Builds the classpath for the application. Returns a list with one classpath entry per element; + * each entry is formatted in the way expected by java.net.URLClassLoader (more + * specifically, with trailing slashes for directories). + */ + List buildClassPath(String appClassPath) throws IOException { + String sparkHome = getSparkHome(); + String scala = getScalaVersion(); + + List cp = new ArrayList(); + addToClassPath(cp, getenv("SPARK_CLASSPATH")); + addToClassPath(cp, appClassPath); + + addToClassPath(cp, getConfDir()); + + boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); + boolean isTesting = "1".equals(getenv("SPARK_TESTING")); + if (prependClasses || isTesting) { + List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", + "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", + "yarn", "launcher"); + if (prependClasses) { + System.err.println( + "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + + "assembly."); + for (String project : projects) { + addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project, + scala)); + } + } + if (isTesting) { + for (String project : projects) { + addToClassPath(cp, String.format("%s/%s/target/scala-%s/test-classes", sparkHome, + project, scala)); + } + } + + // Add this path to include jars that are shaded in the final deliverable created during + // the maven build. These jars are copied to this directory during the build. + addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); + } + + String assembly = findAssembly(scala); + addToClassPath(cp, assembly); + + // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus + // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt + // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built + // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark + // assembly is built for Hive, before actually populating the CLASSPATH with the jars. + // + // This block also serves as a check for SPARK-1703, when the assembly jar is built with + // Java 7 and ends up with too many files, causing issues with other JDK versions. + boolean needsDataNucleus = false; + JarFile assemblyJar = null; + try { + assemblyJar = new JarFile(assembly); + needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; + } catch (IOException ioe) { + if (ioe.getMessage().indexOf("invalid CEN header") >= 0) { + System.err.println( + "Loading Spark jar failed.\n" + + "This is likely because Spark was compiled with Java 7 and run\n" + + "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + + "or build Spark with Java 6."); + System.exit(1); + } else { + throw ioe; + } + } finally { + if (assemblyJar != null) { + try { + assemblyJar.close(); + } catch (IOException e) { + // Ignore. + } + } + } + + if (needsDataNucleus) { + System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " + + "in classpath."); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + } else { + libdir = new File(sparkHome, "lib_managed/jars"); + } + + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + for (File jar : libdir.listFiles()) { + if (jar.getName().startsWith("datanucleus-")) { + addToClassPath(cp, jar.getAbsolutePath()); + } + } + } + + addToClassPath(cp, getenv("HADOOP_CONF_DIR")); + addToClassPath(cp, getenv("YARN_CONF_DIR")); + addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH")); + return cp; + } + + /** + * Adds entries to the classpath. + * + * @param cp List where to appended the new classpath entries. + * @param entries New classpath entries (separated by File.pathSeparator). + */ + private void addToClassPath(List cp, String entries) { + if (isEmpty(entries)) { + return; + } + String[] split = entries.split(Pattern.quote(File.pathSeparator)); + for (String entry : split) { + if (!isEmpty(entry)) { + if (new File(entry).isDirectory() && !entry.endsWith(File.separator)) { + entry += File.separator; + } + cp.add(entry); + } + } + } + + String getScalaVersion() { + String scala = getenv("SPARK_SCALA_VERSION"); + if (scala != null) { + return scala; + } + + String sparkHome = getSparkHome(); + File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); + File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); + if (scala210.isDirectory() && scala211.isDirectory()) { + checkState(false, + "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); + } else if (scala210.isDirectory()) { + return "2.10"; + } else { + checkState(scala211.isDirectory(), "Cannot find any assembly build directories."); + return "2.11"; + } + + throw new IllegalStateException("Should not reach here."); + } + + void addSparkArgs(String... args) { + for (String arg : args) { + sparkArgs.add(arg); + } + } + + String getSparkHome() { + String path = getenv(ENV_SPARK_HOME); + checkState(path != null, + "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); + return path; + } + + /** + * Loads the configuration file for the application, if it exists. This is either the + * user-specified properties file, or the spark-defaults.conf file under the Spark configuration + * directory. + */ + Properties loadPropertiesFile() throws IOException { + Properties props = new Properties(); + File propsFile; + if (propertiesFile != null) { + propsFile = new File(propertiesFile); + checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile); + } else { + propsFile = new File(getConfDir(), DEFAULT_PROPERTIES_FILE); + } + + if (propsFile.isFile()) { + FileInputStream fd = null; + try { + fd = new FileInputStream(propsFile); + props.load(new InputStreamReader(fd, "UTF-8")); + } finally { + if (fd != null) { + try { + fd.close(); + } catch (IOException e) { + // Ignore. + } + } + } + } + + return props; + } + + String getenv(String key) { + return firstNonEmpty(childEnv.get(key), System.getenv(key)); + } + + private String findAssembly(String scalaVersion) { + String sparkHome = getSparkHome(); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + } else { + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion)); + } + + final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); + FileFilter filter = new FileFilter() { + @Override + public boolean accept(File file) { + return file.isFile() && re.matcher(file.getName()).matches(); + } + }; + File[] assemblies = libdir.listFiles(filter); + checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); + checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); + return assemblies[0].getAbsolutePath(); + } + + private String getConfDir() { + String confDir = getenv("SPARK_CONF_DIR"); + return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java deleted file mode 100644 index 68838f21692d..000000000000 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilder.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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 { - - /** - * Builds the command like to execute. - * - * @param env A map containing environment variables for the child process. It may already contain - * entries defined by the user (such as SPARK_HOME, or those defined by the - * SparkLauncher constructor that takes an environment), and may be modified to - * include other variables needed by the process to be executed. - */ - List buildCommand(Map env) throws IOException; - -} diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index fb1bba179237..e8e7faf730f6 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -27,6 +27,10 @@ */ class CommandBuilderUtils { + static final String DEFAULT_MEM = "512m"; + static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; + static final String ENV_SPARK_HOME = "SPARK_HOME"; + /** Returns whether the given string is null or empty. */ static boolean isEmpty(String s) { return s == null || s.isEmpty(); diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index 21f2f303d7df..bb84f70c3d44 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -54,7 +54,7 @@ public static void main(String[] argsArray) throws Exception { String className = args.remove(0); boolean printLaunchCommand; - CommandBuilder builder; + AbstractCommandBuilder builder; try { if (className.equals("org.apache.spark.deploy.SparkSubmit")) { builder = new SparkSubmitCommandBuilder(args); @@ -169,7 +169,7 @@ private static String quoteForBatchScript(String arg) { * 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 implements CommandBuilder { + private static class UsageLauncher extends AbstractCommandBuilder { @Override public List buildCommand(Map env) { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java index 95c5df17508b..20871d5f1ca5 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -32,7 +32,7 @@ * This class handles building the command to launch all internal Spark classes except for * SparkSubmit (which is handled by {@link SparkSubmitCommandBuilder} class. */ -class SparkClassCommandBuilder extends SparkLauncher implements CommandBuilder { +class SparkClassCommandBuilder extends AbstractCommandBuilder { private final String className; private final List classArgs; @@ -145,9 +145,9 @@ protected void handleExtraArgs(List extra) { sparkSubmitArgs.add(className); SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(true, sparkSubmitArgs); - builder.setAppResource("spark-internal"); + builder.appResource = "spark-internal"; for (String arg: appArgs) { - builder.addAppArgs(arg); + builder.appArgs.add(arg); } return builder.buildCommand(env); } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 6ca63c7580a3..92a29e979c8d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -17,21 +17,11 @@ package org.apache.spark.launcher; -import java.io.BufferedReader; import java.io.File; -import java.io.FileFilter; -import java.io.FileInputStream; -import java.io.InputStreamReader; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Properties; -import java.util.jar.JarFile; -import java.util.regex.Pattern; import static org.apache.spark.launcher.CommandBuilderUtils.*; @@ -69,38 +59,22 @@ public class SparkLauncher { /** Configuration key for the number of executor CPU cores. */ public static final String EXECUTOR_CORES = "spark.executor.cores"; - private static final String ENV_SPARK_HOME = "SPARK_HOME"; - private static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; - static final String DEFAULT_MEM = "512m"; - - boolean verbose; - String appName; - String appResource; - String deployMode; - String javaHome; - String mainClass; - String master; - String propertiesFile; - final List sparkArgs; - final List appArgs; - final List jars; - final List files; - final List pyFiles; - final Map childEnv; - final Map conf; + private final SparkSubmitCommandBuilder builder; public SparkLauncher() { - this(Collections.emptyMap()); + this(null); } + /** + * Creates a launcher that will set the given environment variables in the child. + * + * @param env Environment variables to set. + */ public SparkLauncher(Map env) { - this.appArgs = new ArrayList(); - this.childEnv = new HashMap(env); - this.conf = new HashMap(); - this.files = new ArrayList(); - this.jars = new ArrayList(); - this.pyFiles = new ArrayList(); - this.sparkArgs = new ArrayList(); + this.builder = new SparkSubmitCommandBuilder(); + if (env != null) { + this.builder.childEnv.putAll(env); + } } /** @@ -111,7 +85,7 @@ public SparkLauncher(Map env) { */ public SparkLauncher setJavaHome(String javaHome) { checkNotNull(javaHome, "javaHome"); - this.javaHome = javaHome; + builder.javaHome = javaHome; return this; } @@ -123,7 +97,7 @@ public SparkLauncher setJavaHome(String javaHome) { */ public SparkLauncher setSparkHome(String sparkHome) { checkNotNull(sparkHome, "sparkHome"); - childEnv.put(ENV_SPARK_HOME, sparkHome); + builder.childEnv.put(ENV_SPARK_HOME, sparkHome); return this; } @@ -135,7 +109,7 @@ public SparkLauncher setSparkHome(String sparkHome) { */ public SparkLauncher setPropertiesFile(String path) { checkNotNull(path, "path"); - this.propertiesFile = path; + builder.propertiesFile = path; return this; } @@ -150,7 +124,7 @@ public SparkLauncher setConf(String key, String value) { checkNotNull(key, "key"); checkNotNull(value, "value"); checkArgument(key.startsWith("spark."), "'key' must start with 'spark.'"); - conf.put(key, value); + builder.conf.put(key, value); return this; } @@ -162,7 +136,7 @@ public SparkLauncher setConf(String key, String value) { */ public SparkLauncher setAppName(String appName) { checkNotNull(appName, "appName"); - this.appName = appName; + builder.appName = appName; return this; } @@ -174,7 +148,7 @@ public SparkLauncher setAppName(String appName) { */ public SparkLauncher setMaster(String master) { checkNotNull(master, "master"); - this.master = master; + builder.master = master; return this; } @@ -186,7 +160,7 @@ public SparkLauncher setMaster(String master) { */ public SparkLauncher setDeployMode(String mode) { checkNotNull(mode, "mode"); - this.deployMode = mode; + builder.deployMode = mode; return this; } @@ -199,7 +173,7 @@ public SparkLauncher setDeployMode(String mode) { */ public SparkLauncher setAppResource(String resource) { checkNotNull(resource, "resource"); - this.appResource = resource; + builder.appResource = resource; return this; } @@ -211,7 +185,7 @@ public SparkLauncher setAppResource(String resource) { */ public SparkLauncher setMainClass(String mainClass) { checkNotNull(mainClass, "mainClass"); - this.mainClass = mainClass; + builder.mainClass = mainClass; return this; } @@ -224,7 +198,7 @@ public SparkLauncher setMainClass(String mainClass) { public SparkLauncher addAppArgs(String... args) { for (String arg : args) { checkNotNull(arg, "arg"); - appArgs.add(arg); + builder.appArgs.add(arg); } return this; } @@ -237,7 +211,7 @@ public SparkLauncher addAppArgs(String... args) { */ public SparkLauncher addJar(String jar) { checkNotNull(jar, "jar"); - jars.add(jar); + builder.jars.add(jar); return this; } @@ -249,7 +223,7 @@ public SparkLauncher addJar(String jar) { */ public SparkLauncher addFile(String file) { checkNotNull(file, "file"); - files.add(file); + builder.files.add(file); return this; } @@ -261,7 +235,7 @@ public SparkLauncher addFile(String file) { */ public SparkLauncher addPyFile(String file) { checkNotNull(file, "file"); - pyFiles.add(file); + builder.pyFiles.add(file); return this; } @@ -272,7 +246,7 @@ public SparkLauncher addPyFile(String file) { * @return This launcher. */ public SparkLauncher setVerbose(boolean verbose) { - this.verbose = verbose; + builder.verbose = verbose; return this; } @@ -283,362 +257,14 @@ public SparkLauncher setVerbose(boolean verbose) { */ public Process launch() throws IOException { List cmd = new ArrayList(); - cmd.add(join(File.separator, getSparkHome(), "bin", "spark-submit")); - cmd.addAll(buildSparkSubmitArgs()); + cmd.add(join(File.separator, builder.getSparkHome(), "bin", "spark-submit")); + cmd.addAll(builder.buildSparkSubmitArgs()); ProcessBuilder pb = new ProcessBuilder(cmd.toArray(new String[cmd.size()])); - for (Map.Entry e : childEnv.entrySet()) { + for (Map.Entry e : builder.childEnv.entrySet()) { pb.environment().put(e.getKey(), e.getValue()); } return pb.start(); } - List buildJavaCommand(String extraClassPath) throws IOException { - List cmd = new ArrayList(); - if (javaHome == null) { - cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); - } else { - cmd.add(join(File.separator, javaHome, "bin", "java")); - } - - // Load extra JAVA_OPTS from conf/java-opts, if it exists. - File javaOpts = new File(join(File.separator, getConfDir(), "java-opts")); - if (javaOpts.isFile()) { - BufferedReader br = new BufferedReader(new InputStreamReader( - new FileInputStream(javaOpts), "UTF-8")); - try { - String line; - while ((line = br.readLine()) != null) { - addOptionString(cmd, line); - } - } finally { - br.close(); - } - } - - cmd.add("-cp"); - cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); - return cmd; - } - - /** - * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't - * set it. - */ - void addPermGenSizeOpt(List cmd) { - // Don't set MaxPermSize for Java 8 and later. - String[] version = System.getProperty("java.version").split("\\."); - if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) { - return; - } - - for (String arg : cmd) { - if (arg.startsWith("-XX:MaxPermSize=")) { - return; - } - } - - cmd.add("-XX:MaxPermSize=128m"); - } - - void addOptionString(List cmd, String options) { - if (!isEmpty(options)) { - for (String opt : parseOptionString(options)) { - cmd.add(opt); - } - } - } - - /** - * Builds the classpath for the application. Returns a list with one classpath entry per element; - * each entry is formatted in the way expected by java.net.URLClassLoader (more - * specifically, with trailing slashes for directories). - */ - List buildClassPath(String appClassPath) throws IOException { - String sparkHome = getSparkHome(); - String scala = getScalaVersion(); - - List cp = new ArrayList(); - addToClassPath(cp, getenv("SPARK_CLASSPATH")); - addToClassPath(cp, appClassPath); - - addToClassPath(cp, getConfDir()); - - boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); - boolean isTesting = "1".equals(getenv("SPARK_TESTING")); - if (prependClasses || isTesting) { - List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", - "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", - "yarn", "launcher"); - if (prependClasses) { - System.err.println( - "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + - "assembly."); - for (String project : projects) { - addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project, - scala)); - } - } - if (isTesting) { - for (String project : projects) { - addToClassPath(cp, String.format("%s/%s/target/scala-%s/test-classes", sparkHome, - project, scala)); - } - } - - // Add this path to include jars that are shaded in the final deliverable created during - // the maven build. These jars are copied to this directory during the build. - addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); - } - - String assembly = findAssembly(scala); - addToClassPath(cp, assembly); - - // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus - // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt - // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built - // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark - // assembly is built for Hive, before actually populating the CLASSPATH with the jars. - // - // This block also serves as a check for SPARK-1703, when the assembly jar is built with - // Java 7 and ends up with too many files, causing issues with other JDK versions. - boolean needsDataNucleus = false; - JarFile assemblyJar = null; - try { - assemblyJar = new JarFile(assembly); - needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; - } catch (IOException ioe) { - if (ioe.getMessage().indexOf("invalid CEN header") >= 0) { - System.err.println( - "Loading Spark jar failed.\n" + - "This is likely because Spark was compiled with Java 7 and run\n" + - "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + - "or build Spark with Java 6."); - System.exit(1); - } else { - throw ioe; - } - } finally { - if (assemblyJar != null) { - try { - assemblyJar.close(); - } catch (IOException e) { - // Ignore. - } - } - } - - if (needsDataNucleus) { - System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " + - "in classpath."); - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - } else { - libdir = new File(sparkHome, "lib_managed/jars"); - } - - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - for (File jar : libdir.listFiles()) { - if (jar.getName().startsWith("datanucleus-")) { - addToClassPath(cp, jar.getAbsolutePath()); - } - } - } - - addToClassPath(cp, getenv("HADOOP_CONF_DIR")); - addToClassPath(cp, getenv("YARN_CONF_DIR")); - addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH")); - return cp; - } - - /** - * Adds entries to the classpath. - * - * @param cp List where to appended the new classpath entries. - * @param entries New classpath entries (separated by File.pathSeparator). - */ - private void addToClassPath(List cp, String entries) { - if (isEmpty(entries)) { - return; - } - String[] split = entries.split(Pattern.quote(File.pathSeparator)); - for (String entry : split) { - if (!isEmpty(entry)) { - if (new File(entry).isDirectory() && !entry.endsWith(File.separator)) { - entry += File.separator; - } - cp.add(entry); - } - } - } - - String getScalaVersion() { - String scala = getenv("SPARK_SCALA_VERSION"); - if (scala != null) { - return scala; - } - - String sparkHome = getSparkHome(); - File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); - File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); - if (scala210.isDirectory() && scala211.isDirectory()) { - checkState(false, - "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + - "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); - } else if (scala210.isDirectory()) { - return "2.10"; - } else { - checkState(scala211.isDirectory(), "Cannot find any assembly build directories."); - return "2.11"; - } - - throw new IllegalStateException("Should not reach here."); - } - - SparkLauncher addSparkArgs(String... args) { - for (String arg : args) { - sparkArgs.add(arg); - } - return this; - } - - // Visible for testing. - List buildSparkSubmitArgs() { - List args = new ArrayList(); - SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); - - if (verbose) { - args.add(parser.VERBOSE); - } - - if (master != null) { - args.add(parser.MASTER); - args.add(master); - } - - if (deployMode != null) { - args.add(parser.DEPLOY_MODE); - args.add(deployMode); - } - - if (appName != null) { - args.add(parser.NAME); - args.add(appName); - } - - for (Map.Entry e : conf.entrySet()) { - args.add(parser.CONF); - args.add(String.format("%s=%s", e.getKey(), e.getValue())); - } - - if (propertiesFile != null) { - args.add(parser.PROPERTIES_FILE); - args.add(propertiesFile); - } - - if (!jars.isEmpty()) { - args.add(parser.JARS); - args.add(join(",", jars)); - } - - if (!files.isEmpty()) { - args.add(parser.FILES); - args.add(join(",", files)); - } - - if (!pyFiles.isEmpty()) { - args.add(parser.PY_FILES); - args.add(join(",", pyFiles)); - } - - if (mainClass != null) { - args.add(parser.CLASS); - args.add(mainClass); - } - - args.addAll(sparkArgs); - if (appResource != null) { - args.add(appResource); - } - args.addAll(appArgs); - - return args; - } - - String getSparkHome() { - String path = getenv(ENV_SPARK_HOME); - checkState(path != null, - "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); - return path; - } - - /** - * Loads the configuration file for the application, if it exists. This is either the - * user-specified properties file, or the spark-defaults.conf file under the Spark configuration - * directory. - */ - Properties loadPropertiesFile() throws IOException { - Properties props = new Properties(); - File propsFile; - if (propertiesFile != null) { - propsFile = new File(propertiesFile); - checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile); - } else { - propsFile = new File(getConfDir(), DEFAULT_PROPERTIES_FILE); - } - - if (propsFile.isFile()) { - FileInputStream fd = null; - try { - fd = new FileInputStream(propsFile); - props.load(new InputStreamReader(fd, "UTF-8")); - } finally { - if (fd != null) { - try { - fd.close(); - } catch (IOException e) { - // Ignore. - } - } - } - } - - return props; - } - - String getenv(String key) { - return firstNonEmpty(childEnv.get(key), System.getenv(key)); - } - - private String findAssembly(String scalaVersion) { - String sparkHome = getSparkHome(); - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - } else { - libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion)); - } - - final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); - FileFilter filter = new FileFilter() { - @Override - public boolean accept(File file) { - return file.isFile() && re.matcher(file.getName()).matches(); - } - }; - File[] assemblies = libdir.listFiles(filter); - checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); - checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); - return assemblies[0].getAbsolutePath(); - } - - private String getConfDir() { - String confDir = getenv("SPARK_CONF_DIR"); - return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); - } - } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 098c246deabe..ac0069e55720 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -36,7 +37,7 @@ *

    * This class has also some special features to aid PySparkLauncher. */ -class SparkSubmitCommandBuilder extends SparkLauncher implements CommandBuilder { +class SparkSubmitCommandBuilder extends AbstractCommandBuilder { /** * Name of the app resource used to identify the PySpark shell. The command line parser expects @@ -62,6 +63,10 @@ class SparkSubmitCommandBuilder extends SparkLauncher implements CommandBuilder private final List driverArgs; private boolean hasMixedArguments; + SparkSubmitCommandBuilder() { + this.driverArgs = Collections.emptyList(); + } + SparkSubmitCommandBuilder(List args) { this(false, args); } @@ -72,7 +77,7 @@ class SparkSubmitCommandBuilder extends SparkLauncher implements CommandBuilder List submitArgs = args; if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { this.hasMixedArguments = true; - setAppResource(PYSPARK_SHELL); + appResource = PYSPARK_SHELL; submitArgs = args.subList(1, args.size()); } else { this.hasMixedArguments = hasMixedArguments; @@ -90,13 +95,76 @@ public List buildCommand(Map env) throws IOException { } } + List buildSparkSubmitArgs() { + List args = new ArrayList(); + SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); + + if (verbose) { + args.add(parser.VERBOSE); + } + + if (master != null) { + args.add(parser.MASTER); + args.add(master); + } + + if (deployMode != null) { + args.add(parser.DEPLOY_MODE); + args.add(deployMode); + } + + if (appName != null) { + args.add(parser.NAME); + args.add(appName); + } + + for (Map.Entry e : conf.entrySet()) { + args.add(parser.CONF); + args.add(String.format("%s=%s", e.getKey(), e.getValue())); + } + + if (propertiesFile != null) { + args.add(parser.PROPERTIES_FILE); + args.add(propertiesFile); + } + + if (!jars.isEmpty()) { + args.add(parser.JARS); + args.add(join(",", jars)); + } + + if (!files.isEmpty()) { + args.add(parser.FILES); + args.add(join(",", files)); + } + + if (!pyFiles.isEmpty()) { + args.add(parser.PY_FILES); + args.add(join(",", pyFiles)); + } + + if (mainClass != null) { + args.add(parser.CLASS); + args.add(mainClass); + } + + args.addAll(sparkArgs); + if (appResource != null) { + args.add(appResource); + } + args.addAll(appArgs); + + return args; + } + private List buildSparkSubmitCommand(Map env) throws IOException { // Load the properties file and check whether spark-submit will be running the app's driver // or just launching a cluster app. When running the driver, the JVM's argument will be // modified to cover the driver's configuration. Properties props = loadPropertiesFile(); boolean isClientMode = isClientMode(props); - String extraClassPath = isClientMode ? find(DRIVER_EXTRA_CLASSPATH, conf, props) : null; + String extraClassPath = isClientMode ? + find(SparkLauncher.DRIVER_EXTRA_CLASSPATH, conf, props) : null; List cmd = buildJavaCommand(extraClassPath); addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); @@ -110,12 +178,13 @@ private List buildSparkSubmitCommand(Map env) throws IOE // - SPARK_DRIVER_MEMORY env variable // - SPARK_MEM env variable // - default value (512m) - String memory = firstNonEmpty(find(DRIVER_MEMORY, conf, props), + String memory = firstNonEmpty(find(SparkLauncher.DRIVER_MEMORY, conf, props), System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); cmd.add("-Xms" + memory); cmd.add("-Xmx" + memory); - addOptionString(cmd, find(DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); - mergeEnvPathList(env, getLibPathEnvName(), find(DRIVER_EXTRA_LIBRARY_PATH, conf, props)); + addOptionString(cmd, find(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); + mergeEnvPathList(env, getLibPathEnvName(), + find(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); } addPermGenSizeOpt(cmd); @@ -131,7 +200,7 @@ private List buildPySparkShellCommand(Map env) throws IO System.err.println( "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + "Use ./bin/spark-submit "); - setAppResource(appArgs.get(0)); + appResource = appArgs.get(0); appArgs.remove(0); return buildCommand(env); } @@ -142,7 +211,8 @@ private List buildPySparkShellCommand(Map env) throws IO checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); Properties props = loadPropertiesFile(); - mergeEnvPathList(env, getLibPathEnvName(), find(DRIVER_EXTRA_LIBRARY_PATH, conf, props)); + mergeEnvPathList(env, getLibPathEnvName(), + find(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); StringBuilder submitArgs = new StringBuilder(); for (String arg : sparkArgs) { @@ -171,7 +241,7 @@ private List buildPySparkShellCommand(Map env) throws IO } private boolean isClientMode(Properties userProps) { - String userMaster = firstNonEmpty(master, (String) userProps.get(SPARK_MASTER)); + String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER)); return userMaster == null || "client".equals(deployMode) || "yarn-client".equals(userMaster) || @@ -196,40 +266,48 @@ private String quote(String s) { private class OptionParser extends SparkSubmitOptionParser { + private final List driverJvmKeys = Arrays.asList( + SparkLauncher.DRIVER_EXTRA_CLASSPATH, + SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, + SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, + SparkLauncher.DRIVER_MEMORY); + @Override protected boolean handle(String opt, String value) { if (opt.equals(MASTER)) { - setMaster(value); + master = value; driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DEPLOY_MODE)) { - setDeployMode(value); + deployMode = value; driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(PROPERTIES_FILE)) { - setPropertiesFile(value); + propertiesFile = value; driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_MEMORY)) { - setConf(SparkLauncher.DRIVER_MEMORY, value); + conf.put(SparkLauncher.DRIVER_MEMORY, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_JAVA_OPTIONS)) { - setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value); + conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_LIBRARY_PATH)) { - setConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value); + conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(DRIVER_CLASS_PATH)) { - setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value); + conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value); driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(CONF)) { - String[] conf = value.split("=", 2); - checkArgument(conf.length == 2, "Invalid argument to %s: %s", CONF, value); - handleConf(conf[0], conf[1]); + String[] setConf = value.split("=", 2); + checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); + if (driverJvmKeys.contains(setConf[0])) { + conf.put(setConf[0], setConf[1]); + } driverArgs.add(opt); driverArgs.add(value); } else if (opt.equals(CLASS)) { @@ -237,14 +315,14 @@ protected boolean handle(String opt, String value) { // mixing spark-submit arguments with arguments that should be propagated to the shell // itself. Note that for this to work, the "--class" argument must come before any // non-spark-submit arguments. - setMainClass(value); + mainClass = value; if (specialClasses.containsKey(value)) { hasMixedArguments = true; - setAppResource(specialClasses.get(value)); + appResource = specialClasses.get(value); } } else if (opt.equals(PYSPARK_SHELL)) { hasMixedArguments = true; - setAppResource(opt); + appResource = opt; } else { addSparkArgs(opt, value); } @@ -257,7 +335,7 @@ protected boolean handleUnknown(String opt) { // In normal mode, any unrecognized parameter triggers the end of command line parsing. // The remaining params will be appended to the list of SparkSubmit arguments. if (hasMixedArguments) { - addAppArgs(opt); + appArgs.add(opt); return true; } else { addSparkArgs(opt); @@ -272,18 +350,6 @@ protected void handleExtraArgs(List extra) { } } - private void handleConf(String key, String value) { - List driverJvmKeys = Arrays.asList( - SparkLauncher.DRIVER_EXTRA_CLASSPATH, - SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, - SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, - SparkLauncher.DRIVER_MEMORY); - - if (driverJvmKeys.contains(key)) { - setConf(key, value); - } - } - } } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index bd713dde4e19..2e994d39b420 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -149,19 +149,21 @@ private void testCmdBuilder(boolean isDriver) throws Exception { SparkSubmitCommandBuilder launcher = new SparkSubmitCommandBuilder(Collections.emptyList()); - launcher.setSparkHome(System.getProperty("spark.test.home")) - .setMaster("yarn") - .setDeployMode(deployMode) - .setAppResource("/foo") - .setAppName("MyApp") - .setMainClass("my.Class") - .setPropertiesFile(dummyPropsFile.getAbsolutePath()) - .addAppArgs("foo", "bar") - .setConf(SparkLauncher.DRIVER_MEMORY, "1g") - .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver") - .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver -XX:MaxPermSize=256m") - .setConf(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native") - .setConf("spark.foo", "foo"); + launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, + System.getProperty("spark.test.home")); + launcher.master = "yarn"; + launcher.deployMode = deployMode; + launcher.appResource = "/foo"; + launcher.appName = "MyApp"; + launcher.mainClass = "my.Class"; + launcher.propertiesFile = dummyPropsFile.getAbsolutePath(); + launcher.appArgs.add("foo"); + launcher.appArgs.add("bar"); + launcher.conf.put(SparkLauncher.DRIVER_MEMORY, "1g"); + launcher.conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver"); + launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver -XX:MaxPermSize=256m"); + launcher.conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native"); + launcher.conf.put("spark.foo", "foo"); Map env = new HashMap(); List cmd = launcher.buildCommand(env); @@ -268,7 +270,7 @@ private boolean findInStringList(String list, String sep, String needle) { private List buildCommand(List args, Map env) throws Exception { SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); - builder.setSparkHome(System.getProperty("spark.test.home")); + builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); return builder.buildCommand(env); } From e2d4d717cee78de21aa9bdb60d524bbfd3aeb5fa Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 24 Feb 2015 10:24:34 -0800 Subject: [PATCH 38/51] Simplify some code used to launch pyspark. --- .../launcher/AbstractCommandBuilder.java | 8 --- .../launcher/SparkSubmitCommandBuilder.java | 50 ++++++------------- .../SparkSubmitCommandBuilderSuite.java | 3 +- python/pyspark/java_gateway.py | 2 +- 4 files changed, 18 insertions(+), 45 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 2c3719690c8a..07edecd03786 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -48,7 +48,6 @@ abstract class AbstractCommandBuilder { String mainClass; String master; String propertiesFile; - final List sparkArgs; final List appArgs; final List jars; final List files; @@ -67,7 +66,6 @@ public AbstractCommandBuilder(Map env) { this.files = new ArrayList(); this.jars = new ArrayList(); this.pyFiles = new ArrayList(); - this.sparkArgs = new ArrayList(); } /** @@ -284,12 +282,6 @@ String getScalaVersion() { throw new IllegalStateException("Should not reach here."); } - void addSparkArgs(String... args) { - for (String arg : args) { - sparkArgs.add(arg); - } - } - String getSparkHome() { String path = getenv(ENV_SPARK_HOME); checkState(path != null, diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index ac0069e55720..abda7071418f 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -48,6 +48,11 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { */ static final String PYSPARK_SHELL = "pyspark-shell-main"; + /** + * This is the actual resource name that identifies the PySpark shell to SparkSubmit. + */ + static final String PYSPARK_SHELL_RESOURCE = "pyspark-shell"; + /** * This map must match the class names for available special classes, since this modifies the way * command line parsing works. This maps the class name to the resource to use when calling @@ -60,11 +65,11 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { "spark-internal"); } - private final List driverArgs; + private final List sparkArgs; private boolean hasMixedArguments; SparkSubmitCommandBuilder() { - this.driverArgs = Collections.emptyList(); + this.sparkArgs = new ArrayList(); } SparkSubmitCommandBuilder(List args) { @@ -72,12 +77,11 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { } SparkSubmitCommandBuilder(boolean hasMixedArguments, List args) { - this.driverArgs = new ArrayList(); - + this(); List submitArgs = args; if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { this.hasMixedArguments = true; - appResource = PYSPARK_SHELL; + appResource = PYSPARK_SHELL_RESOURCE; submitArgs = args.subList(1, args.size()); } else { this.hasMixedArguments = hasMixedArguments; @@ -88,7 +92,7 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { @Override public List buildCommand(Map env) throws IOException { - if (PYSPARK_SHELL.equals(appResource)) { + if (PYSPARK_SHELL_RESOURCE.equals(appResource)) { return buildPySparkShellCommand(env); } else { return buildSparkSubmitCommand(env); @@ -215,13 +219,7 @@ private List buildPySparkShellCommand(Map env) throws IO find(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); StringBuilder submitArgs = new StringBuilder(); - for (String arg : sparkArgs) { - if (submitArgs.length() > 0) { - submitArgs.append(" "); - } - submitArgs.append(quote(arg)); - } - for (String arg : driverArgs) { + for (String arg : buildSparkSubmitArgs()) { if (submitArgs.length() > 0) { submitArgs.append(" "); } @@ -276,40 +274,24 @@ private class OptionParser extends SparkSubmitOptionParser { protected boolean handle(String opt, String value) { if (opt.equals(MASTER)) { master = value; - driverArgs.add(opt); - driverArgs.add(value); } else if (opt.equals(DEPLOY_MODE)) { deployMode = value; - driverArgs.add(opt); - driverArgs.add(value); } else if (opt.equals(PROPERTIES_FILE)) { propertiesFile = value; - driverArgs.add(opt); - driverArgs.add(value); } else if (opt.equals(DRIVER_MEMORY)) { conf.put(SparkLauncher.DRIVER_MEMORY, value); - driverArgs.add(opt); - driverArgs.add(value); } else if (opt.equals(DRIVER_JAVA_OPTIONS)) { conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value); - driverArgs.add(opt); - driverArgs.add(value); } else if (opt.equals(DRIVER_LIBRARY_PATH)) { conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value); - driverArgs.add(opt); - driverArgs.add(value); } else if (opt.equals(DRIVER_CLASS_PATH)) { conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value); - driverArgs.add(opt); - driverArgs.add(value); } else if (opt.equals(CONF)) { String[] setConf = value.split("=", 2); checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); if (driverJvmKeys.contains(setConf[0])) { conf.put(setConf[0], setConf[1]); } - driverArgs.add(opt); - driverArgs.add(value); } else if (opt.equals(CLASS)) { // The special classes require some special command line handling, since they allow // mixing spark-submit arguments with arguments that should be propagated to the shell @@ -320,11 +302,9 @@ protected boolean handle(String opt, String value) { hasMixedArguments = true; appResource = specialClasses.get(value); } - } else if (opt.equals(PYSPARK_SHELL)) { - hasMixedArguments = true; - appResource = opt; } else { - addSparkArgs(opt, value); + sparkArgs.add(opt); + sparkArgs.add(value); } return true; } @@ -338,7 +318,7 @@ protected boolean handleUnknown(String opt) { appArgs.add(opt); return true; } else { - addSparkArgs(opt); + sparkArgs.add(opt); return false; } } @@ -346,7 +326,7 @@ protected boolean handleUnknown(String opt) { @Override protected void handleExtraArgs(List extra) { for (String arg : extra) { - addSparkArgs(arg); + sparkArgs.add(arg); } } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 2e994d39b420..815edc4e4971 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -123,7 +123,8 @@ public void testPySparkLauncher() throws Exception { List cmd = buildCommand(sparkSubmitArgs, env); assertEquals("python", cmd.get(cmd.size() - 1)); assertEquals( - String.format("\"%s\" \"foo\" \"%s\" \"bar\"", parser.MASTER, parser.DEPLOY_MODE), + String.format("\"%s\" \"foo\" \"%s\" \"bar\" \"%s\"", + parser.MASTER, parser.DEPLOY_MODE, SparkSubmitCommandBuilder.PYSPARK_SHELL_RESOURCE), env.get("PYSPARK_SUBMIT_ARGS")); } diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index ea8f12074d98..43d2cf517188 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -41,7 +41,7 @@ def launch_gateway(): submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS") submit_args = submit_args if submit_args is not None else "" submit_args = shlex.split(submit_args) - command = [os.path.join(SPARK_HOME, script)] + submit_args + ["pyspark-shell"] + command = [os.path.join(SPARK_HOME, script)] + submit_args # Start a socket that will be used by PythonGatewayServer to communicate its port to us callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) From c12d84bdf6b62ac5281833c6ce46ef867e34ce04 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 24 Feb 2015 12:30:04 -0800 Subject: [PATCH 39/51] Review feedback. And fix spark-submit on Windows. --- bin/spark-class | 8 ++++---- bin/spark-class2.cmd | 2 ++ bin/spark-submit.cmd | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 4aec4660ed64..cd35095db27b 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -41,8 +41,7 @@ else fi # Look for the launcher. In non-release mode, add the compiled classes directly to the classpath -# instead of looking for a jar file, since it's very common for people using sbt to use the -# "assembly" target instead of "package". +# instead of looking for a jar file. SPARK_LAUNCHER_CP= if [ -f $SPARK_HOME/RELEASE ]; then LAUNCHER_DIR="$SPARK_HOME/lib" @@ -72,8 +71,9 @@ else SPARK_LAUNCHER_CP="$LAUNCHER_DIR/classes" fi -# The launcher library will print arguments separated by a NULL character. Read that in a while -# loop, populating an array that will be used to exec the final command. +# The launcher library will print arguments separated by a NULL character, to allow arguments with +# characters that would be otherwise interpreted by the shell. Read that in a while loop, populating +# an array that will be used to exec the final command. CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 99632e634a61..37d22215a0e7 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -62,6 +62,8 @@ rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java +rem The launcher library prints the command to be executed in a single line suitable for being +rem executed by the batch interpreter. So read all the output of the launcher into a variable. for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCHER_CP% org.apache.spark.launcher.Main %*"') do ( set SPARK_CMD=%%i ) diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd index 8f3b84c7b971..0c7aa8bee0f8 100644 --- a/bin/spark-submit.cmd +++ b/bin/spark-submit.cmd @@ -20,4 +20,4 @@ rem rem This is the entry point for running Spark submit. To avoid polluting the rem environment, it just launches a new cmd to do the real work. -cmd /V /E /C %~dp0spark-submit2.cmd %* +cmd /V /E /C %~dp0spark-class2.cmd org.apache.spark.deploy.SparkSubmit %* From 0031a8e05c9d70d086d5e24171b221c4bbbdf230 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 24 Feb 2015 20:33:22 -0800 Subject: [PATCH 40/51] Review feedback. --- .../spark/deploy/SparkSubmitArguments.scala | 5 +- .../launcher/SparkSubmitArgumentsParser.scala | 7 +- .../spark/launcher/WorkerCommandBuilder.scala | 7 +- .../launcher/AbstractCommandBuilder.java | 33 ++++---- .../spark/launcher/CommandBuilderUtils.java | 72 +++++++++++++++-- .../java/org/apache/spark/launcher/Main.java | 37 ++------- .../launcher/SparkClassCommandBuilder.java | 47 ----------- .../launcher/SparkSubmitCommandBuilder.java | 63 +++++++-------- .../apache/spark/launcher/package-info.java | 2 +- .../launcher/CommandBuilderUtilsSuite.java | 15 ++++ sbin/spark-daemon.sh | 77 ++++++++++++------- sbin/start-thriftserver.sh | 2 +- 12 files changed, 191 insertions(+), 176 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 38b9477a603b..94e4bdbfb7d7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.Utils * The env argument is used for testing. */ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) - extends SparkSubmitArgumentsParser { + extends SparkSubmitArgumentsParser { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -397,6 +397,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } /** + * Handle unrecognized command line options. + * * The first unrecognized option is treated as the "primary resource". Everything else is * treated as application arguments. */ @@ -419,7 +421,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St childArgs ++= extra } - private def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = { val outStream = SparkSubmit.printStream if (unknownParam != null) { diff --git a/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala b/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala index 7db086c550a0..a83501253105 100644 --- a/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala +++ b/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala @@ -19,8 +19,7 @@ package org.apache.spark.launcher /** * This class makes SparkSubmitOptionParser visible for Spark code outside of the `launcher` - * package. + * package, since Java doesn't have a feature similar to `private[spark]`, and we don't want + * that class to be public. */ -private[spark] abstract class SparkSubmitArgumentsParser extends SparkSubmitOptionParser { - -} +private[spark] abstract class SparkSubmitArgumentsParser extends SparkSubmitOptionParser diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala index 4fe73cfc4fce..9be98723aed1 100644 --- a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala +++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala @@ -25,8 +25,9 @@ import scala.collection.JavaConversions._ import org.apache.spark.deploy.Command /** - * This class is used by CommandUtils. It uses some package-private APIs in SparkLauncher and so - * needs to live in the same package as the rest of the library. + * This class is used by CommandUtils. It uses some package-private APIs in SparkLauncher, and since + * Java doesn't have a feature similar to `private[spark]`, and we don't want that class to be + * public, needs to live in the same package as the rest of the library. */ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, command: Command) extends AbstractCommandBuilder { @@ -38,7 +39,7 @@ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, comm val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator)) cmd.add(s"-Xms${memoryMb}M") cmd.add(s"-Xmx${memoryMb}M") - command.javaOpts.foreach { cmd.add } + command.javaOpts.foreach(cmd.add) addPermGenSizeOpt(cmd) addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) cmd diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 07edecd03786..dc90e9e98723 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -36,7 +36,7 @@ import static org.apache.spark.launcher.CommandBuilderUtils.*; /** - * Abstract command builder that defines common functionality for all builders. + * Abstract Spark command builder that defines common functionality. */ abstract class AbstractCommandBuilder { @@ -56,12 +56,8 @@ abstract class AbstractCommandBuilder { final Map conf; public AbstractCommandBuilder() { - this(Collections.emptyMap()); - } - - public AbstractCommandBuilder(Map env) { this.appArgs = new ArrayList(); - this.childEnv = new HashMap(env); + this.childEnv = new HashMap(); this.conf = new HashMap(); this.files = new ArrayList(); this.jars = new ArrayList(); @@ -69,7 +65,7 @@ public AbstractCommandBuilder(Map env) { } /** - * Builds the command like to execute. + * Builds the command to execute. * * @param env A map containing environment variables for the child process. It may already contain * entries defined by the user (such as SPARK_HOME, or those defined by the @@ -78,6 +74,16 @@ public AbstractCommandBuilder(Map env) { */ abstract List buildCommand(Map env) throws IOException; + /** + * Builds a list of arguments to run java. + * + * This method finds the java executable to use and appends JVM-specific options for running a + * class with Spark in the classpath. It also loads options from the "java-opts" file in the + * configuration directory being used. + * + * Callers should still add at least the class to run, as well as any arguments to pass to the + * class. + */ List buildJavaCommand(String extraClassPath) throws IOException { List cmd = new ArrayList(); if (javaHome == null) { @@ -241,7 +247,7 @@ List buildClassPath(String appClassPath) throws IOException { /** * Adds entries to the classpath. * - * @param cp List where to appended the new classpath entries. + * @param cp List to which the new entries are appended. * @param entries New classpath entries (separated by File.pathSeparator). */ private void addToClassPath(List cp, String entries) { @@ -268,18 +274,15 @@ String getScalaVersion() { String sparkHome = getSparkHome(); File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); - if (scala210.isDirectory() && scala211.isDirectory()) { - checkState(false, - "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + - "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); - } else if (scala210.isDirectory()) { + checkState(!scala210.isDirectory() || !scala211.isDirectory(), + "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); + if (scala210.isDirectory()) { return "2.10"; } else { checkState(scala211.isDirectory(), "Cannot find any assembly build directories."); return "2.11"; } - - throw new IllegalStateException("Should not reach here."); } String getSparkHome() { diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index e8e7faf730f6..492124b5bada 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -64,8 +64,10 @@ static String join(String sep, Iterable elements) { return sb.toString(); } - /** Returns the first value mapped to the given key in the given maps. */ - static String find(String key, Map... maps) { + /** + * Returns the first non-empty value mapped to the given key in the given maps, or null otherwise. + */ + static String firstNonEmptyValue(String key, Map... maps) { for (Map map : maps) { String value = (String) map.get(key); if (!isEmpty(value)) { @@ -75,7 +77,7 @@ static String find(String key, Map... maps) { return null; } - /** Returns the first non-empty, non-null string in the given list. */ + /** Returns the first non-empty, non-null string in the given list, or null otherwise. */ static String firstNonEmpty(String... candidates) { for (String s : candidates) { if (!isEmpty(s)) { @@ -106,8 +108,8 @@ static boolean isWindows() { } /** - * Updates the user environment to contain the merged value of "envKey" after appending - * the given path list. + * Updates the user environment, appending the given pathList to the existing value of the given + * environment variable (or setting it if it hasn't yet been set). */ static void mergeEnvPathList(Map userEnv, String envKey, String pathList) { if (!isEmpty(pathList)) { @@ -117,8 +119,8 @@ static void mergeEnvPathList(Map userEnv, String envKey, String } /** - * Parse a string as if it were a list of arguments, in the way that a shell would. - * This tries to follow the way bash parses strings. For example: + * Parse a string as if it were a list of arguments, following bash semantics. + * For example: * * Input: "\"ab cd\" efgh 'i \" j'" * Output: [ "ab cd", "efgh", "i \" j" ] @@ -130,6 +132,8 @@ static List parseOptionString(String s) { boolean inSingleQuote = false; boolean inDoubleQuote = false; boolean escapeNext = false; + + // This is needed to detect when a quoted empty string is used as an argument ("" or ''). boolean hasData = false; for (int i = 0; i < s.length(); i++) { @@ -161,7 +165,7 @@ static List parseOptionString(String s) { } break; default: - if (inSingleQuote || inDoubleQuote || !Character.isWhitespace(c)) { + if (!Character.isWhitespace(c) || inSingleQuote || inDoubleQuote) { opt.appendCodePoint(c); } else { opts.add(opt.toString()); @@ -225,4 +229,56 @@ static void checkState(boolean check, String msg, Object... args) { } } + /** + * 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. + * + * For example: + * original single argument: ab "cde" fgh + * quoted: "ab ""cde"" fgh" + */ + static String quoteForBatchScript(String arg) { + boolean needsQuotes = false; + for (int i = 0; i < arg.length(); i++) { + if (Character.isWhitespace(arg.codePointAt(i)) || 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(); + } + + /** + * Quotes a string so that it can be used in a command string and be parsed back into a single + * argument by python's "shlex.split()" function. + * + * Basically, just add simple escapes. E.g.: + * original single argument : ab "cd" ef + * after: "ab \"cd\" ef" + */ + static String quoteForPython(String s) { + StringBuilder quoted = new StringBuilder().append('"'); + for (int i = 0; i < s.length(); i++) { + int cp = s.codePointAt(i); + if (cp == '"' || cp == '\\') { + quoted.appendCodePoint('\\'); + } + quoted.appendCodePoint(cp); + } + return quoted.append('"').toString(); + } + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index bb84f70c3d44..e3f7e854f10a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -75,9 +75,10 @@ public static void main(String[] argsArray) throws Exception { } if (isWindows()) { - System.out.println(prepareForWindows(cmd, env)); + System.out.println(prepareWindowsCommand(cmd, env)); } else { - List bashCmd = prepareForBash(cmd, env); + // In bash, use NULL as the arg separator since it cannot be used in an argument. + List bashCmd = prepareBashCommand(cmd, env); for (String c : bashCmd) { System.out.print(c); System.out.print('\0'); @@ -95,7 +96,7 @@ public static void main(String[] argsArray) throws Exception { * The command is executed using "cmd /c" and formatted in single line, since that's the * easiest way to consume this from a batch script (see spark-class2.cmd). */ - private static String prepareForWindows(List cmd, Map childEnv) { + private static String prepareWindowsCommand(List cmd, Map childEnv) { StringBuilder cmdline = new StringBuilder("cmd /c \""); for (Map.Entry e : childEnv.entrySet()) { cmdline.append(String.format("set %s=%s", e.getKey(), e.getValue())); @@ -113,7 +114,7 @@ private static String prepareForWindows(List cmd, Map ch * 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 prepareForBash(List cmd, Map childEnv) { + private static List prepareBashCommand(List cmd, Map childEnv) { if (childEnv.isEmpty()) { return cmd; } @@ -128,34 +129,6 @@ private static List prepareForBash(List cmd, Map 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: diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java index 20871d5f1ca5..e601a0a19f36 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -89,9 +89,6 @@ public List buildCommand(Map env) throws IOException { toolsDir.getAbsolutePath(), className); javaOptsKeys.add("SPARK_JAVA_OPTS"); - } else { - // Any classes not explicitly listed above are submitted using SparkSubmit. - return createSparkSubmitCommand(env); } List cmd = buildJavaCommand(extraClassPath); @@ -108,48 +105,4 @@ public List buildCommand(Map env) throws IOException { return cmd; } - private List createSparkSubmitCommand(Map env) throws IOException { - final List sparkSubmitArgs = new ArrayList(); - final List appArgs = new ArrayList(); - - // Parse the command line and special-case the HELP command line argument, allowing it to be - // propagated to the app being launched. - SparkSubmitOptionParser parser = new SparkSubmitOptionParser() { - - @Override - protected boolean handle(String opt, String value) { - if (opt.equals(HELP)) { - appArgs.add(opt); - } else { - sparkSubmitArgs.add(opt); - sparkSubmitArgs.add(value); - } - return true; - } - - @Override - protected boolean handleUnknown(String opt) { - appArgs.add(opt); - return true; - } - - @Override - protected void handleExtraArgs(List extra) { - appArgs.addAll(extra); - } - - }; - - parser.parse(classArgs); - sparkSubmitArgs.add(parser.CLASS); - sparkSubmitArgs.add(className); - - SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(true, sparkSubmitArgs); - builder.appResource = "spark-internal"; - for (String arg: appArgs) { - builder.appArgs.add(arg); - } - return builder.buildCommand(env); - } - } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index abda7071418f..17f94a4798c2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -63,28 +63,32 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { specialClasses.put("org.apache.spark.repl.Main", "spark-shell"); specialClasses.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver", "spark-internal"); + specialClasses.put("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2", + "spark-internal"); } private final List sparkArgs; - private boolean hasMixedArguments; + + /** + * Controls whether mixing spark-submit arguments with app arguments is allowed. This is needed + * to parse the command lines for things like bin/spark-shell, which allows users to mix and + * match arguments (e.g. "bin/spark-shell SparkShellArg --master foo"). + */ + private boolean isSpecialClass; SparkSubmitCommandBuilder() { this.sparkArgs = new ArrayList(); } SparkSubmitCommandBuilder(List args) { - this(false, args); - } - - SparkSubmitCommandBuilder(boolean hasMixedArguments, List args) { this(); List submitArgs = args; if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { - this.hasMixedArguments = true; + this.isSpecialClass = true; appResource = PYSPARK_SHELL_RESOURCE; submitArgs = args.subList(1, args.size()); } else { - this.hasMixedArguments = hasMixedArguments; + this.isSpecialClass = false; } new OptionParser().parse(submitArgs); @@ -168,7 +172,7 @@ private List buildSparkSubmitCommand(Map env) throws IOE Properties props = loadPropertiesFile(); boolean isClientMode = isClientMode(props); String extraClassPath = isClientMode ? - find(SparkLauncher.DRIVER_EXTRA_CLASSPATH, conf, props) : null; + firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_CLASSPATH, conf, props) : null; List cmd = buildJavaCommand(extraClassPath); addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); @@ -182,13 +186,13 @@ private List buildSparkSubmitCommand(Map env) throws IOE // - SPARK_DRIVER_MEMORY env variable // - SPARK_MEM env variable // - default value (512m) - String memory = firstNonEmpty(find(SparkLauncher.DRIVER_MEMORY, conf, props), + String memory = firstNonEmpty(firstNonEmptyValue(SparkLauncher.DRIVER_MEMORY, conf, props), System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); cmd.add("-Xms" + memory); cmd.add("-Xmx" + memory); - addOptionString(cmd, find(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); + addOptionString(cmd, firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); mergeEnvPathList(env, getLibPathEnvName(), - find(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); + firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); } addPermGenSizeOpt(cmd); @@ -216,16 +220,17 @@ private List buildPySparkShellCommand(Map env) throws IO Properties props = loadPropertiesFile(); mergeEnvPathList(env, getLibPathEnvName(), - find(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); + firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); + // Store spark-submit arguments in an environment variable, since there's no way to pass + // them to shell.py on the comand line. StringBuilder submitArgs = new StringBuilder(); for (String arg : buildSparkSubmitArgs()) { if (submitArgs.length() > 0) { submitArgs.append(" "); } - submitArgs.append(quote(arg)); + submitArgs.append(quoteForPython(arg)); } - env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); List pyargs = new ArrayList(); @@ -242,24 +247,7 @@ private boolean isClientMode(Properties userProps) { String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER)); return userMaster == null || "client".equals(deployMode) || - "yarn-client".equals(userMaster) || - (deployMode == null && !userMaster.startsWith("yarn-")); - } - - /** - * Quotes a string so that it can be used in a command string and be parsed back into a single - * argument by python's "shlex.split()" function. - */ - private String quote(String s) { - StringBuilder quoted = new StringBuilder().append('"'); - for (int i = 0; i < s.length(); i++) { - int cp = s.codePointAt(i); - if (cp == '"' || cp == '\\') { - quoted.appendCodePoint('\\'); - } - quoted.appendCodePoint(cp); - } - return quoted.append('"').toString(); + (!userMaster.equals("yarn-cluster") && deployMode == null); } private class OptionParser extends SparkSubmitOptionParser { @@ -299,7 +287,7 @@ protected boolean handle(String opt, String value) { // non-spark-submit arguments. mainClass = value; if (specialClasses.containsKey(value)) { - hasMixedArguments = true; + isSpecialClass = true; appResource = specialClasses.get(value); } } else { @@ -311,10 +299,11 @@ protected boolean handle(String opt, String value) { @Override protected boolean handleUnknown(String opt) { - // When mixing arguments, add unrecognized parameters directly to the user arguments list. - // In normal mode, any unrecognized parameter triggers the end of command line parsing. - // The remaining params will be appended to the list of SparkSubmit arguments. - if (hasMixedArguments) { + // When mixing arguments, add unrecognized parameters directly to the user arguments list. In + // normal mode, any unrecognized parameter triggers the end of command line parsing, and the + // parameter itself will be interpreted by SparkSubmit as the application resource. The + // remaining params will be appended to the list of SparkSubmit arguments. + if (isSpecialClass) { appArgs.add(opt); return true; } else { diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java index 96f33ecafedd..7ed756f4b859 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java +++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java @@ -42,4 +42,4 @@ * } * */ -package org.apache.spark.launcher; \ No newline at end of file +package org.apache.spark.launcher; diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java index 7854bd826239..b9d1f379c35e 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -68,6 +68,21 @@ public void testInvalidOptionStrings() { testInvalidOpt("'abcde"); } + @Test + public void testWindowsBatchQuoting() { + assertEquals("abc", quoteForBatchScript("abc")); + assertEquals("\"a b c\"", quoteForBatchScript("a b c")); + assertEquals("\"a \"\"b\"\" c\"", quoteForBatchScript("a \"b\" c")); + assertEquals("\"a\"\"b\"\"c\"", quoteForBatchScript("a\"b\"c")); + } + + @Test + public void testPythonArgQuoting() { + assertEquals("\"abc\"", quoteForPython("abc")); + assertEquals("\"a b c\"", quoteForPython("a b c")); + assertEquals("\"a \\\"b\\\" c\"", quoteForPython("a \"b\" c")); + } + private void testOpt(String opts, List expected) { assertEquals(String.format("test string failed to parse: [[ %s ]]", opts), expected, parseOptionString(opts)); diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index cc23c348f3b6..a792d36b6990 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -121,38 +121,63 @@ if [ "$SPARK_NICENESS" = "" ]; then export SPARK_NICENESS=0 fi +run_command() { + mode=$1 + shift -case $option in + mkdir -p "$SPARK_PID_DIR" - (start) + if [ -f $pid ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then + echo "$command running as process $TARGET_ID. Stop it first." + exit 1 + fi + fi - mkdir -p "$SPARK_PID_DIR" + if [ "$SPARK_MASTER" != "" ]; then + echo rsync from "$SPARK_MASTER" + rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME" + fi - if [ -f $pid ]; then - TARGET_ID="$(cat "$pid")" - if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then - echo "$command running as process $TARGET_ID. Stop it first." - exit 1 - fi - fi + spark_rotate_log "$log" + echo "starting $command, logging to $log" + + case $mode in + (class) + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + newpid=$! + ;; + + (submit) + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-submit --class $command "$@" >> "$log" 2>&1 < /dev/null & + newpid=$! + ;; + + (*) + echo "unknown mode: $mode" + exit 1 + ;; + esac + + echo $newpid > $pid + sleep 2 + # Check if the process has died; in that case we'll tail the log so the user can see + if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then + echo "failed to launch $command:" + tail -2 "$log" | sed 's/^/ /' + echo "full log in $log" + fi +} - if [ "$SPARK_MASTER" != "" ]; then - echo rsync from "$SPARK_MASTER" - rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME" - fi +case $option in - spark_rotate_log "$log" - echo "starting $command, logging to $log" - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & - newpid=$! - echo $newpid > $pid - sleep 2 - # Check if the process has died; in that case we'll tail the log so the user can see - if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then - echo "failed to launch $command:" - tail -2 "$log" | sed 's/^/ /' - echo "full log in $log" - fi + (submit) + run_command submit "$@" + ;; + + (start) + run_command class "$@" ;; (stop) diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index e730329d7b0b..5b0aeb177fff 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -52,4 +52,4 @@ fi export SUBMIT_USAGE_FUNCTION=usage -exec "$FWDIR"/sbin/spark-daemon.sh start $CLASS 1 "$@" +exec "$FWDIR"/sbin/spark-daemon.sh submit $CLASS 1 "$@" From 7e66c18bad6e672c60ec995bd968b3a7db32ee73 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 25 Feb 2015 12:51:12 -0800 Subject: [PATCH 41/51] Fix pyspark tests. --- bin/pyspark | 1 + 1 file changed, 1 insertion(+) diff --git a/bin/pyspark b/bin/pyspark index be88f7a6e743..d09d1ba8a6ce 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -86,6 +86,7 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR + export PYSPARK_SUBMIT_ARGS=pyspark-shell if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else From 4c19196f495813dc2dcd6116b5831932ec406803 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 25 Feb 2015 17:23:10 -0800 Subject: [PATCH 42/51] Update comment. --- .../java/org/apache/spark/launcher/SparkSubmitOptionParser.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 7bb6e3f4fa5b..8526d2e7cfa3 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -23,8 +23,6 @@ /** * Parser for spark-submit command line options. - *

    - * This class, although public, is not designed to be used outside of Spark. *

    * This class encapsulates the parsing code for spark-submit command line options, so that there * is a single list of options that needs to be maintained (well, sort of, but it makes it harder From 6184c07d1f7ddb9bf74e898155f28340b7a33f53 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 25 Feb 2015 17:41:33 -0800 Subject: [PATCH 43/51] Rename field. --- .../spark/launcher/SparkSubmitCommandBuilder.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 17f94a4798c2..c9da7ec90e1a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -74,7 +74,7 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { * to parse the command lines for things like bin/spark-shell, which allows users to mix and * match arguments (e.g. "bin/spark-shell SparkShellArg --master foo"). */ - private boolean isSpecialClass; + private boolean allowsMixedArguments; SparkSubmitCommandBuilder() { this.sparkArgs = new ArrayList(); @@ -84,11 +84,11 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { this(); List submitArgs = args; if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { - this.isSpecialClass = true; + this.allowsMixedArguments = true; appResource = PYSPARK_SHELL_RESOURCE; submitArgs = args.subList(1, args.size()); } else { - this.isSpecialClass = false; + this.allowsMixedArguments = false; } new OptionParser().parse(submitArgs); @@ -287,7 +287,7 @@ protected boolean handle(String opt, String value) { // non-spark-submit arguments. mainClass = value; if (specialClasses.containsKey(value)) { - isSpecialClass = true; + allowsMixedArguments = true; appResource = specialClasses.get(value); } } else { @@ -303,7 +303,7 @@ protected boolean handleUnknown(String opt) { // normal mode, any unrecognized parameter triggers the end of command line parsing, and the // parameter itself will be interpreted by SparkSubmit as the application resource. The // remaining params will be appended to the list of SparkSubmit arguments. - if (isSpecialClass) { + if (allowsMixedArguments) { appArgs.add(opt); return true; } else { From 92a9cfb6721d2f70b0fac62bf2142a6dfad6d8cb Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 26 Feb 2015 11:27:30 -0800 Subject: [PATCH 44/51] Fix Win32 launcher, usage. Some more characters need to be espaced now that SparkLauncher is running a batch script as a a child. Also some fixes for the handling of usage errors. --- bin/spark-submit | 7 ++++++ bin/spark-submit.cmd | 13 +++++++++- .../spark/launcher/CommandBuilderUtils.java | 24 ++++++++++++++----- .../java/org/apache/spark/launcher/Main.java | 4 ++-- .../apache/spark/launcher/SparkLauncher.java | 14 ++++++++++- .../launcher/SparkSubmitCommandBuilder.java | 5 +++- .../launcher/CommandBuilderUtilsSuite.java | 1 + 7 files changed, 57 insertions(+), 11 deletions(-) diff --git a/bin/spark-submit b/bin/spark-submit index dc1773148c33..2bb7024e9dc0 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -18,4 +18,11 @@ # SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + +usage() { + "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit --help + exit $1 +} +export -f usage + exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "$@" diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd index 0c7aa8bee0f8..784cd96b1763 100644 --- a/bin/spark-submit.cmd +++ b/bin/spark-submit.cmd @@ -20,4 +20,15 @@ rem rem This is the entry point for running Spark submit. To avoid polluting the rem environment, it just launches a new cmd to do the real work. -cmd /V /E /C %~dp0spark-class2.cmd org.apache.spark.deploy.SparkSubmit %* +set CLASS=org.apache.spark.deploy.SparkSubmit +call %~dp0spark-class2.cmd %CLASS% %* +set SPARK_ERROR_LEVEL=%ERRORLEVEL% +if "%SPARK_LAUNCHER_USAGE_ERROR%"=="1" ( + call :usage + exit /b 1 +) +exit /b %SPARK_ERROR_LEVEL% + +:usage +call %SPARK_HOME%\bin\spark-class2.cmd %CLASS% --help +goto :eof diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index 492124b5bada..9b04732afee1 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -231,16 +231,19 @@ static void checkState(boolean check, String msg, Object... args) { /** * 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. + * needs quoting. Arguments only seem to need quotes in batch scripts if they have certain + * special characters, some of which need extra (and different) escaping. * * For example: - * original single argument: ab "cde" fgh - * quoted: "ab ""cde"" fgh" + * original single argument: ab="cde fgh" + * quoted: "ab^=""cde fgh""" */ static String quoteForBatchScript(String arg) { + boolean needsQuotes = false; for (int i = 0; i < arg.length(); i++) { - if (Character.isWhitespace(arg.codePointAt(i)) || arg.codePointAt(i) == '"') { + int c = arg.codePointAt(i); + if (Character.isWhitespace(c) || c == '"' || c == '=') { needsQuotes = true; break; } @@ -252,8 +255,17 @@ static String quoteForBatchScript(String arg) { quoted.append("\""); for (int i = 0; i < arg.length(); i++) { int cp = arg.codePointAt(i); - if (cp == '\"') { - quoted.append("\""); + switch (cp) { + case '"': + quoted.append('"'); + break; + + case '=': + quoted.append('^'); + break; + + default: + break; } quoted.appendCodePoint(cp); } diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index e3f7e854f10a..b3015985b334 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -147,9 +147,9 @@ private static class UsageLauncher extends AbstractCommandBuilder { @Override public List buildCommand(Map env) { if (isWindows()) { - return Arrays.asList("set SPARK_LAUNCHER_USAGE_ERROR=1"); + return Arrays.asList("set", "SPARK_LAUNCHER_USAGE_ERROR=1"); } else { - return Arrays.asList("usage 1"); + return Arrays.asList("usage", "1"); } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 92a29e979c8d..81ddbe9194ef 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -257,9 +257,21 @@ public SparkLauncher setVerbose(boolean verbose) { */ public Process launch() throws IOException { List cmd = new ArrayList(); - cmd.add(join(File.separator, builder.getSparkHome(), "bin", "spark-submit")); + String script = isWindows() ? "spark-submit.cmd" : "spark-submit"; + cmd.add(join(File.separator, builder.getSparkHome(), "bin", script)); cmd.addAll(builder.buildSparkSubmitArgs()); + // Since the child process is a batch script, let's quote things so that special characters are + // preserved, otherwise the batch interpreter will mess up the arguments. Batch scripts are + // weird. + if (isWindows()) { + List winCmd = new ArrayList(); + for (String arg : cmd) { + winCmd.add(quoteForBatchScript(arg)); + } + cmd = winCmd; + } + ProcessBuilder pb = new ProcessBuilder(cmd.toArray(new String[cmd.size()])); for (Map.Entry e : builder.childEnv.entrySet()) { pb.environment().put(e.getKey(), e.getValue()); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index c9da7ec90e1a..01e99612fb40 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -245,6 +245,7 @@ private List buildPySparkShellCommand(Map env) throws IO private boolean isClientMode(Properties userProps) { String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER)); + // Default master is "local[*]", so assume client mode in that case. return userMaster == null || "client".equals(deployMode) || (!userMaster.equals("yarn-cluster") && deployMode == null); @@ -292,7 +293,9 @@ protected boolean handle(String opt, String value) { } } else { sparkArgs.add(opt); - sparkArgs.add(value); + if (value != null) { + sparkArgs.add(value); + } } return true; } diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java index b9d1f379c35e..dba020386737 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -74,6 +74,7 @@ public void testWindowsBatchQuoting() { assertEquals("\"a b c\"", quoteForBatchScript("a b c")); assertEquals("\"a \"\"b\"\" c\"", quoteForBatchScript("a \"b\" c")); assertEquals("\"a\"\"b\"\"c\"", quoteForBatchScript("a\"b\"c")); + assertEquals("\"ab^=\"\"cd\"\"\"", quoteForBatchScript("ab=\"cd\"")); } @Test From 5f4ddccbcd15e712cbd588ee958c6302ce95e352 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 26 Feb 2015 13:11:46 -0800 Subject: [PATCH 45/51] Better usage messages. Print the actual error message before the usual usage string. --- bin/pyspark | 5 ++- bin/spark-shell | 5 ++- bin/spark-shell2.cmd | 3 +- bin/spark-sql | 7 +++- bin/spark-submit | 16 +++++--- bin/spark-submit.cmd | 13 +------ bin/spark-submit2.cmd | 35 +++++++++++++++++ .../java/org/apache/spark/launcher/Main.java | 39 +++++++++++++------ 8 files changed, 89 insertions(+), 34 deletions(-) create mode 100644 bin/spark-submit2.cmd diff --git a/bin/pyspark b/bin/pyspark index d09d1ba8a6ce..e7f6a1a072c2 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -23,9 +23,12 @@ export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" source "$SPARK_HOME"/bin/load-spark-env.sh function usage() { + if [ -n "$1" ]; then + echo $1 + fi echo "Usage: ./bin/pyspark [options]" 1>&2 "$SPARK_HOME"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 + exit $2 } export -f usage diff --git a/bin/spark-shell b/bin/spark-shell index 80cab8ad1d3c..e2da80a962d5 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -31,9 +31,12 @@ set -o posix export FWDIR="$(cd "`dirname "$0"`"/..; pwd)" usage() { + if [ -n "$1" ]; then + echo $1 + fi echo "Usage: ./bin/spark-shell [options]" "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit $1 + exit $2 } export -f usage diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index d59bff71f02c..02f51fe59a91 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -39,13 +39,14 @@ set SPARK_SUBMIT_OPTS="%SPARK_SUBMIT_OPTS% -Dscala.usejavacp=true" :run_shell call %SPARK_HOME%\bin\spark-submit2.cmd --class org.apache.spark.repl.Main %* set SPARK_ERROR_LEVEL=%ERRORLEVEL% -if "%SPARK_LAUNCHER_USAGE_ERROR%"=="1" ( +if not "x%SPARK_LAUNCHER_USAGE_ERROR%"=="x" ( call :usage exit /b 1 ) exit /b %SPARK_ERROR_LEVEL% :usage +echo %SPARK_LAUNCHER_USAGE_ERROR% echo "Usage: .\bin\spark-shell.cmd [options]" >&2 call %SPARK_HOME%\bin\spark-submit2.cmd --help 2>&1 | findstr /V "Usage" 1>&2 goto :eof diff --git a/bin/spark-sql b/bin/spark-sql index 70f3d3d96343..b67b16c03639 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -31,6 +31,9 @@ export CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" export FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage { + if [ -n "$1" ]; then + echo $1 + fi echo "Usage: ./bin/spark-sql [options] [cli option]" pattern="usage" pattern+="\|Spark assembly has been built with Hive" @@ -43,12 +46,12 @@ function usage { echo echo "CLI options:" "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 - exit $1 + exit $2 } export -f usage if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - usage 0 + usage "" 0 fi exec "$FWDIR"/bin/spark-submit --class $CLASS "$@" diff --git a/bin/spark-submit b/bin/spark-submit index 2bb7024e9dc0..91a40cf3cdcf 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -19,10 +19,16 @@ SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -usage() { - "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit --help - exit $1 -} -export -f usage +# Only define a usage function if an upstream script hasn't done so. +if ! type -t usage >/dev/null 2>&1; then + usage() { + if [ -n "$1" ]; then + echo $1 + fi + "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit --help + exit $2 + } + export -f usage +fi exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "$@" diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd index 784cd96b1763..8f3b84c7b971 100644 --- a/bin/spark-submit.cmd +++ b/bin/spark-submit.cmd @@ -20,15 +20,4 @@ rem rem This is the entry point for running Spark submit. To avoid polluting the rem environment, it just launches a new cmd to do the real work. -set CLASS=org.apache.spark.deploy.SparkSubmit -call %~dp0spark-class2.cmd %CLASS% %* -set SPARK_ERROR_LEVEL=%ERRORLEVEL% -if "%SPARK_LAUNCHER_USAGE_ERROR%"=="1" ( - call :usage - exit /b 1 -) -exit /b %SPARK_ERROR_LEVEL% - -:usage -call %SPARK_HOME%\bin\spark-class2.cmd %CLASS% --help -goto :eof +cmd /V /E /C %~dp0spark-submit2.cmd %* diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd new file mode 100644 index 000000000000..08ddb185742d --- /dev/null +++ b/bin/spark-submit2.cmd @@ -0,0 +1,35 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This is the entry point for running Spark submit. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. + +set CLASS=org.apache.spark.deploy.SparkSubmit +call %~dp0spark-class2.cmd %CLASS% %* +set SPARK_ERROR_LEVEL=%ERRORLEVEL% +if not "x%SPARK_LAUNCHER_USAGE_ERROR%"=="x" ( + call :usage + exit /b 1 +) +exit /b %SPARK_ERROR_LEVEL% + +:usage +echo %SPARK_LAUNCHER_USAGE_ERROR% +call %SPARK_HOME%\bin\spark-class2.cmd %CLASS% --help +goto :eof diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index b3015985b334..2013d959f2bd 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -54,6 +54,7 @@ public static void main(String[] argsArray) throws Exception { String className = args.remove(0); boolean printLaunchCommand; + boolean printUsage; AbstractCommandBuilder builder; try { if (className.equals("org.apache.spark.deploy.SparkSubmit")) { @@ -62,9 +63,11 @@ public static void main(String[] argsArray) throws Exception { builder = new SparkClassCommandBuilder(className, args); } printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); + printUsage = false; } catch (IllegalArgumentException e) { - builder = new UsageLauncher(); + builder = new UsageCommandBuilder(e.getMessage()); printLaunchCommand = false; + printUsage = true; } Map env = new HashMap(); @@ -75,7 +78,13 @@ public static void main(String[] argsArray) throws Exception { } if (isWindows()) { - System.out.println(prepareWindowsCommand(cmd, env)); + // When printing the usage message, we can't use "cmd /v" since that prevents the env + // variable from being seen in the caller script. So do not call prepareWindowsCommand(). + if (printUsage) { + System.out.println(join(" ", cmd)); + } else { + System.out.println(prepareWindowsCommand(cmd, env)); + } } else { // In bash, use NULL as the arg separator since it cannot be used in an argument. List bashCmd = prepareBashCommand(cmd, env); @@ -133,23 +142,29 @@ private static List prepareBashCommand(List cmd, Map buildCommand(Map env) { if (isWindows()) { - return Arrays.asList("set", "SPARK_LAUNCHER_USAGE_ERROR=1"); + return Arrays.asList("set", "SPARK_LAUNCHER_USAGE_ERROR=" + message); } else { - return Arrays.asList("usage", "1"); + return Arrays.asList("usage", message, "1"); } } From 00505f99ca5b1a7ee8d7955bde54f7c1fcf27689 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 26 Feb 2015 13:16:18 -0800 Subject: [PATCH 46/51] Add blurb about new API in the programming guide. --- docs/programming-guide.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 4e4af7631686..5575ae53a376 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1366,6 +1366,11 @@ The [application submission guide](submitting-applications.html) describes how t In short, once you package your application into a JAR (for Java/Scala) or a set of `.py` or `.zip` files (for Python), the `bin/spark-submit` script lets you submit it to any supported cluster manager. +# Launching Spark jobs from Java / Scala + +The [org.apache.spark.launcher](api/java/index.html?org/apache/spark/launcher/package-summary.html) +package provides classes for launching Spark jobs as child processes using a simple Java API. + # Unit Testing Spark is friendly to unit testing with any popular unit test framework. From 28cd35eb7d7c065f679ef4749e599ad4d31ee5cf Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 2 Mar 2015 15:19:34 -0800 Subject: [PATCH 47/51] Remove stale comment. --- .../src/main/java/org/apache/spark/launcher/SparkLauncher.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 81ddbe9194ef..b566507ee606 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -30,9 +30,6 @@ *

    * Use this class to start Spark applications programmatically. The class uses a builder pattern * to allow clients to configure the Spark application and launch it as a child process. - *

    - * Note that launching Spark applications using this class will not automatically load environment - * variables from the "spark-env.sh" or "spark-env.cmd" scripts in the configuration directory. */ public class SparkLauncher { From 897141fa7bc521f95f42729fca89467b46fbf412 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 9 Mar 2015 13:28:45 -0700 Subject: [PATCH 48/51] Review feedback. --- bin/spark-shell | 2 +- .../src/main/java/org/apache/spark/launcher/Main.java | 6 +++--- .../spark/launcher/SparkSubmitCommandBuilder.java | 10 +++++----- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index e2da80a962d5..ed436986b21d 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -41,7 +41,7 @@ usage() { export -f usage if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - usage 0 + usage "" 0 fi # SPARK-4161: scala does not assume use of the java classpath, diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index 2013d959f2bd..206acfb514d8 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -44,8 +44,8 @@ class Main { * "bin/spark-class2.cmd" batch script on Windows to execute the final command. *

    * On Unix-like systems, the output is a list of command arguments, separated by the NULL - * character. On Windows, the output is single command line suitable for direct execution - * form the script. + * character. On Windows, the output is a command line suitable for direct execution from the + * script. */ public static void main(String[] argsArray) throws Exception { checkArgument(argsArray.length > 0, "Not enough arguments: missing class name."); @@ -139,7 +139,7 @@ private static List prepareBashCommand(List cmd, Map - * This launcher extends SparkLauncher to add command line parsing compatible with - * SparkSubmit. It handles setting driver-side options and special parsing needed - * for the different specialClasses. + * This builder adds command line parsing compatible with SparkSubmit. It handles setting + * driver-side options and special parsing behavior needed for the special-casing certain internal + * Spark applications. *

    - * This class has also some special features to aid PySparkLauncher. + * This class has also some special features to aid launching pyspark. */ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { From 3b28a7537ed184c88994be176fa1c66d47b43a8e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 9 Mar 2015 13:41:42 -0700 Subject: [PATCH 49/51] Update new pom. --- launcher/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/launcher/pom.xml b/launcher/pom.xml index 2202de4a84a9..ccbd9d0419a9 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -21,7 +21,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml From 2ce741ff59c17f64ffea1a0fd91eb922e4e87204 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 9 Mar 2015 15:17:46 -0700 Subject: [PATCH 50/51] Add lots of quotes. Even to places that weren't really changes I made. --- bin/spark-class | 2 +- bin/spark-shell | 4 ++-- bin/spark-sql | 8 ++++---- bin/spark-submit | 4 ++-- sbin/spark-daemon.sh | 18 +++++++++--------- 5 files changed, 18 insertions(+), 18 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index cd35095db27b..e29b234afaf9 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -77,7 +77,7 @@ fi CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") -done < <($RUNNER -cp "$SPARK_LAUNCHER_CP" org.apache.spark.launcher.Main "$@") +done < <("$RUNNER" -cp "$SPARK_LAUNCHER_CP" org.apache.spark.launcher.Main "$@") if [ "${CMD[0]}" = "usage" ]; then "${CMD[@]}" diff --git a/bin/spark-shell b/bin/spark-shell index ed436986b21d..b3761b5e1375 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -32,11 +32,11 @@ export FWDIR="$(cd "`dirname "$0"`"/..; pwd)" usage() { if [ -n "$1" ]; then - echo $1 + echo "$1" fi echo "Usage: ./bin/spark-shell [options]" "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit $2 + exit "$2" } export -f usage diff --git a/bin/spark-sql b/bin/spark-sql index b67b16c03639..ca1729f4cfcb 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -32,7 +32,7 @@ export FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage { if [ -n "$1" ]; then - echo $1 + echo "$1" fi echo "Usage: ./bin/spark-sql [options] [cli option]" pattern="usage" @@ -45,8 +45,8 @@ function usage { "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo echo "CLI options:" - "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 - exit $2 + "$FWDIR"/bin/spark-class "$CLASS" --help 2>&1 | grep -v "$pattern" 1>&2 + exit "$2" } export -f usage @@ -54,4 +54,4 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage "" 0 fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "$@" +exec "$FWDIR"/bin/spark-submit --class "$CLASS" "$@" diff --git a/bin/spark-submit b/bin/spark-submit index 91a40cf3cdcf..bcff78edd51c 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -23,10 +23,10 @@ SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" if ! type -t usage >/dev/null 2>&1; then usage() { if [ -n "$1" ]; then - echo $1 + echo "$1" fi "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit --help - exit $2 + exit "$2" } export -f usage fi diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 325fe2935db5..92e76a3fe6ca 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -122,12 +122,12 @@ if [ "$SPARK_NICENESS" = "" ]; then fi run_command() { - mode=$1 + mode="$1" shift mkdir -p "$SPARK_PID_DIR" - if [ -f $pid ]; then + if [ -f "$pid" ]; then TARGET_ID="$(cat "$pid")" if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then echo "$command running as process $TARGET_ID. Stop it first." @@ -137,21 +137,21 @@ run_command() { if [ "$SPARK_MASTER" != "" ]; then echo rsync from "$SPARK_MASTER" - rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME" + rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' "$SPARK_MASTER/" "$SPARK_HOME" fi spark_rotate_log "$log" echo "starting $command, logging to $log" - case $mode in + case "$mode" in (class) - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & - newpid=$! + nohup nice -n "$SPARK_NICENESS" "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + newpid="$!" ;; (submit) - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-submit --class $command "$@" >> "$log" 2>&1 < /dev/null & - newpid=$! + nohup nice -n "$SPARK_NICENESS" "$SPARK_PREFIX"/bin/spark-submit --class $command "$@" >> "$log" 2>&1 < /dev/null & + newpid="$!" ;; (*) @@ -160,7 +160,7 @@ run_command() { ;; esac - echo $newpid > $pid + echo "$newpid" > "$pid" sleep 2 # Check if the process has died; in that case we'll tail the log so the user can see if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then From 18c7e4db3b9713c4bc13487e3a15e59b6bf2dc58 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 10 Mar 2015 16:22:40 -0700 Subject: [PATCH 51/51] Fix make-distribution.sh. --- make-distribution.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 11023a3b4654..82d33408cd5e 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -127,6 +127,7 @@ if [ ! $(command -v "$MVN") ] ; then fi VERSION=$("$MVN" help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version 2>/dev/null | grep -v "INFO" | tail -n 1) SPARK_HADOOP_VERSION=$("$MVN" help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ | grep -v "INFO"\ | tail -n 1) @@ -196,7 +197,7 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy jars cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" -cp "$SPARK_HOME"/launcher/target/scala*/spark-launcher*.jar "$DISTDIR/lib/" +cp "$SPARK_HOME"/launcher/target/spark-launcher_$SCALA_VERSION-$VERSION.jar "$DISTDIR/lib/" # This will fail if the -Pyarn profile is not provided # In this case, silence the error and ignore the return code of this command cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :