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