From ac2d65e9299109759ee9b46687acee2cac5b276c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 31 Mar 2014 13:16:46 -0700 Subject: [PATCH 01/31] Change spark.local.dir -> SPARK_LOCAL_DIRS --- conf/spark-env.sh.template | 2 ++ core/src/main/scala/org/apache/spark/SparkConf.scala | 9 +++++++++ .../main/scala/org/apache/spark/executor/Executor.scala | 5 +++-- docs/configuration.md | 6 ++++-- 4 files changed, 18 insertions(+), 4 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 6432a566089be..52ffaaa64befb 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -5,6 +5,8 @@ # # The following variables can be set in this file: # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public dns name of the driver program +# - SPARK_LOCAL_DIRS, shuffle directories to use on this node # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that # we recommend setting app-wide options in the application's driver program. diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b947feb891ee6..4135d6bb666d0 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -208,6 +208,15 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } + /** Print any necessary deprecation warnings based on the values set in this configuration. */ + private[spark] def printDeprecationWarnings() { + if (settings.contains("spark.local.dir")) { + val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + + "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone and LOCAL_DIRS in YARN)." + logWarning(msg) + } + } + /** * Return a string listing all keys and values, one per line. This is useful to print the * configuration out for debugging. 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 f89b2bffd1676..2bfb9c387e1c9 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -64,9 +64,10 @@ private[spark] class Executor( // to what Yarn on this system said was available. This will be used later when SparkEnv // created. if (java.lang.Boolean.valueOf( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) - { + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { conf.set("spark.local.dir", getYarnLocalDirs()) + } else if (sys.env.contains("SPARK_LOCAL_DIRS")) { + conf.set("spark.local.dir", sys.env("SPARK_LOCAL_DIRS")) } if (!isLocal) { diff --git a/docs/configuration.md b/docs/configuration.md index f3bfd036f4164..bc5e2f9e8167f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -73,6 +73,9 @@ there are at least five properties that you will commonly want to control: Directory to use for "scratch" space in Spark, including map output files and RDDs that get stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. + + NOTE: In Spark 1.0 and later this will be overriden by SPARK_LOCAL_DIRS (Standalone, Mesos) or + LOCAL_DIRS (YARN) envrionment variables set by the cluster manager. @@ -671,8 +674,7 @@ The following variables can be set in `spark-env.sh`: Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend doing that when possible. * `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system - properties that you'd like to pass with `-D`. One use case is to set some Spark properties differently on this - machine, e.g., `-Dspark.local.dir=/disk1,/disk2`. + properties that you'd like to pass with `-D`. * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. From 0faa3b6ff0e0b37b18fdde3d6a6110459ffd8f28 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 00:13:52 -0700 Subject: [PATCH 02/31] Stash of adding config options in submit script and YARN --- .../org/apache/spark/deploy/SparkSubmit.scala | 36 +++++++++++++++---- .../spark/deploy/SparkSubmitArguments.scala | 18 +++++++++- .../apache/spark/deploy/yarn/ClientBase.scala | 16 +++++++-- .../deploy/yarn/ExecutorRunnableUtil.scala | 6 ++-- .../cluster/YarnClientClusterScheduler.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +-- .../spark/deploy/yarn/ExecutorLauncher.scala | 2 +- 7 files changed, 69 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e05fbfe321495..33444421a2cf6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,11 +17,13 @@ package org.apache.spark.deploy -import java.io.{PrintStream, File} +import java.io.{FileInputStream, PrintStream, File} import java.net.URL +import java.util.Properties import org.apache.spark.executor.ExecutorURLClassLoader +import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.Map @@ -108,6 +110,21 @@ object SparkSubmit { val sysProps = new HashMap[String, String]() var childMainClass = "" + // Load system properties by default from the file, if present + if (appArgs.verbose) printStream.println(s"Using properties file: ${appArgs.propertiesFile}") + Option(appArgs.propertiesFile).map { filename => + val file = new File(filename) + getDefaultProperties(file).foreach { case (k, v) => + if (k.startsWith("spark")) { + sysProps(k) = v + if (appArgs.verbose) printStream.println(s"Adding default property: $k=$v") + } + else { + printWarning(s"Ignoring non-spark config property: $k=$v") + } + } + } + if (clusterManager == MESOS && deployOnCluster) { printErrorAndExit("Mesos does not support running the driver on the cluster") } @@ -191,11 +208,11 @@ object SparkSubmit { sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { if (verbose) { - System.err.println(s"Main class:\n$childMainClass") - System.err.println(s"Arguments:\n${childArgs.mkString("\n")}") - System.err.println(s"System properties:\n${sysProps.mkString("\n")}") - System.err.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") - System.err.println("\n") + printStream.println(s"Main class:\n$childMainClass") + printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") + printStream.println(s"System properties:\n${sysProps.mkString("\n")}") + printStream.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") + printStream.println("\n") } val loader = new ExecutorURLClassLoader(new Array[URL](0), @@ -224,6 +241,13 @@ object SparkSubmit { val url = localJarFile.getAbsoluteFile.toURI.toURL loader.addURL(url) } + + private def getDefaultProperties(file: File): Seq[(String, String)] = { + val inputStream = new FileInputStream(file) + val properties = new Properties() + properties.load(inputStream) + properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) + } } private[spark] class OptionAssigner(val value: String, 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 834b3df2f164b..fe69282c03019 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy import scala.collection.mutable.ArrayBuffer +import java.io.File /** * Parses and encapsulates arguments from the spark-submit script. @@ -28,6 +29,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { var executorMemory: String = null var executorCores: String = null var totalExecutorCores: String = null + var propertiesFile: String = null var driverMemory: String = null var driverCores: String = null var supervise: Boolean = false @@ -49,6 +51,15 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (args.length == 0) printUsageAndExit(-1) if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + if (propertiesFile == null) { + val sparkHome = sys.env("SPARK_HOME") // defined via `spark-class` + val sep = File.separator + val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.properties" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } override def toString = { s"""Parsed arguments: @@ -57,8 +68,9 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | executorMemory $executorMemory | executorCores $executorCores | totalExecutorCores $totalExecutorCores + | propertiesFile $propertiesFile | driverMemory $driverMemory - | drivercores $driverCores + | driverCores $driverCores | supervise $supervise | queue $queue | numExecutors $numExecutors @@ -122,6 +134,10 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { driverCores = value parseOpts(tail) + case ("--properties-file") :: value :: tail => + propertiesFile = value + parseOpts(tail) + case ("--supervise") :: tail => supervise = true parseOpts(tail) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index eb42922aea228..1047630e69f95 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.util.{Records, Apps} import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.ExecutorLauncher import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -340,8 +341,19 @@ trait ClientBase extends Logging { JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " } - if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") + + if (args.amClass == classOf[ExecutorLauncher].getName) { + // If we are being launched in client mode, forward the spark-conf options + // onto the executor launcher + for ((k, v) <- sparkConf.getAll) { + JAVA_OPTS += s"-D$k=$v" + } + } else { + // If we are being launched in standalone mode, capture and forward any spark + // system properties (e.g. set by spark-class). + for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { + JAVA_OPTS += s"-D$k=$v" + } } if (!localResources.contains(ClientBase.LOG4J_PROP)) { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index b3696c5fe7183..07d6fc66b2582 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -57,8 +57,10 @@ trait ExecutorRunnableUtil extends Logging { // Set the JVM memory val executorMemoryString = executorMemory + "m" JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " - if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + + /* Pass on Spark properties to the driver. */ + for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { + JAVA_OPTS += s"-D$k=$v" } JAVA_OPTS += " -Djava.io.tmpdir=" + diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala index 6b91e6b9eb899..17b1632ab0174 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils */ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) { - def this(sc: SparkContext) = this(sc, new Configuration()) + def this(sc: SparkContext) = this(sc, sc.getConf) // By default, rack is unknown override def getRackForHost(hostPort: String): Option[String] = { diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 161918859e7c4..ce2dde0631ed9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments} +import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -54,7 +54,7 @@ private[spark] class YarnClientSchedulerBackend( "--class", "notused", "--jar", null, "--args", hostport, - "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher" + "--am-class", classOf[ExecutorLauncher].getName ) // process any optional arguments, given either as environment variables diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index b697f103914fd..c0b130025c3af 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -237,7 +237,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } def finishApplicationMaster(status: FinalApplicationStatus) { - logInfo("finish ApplicationMaster with " + status) + logInfo("finish ApplicationEMaster with " + status) amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */) } From 6eaf7d08ee8d98d4d17f4e09745be17b81150cfa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 15:00:42 -0700 Subject: [PATCH 03/31] executorJavaOpts --- bin/compute-classpath.sh | 3 ++- .../main/scala/org/apache/spark/SparkConf.scala | 15 +++++++++++++-- .../cluster/SparkDeploySchedulerBackend.scala | 6 ++++-- .../mesos/CoarseMesosSchedulerBackend.scala | 10 ++++++---- docs/configuration.md | 9 +++++++++ .../spark/deploy/yarn/ExecutorRunnableUtil.scala | 7 +++---- 6 files changed, 37 insertions(+), 13 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 2a2bb376fd71f..b2657f205e7f1 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -32,9 +32,10 @@ CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" + +CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" # First check if we have a dependencies jar. If so, include binary classes with the deps jar if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then - CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 4135d6bb666d0..607ea04dda696 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -208,13 +208,24 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } - /** Print any necessary deprecation warnings based on the values set in this configuration. */ - private[spark] def printDeprecationWarnings() { + /** Checks for illegal or deprecated config settings. Throws an exception for the former. */ + private[spark] def validateSettings() { if (settings.contains("spark.local.dir")) { val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone and LOCAL_DIRS in YARN)." logWarning(msg) } + val executorOptsKey = "spark.executor.extraJavaOptions" + settings.get(executorOptsKey).map { javaOpts => + if (javaOpts.contains("-Dspark")) { + val msg = s"$executorOptsKey is not allowed to set Spark options. Was '$javaOpts'" + throw new Exception(msg) + } + if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { + val msg = s"$executorOptsKey is not allowed to alter memory settings. Was '$javaOpts'" + throw new Exception(msg) + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 936e9db80573d..3d3067b130069 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -42,9 +42,11 @@ private[spark] class SparkDeploySchedulerBackend( // The endpoint for executors to talk to us val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val extraOpts = sc.conf.get("spark.executor.extraJavaOptions", "null") + val args = Seq(extraOpts, driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", + "{{CORES}}", "{{WORKER_URL}}") val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 06b041e1fd9a9..230f9e10d3096 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -123,20 +123,22 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) + val extraOpts = conf.get("spark.executor.extraJavaOptions") val uri = conf.get("spark.executor.uri", null) if (uri == null) { val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d".format( + runScript, extraOpts, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( ("cd %s*; " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d") - .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d") + .format(basename, extraOpts, driverUrl, offer.getSlaveId.getValue, + offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } command.build() diff --git a/docs/configuration.md b/docs/configuration.md index bc5e2f9e8167f..cd9f1a2d2cf7f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -646,6 +646,15 @@ Apart from these, the following properties are also available, and may be useful Number of cores to allocate for each task. + + spark.executor.extraJavaOptions + (none) + + A string of extra JVM options to pass to executors. For instance, GC settings. Note that + it is illegal to set Spark properties or heap size settings with this flag. + + + ## Viewing Spark Properties diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 07d6fc66b2582..37c38a6b5b775 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -58,10 +58,9 @@ trait ExecutorRunnableUtil extends Logging { val executorMemoryString = executorMemory + "m" JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " - /* Pass on Spark properties to the driver. */ - for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { - JAVA_OPTS += s"-D$k=$v" - } + // Set extra Java options for the executor + val executorOpts = sys.props.find(_._1.contains("spark.executor.extraJavaOptions")) + JAVA_OPTS += executorOpts JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " From 49823312ed64a15b6ec9b58e50028c296b09d3b1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 15:10:31 -0700 Subject: [PATCH 04/31] Remove SPARK_LIBRARY_PATH --- bin/run-example | 1 - bin/spark-class | 1 - docs/configuration.md | 6 +++--- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/bin/run-example b/bin/run-example index 5af95a08c6c41..b2999198a8d41 100755 --- a/bin/run-example +++ b/bin/run-example @@ -75,7 +75,6 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$SPARK_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" # 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`" diff --git a/bin/spark-class b/bin/spark-class index 1b0d309cc5b1c..ea97e3d7768a7 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -98,7 +98,6 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" 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 diff --git a/docs/configuration.md b/docs/configuration.md index cd9f1a2d2cf7f..642d4644b37b0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -650,8 +650,9 @@ Apart from these, the following properties are also available, and may be useful spark.executor.extraJavaOptions (none) - A string of extra JVM options to pass to executors. For instance, GC settings. Note that - it is illegal to set Spark properties or heap size settings with this flag. + A string of extra JVM options to pass to executors. For instance, GC settings or custom + paths for native code. Note that it is illegal to set Spark properties or heap size + settings with this option. @@ -678,7 +679,6 @@ The following variables can be set in `spark-env.sh`: * `JAVA_HOME`, the location where Java is installed (if it's not on your default `PATH`) * `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. -* `SPARK_LIBRARY_PATH`, to add search directories for native libraries. * `SPARK_CLASSPATH`, to add elements to Spark's classpath that you want to be present for _all_ applications. Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend doing that when possible. From 1f75238f8f9e462a97a906109fdacad513ce8b20 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 15:18:42 -0700 Subject: [PATCH 05/31] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings --- conf/spark-env.sh.template | 1 + docs/configuration.md | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 52ffaaa64befb..b91df8c7f2374 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -16,6 +16,7 @@ # If using the standalone deploy mode, you can also set variables for it here: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports +# - SPARK_MASTER_OPTS, to set config properties at the master (e.g "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT diff --git a/docs/configuration.md b/docs/configuration.md index 642d4644b37b0..0d872010d5e3e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -575,7 +575,7 @@ Apart from these, the following properties are also available, and may be useful to consolidate them onto as few nodes as possible. Spreading out is usually better for data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. @@ -588,7 +588,7 @@ Apart from these, the following properties are also available, and may be useful Set this lower on a shared cluster to prevent users from grabbing the whole cluster by default.
Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. From 84cc5e504b2b35611f95601554ea59fe98cb70a3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 16:31:11 -0700 Subject: [PATCH 06/31] Small clean-up --- core/src/main/scala/org/apache/spark/SparkConf.scala | 4 +++- .../scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 607ea04dda696..e81fd68f18713 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -52,6 +52,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } } + validateSettings() + /** Set a configuration variable. */ def set(key: String, value: String): SparkConf = { if (key == null) { @@ -209,7 +211,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } /** Checks for illegal or deprecated config settings. Throws an exception for the former. */ - private[spark] def validateSettings() { + private def validateSettings() { if (settings.contains("spark.local.dir")) { val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone and LOCAL_DIRS in YARN)." diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index c0b130025c3af..b697f103914fd 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -237,7 +237,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } def finishApplicationMaster(status: FinalApplicationStatus) { - logInfo("finish ApplicationEMaster with " + status) + logInfo("finish ApplicationMaster with " + status) amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */) } From 5b0ba8eddd97e8a718f0f24e15aea2a0a0386b8c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 17:35:13 -0700 Subject: [PATCH 07/31] Don't ship executor envs --- core/src/main/scala/org/apache/spark/SparkContext.scala | 5 +---- .../org/apache/spark/deploy/worker/CommandUtils.scala | 8 +------- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 5 +++-- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5a36e6f5c19a9..0382a34e28074 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -248,10 +248,7 @@ class SparkContext(config: SparkConf) extends Logging { // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() - for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS"); - value <- Option(System.getenv(key))) { - executorEnvs(key) = value - } + // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) 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 0c761dfc93a1f..0a7d5b25810a5 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 @@ -46,12 +46,6 @@ object CommandUtils extends Logging { * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command) - .map(p => List("-Djava.library.path=" + p)) - .getOrElse(Nil) - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")) - .map(Utils.splitCommandString).getOrElse(Nil) - val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") // Figure out our classpath with the external compute-classpath script @@ -60,7 +54,7 @@ object CommandUtils extends Logging { Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=command.environment) - Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts + Seq("-cp", classPath) ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 3d3067b130069..14cf8780feb15 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -44,9 +44,10 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val extraOpts = sc.conf.get("spark.executor.extraJavaOptions", "null") - val args = Seq(extraOpts, driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", + val args = sc.conf.get("spark.executor.extraJavaOptions").split(" ") ++ + Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() From 7cc70e4b1fbe2b00c160510ddd04653658a0d776 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 18:02:43 -0700 Subject: [PATCH 08/31] Clean up terminology inside of spark-env script --- conf/spark-env.sh.template | 23 +++++++++++++---------- docs/configuration.md | 10 ++++------ 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index b91df8c7f2374..b4e24ba720f56 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -1,19 +1,22 @@ #!/usr/bin/env bash -# This file contains environment variables required to run Spark. Copy it as -# spark-env.sh and edit that to configure Spark for your site. -# -# The following variables can be set in this file: +# This file is sourced when running various Spark classes. +# Copy it as spark-env.sh and edit that to configure Spark for your site. + +# Options read when launching programs locally with +# ./bin/spark-example or ./bin/spark-submit # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public dns name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append + +# Options read by executors and drivers running inside the cluster +# - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program # - SPARK_LOCAL_DIRS, shuffle directories to use on this node # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos -# - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that -# we recommend setting app-wide options in the application's driver program. -# Examples of node-specific options : -Dspark.local.dir, GC options -# Examples of app-wide options : -Dspark.serializer -# -# If using the standalone deploy mode, you can also set variables for it here: +# - SPARK_CLASSPATH, default classpath entries to append + +# Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports # - SPARK_MASTER_OPTS, to set config properties at the master (e.g "-Dx=y") diff --git a/docs/configuration.md b/docs/configuration.md index 0d872010d5e3e..4d378b2e2b195 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -666,10 +666,9 @@ This is a useful place to check to make sure that your properties have been set # Environment Variables Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh` -script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). These variables are meant to be for machine-specific settings, such -as library search paths. While Spark properties can also be set there through `SPARK_JAVA_OPTS`, for per-application settings, we recommend setting -these properties within the application instead of in `spark-env.sh` so that different applications can use different -settings. +script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). In Standalone and Mesos modes, +this file can give machine specific information such as hostnames. It is also sourced when running local +Spark applications or submission scripts. Note that `conf/spark-env.sh` does not exist by default when Spark is installed. However, you can copy `conf/spark-env.sh.template` to create it. Make sure you make the copy executable. @@ -682,8 +681,7 @@ The following variables can be set in `spark-env.sh`: * `SPARK_CLASSPATH`, to add elements to Spark's classpath that you want to be present for _all_ applications. Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend doing that when possible. -* `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system - properties that you'd like to pass with `-D`. +* `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. From 761ebcd042123484f4dbd243b18978d86f1cdf86 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 22:45:59 -0700 Subject: [PATCH 09/31] Library path and classpath for drivers --- bin/spark-class | 1 + bin/spark-submit | 7 ++- conf/spark-env.sh.template | 11 ++++ .../org/apache/spark/deploy/Client.scala | 9 ++- .../org/apache/spark/deploy/Command.scala | 5 +- .../org/apache/spark/deploy/SparkSubmit.scala | 11 +++- .../spark/deploy/SparkSubmitArguments.scala | 62 +++++++++++++------ .../spark/deploy/client/TestClient.scala | 4 +- .../spark/deploy/worker/CommandUtils.scala | 11 +++- .../spark/deploy/worker/DriverRunner.scala | 16 +++-- .../spark/deploy/worker/ExecutorRunner.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 4 +- .../spark/deploy/JsonProtocolSuite.scala | 4 +- .../apache/spark/deploy/yarn/ClientBase.scala | 3 + 14 files changed, 115 insertions(+), 37 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index ea97e3d7768a7..6871e180c9fa8 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -98,6 +98,7 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" +JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$_SPARK_LIBRARY_PATH" 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 diff --git a/bin/spark-submit b/bin/spark-submit index d92d55a032bd5..498d0b27bacdf 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -25,8 +25,13 @@ while (($#)); do DEPLOY_MODE=$2 elif [ $1 = "--driver-memory" ]; then DRIVER_MEMORY=$2 + elif [ $1 = "--driver-library-path" ]; then + export _SPARK_LIBRARY_PATH=$2 + elif [ $1 = "--driver-class-path" ]; then + export SPARK_CLASSPATH="$SPARK_CLASSPATH:$2" + elif [ $1 = "--driver-java-options" ]; then + export SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $2" fi - shift done diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index b4e24ba720f56..e46cabf825e61 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -16,6 +16,17 @@ # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # - SPARK_CLASSPATH, default classpath entries to append +# Options read in YARN client mode +# - SPARK_YARN_APP_JAR, Path to your application’s JAR file (required) +# - SPARK_WORKER_INSTANCES, Number of workers to start (Default: 2) +# - SPARK_WORKER_CORES, Number of cores for the workers (Default: 1). +# - SPARK_WORKER_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) +# - SPARK_MASTER_MEMORY, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +# - SPARK_YARN_APP_NAME, The name of your application (Default: Spark) +# - SPARK_YARN_QUEUE, The hadoop queue to use for allocation requests (Default: ‘default’) +# - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job. +# - SPARK_YARN_DIST_ARCHIVES, Comma separated list of archives to be distributed with the job. + # Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 8fd2c7e95b966..98cdb010e9b65 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -54,8 +54,15 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends System.getenv().foreach{case (k, v) => env(k) = v} val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" + val classPathEntries = sys.props.get("spark.driver.classPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val libraryPathEntries = sys.props.get("spark.driver.libraryPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val javaOpts = sys.props.get("spark.driver.javaOpts").toSeq val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ - driverArgs.driverOptions, env) + driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) val driverDescription = new DriverDescription( driverArgs.jarUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala index fa8af9a646750..9ef554f2bd9b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -22,5 +22,8 @@ import scala.collection.Map private[spark] case class Command( mainClass: String, arguments: Seq[String], - environment: Map[String, String]) { + environment: Map[String, String], + classPathEntries: Seq[String], + libraryPathEntries: Seq[String], + javaOptions: Seq[String]) { } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 33444421a2cf6..d09eef210dabe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -65,7 +65,8 @@ object SparkSubmit { /** * @return * a tuple containing the arguments for the child, a list of classpath - * entries for the child, and the main class for the child + * entries for the child, a list of system propertes, a list of env vars + * and the main class for the child */ private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { @@ -140,6 +141,14 @@ object SparkSubmit { val options = List[OptionAssigner]( new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + + new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, + sysProp = "spark.driver.classPath"), + new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, + sysProp = "spark.driver.javaOpts"), + new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, + sysProp = "spark.driver.libraryPath"), + new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), 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 fe69282c03019..d4747e64980ee 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -31,6 +31,9 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { var totalExecutorCores: String = null var propertiesFile: String = null var driverMemory: String = null + var driverExtraClassPath: String = null + var driverExtraLibraryPath: String = null + var driverExtraJavaOptions: String = null var driverCores: String = null var supervise: Boolean = false var queue: String = null @@ -63,25 +66,28 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { override def toString = { s"""Parsed arguments: - | master $master - | deployMode $deployMode - | executorMemory $executorMemory - | executorCores $executorCores - | totalExecutorCores $totalExecutorCores - | propertiesFile $propertiesFile - | driverMemory $driverMemory - | driverCores $driverCores - | supervise $supervise - | queue $queue - | numExecutors $numExecutors - | files $files - | archives $archives - | mainClass $mainClass - | primaryResource $primaryResource - | name $name - | childArgs [${childArgs.mkString(" ")}] - | jars $jars - | verbose $verbose + | master $master + | deployMode $deployMode + | executorMemory $executorMemory + | executorCores $executorCores + | totalExecutorCores $totalExecutorCores + | propertiesFile $propertiesFile + | driverMemory $driverMemory + | driverCores $driverCores + | driverExtraClassPath $driverExtraClassPath + | driverExtraLibraryPath $driverExtraLibraryPath + | driverExtraJavaOptions $driverExtraJavaOptions + | supervise $supervise + | queue $queue + | numExecutors $numExecutors + | files $files + | archives $archives + | mainClass $mainClass + | primaryResource $primaryResource + | name $name + | childArgs [${childArgs.mkString(" ")}] + | jars $jars + | verbose $verbose """.stripMargin } @@ -134,6 +140,18 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { driverCores = value parseOpts(tail) + case ("--driver-class-path") :: value :: tail => + driverExtraClassPath = value + parseOpts(tail) + + case ("--driver-java-opts") :: value :: tail => + driverExtraJavaOptions = value + parseOpts(tail) + + case ("--driver-library-path") :: value :: tail => + driverExtraLibraryPath = value + parseOpts(tail) + case ("--properties-file") :: value :: tail => propertiesFile = value parseOpts(tail) @@ -194,11 +212,15 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | --class CLASS_NAME Name of your app's main class (required for Java apps). | --arg ARG Argument to be passed to your application's main class. This | option can be specified multiple times for multiple args. - | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). | --name NAME The name of your application (Default: 'Spark'). | --jars JARS A comma-separated list of local jars to include on the | driver classpath and that SparkContext.addJar will work | with. Doesn't work on standalone with 'cluster' deploy mode. + | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). + | --driver-java-opts Extra Java options to pass to the driver + | --driver-library-path Extra library path entries to pass to the driver + | --driver-class-path Extra class path entries to pass to the driver + | | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 63f166d401059..f2991e1bd2359 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -49,8 +49,8 @@ private[spark] object TestClient { val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription( - "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), - Some("dummy-spark-home"), "ignored") + "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), + Seq(), Seq()), Some("dummy-spark-home"), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() 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 0a7d5b25810a5..053828111fd96 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 @@ -47,14 +47,23 @@ object CommandUtils extends Logging { */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") + val libraryOpts = + if (command.libraryPathEntries.size > 0) { + val joined = command.libraryPathEntries.mkString(File.pathSeparator) + Seq(s"-Djava.library.path=$joined") + } else { + Seq() + } // 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.mkString(File.pathSeparator) + val classPathWithUser = classPath + File.pathSeparator + userClassPath - Seq("-cp", classPath) ++ memoryOpts + Seq("-cp", classPathWithUser) ++ libraryOpts ++ memoryOpts ++ command.javaOptions } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index b4df1a0dd4718..eed9ed93480b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker import java.io._ import scala.collection.JavaConversions._ -import scala.collection.mutable.Map +import scala.collection.Map import akka.actor.ActorRef import com.google.common.base.Charsets @@ -74,13 +74,17 @@ private[spark] class DriverRunner( // Make sure user application jar is on the classpath // TODO: If we add ability to submit multiple jars they should also be added here - val env = Map(driverDesc.command.environment.toSeq: _*) - env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename" - val newCommand = Command(driverDesc.command.mainClass, - driverDesc.command.arguments.map(substituteVariables), env) + val classPath = driverDesc.command.classPathEntries ++ Seq(s":$localJarFilename") + val newCommand = Command( + driverDesc.command.mainClass, + driverDesc.command.arguments.map(substituteVariables), + driverDesc.command.environment, + classPath, + driverDesc.command.libraryPathEntries, + driverDesc.command.javaOptions) val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) - launchDriver(command, env, driverDir, driverDesc.supervise) + launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) } catch { case e: Exception => finalException = Some(e) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2edd921066876..2348c84807826 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -99,7 +99,9 @@ private[spark] class ExecutorRunner( def getCommandSeq = { val command = Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment) + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, + appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, + appDesc.command.javaOptions) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 14cf8780feb15..d8b2f9288054d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -48,8 +48,10 @@ private[spark] class SparkDeploySchedulerBackend( Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + // TODO (pwendell) LOOK AT THIS val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, + Seq(), Seq(), Seq()) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 9f2924c23b73c..d563db9ca06b9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -87,7 +87,7 @@ class JsonProtocolSuite extends FunSuite { } def createAppDesc(): ApplicationDescription = { - val cmd = new Command("mainClass", List("arg1", "arg2"), Map()) + val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq()) new ApplicationDescription("name", Some(4), 1234, cmd, Some("sparkHome"), "appUiUrl") } @@ -100,7 +100,7 @@ class JsonProtocolSuite extends FunSuite { def createDriverCommand() = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), - Map(("K1", "V1"), ("K2", "V2")) + Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Seq("-Dfoo") ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 1047630e69f95..19f161e274275 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -354,6 +354,9 @@ trait ClientBase extends Logging { for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { JAVA_OPTS += s"-D$k=$v" } + // TODO: honor driver classpath here: sys.props.get("spark.driver.classPath") + sys.props.get("spark.driver.javaOpts").map(opts => JAVA_OPTS += opts) + sys.props.get("spark.driver.libraryPath").map(p => JAVA_OPTS + s"-Djava.library.path=$p") } if (!localResources.contains(ClientBase.LOG4J_PROP)) { From 437aed140ca15528a4e413e200624361d33cadf5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 1 Apr 2014 23:14:45 -0700 Subject: [PATCH 10/31] Small fix --- bin/compute-classpath.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index b2657f205e7f1..2a2bb376fd71f 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -32,10 +32,9 @@ CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" - -CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" # First check if we have a dependencies jar. If so, include binary classes with the deps jar if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then + CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" From 46555c19f43ca63ba457f1de236404a4e287ba56 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 13 Apr 2014 13:27:27 -0700 Subject: [PATCH 11/31] Review feedback and import clean-ups --- docs/configuration.md | 6 ++--- .../apache/spark/deploy/yarn/ClientBase.scala | 23 +++++-------------- .../deploy/yarn/ExecutorRunnableUtil.scala | 19 +++++---------- 3 files changed, 15 insertions(+), 33 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 4d378b2e2b195..afd44fc7fe980 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -650,9 +650,9 @@ Apart from these, the following properties are also available, and may be useful spark.executor.extraJavaOptions (none) - A string of extra JVM options to pass to executors. For instance, GC settings or custom - paths for native code. Note that it is illegal to set Spark properties or heap size - settings with this option. + A string of extra JVM options to pass to executors. For instance, GC settings or other + logging. Note that it is illegal to set Spark properties or heap size settings with this + option. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 19f161e274275..91d478f7664a6 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -17,20 +17,16 @@ package org.apache.spark.deploy.yarn -import java.net.{InetAddress, UnknownHostException, URI} -import java.nio.ByteBuffer +import java.net.{InetAddress, URI, UnknownHostException} import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import scala.collection.mutable.{HashMap, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ @@ -38,15 +34,8 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Records, Apps} - +import org.apache.hadoop.yarn.util.{Apps, Records} import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.ExecutorLauncher -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment - /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The @@ -355,8 +344,8 @@ trait ClientBase extends Logging { JAVA_OPTS += s"-D$k=$v" } // TODO: honor driver classpath here: sys.props.get("spark.driver.classPath") - sys.props.get("spark.driver.javaOpts").map(opts => JAVA_OPTS += opts) - sys.props.get("spark.driver.libraryPath").map(p => JAVA_OPTS + s"-Djava.library.path=$p") + sys.props.get("spark.driver.javaOpts").foreach(opts => JAVA_OPTS += opts) + sys.props.get("spark.driver.libraryPath").foreach(p => JAVA_OPTS += s"-Djava.library.path=$p") } if (!localResources.contains(ClientBase.LOG4J_PROP)) { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 37c38a6b5b775..6b29e3249b835 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -18,26 +18,18 @@ package org.apache.spark.deploy.yarn import java.net.URI -import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} - -import org.apache.spark.{SparkConf, Logging} import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.spark.{Logging, SparkConf} trait ExecutorRunnableUtil extends Logging { @@ -58,9 +50,10 @@ trait ExecutorRunnableUtil extends Logging { val executorMemoryString = executorMemory + "m" JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " - // Set extra Java options for the executor - val executorOpts = sys.props.find(_._1.contains("spark.executor.extraJavaOptions")) - JAVA_OPTS += executorOpts + // Set extra Java options for the executor, if defined + sys.props.get("spark.executor.extraJavaOptions").foreach { opts => + JAVA_OPTS += opts + } JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " From b72d183154c3e40f2d1e531acef4ac454554ac81 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 13 Apr 2014 13:35:49 -0700 Subject: [PATCH 12/31] Review feedback for spark env file --- conf/spark-env.sh.template | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index e46cabf825e61..dd65669b94149 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -12,9 +12,9 @@ # Options read by executors and drivers running inside the cluster # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append # - SPARK_LOCAL_DIRS, shuffle directories to use on this node # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos -# - SPARK_CLASSPATH, default classpath entries to append # Options read in YARN client mode # - SPARK_YARN_APP_JAR, Path to your application’s JAR file (required) @@ -30,10 +30,13 @@ # Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports -# - SPARK_MASTER_OPTS, to set config properties at the master (e.g "-Dx=y") +# - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine -# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) +# - SPARK_WORKER_MEMORY, to set how much total memory to workers have to give executors (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes -# - SPARK_PUBLIC_DNS, to set the public dns name of the master +# - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") +# - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") +# - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y") +# - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers \ No newline at end of file From ace4ead12f55c95c6f3036ce343e9be4579e8039 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 13 Apr 2014 14:30:56 -0700 Subject: [PATCH 13/31] Responses to review feedback. --- conf/spark-env.sh.template | 14 +++++++------- .../main/scala/org/apache/spark/SparkConf.scala | 3 ++- .../scala/org/apache/spark/deploy/Client.scala | 2 +- .../scala/org/apache/spark/deploy/Command.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 10 ++++++++-- .../apache/spark/deploy/client/TestClient.scala | 2 +- .../apache/spark/deploy/worker/CommandUtils.scala | 4 +++- .../apache/spark/deploy/worker/DriverRunner.scala | 4 ++-- .../spark/deploy/worker/ExecutorRunner.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 6 +++--- .../org/apache/spark/deploy/yarn/ClientBase.scala | 3 +-- .../cluster/YarnClientClusterScheduler.scala | 2 +- 12 files changed, 31 insertions(+), 23 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index dd65669b94149..975501e6fbe4d 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -1,10 +1,10 @@ #!/usr/bin/env bash -# This file is sourced when running various Spark classes. +# This file is sourced when running various Spark programs. # Copy it as spark-env.sh and edit that to configure Spark for your site. # Options read when launching programs locally with -# ./bin/spark-example or ./bin/spark-submit +# ./bin/run-example or ./bin/spark-submit # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public dns name of the driver program # - SPARK_CLASSPATH, default classpath entries to append @@ -13,15 +13,15 @@ # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program # - SPARK_CLASSPATH, default classpath entries to append -# - SPARK_LOCAL_DIRS, shuffle directories to use on this node +# - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in YARN client mode # - SPARK_YARN_APP_JAR, Path to your application’s JAR file (required) -# - SPARK_WORKER_INSTANCES, Number of workers to start (Default: 2) -# - SPARK_WORKER_CORES, Number of cores for the workers (Default: 1). -# - SPARK_WORKER_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) -# - SPARK_MASTER_MEMORY, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +# - SPARK_EXECUTOR_INSTANCES, Number of workers to start (Default: 2) +# - SPARK_EXECUTOR_CORES, Number of cores for the workers (Default: 1). +# - SPARK_EXECUTOR_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) +# - SPARK_DRIVER_MEMORY, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) # - SPARK_YARN_APP_NAME, The name of your application (Default: Spark) # - SPARK_YARN_QUEUE, The hadoop queue to use for allocation requests (Default: ‘default’) # - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job. diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index e81fd68f18713..714f1f54b7a58 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -224,7 +224,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { throw new Exception(msg) } if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { - val msg = s"$executorOptsKey is not allowed to alter memory settings. Was '$javaOpts'" + val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). Please use " + + "spark.executor.memory." throw new Exception(msg) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 98cdb010e9b65..e6a0d404571e5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -60,7 +60,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends val libraryPathEntries = sys.props.get("spark.driver.libraryPath").toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val javaOpts = sys.props.get("spark.driver.javaOpts").toSeq + val javaOpts = sys.props.get("spark.driver.extraJavaOptions") val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala index 9ef554f2bd9b4..32f3ba385084f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -25,5 +25,5 @@ private[spark] case class Command( environment: Map[String, String], classPathEntries: Seq[String], libraryPathEntries: Seq[String], - javaOptions: Seq[String]) { + extraJavaOptions: Option[String] = None) { } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index d09eef210dabe..ff51ae0a37e24 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import java.io.{FileInputStream, PrintStream, File} +import java.io.{IOException, FileInputStream, PrintStream, File} import java.net.URL import java.util.Properties @@ -27,6 +27,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.Map +import org.apache.spark.SparkException /** * Scala code behind the spark-submit script. The script handles setting up the classpath with @@ -252,9 +253,14 @@ object SparkSubmit { } private def getDefaultProperties(file: File): Seq[(String, String)] = { + require(file.exists(), s"Default properties file ${file.getName} does not exist") val inputStream = new FileInputStream(file) val properties = new Properties() - properties.load(inputStream) + try { + properties.load(inputStream) + } catch { + case e: IOException => throw new SparkException(s"Failed when loading Spark properties file ${file.getName}", e) + } properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index f2991e1bd2359..888dd45e93c6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -50,7 +50,7 @@ private[spark] object TestClient { conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription( "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), - Seq(), Seq()), Some("dummy-spark-home"), "ignored") + Seq()), Some("dummy-spark-home"), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() 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 053828111fd96..9103c885fa96c 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 @@ -47,6 +47,8 @@ object CommandUtils extends Logging { */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") + // Note, this will coalesce multiple options into a single command component + val extraOpts = command.extraJavaOptions.toSeq val libraryOpts = if (command.libraryPathEntries.size > 0) { val joined = command.libraryPathEntries.mkString(File.pathSeparator) @@ -63,7 +65,7 @@ object CommandUtils extends Logging { val userClassPath = command.classPathEntries.mkString(File.pathSeparator) val classPathWithUser = classPath + File.pathSeparator + userClassPath - Seq("-cp", classPathWithUser) ++ libraryOpts ++ memoryOpts ++ command.javaOptions + Seq("-cp", classPathWithUser) ++ libraryOpts ++ extraOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index eed9ed93480b3..f918b42c83bc6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -74,14 +74,14 @@ private[spark] class DriverRunner( // Make sure user application jar is on the classpath // TODO: If we add ability to submit multiple jars they should also be added here - val classPath = driverDesc.command.classPathEntries ++ Seq(s":$localJarFilename") + val classPath = driverDesc.command.classPathEntries ++ Seq(s"$localJarFilename") val newCommand = Command( driverDesc.command.mainClass, driverDesc.command.arguments.map(substituteVariables), driverDesc.command.environment, classPath, driverDesc.command.libraryPathEntries, - driverDesc.command.javaOptions) + driverDesc.command.extraJavaOptions) val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2348c84807826..f94cd685e8eb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -101,7 +101,7 @@ private[spark] class ExecutorRunner( val command = Command(appDesc.command.mainClass, appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, - appDesc.command.javaOptions) + appDesc.command.extraJavaOptions) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index d8b2f9288054d..fe95b962d6ca2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -44,14 +44,14 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = sc.conf.get("spark.executor.extraJavaOptions").split(" ") ++ - Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") // TODO (pwendell) LOOK AT THIS val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, - Seq(), Seq(), Seq()) + Seq(), Seq(), extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 91d478f7664a6..54d77da196ed4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -330,7 +330,6 @@ trait ClientBase extends Logging { JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " } - if (args.amClass == classOf[ExecutorLauncher].getName) { // If we are being launched in client mode, forward the spark-conf options // onto the executor launcher @@ -344,7 +343,7 @@ trait ClientBase extends Logging { JAVA_OPTS += s"-D$k=$v" } // TODO: honor driver classpath here: sys.props.get("spark.driver.classPath") - sys.props.get("spark.driver.javaOpts").foreach(opts => JAVA_OPTS += opts) + sys.props.get("spark.driver.extraJavaOptions").foreach(opts => JAVA_OPTS += opts) sys.props.get("spark.driver.libraryPath").foreach(p => JAVA_OPTS += s"-Djava.library.path=$p") } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala index 17b1632ab0174..6b91e6b9eb899 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils */ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) { - def this(sc: SparkContext) = this(sc, sc.getConf) + def this(sc: SparkContext) = this(sc, new Configuration()) // By default, rack is unknown override def getRackForHost(hostPort: String): Option[String] = { From b08893b2257e132161400077aa3f3e92491d427e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 13 Apr 2014 16:43:49 -0700 Subject: [PATCH 14/31] Additional improvements. - Made driver & executor options consistent. - Some doc improvements to YARN code. - Handled special flags on YARN. --- .../scala/org/apache/spark/SparkConf.scala | 4 +- .../scala/org/apache/spark/SparkContext.scala | 15 ++--- .../org/apache/spark/deploy/Client.scala | 4 +- .../org/apache/spark/deploy/SparkSubmit.scala | 6 +- .../cluster/SparkDeploySchedulerBackend.scala | 11 +++- .../mesos/CoarseMesosSchedulerBackend.scala | 12 +++- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- docs/configuration.md | 16 ++++++ .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../apache/spark/deploy/yarn/ClientBase.scala | 56 +++++++++---------- .../deploy/yarn/ExecutorRunnableUtil.scala | 3 +- .../spark/deploy/yarn/ApplicationMaster.scala | 3 + .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../spark/deploy/yarn/ExecutorLauncher.scala | 5 ++ .../deploy/yarn/YarnAllocationHandler.scala | 4 ++ 15 files changed, 96 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 714f1f54b7a58..ed9b1904e9346 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -224,8 +224,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { throw new Exception(msg) } if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { - val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). Please use " + - "spark.executor.memory." + val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). Use " + + "spark.executor.memory instead." throw new Exception(msg) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0382a34e28074..61bed816b5905 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -246,19 +246,20 @@ class SparkContext(config: SparkConf) extends Logging { .map(Utils.memoryStringToMb) .getOrElse(512) - // Environment variables to pass to our executors - private[spark] val executorEnvs = HashMap[String, String]() + // Environment variables to pass to our executors. + // NOTE: This should only be used for test related settings. + private[spark] val testExecutorEnvs = HashMap[String, String]() // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) + for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - executorEnvs(envKey) = value + testExecutorEnvs(envKey) = value } // The Mesos scheduler backend relies on this environment variable to set executor memory. // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - executorEnvs ++= conf.getExecutorEnv + testExecutorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + testExecutorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -266,7 +267,7 @@ class SparkContext(config: SparkConf) extends Logging { }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } - executorEnvs("SPARK_USER") = sparkUser + testExecutorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index e6a0d404571e5..834c382b57c51 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -54,10 +54,10 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends System.getenv().foreach{case (k, v) => env(k) = v} val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" - val classPathEntries = sys.props.get("spark.driver.classPath").toSeq.flatMap { cp => + val classPathEntries = sys.props.get("spark.driver.extraClassPath").toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val libraryPathEntries = sys.props.get("spark.driver.libraryPath").toSeq.flatMap { cp => + val libraryPathEntries = sys.props.get("spark.driver.extraLibraryPath").toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } val javaOpts = sys.props.get("spark.driver.extraJavaOptions") diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index ff51ae0a37e24..fa0c81b6bb068 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -144,11 +144,11 @@ object SparkSubmit { new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, - sysProp = "spark.driver.classPath"), + sysProp = "spark.driver.extraClassPath"), new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, - sysProp = "spark.driver.javaOpts"), + sysProp = "spark.driver.extraJavaOptions"), new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, - sysProp = "spark.driver.libraryPath"), + sysProp = "spark.driver.extraLibraryPath"), new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index fe95b962d6ca2..948477aaee861 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -47,11 +47,16 @@ private[spark] class SparkDeploySchedulerBackend( val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + val classPathEntries = sys.props.get("spark.executor.extraClassPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val libraryPathEntries = sys.props.get("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } - // TODO (pwendell) LOOK AT THIS val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, - Seq(), Seq(), extraJavaOpts) + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.testExecutorEnvs, + classPathEntries, libraryPathEntries, extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 230f9e10d3096..bb1b86bbd2e6c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -111,7 +111,15 @@ private[spark] class CoarseMesosSchedulerBackend( def createCommand(offer: Offer, numCores: Int): CommandInfo = { val environment = Environment.newBuilder() - sc.executorEnvs.foreach { case (key, value) => + val extraClassPath = conf.getOption("spark.executor.extraClassPath") + extraClassPath.foreach { cp => + envrionment.addVariables(Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) + } + val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions", "") + val extraLibraryPath = conf.getOption("spark.executor.extraLibraryPath").map(p => s"-Djava.library.path=$p") + val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + + sc.testExecutorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) @@ -123,7 +131,7 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val extraOpts = conf.get("spark.executor.extraJavaOptions") + val uri = conf.get("spark.executor.uri", null) if (uri == null) { val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index dfdcafe19fb93..c975f312324ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -90,7 +90,7 @@ private[spark] class MesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val environment = Environment.newBuilder() - sc.executorEnvs.foreach { case (key, value) => + sc.testExecutorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) diff --git a/docs/configuration.md b/docs/configuration.md index afd44fc7fe980..c4a169a510938 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -655,6 +655,22 @@ Apart from these, the following properties are also available, and may be useful option. + + spark.executor.extraClassPath + (none) + + Extra classpath entries to append to the classpath of executors. This exists primarily + for backwards-compatiblity with older versions of Spark. Users typically should not need + to set this option. + + + + spark.executor.extraLibraryPath + (none) + + Set a special library path to use when launching executor JVM's. + + diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 0179b0600c61f..00c7649e68e13 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -33,7 +33,9 @@ import org.apache.hadoop.yarn.util.{Apps, Records} import org.apache.spark.{Logging, SparkConf} - +/** + * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. + */ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 54d77da196ed4..2c66cc5adf9e2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -39,8 +39,11 @@ import org.apache.spark.{Logging, SparkConf} /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, - * which will launch a Spark master process and negotiate resources throughout its duration. + * Client submits an application to the YARN ResourceManager. + * + * Depending on the deployment mode this will launch one of two application master classes: + * 1. In standalone mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] which embeds a driver. + * 2. In client mode, it will launch an [[org.apache.spark.deploy.yarn.ExecutorLauncher]]. */ trait ClientBase extends Logging { val args: ClientArguments @@ -259,8 +262,8 @@ trait ClientBase extends Logging { val env = new HashMap[String, String]() - ClientBase.populateClasspath(yarnConf, sparkConf, localResources.contains(ClientBase.LOG4J_PROP), - env) + val extraCp = sparkConf.getOption("spark.driver.extraClassPath") + ClientBase.populateClasspath(yarnConf, sparkConf, localResources.contains(ClientBase.LOG4J_PROP), env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -272,9 +275,6 @@ trait ClientBase extends Logging { // Allow users to specify some environment variables. Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) - // Add each SPARK_* key to the environment. - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } - env } @@ -427,29 +427,29 @@ object ClientBase { } } - def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) + def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String], + extraClassPath: Option[String] = None) { + + /** Add entry to the classpath. */ + def addClasspathEntry(entry: String) = pps.addToEnvironment(env, Environment.CLASSPATH.name, entry) + /** Add entry to the classpath. Interpreted as a path relative to the working directory. */ + def addPwdClasspathEntry(path: String) = addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) + + extraClassPath.foreach(addClasspathEntry) + + addClasspathEntry(Environment.PWD.$()) // If log4j present, ensure ours overrides all others - if (addLog4j) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + - Path.SEPARATOR + LOG4J_PROP) - } + if (addLog4j) addPwdClasspathEntry(LOG4J_PROP) // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false") - .toBoolean - if (userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + - Path.SEPARATOR + APP_JAR) - } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + - Path.SEPARATOR + SPARK_JAR) - ClientBase.populateHadoopClasspath(conf, env) - - if (!userClasspathFirst) { - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + - Path.SEPARATOR + APP_JAR) + if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { + addPwdClasspathEntry(APP_JAR) + addPwdClasspathEntry(SPARK_JAR) + ClientBase.populateHadoopClasspath(conf, env) + } else { + addPwdClasspathEntry(SPARK_JAR) + ClientBase.populateHadoopClasspath(conf, env) + addPwdClasspathEntry(APP_JAR) } - Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + - Path.SEPARATOR + "*") + addPwdClasspathEntry("*") } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 6b29e3249b835..64cf54ddbf7c4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -158,7 +158,8 @@ trait ExecutorRunnableUtil extends Logging { def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() - ClientBase.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) + val extraCp = sparkConf.getOption("spark.executor.extraClassPath") + ClientBase.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env, extraCp) // Allow users to specify some environment variables Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 61af0f9ac5ca0..af23885dd8574 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -42,6 +42,9 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils +/** + * An application master that runs the users driver program and allocates executors. + */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 77eb1276a0c4e..2e2fb5d4fa787 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -34,9 +34,7 @@ import org.apache.spark.{Logging, SparkConf} /** - * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, - * which will launch a Spark master process and negotiate resources throughout its duration. + * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API. */ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index b697f103914fd..350c242456042 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -35,6 +35,11 @@ import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +/** + * An application master that allocates executors on behalf of a driver that is running outside the cluster. + * + * This is used only in yarn-client mode. + */ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index e31c4060e8452..bf4c647fcafbc 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -56,6 +56,10 @@ object AllocationType extends Enumeration { // Note that right now, we assume all node asks as uniform in terms of capabilities and priority // Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for // more info on how we are requesting for containers. + +/** + * Acquires resources for executors from a ResourceMAnager and launches executors in new containers. + */ private[yarn] class YarnAllocationHandler( val conf: Configuration, val amClient: AMRMClient[ContainerRequest], From afc9ed8deb012435f74ba1331eef33b76614d78c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 13 Apr 2014 17:30:06 -0700 Subject: [PATCH 15/31] Cleaning up line limits and two compile errors. --- conf/spark-env.sh.template | 4 +- .../scala/org/apache/spark/SparkConf.scala | 64 +++++++++++++++++-- .../scala/org/apache/spark/SparkContext.scala | 1 + .../org/apache/spark/deploy/Client.scala | 12 +++- .../org/apache/spark/deploy/SparkSubmit.scala | 6 +- .../mesos/CoarseMesosSchedulerBackend.scala | 9 ++- docs/configuration.md | 7 +- .../apache/spark/deploy/yarn/ClientBase.scala | 10 +-- .../deploy/yarn/ExecutorRunnableUtil.scala | 3 +- 9 files changed, 91 insertions(+), 25 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 975501e6fbe4d..8c662f4fc6cba 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -32,11 +32,11 @@ # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports # - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine -# - SPARK_WORKER_MEMORY, to set how much total memory to workers have to give executors (e.g. 1000m, 2g) +# - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") # - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") # - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y") -# - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers \ No newline at end of file +# - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index ed9b1904e9346..6be275550c3a5 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -52,8 +52,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } } - validateSettings() - /** Set a configuration variable. */ def set(key: String, value: String): SparkConf = { if (key == null) { @@ -210,25 +208,79 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } - /** Checks for illegal or deprecated config settings. Throws an exception for the former. */ - private def validateSettings() { + /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not + * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */ + private[spark] def validateSettings() { if (settings.contains("spark.local.dir")) { val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone and LOCAL_DIRS in YARN)." logWarning(msg) } + val executorOptsKey = "spark.executor.extraJavaOptions" + val executorClasspathKey = "spark.executor.extraClassPath" + val driverOptsKey = "spark.driver.extraJavaOptions" + val driverClassPathKey = "spark.driver.extraClassPath" + + // Validate spark.executor.extraJavaOptions settings.get(executorOptsKey).map { javaOpts => if (javaOpts.contains("-Dspark")) { val msg = s"$executorOptsKey is not allowed to set Spark options. Was '$javaOpts'" throw new Exception(msg) } if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { - val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). Use " + - "spark.executor.memory instead." + val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). " + + "Use spark.executor.memory instead." throw new Exception(msg) } } + + // Check for legacy configs + sys.env.get("SPARK_JAVA_OPTS").foreach { value => + val error = + s""" + |SPARK_JAVA_OPTS was detected (set to '$value'). + |This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with conf/spark-defaults.conf to set properties for an application + | - ./spark-submit with --driver-java-options to set -X options for a driver + | - spark.executor.executor.extraJavaOptions to set -X options for executors + | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) + """.stripMargin + logError(error) + + for (key <- Seq(executorOptsKey, driverOptsKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + + sys.env.get("SPARK_CLASSPATH").foreach { value => + val error = + s""" + |SPARK_CLASSPATH was detected (set to '$value'). + | This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with --driver-class-path to augment the driver classpath + | - spark.executor.executor.extraClassPath to augment the executor classpath + """.stripMargin + logError(error) + + for (key <- Seq(executorClasspathKey, driverClassPathKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_CLASSPATH. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 61bed816b5905..a308f4d56223e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -147,6 +147,7 @@ class SparkContext(config: SparkConf) extends Logging { this(master, appName, sparkHome, jars, Map(), Map()) private[spark] val conf = config.clone() + conf.validateSettings() /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 834c382b57c51..7ead1171525d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -54,13 +54,19 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends System.getenv().foreach{case (k, v) => env(k) = v} val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" - val classPathEntries = sys.props.get("spark.driver.extraClassPath").toSeq.flatMap { cp => + + val classPathConf = "spark.driver.extraClassPath" + val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val libraryPathEntries = sys.props.get("spark.driver.extraLibraryPath").toSeq.flatMap { cp => + + val libraryPathConf = "spark.driver.extraLibraryPath" + val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val javaOpts = sys.props.get("spark.driver.extraJavaOptions") + + val javaOptionsConf = "spark.driver.extraJavaOptions" + val javaOpts = sys.props.get(javaOptionsConf) val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index fa0c81b6bb068..755160b6c4804 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -146,7 +146,7 @@ object SparkSubmit { new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraClassPath"), new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, - sysProp = "spark.driver.extraJavaOptions"), + sysProp = "spark.driver.extraJavaOpts"), new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraLibraryPath"), @@ -259,7 +259,9 @@ object SparkSubmit { try { properties.load(inputStream) } catch { - case e: IOException => throw new SparkException(s"Failed when loading Spark properties file ${file.getName}", e) + case e: IOException => + val message = s"Failed when loading Spark properties file ${file.getName}" + throw new SparkException(message, e) } properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index bb1b86bbd2e6c..2cd9d6c12eaf7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -113,10 +113,13 @@ private[spark] class CoarseMesosSchedulerBackend( val environment = Environment.newBuilder() val extraClassPath = conf.getOption("spark.executor.extraClassPath") extraClassPath.foreach { cp => - envrionment.addVariables(Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) + environment.addVariables( + Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions", "") - val extraLibraryPath = conf.getOption("spark.executor.extraLibraryPath").map(p => s"-Djava.library.path=$p") + val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions") + + val libraryPathOption = "spark.executor.extraLibraryPath" + val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") sc.testExecutorEnvs.foreach { case (key, value) => diff --git a/docs/configuration.md b/docs/configuration.md index c4a169a510938..bb6ad67c10e02 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -652,7 +652,9 @@ Apart from these, the following properties are also available, and may be useful A string of extra JVM options to pass to executors. For instance, GC settings or other logging. Note that it is illegal to set Spark properties or heap size settings with this - option. + option. Spark properties should be set using a SparkConf object or the + spark-defaults.conf file used with the spark-submit script. Heap size settings can be set + with spark.executor.memory. @@ -694,9 +696,6 @@ The following variables can be set in `spark-env.sh`: * `JAVA_HOME`, the location where Java is installed (if it's not on your default `PATH`) * `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. -* `SPARK_CLASSPATH`, to add elements to Spark's classpath that you want to be present for _all_ applications. - Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend - doing that when possible. * `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 2c66cc5adf9e2..c7b0bf81b4f99 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -42,8 +42,10 @@ import org.apache.spark.{Logging, SparkConf} * Client submits an application to the YARN ResourceManager. * * Depending on the deployment mode this will launch one of two application master classes: - * 1. In standalone mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] which embeds a driver. - * 2. In client mode, it will launch an [[org.apache.spark.deploy.yarn.ExecutorLauncher]]. + * 1. In standalone mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] + * which launches a driver program inside of the cluster. + * 2. In client mode, it will launch an [[org.apache.spark.deploy.yarn.ExecutorLauncher]] to + * request executors on behalf of a driver running outside of the cluster. */ trait ClientBase extends Logging { val args: ClientArguments @@ -263,7 +265,8 @@ trait ClientBase extends Logging { val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - ClientBase.populateClasspath(yarnConf, sparkConf, localResources.contains(ClientBase.LOG4J_PROP), env, extraCp) + ClientBase.populateClasspath(yarnConf, sparkConf, + localResources.contains(ClientBase.LOG4J_PROP), env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -342,7 +345,6 @@ trait ClientBase extends Logging { for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { JAVA_OPTS += s"-D$k=$v" } - // TODO: honor driver classpath here: sys.props.get("spark.driver.classPath") sys.props.get("spark.driver.extraJavaOptions").foreach(opts => JAVA_OPTS += opts) sys.props.get("spark.driver.libraryPath").foreach(p => JAVA_OPTS += s"-Djava.library.path=$p") } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 64cf54ddbf7c4..c8d747bd463c1 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -159,7 +159,8 @@ trait ExecutorRunnableUtil extends Logging { val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.executor.extraClassPath") - ClientBase.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env, extraCp) + ClientBase.populateClasspath(yarnConf, sparkConf, + System.getenv("SPARK_YARN_LOG4J_PATH") != null, env, extraCp) // Allow users to specify some environment variables Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) From 4ee6f9d1387793fb081e9384f0c1aec2144e5806 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 14 Apr 2014 13:54:49 -0700 Subject: [PATCH 16/31] Making YARN doc changes consistent --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 +++ .../org/apache/spark/deploy/yarn/ExecutorLauncher.scala | 6 ++++++ .../apache/spark/deploy/yarn/YarnAllocationHandler.scala | 4 ++++ .../org/apache/spark/deploy/yarn/ExecutorLauncher.scala | 3 ++- .../apache/spark/deploy/yarn/YarnAllocationHandler.scala | 2 +- 5 files changed, 16 insertions(+), 2 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 67ec95c8fc04f..7e75e5b602665 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -39,6 +39,9 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils +/** + * An application master that runs the users driver program and allocates executors. + */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 7b0e020263835..485e2d2061b7d 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -34,6 +34,12 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo +/** + * An application master that allocates executors on behalf of a driver that is running outside + * the cluster. + * + * This is used only in yarn-client mode. + */ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 2056667af50cb..92fe297808375 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -54,6 +54,10 @@ object AllocationType extends Enumeration { // Note that right now, we assume all node asks as uniform in terms of capabilities and priority // Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for // more info on how we are requesting for containers. + +/** + * Acquires resources for executors from a ResourceManager and launches executors in new containers. + */ private[yarn] class YarnAllocationHandler( val conf: Configuration, val resourceManager: AMRMProtocol, diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 350c242456042..f41cef18cc9a7 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -36,7 +36,8 @@ import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest /** - * An application master that allocates executors on behalf of a driver that is running outside the cluster. + * An application master that allocates executors on behalf of a driver that is running outside + * the cluster. * * This is used only in yarn-client mode. */ diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index bf4c647fcafbc..3c80dce2d702d 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -58,7 +58,7 @@ object AllocationType extends Enumeration { // more info on how we are requesting for containers. /** - * Acquires resources for executors from a ResourceMAnager and launches executors in new containers. + * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ private[yarn] class YarnAllocationHandler( val conf: Configuration, From c2a2909eeffcb8354fa6a9ce8760e36ab27c7f05 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 14 Apr 2014 15:21:48 -0700 Subject: [PATCH 17/31] Test compile fixes --- .../test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 2 +- .../org/apache/spark/deploy/worker/DriverRunnerTest.scala | 2 +- .../org/apache/spark/deploy/worker/ExecutorRunnerTest.scala | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index d563db9ca06b9..1375fad52d9b0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -100,7 +100,7 @@ class JsonProtocolSuite extends FunSuite { def createDriverCommand() = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), - Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Seq("-Dfoo") + Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Some("-Dfoo") ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index a2c131b0c9787..4633bc3f7f25e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -29,7 +29,7 @@ import org.apache.spark.deploy.{Command, DriverDescription} class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { - val command = new Command("mainClass", Seq(), Map()) + val command = new Command("mainClass", Seq(), Map(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription, null, "akka://1.2.3.4/worker/") diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 3cab8e7b37934..8ae387fa0be6f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -27,7 +27,8 @@ class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")) - val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()), + val appDesc = new ApplicationDescription("app name", Some(8), 500, + Command("foo", Seq(), Map(), Seq(), Seq()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome.getOrElse(".")), From be42f35c48179b34d1d53b118ff50f5b15733a1a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 14 Apr 2014 18:26:07 -0700 Subject: [PATCH 18/31] Handle case where SPARK_HOME is not set --- .../spark/deploy/SparkSubmitArguments.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index d4747e64980ee..7ac993a51d87d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -55,13 +55,14 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") if (propertiesFile == null) { - val sparkHome = sys.env("SPARK_HOME") // defined via `spark-class` - val sep = File.separator - val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.properties" - val file = new File(defaultPath) - if (file.exists()) { - propertiesFile = file.getAbsolutePath - } + sys.env.get("SPARK_HOME").foreach { sparkHome => + val sep = File.separator + val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.properties" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } } override def toString = { From e83cd8f6b667954471c271cdb03657d5d8bf8fa9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 14 Apr 2014 18:41:08 -0700 Subject: [PATCH 19/31] Changes to allow re-use of test applications --- dev/audit-release/audit_release.py | 1 + .../sbt_app_core/src/main/scala/SparkApp.scala | 7 ++++++- .../sbt_app_graphx/src/main/scala/GraphxApp.scala | 10 ++++++++-- .../src/main/scala/StreamingApp.scala | 8 +++++--- 4 files changed, 20 insertions(+), 6 deletions(-) diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index fa2f02dfecc75..4a816d4101e57 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -114,6 +114,7 @@ def ensure_path_not_present(x): os.environ["SPARK_VERSION"] = RELEASE_VERSION os.environ["SCALA_VERSION"] = SCALA_VERSION os.environ["SPARK_RELEASE_REPOSITORY"] = RELEASE_REPOSITORY +os.environ["SPARK_AUDIT_MASTER"] = "local" for module in modules: os.environ["SPARK_MODULE"] = module ret = run_cmd("sbt clean update", exit_on_failure=False) diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index 53fe43215e40e..a89b0d7d38bf1 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -24,8 +24,13 @@ import org.apache.spark.SparkContext._ object SimpleApp { def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Spark App").setMaster(master) + case None => new SparkConf().setAppName("Simple Spark App") + } val logFile = "input.txt" - val sc = new SparkContext("local", "Simple App") + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() diff --git a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala index da08e014ebd94..24c7f8d667296 100644 --- a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala +++ b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala @@ -17,14 +17,20 @@ package main.scala -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD object GraphXApp { def main(args: Array[String]) { - val sc = new SparkContext("local", "Simple GraphX App") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple GraphX App").setMaster(master) + case None => new SparkConf().setAppName("Simple Graphx App") + } + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) + val users: RDD[(VertexId, (String, String))] = sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala index 3d0722d2ac45e..a1d8971abe9a4 100644 --- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala +++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala @@ -27,10 +27,12 @@ import org.apache.spark.streaming._ object SparkStreamingExample { def main(args: Array[String]) { - val conf = new SparkConf(true) - .setMaster("local[2]") - .setAppName("Streaming test") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Streaming App").setMaster(master) + case None => new SparkConf().setAppName("Simple Streaming App") + } val ssc = new StreamingContext(conf, Seconds(1)) + SparkContext.jarOfClass(this.getClass).foreach(ssc.sparkContext.addJar) val seen = ListBuffer[RDD[Int]]() val rdd1 = ssc.sparkContext.makeRDD(1 to 100, 10) From 308f1f60f7cc5df80806c232a33e1aafc710d8f4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 15 Apr 2014 00:06:44 -0700 Subject: [PATCH 20/31] Properly escape quotes and other clean-up for YARN --- .../apache/spark/deploy/yarn/ClientBase.scala | 62 +++++++++---------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index c7b0bf81b4f99..19016849784ef 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn import java.net.{InetAddress, URI, UnknownHostException} import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, Map} +import scala.collection.mutable.{HashMap, ListBuffer, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.{Apps, Records} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkContext} /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The @@ -64,7 +64,6 @@ trait ClientBase extends Logging { // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { Map( - (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", ((args.userJar == null && args.amClass == classOf[ApplicationMaster].getName) -> "Error: You must specify a user jar when running in standalone mode!"), (args.userClass == null) -> "Error: You must specify a user class!", @@ -202,7 +201,7 @@ trait ClientBase extends Logging { val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() Map( - ClientBase.SPARK_JAR -> System.getenv("SPARK_JAR"), ClientBase.APP_JAR -> args.userJar, + ClientBase.SPARK_JAR -> ClientBase.getSparkJar, ClientBase.APP_JAR -> args.userJar, ClientBase.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF") ).foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" @@ -306,13 +305,13 @@ trait ClientBase extends Logging { val amMemory = calculateAMMemory(newApp) - var JAVA_OPTS = "" + var JAVA_OPTS = ListBuffer[String]() // Add Xmx for AM memory JAVA_OPTS += "-Xmx" + amMemory + "m" val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) - JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir + JAVA_OPTS += "-Djava.io.tmpdir=" + tmpDir // TODO: Remove once cpuset version is pushed out. // The context is, default gc for server class machines ends up using all cores to do gc - @@ -326,24 +325,24 @@ trait ClientBase extends Logging { if (useConcurrentAndIncrementalGC) { // In our expts, using (default) throughput collector has severe perf ramifications in // multi-tenant machines - JAVA_OPTS += " -XX:+UseConcMarkSweepGC " - JAVA_OPTS += " -XX:+CMSIncrementalMode " - JAVA_OPTS += " -XX:+CMSIncrementalPacing " - JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " - JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + JAVA_OPTS += "-XX:+UseConcMarkSweepGC" + JAVA_OPTS += "-XX:+CMSIncrementalMode" + JAVA_OPTS += "-XX:+CMSIncrementalPacing" + JAVA_OPTS += "-XX:CMSIncrementalDutyCycleMin=0" + JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10" } if (args.amClass == classOf[ExecutorLauncher].getName) { // If we are being launched in client mode, forward the spark-conf options // onto the executor launcher for ((k, v) <- sparkConf.getAll) { - JAVA_OPTS += s"-D$k=$v" + JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" } } else { // If we are being launched in standalone mode, capture and forward any spark // system properties (e.g. set by spark-class). for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { - JAVA_OPTS += s"-D$k=$v" + JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" } sys.props.get("spark.driver.extraJavaOptions").foreach(opts => JAVA_OPTS += opts) sys.props.get("spark.driver.libraryPath").foreach(p => JAVA_OPTS += s"-Djava.library.path=$p") @@ -354,22 +353,21 @@ trait ClientBase extends Logging { } // Command for the ApplicationMaster - val commands = List[String]( - Environment.JAVA_HOME.$() + "/bin/java" + - " -server " + - JAVA_OPTS + - " " + args.amClass + - " --class " + args.userClass + - " --jar " + args.userJar + - userArgsToString(args) + - " --executor-memory " + args.executorMemory + - " --executor-cores " + args.executorCores + - " --num-executors " + args.numExecutors + - " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + - " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - logInfo("Command for starting the Spark ApplicationMaster: " + commands(0)) - amContainer.setCommands(commands) + val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ + JAVA_OPTS ++ + Seq(args.amClass, "--class", args.userClass, "--jar ", args.userJar, + userArgsToString(args), + "--executor-memory", args.executorMemory.toString, + "--executor-cores", args.executorCores.toString, + "--num-executors ", args.numExecutors.toString, + "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", + "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + logInfo("Command for starting the Spark ApplicationMaster: " + commands) + + // TODO: it would be nicer to just make sure there are no null commands here + val printableCommands = commands.map(s => if (s == null) "null" else s).toList + amContainer.setCommands(printableCommands) setupSecurityToken(amContainer) amContainer @@ -381,6 +379,8 @@ object ClientBase { val APP_JAR: String = "app.jar" val LOG4J_PROP: String = "log4j.properties" + def getSparkJar = sys.env.get("SPARK_JAR").getOrElse(SparkContext.jarOfClass(this.getClass).head) + // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { val classpathEntries = Option(conf.getStrings( @@ -433,9 +433,9 @@ object ClientBase { extraClassPath: Option[String] = None) { /** Add entry to the classpath. */ - def addClasspathEntry(entry: String) = pps.addToEnvironment(env, Environment.CLASSPATH.name, entry) + def addClasspathEntry(path: String) = Apps.addToEnvironment(env, Environment.CLASSPATH.name, path) /** Add entry to the classpath. Interpreted as a path relative to the working directory. */ - def addPwdClasspathEntry(path: String) = addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) + def addPwdClasspathEntry(entry: String) = addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) extraClassPath.foreach(addClasspathEntry) From fda0301a7ca28d684f03111020c8a46be7470024 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 15 Apr 2014 00:35:12 -0700 Subject: [PATCH 21/31] Note --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 19016849784ef..76537cf1c8736 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -332,6 +332,8 @@ trait ClientBase extends Logging { JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10" } + // TODO: it might be nicer to pass these as an internal environment variable rather than + // as Java options, due to complications with string parsing of nested quotes. if (args.amClass == classOf[ExecutorLauncher].getName) { // If we are being launched in client mode, forward the spark-conf options // onto the executor launcher From ffa00febea3777c7c662053aabf10299f29fdd76 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 17 Apr 2014 21:31:38 -0700 Subject: [PATCH 22/31] Review feedback --- .../main/scala/org/apache/spark/SparkConf.scala | 7 ++++--- .../org/apache/spark/deploy/SparkSubmit.scala | 16 ++++++---------- .../cluster/SparkDeploySchedulerBackend.scala | 3 +-- docs/configuration.md | 2 +- .../apache/spark/deploy/yarn/ClientBase.scala | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- 6 files changed, 14 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 6be275550c3a5..811dac5336277 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -225,7 +225,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { // Validate spark.executor.extraJavaOptions settings.get(executorOptsKey).map { javaOpts => if (javaOpts.contains("-Dspark")) { - val msg = s"$executorOptsKey is not allowed to set Spark options. Was '$javaOpts'" + val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts)'. " + + "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." throw new Exception(msg) } if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { @@ -245,7 +246,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { |Please instead use: | - ./spark-submit with conf/spark-defaults.conf to set properties for an application | - ./spark-submit with --driver-java-options to set -X options for a driver - | - spark.executor.executor.extraJavaOptions to set -X options for executors + | - spark.executor.extraJavaOptions to set -X options for executors | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) """.stripMargin logError(error) @@ -268,7 +269,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { | |Please instead use: | - ./spark-submit with --driver-class-path to augment the driver classpath - | - spark.executor.executor.extraClassPath to augment the executor classpath + | - spark.executor.extraClassPath to augment the executor classpath """.stripMargin logError(error) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 755160b6c4804..f243d078c3c26 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,17 +17,15 @@ package org.apache.spark.deploy -import java.io.{IOException, FileInputStream, PrintStream, File} +import java.io.{File, FileInputStream, IOException, PrintStream} import java.net.URL import java.util.Properties -import org.apache.spark.executor.ExecutorURLClassLoader - import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} + import org.apache.spark.SparkException +import org.apache.spark.executor.ExecutorURLClassLoader /** * Scala code behind the spark-submit script. The script handles setting up the classpath with @@ -114,7 +112,7 @@ object SparkSubmit { // Load system properties by default from the file, if present if (appArgs.verbose) printStream.println(s"Using properties file: ${appArgs.propertiesFile}") - Option(appArgs.propertiesFile).map { filename => + Option(appArgs.propertiesFile).foreach { filename => val file = new File(filename) getDefaultProperties(file).foreach { case (k, v) => if (k.startsWith("spark")) { @@ -142,14 +140,12 @@ object SparkSubmit { val options = List[OptionAssigner]( new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), - new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraClassPath"), new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, - sysProp = "spark.driver.extraJavaOpts"), + sysProp = "spark.driver.extraJavaOptions"), new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraLibraryPath"), - new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 948477aaee861..9544ca05dca70 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -44,8 +44,7 @@ private[spark] class SparkDeploySchedulerBackend( val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", - "{{CORES}}", "{{WORKER_URL}}") + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") val classPathEntries = sys.props.get("spark.executor.extraClassPath").toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) diff --git a/docs/configuration.md b/docs/configuration.md index bb6ad67c10e02..15034e65dfaa4 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -662,7 +662,7 @@ Apart from these, the following properties are also available, and may be useful (none) Extra classpath entries to append to the classpath of executors. This exists primarily - for backwards-compatiblity with older versions of Spark. Users typically should not need + for backwards-compatibility with older versions of Spark. Users typically should not need to set this option. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 76537cf1c8736..99615247104b5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -305,7 +305,7 @@ trait ClientBase extends Logging { val amMemory = calculateAMMemory(newApp) - var JAVA_OPTS = ListBuffer[String]() + val JAVA_OPTS = ListBuffer[String]() // Add Xmx for AM memory JAVA_OPTS += "-Xmx" + amMemory + "m" diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index af23885dd8574..b96140290c61f 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -43,7 +43,7 @@ import org.apache.spark.util.Utils /** - * An application master that runs the users driver program and allocates executors. + * An application master that runs the user's driver program and allocates executors. */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { From a762901ee3b6884053bfe0d3bf45a563cc561cd9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 17 Apr 2014 22:05:55 -0700 Subject: [PATCH 23/31] Fixing test failures --- .../org/apache/spark/deploy/JsonProtocolSuite.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 1375fad52d9b0..bfae32dae0dc5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -133,9 +133,12 @@ class JsonProtocolSuite extends FunSuite { def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { val Diff(c, a, d) = validateJson diff expectedJson - assert(c === JNothing, "Json changed") - assert(a === JNothing, "Json added") - assert(d === JNothing, "Json deleted") + val validatePretty = JsonMethods.pretty(validateJson) + val expectedPretty = JsonMethods.pretty(expectedJson) + val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" + assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") + assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") + assert(d === JNothing, s"$errorMessage\nDelected:\n${JsonMethods.pretty(d)}") } } @@ -165,7 +168,7 @@ object JsonConstants { """ |{"name":"name","cores":4,"memoryperslave":1234, |"user":"%s","sparkhome":"sparkHome", - |"command":"Command(mainClass,List(arg1, arg2),Map())"} + |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),None)"} """.format(System.getProperty("user.name", "")).stripMargin val executorRunnerJsonStr = From a56b125c842a070f365b889a7d83788b35bc9e0b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 18 Apr 2014 11:02:35 -0700 Subject: [PATCH 24/31] Responses to Tom's review --- .rat-excludes | 1 + conf/spark-defaults.properties.template | 7 +++++++ conf/spark-env.sh.template | 1 - core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- .../org/apache/spark/deploy/SparkSubmitArguments.scala | 7 +++++-- docs/configuration.md | 2 +- 6 files changed, 15 insertions(+), 5 deletions(-) create mode 100644 conf/spark-defaults.properties.template diff --git a/.rat-excludes b/.rat-excludes index 9e9abb3f10bbf..fe5a55f5e96b6 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -11,6 +11,7 @@ RELEASE control docs fairscheduler.xml.template +spark-defaults.properties.template log4j.properties log4j.properties.template metrics.properties.template diff --git a/conf/spark-defaults.properties.template b/conf/spark-defaults.properties.template new file mode 100644 index 0000000000000..f840ff681d019 --- /dev/null +++ b/conf/spark-defaults.properties.template @@ -0,0 +1,7 @@ +# Default system properties included when running spark-submit. +# This is useful for setting default environmental settings. + +# Example: +# spark.master spark://master:7077 +# spark.eventLog.enabled true +# spark.eventLog.dir hdfs://namenode:8021/directory diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 8c662f4fc6cba..177a21cc0377f 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -17,7 +17,6 @@ # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in YARN client mode -# - SPARK_YARN_APP_JAR, Path to your application’s JAR file (required) # - SPARK_EXECUTOR_INSTANCES, Number of workers to start (Default: 2) # - SPARK_EXECUTOR_CORES, Number of cores for the workers (Default: 1). # - SPARK_EXECUTOR_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 811dac5336277..20aa3d73dabda 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -244,7 +244,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { |This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. | |Please instead use: - | - ./spark-submit with conf/spark-defaults.conf to set properties for an application + | - ./spark-submit with conf/spark-defaults.properties to set defaults for an application | - ./spark-submit with --driver-java-options to set -X options for a driver | - spark.executor.extraJavaOptions to set -X options for executors | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) 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 7ac993a51d87d..9127460a1c504 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -145,7 +145,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { driverExtraClassPath = value parseOpts(tail) - case ("--driver-java-opts") :: value :: tail => + case ("--driver-java-options") :: value :: tail => driverExtraJavaOptions = value parseOpts(tail) @@ -217,8 +217,11 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | --jars JARS A comma-separated list of local jars to include on the | driver classpath and that SparkContext.addJar will work | with. Doesn't work on standalone with 'cluster' deploy mode. + | --properties-file FILE Path to a file from which to load extra properties. If not + | specified, this will look for conf/spark-defaults.properties. + | | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). - | --driver-java-opts Extra Java options to pass to the driver + | --driver-java-options Extra Java options to pass to the driver | --driver-library-path Extra library path entries to pass to the driver | --driver-class-path Extra class path entries to pass to the driver | diff --git a/docs/configuration.md b/docs/configuration.md index 5a4abca2646b6..ce34a367a5bce 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -659,7 +659,7 @@ Apart from these, the following properties are also available, and may be useful A string of extra JVM options to pass to executors. For instance, GC settings or other logging. Note that it is illegal to set Spark properties or heap size settings with this option. Spark properties should be set using a SparkConf object or the - spark-defaults.conf file used with the spark-submit script. Heap size settings can be set + spark-defaults.properties file used with the spark-submit script. Heap size settings can be set with spark.executor.memory. From af0adf71bfec418bf66fe87de3686dc4d8e09608 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 18 Apr 2014 14:08:09 -0700 Subject: [PATCH 25/31] Automatically add user jar --- .../org/apache/spark/deploy/SparkSubmit.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index f39adff247ea4..78c4f915f041e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -116,6 +116,9 @@ object SparkSubmit { val file = new File(filename) getDefaultProperties(file).foreach { case (k, v) => if (k.startsWith("spark")) { + if (k == "spark.master") + throw new Exception("Setting spark.master in spark-defaults.properties is not " + + "supported. Use MASTER environment variable or --master.") sysProps(k) = v if (appArgs.verbose) printStream.println(s"Adding default property: $k=$v") } @@ -165,10 +168,11 @@ object SparkSubmit { new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), - new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars") + new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars"), + new OptionAssigner(appArgs.jars, STANDALONE | YARN | MESOS, true, sysProp = "spark.jars") ) - // more jars + // For client mode make any added jars immediately visible on the classpath if (appArgs.jars != null && !deployOnCluster) { for (jar <- appArgs.jars.split(",")) { childClasspath += jar @@ -186,6 +190,14 @@ object SparkSubmit { } } + // For standalone mode, add the application jar automatically so the user doesn't have to + // call sc.addJar. TODO: Standalone mode in the cluster + if (clusterManager == STANDALONE) { + val existingJars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) + sysProps.put("spark.jars", (existingJars ++ Seq(appArgs.primaryResource)).mkString(",")) + println("SPARK JARS" + sysProps.get("spark.jars")) + } + if (deployOnCluster && clusterManager == STANDALONE) { if (appArgs.supervise) { childArgs += "--supervise" @@ -196,7 +208,7 @@ object SparkSubmit { childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) } - // args + // Arguments to be passed to user program if (appArgs.childArgs != null) { if (!deployOnCluster || clusterManager == STANDALONE) { childArgs ++= appArgs.childArgs From b16e6a29930cab0c087136f357aeabf0ed075c6b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 20 Apr 2014 14:26:31 -0700 Subject: [PATCH 26/31] Cleanup of spark-submit script and Scala quick start guide --- .../scala/org/apache/spark/SparkContext.scala | 14 +- .../org/apache/spark/deploy/SparkSubmit.scala | 47 ++----- .../spark/deploy/SparkSubmitArguments.scala | 128 ++++++++++++++---- .../spark/deploy/SparkSubmitSuite.scala | 22 ++- docs/cluster-overview.md | 8 ++ docs/quick-start.md | 52 +++++-- 6 files changed, 192 insertions(+), 79 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index db152c4917ccb..b9011d238b43c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -171,11 +171,11 @@ class SparkContext(config: SparkConf) extends Logging { conf.setIfMissing("spark.driver.host", Utils.localHostName()) conf.setIfMissing("spark.driver.port", "0") - val jars: Seq[String] = if (conf.contains("spark.jars")) { - conf.get("spark.jars").split(",").filter(_.size != 0) - } else { - null - } + val jars: Seq[String] = + conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten + + val files: Seq[String] = + conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten val master = conf.get("spark.master") val appName = conf.get("spark.app.name") @@ -236,6 +236,10 @@ class SparkContext(config: SparkConf) extends Logging { jars.foreach(addJar) } + if (files != null) { + files.foreach(addFile) + } + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 78c4f915f041e..3dea722a02da5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,14 +17,11 @@ package org.apache.spark.deploy -import java.io.{File, FileInputStream, IOException, PrintStream} +import java.io.{File, PrintStream} import java.net.{URI, URL} -import java.util.Properties -import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} -import org.apache.spark.SparkException import org.apache.spark.executor.ExecutorURLClassLoader /** @@ -110,23 +107,6 @@ object SparkSubmit { val sysProps = new HashMap[String, String]() var childMainClass = "" - // Load system properties by default from the file, if present - if (appArgs.verbose) printStream.println(s"Using properties file: ${appArgs.propertiesFile}") - Option(appArgs.propertiesFile).foreach { filename => - val file = new File(filename) - getDefaultProperties(file).foreach { case (k, v) => - if (k.startsWith("spark")) { - if (k == "spark.master") - throw new Exception("Setting spark.master in spark-defaults.properties is not " + - "supported. Use MASTER environment variable or --master.") - sysProps(k) = v - if (appArgs.verbose) printStream.println(s"Adding default property: $k=$v") - } - else { - printWarning(s"Ignoring non-spark config property: $k=$v") - } - } - } if (clusterManager == MESOS && deployOnCluster) { printErrorAndExit("Mesos does not support running the driver on the cluster") @@ -166,10 +146,11 @@ object SparkSubmit { sysProp = "spark.cores.max"), new OptionAssigner(appArgs.files, YARN, false, sysProp = "spark.yarn.dist.files"), new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), + new OptionAssigner(appArgs.files, STANDALONE | MESOS, true, sysProp = "spark.files"), new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars"), - new OptionAssigner(appArgs.jars, STANDALONE | YARN | MESOS, true, sysProp = "spark.jars") + new OptionAssigner(appArgs.jars, STANDALONE | MESOS, false, sysProp = "spark.jars") ) // For client mode make any added jars immediately visible on the classpath @@ -219,6 +200,10 @@ object SparkSubmit { } } + for ((k, v) <- appArgs.getDefaultSparkProperties) { + if (!sysProps.contains(k)) sysProps(k) = v + } + (childArgs, childClasspath, sysProps, childMainClass) } @@ -259,22 +244,12 @@ object SparkSubmit { val url = localJarFile.getAbsoluteFile.toURI.toURL loader.addURL(url) } - - private def getDefaultProperties(file: File): Seq[(String, String)] = { - require(file.exists(), s"Default properties file ${file.getName} does not exist") - val inputStream = new FileInputStream(file) - val properties = new Properties() - try { - properties.load(inputStream) - } catch { - case e: IOException => - val message = s"Failed when loading Spark properties file ${file.getName}" - throw new SparkException(message, e) - } - properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) - } } +/** + * Provides an indirection layer for passing arguments as system properties or flags to + * the user's driver program or to downstream launcher tools. + */ private[spark] class OptionAssigner(val value: String, val clusterManager: Int, val deployOnCluster: Boolean, 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 9127460a1c504..eba6e981b51d3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,14 +17,19 @@ package org.apache.spark.deploy -import scala.collection.mutable.ArrayBuffer -import java.io.File +import java.io.{File, FileInputStream, IOException} +import java.util.Properties + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, ArrayBuffer} + +import org.apache.spark.SparkException /** * Parses and encapsulates arguments from the spark-submit script. */ private[spark] class SparkSubmitArguments(args: Array[String]) { - var master: String = "local" + var master: String = null var deployMode: String = null var executorMemory: String = null var executorCores: String = null @@ -47,22 +52,70 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { var jars: String = null var verbose: Boolean = false - loadEnvVars() parseOpts(args.toList) + loadDefaults() + checkRequiredArguments() + + /** Return default present in the currently defined defaults file. */ + def getDefaultSparkProperties = { + val defaultProperties = new HashMap[String, String]() + if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") + Option(propertiesFile).foreach { filename => + val file = new File(filename) + SparkSubmitArguments.getPropertiesFromFile(file).foreach { case (k, v) => + if (k.startsWith("spark")) { + defaultProperties(k) = v + if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") + } + else { + SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") + } + } + } + defaultProperties + } - // Sanity checks - if (args.length == 0) printUsageAndExit(-1) - if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") - if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") - if (propertiesFile == null) { - sys.env.get("SPARK_HOME").foreach { sparkHome => - val sep = File.separator - val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.properties" - val file = new File(defaultPath) - if (file.exists()) { - propertiesFile = file.getAbsolutePath - } + /** Fill in any undefined values based on the current properties file or built-in defaults. */ + private def loadDefaults() = { + + // Use common defaults file, if not specified by user + if (propertiesFile == null) { + sys.env.get("SPARK_HOME").foreach { sparkHome => + val sep = File.separator + val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.properties" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } } + + val defaultProperties = getDefaultSparkProperties + // Use properties file as fallback for values which have a direct analog to + // arguments in this script. + master = Option(master).getOrElse(defaultProperties.get("spark.master").orNull) + executorMemory = Option(executorMemory) + .getOrElse(defaultProperties.get("spark.executor.memory").orNull) + executorCores = Option(executorCores) + .getOrElse(defaultProperties.get("spark.executor.cores").orNull) + totalExecutorCores = Option(totalExecutorCores) + .getOrElse(defaultProperties.get("spark.cores.max").orNull) + name = Option(name).getOrElse(defaultProperties.get("spark.app.name").orNull) + jars = Option(jars).getOrElse(defaultProperties.get("spark.jars").orNull) + + // This supports env vars in older versions of Spark + master = Option(master).getOrElse(System.getenv("MASTER")) + deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + + // Global defaults. These should be keep to minimum to avoid confusing behavior. + master = Option(master).getOrElse("local") + } + + /** Ensure that required fields exists. Call this only once all defaults are loaded. */ + private def checkRequiredArguments() = { + if (args.length == 0) printUsageAndExit(-1) + if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") + if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") } override def toString = { @@ -89,14 +142,12 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | childArgs [${childArgs.mkString(" ")}] | jars $jars | verbose $verbose + | + |Default properties from $propertiesFile: + |${getDefaultSparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } - private def loadEnvVars() { - Option(System.getenv("MASTER")).map(master = _) - Option(System.getenv("DEPLOY_MODE")).map(deployMode = _) - } - private def parseOpts(opts: List[String]): Unit = opts match { case ("--name") :: value :: tail => name = value @@ -189,6 +240,18 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { parseOpts(tail) case value :: tail => + if (value.startsWith("-")) { + val errMessage = s"Unrecognized option '$value'." + val suggestion: Option[String] = value match { + case v if v.startsWith("--") && v.contains("=") => + val parts = v.split("=") + Some(s"Perhaps you meant '${parts(0)} ${parts(1)}'?") + case _ => + None + } + SparkSubmit.printErrorAndExit(errMessage + suggestion.map(" " + _).getOrElse("")) + } + if (primaryResource != null) { val error = s"Found two conflicting resources, $value and $primaryResource." + " Expecting only one resource." @@ -217,6 +280,8 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | --jars JARS A comma-separated list of local jars to include on the | driver classpath and that SparkContext.addJar will work | with. Doesn't work on standalone with 'cluster' deploy mode. + | --files FILES Comma separated list of files to be placed in the working dir + | of each executor. | --properties-file FILE Path to a file from which to load extra properties. If not | specified, this will look for conf/spark-defaults.properties. | @@ -225,6 +290,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | --driver-library-path Extra library path entries to pass to the driver | --driver-class-path Extra class path entries to pass to the driver | + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). @@ -235,14 +301,28 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | | YARN-only: | --executor-cores NUM Number of cores per executor (Default: 1). - | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). | --num-executors NUM Number of executors to (Default: 2). - | --files FILES Comma separated list of files to be placed in the working dir - | of each executor. | --archives ARCHIVES Comma separated list of archives to be extracted into the | working dir of each executor.""".stripMargin ) SparkSubmit.exitFn() } } + +object SparkSubmitArguments { + /** Load properties present in the given file. */ + def getPropertiesFromFile(file: File): Seq[(String, String)] = { + require(file.exists(), s"Properties file ${file.getName} does not exist") + val inputStream = new FileInputStream(file) + val properties = new Properties() + try { + properties.load(inputStream) + } catch { + case e: IOException => + val message = s"Failed when loading Spark properties file ${file.getName}" + throw new SparkException(message, e) + } + properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) + } +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 4e489cd9b66a6..115b3b084bbe0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import java.io.{OutputStream, PrintStream} +import java.io.{File, OutputStream, PrintStream} import scala.collection.mutable.ArrayBuffer @@ -25,6 +25,9 @@ import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers import org.apache.spark.deploy.SparkSubmit._ +import org.scalatest.prop.Tables.Table +import org.scalatest.prop.TableDrivenPropertyChecks._ +import org.apache.spark.util.Utils class SparkSubmitSuite extends FunSuite with ShouldMatchers { @@ -71,6 +74,13 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { testPrematureExit(Array("--help"), "Usage: spark-submit") should be (true) } + test("prints error with unrecognized option") { + testPrematureExit(Array("my.jar --blarg"), "Unrecognized option '--blarg'") should be (true) + testPrematureExit(Array("my.jar -bleg"), "Unrecognized option: '-bleg'") should be (true) + testPrematureExit(Array("my.jar --master=abc"), + "Unrecognized option: '--master=abc'. Perhaps you want '--master abc'?") should be (true) + } + test("handles multiple binary definitions") { val adjacentJars = Array("foo.jar", "bar.jar") testPrematureExit(adjacentJars, "error: Found two conflicting resources") should be (true) @@ -175,4 +185,14 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") } + + def runSparkSubmit(args: Seq[String]): String = { + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + Utils.executeAndGetOutput( + Seq("./bin/spark-submit") ++ args, + new File(sparkHome), + Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) + } + + } diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 7f75ea44e4cea..a2ecebf3140f5 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -138,6 +138,14 @@ The following table summarizes terms you'll see used to refer to cluster concept Application User program built on Spark. Consists of a driver program and executors on the cluster. + + Application jar + + A jar containing the user's Spark application. In some cases users will want to create + an "uber jar" containing their application along with its dependencies. The user's jar + should never include Hadoop or Spark libraries, however, these will be added at runtime. + + Driver program The process running the main() function of the application and creating the SparkContext diff --git a/docs/quick-start.md b/docs/quick-start.md index 60e8b1ba0eb46..5b7a6582cd320 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -99,23 +99,29 @@ scala> linesWithSpark.count() res9: Long = 15 {% endhighlight %} -It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). +It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is +that these same functions can be used on very large data sets, even when they are striped across +tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to +a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). # A Standalone App in Scala -Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. +Now say we wanted to write a standalone application using the Spark API. We will walk through a +simple application in both Scala (with SBT), Java (with Maven), and Python. -We'll create a very simple Spark application in Scala. So simple, in fact, that it's named `SimpleApp.scala`: +We'll create a very simple Spark application in Scala. So simple, in fact, that it's +named `SimpleApp.scala`: {% highlight scala %} /*** SimpleApp.scala ***/ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.SparkConf object SimpleApp { def main(args: Array[String]) { - val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system - val sc = new SparkContext("local", "Simple App", "YOUR_SPARK_HOME", - List("target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar")) + val logFile = "YOUR_SPARK_HOME/README.md" // Should be some file on your system + val conf = new SparkConf().setAppName("Simple Application") + val sc = new SparkContext(conf) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() @@ -124,9 +130,17 @@ object SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. +This program just counts the number of lines containing 'a' and the number containing 'b' in the +Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is +installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, +we initialize a SparkContext as part of the program. -This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on: +We pass the SparkContext constructor a SparkConf object which contains information about our +application. We also call sc.addJar to make sure that when our application is launched in cluster +mode, the jar file containing it will be shipped automatically to worker nodes. + +This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` +which explains that Spark is a dependency. This file also adds a repository that Spark depends on: {% highlight scala %} name := "Simple Project" @@ -140,15 +154,19 @@ libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSIO resolvers += "Akka Repository" at "http://repo.akka.io/releases/" {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: +If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on +`hadoop-client` for your version of HDFS: {% highlight scala %} libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "" {% endhighlight %} -Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt/sbt run` to execute our program. +Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` +according to the typical directory structure. Once that is in place, we can create a JAR package +containing the application's code, then use the `spark-submit` script to run our program. {% highlight bash %} +# Your directory layout should look like this $ find . . ./simple.sbt @@ -157,14 +175,22 @@ $ find . ./src/main/scala ./src/main/scala/SimpleApp.scala -$ sbt/sbt package -$ sbt/sbt run +# Package a jar containing your application +$ sbt package +... +[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar + +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit target/scala-2.10/simple-project_2.10-1.0.jar \ + --class "SimpleApp" \ + --master local[4] ... Lines with a: 46, Lines with b: 23 {% endhighlight %} # A Standalone App in Java -Now say we wanted to write a standalone application using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. +Now say we wanted to write a standalone application using the Java API. This example will use +maven to compile an application jar, but any similar build system will work. We'll create a very simple Spark application, `SimpleApp.java`: From af09e3ef2232956d2640d3c983a0352ca2bb8c63 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 20 Apr 2014 20:26:21 -0700 Subject: [PATCH 27/31] Mention config file in docs and clean-up docs --- docs/cluster-overview.md | 127 ++++++++++++++++------------- docs/quick-start.md | 136 ++++++++++++-------------------- docs/scala-programming-guide.md | 46 +++++------ docs/sql-programming-guide.md | 2 +- 4 files changed, 146 insertions(+), 165 deletions(-) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index a2ecebf3140f5..dea766ab30106 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -50,61 +50,78 @@ The system currently supports three cluster managers: In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone cluster on Amazon EC2. -# Launching Applications - -The recommended way to launch a compiled Spark application is through the spark-submit script (located in the -bin directory), which takes care of setting up the classpath with Spark and its dependencies, as well as -provides a layer over the different cluster managers and deploy modes that Spark supports. It's usage is - - spark-submit `` `` - -Where options are any of: - -- **\--class** - The main class to run. -- **\--master** - The URL of the cluster manager master, e.g. spark://host:port, mesos://host:port, yarn, - or local. -- **\--deploy-mode** - "client" to run the driver in the client process or "cluster" to run the driver in - a process on the cluster. For Mesos, only "client" is supported. -- **\--executor-memory** - Memory per executor (e.g. 1000M, 2G). -- **\--executor-cores** - Number of cores per executor. (Default: 2) -- **\--driver-memory** - Memory for driver (e.g. 1000M, 2G) -- **\--name** - Name of the application. -- **\--arg** - Argument to be passed to the application's main class. This option can be specified - multiple times to pass multiple arguments. -- **\--jars** - A comma-separated list of local jars to include on the driver classpath and that - SparkContext.addJar will work with. Doesn't work on standalone with 'cluster' deploy mode. - -The following currently only work for Spark standalone with cluster deploy mode: - -- **\--driver-cores** - Cores for driver (Default: 1). -- **\--supervise** - If given, restarts the driver on failure. - -The following only works for Spark standalone and Mesos only: - -- **\--total-executor-cores** - Total cores for all executors. - -The following currently only work for YARN: - -- **\--queue** - The YARN queue to place the application in. -- **\--files** - Comma separated list of files to be placed in the working dir of each executor. -- **\--archives** - Comma separated list of archives to be extracted into the working dir of each - executor. -- **\--num-executors** - Number of executors (Default: 2). - -The master and deploy mode can also be set with the MASTER and DEPLOY_MODE environment variables. -Values for these options passed via command line will override the environment variables. - -# Shipping Code to the Cluster - -The recommended way to ship your code to the cluster is to pass it through SparkContext's constructor, -which takes a list of JAR files (Java/Scala) or .egg and .zip libraries (Python) to disseminate to -worker nodes. You can also dynamically add new files to be sent to executors with `SparkContext.addJar` -and `addFile`. - -## URIs for addJar / addFile - -- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor - pulls the file from the driver HTTP server +# Bundling and Launching Applications + +### Bundling Your Application's Dependencies +If your code depends on other projects, you will need to package them alongside +your application in order to distribute the code to a Spark cluster. To do this, +to create an assembly jar (or "uber" jar) containing your code and its dependencies. Both +[sbt](https://github.com/sbt/sbt-assembly) and +[Maven](http://maven.apache.org/plugins/maven-shade-plugin/) +have assembly plugins. When creating assembly jars, list Spark and Hadoop +as `provided` dependencies; these need not be bundled since they are provided by +the cluster manager at runtime. Once you have an assembled jar you can call the `bin/spark-submit` +script as shown here while passing your jar. + +For Python, you can use the `pyFiles` argument of SparkContext +or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. + +### Launching Applications with ./bin/spark-submit + +Once a user application is bundled, it can be launched using the `spark-submit` script located in +the bin directory. This script takes care of setting up the classpath with Spark and its +dependencies, and can support different cluster managers and deploy modes that Spark supports. +It's usage is + + ./bin/spark-submit --class path.to.your.Class [other options..] + +To enumerate all options available to `spark-submit` run it with the `--help` flag. +Here are a few examples of common options: + +{% highlight bash %} +# Run application locally +./bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master local[8] + +# Run on a Spark cluster +./bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master spark://mycluster:7077 \ + --executor-memory 20G \ + --total-executor-cores 100 + +# Run on a YARN cluster +HADOOP_CONF_DIR=XX /bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master yarn-cluster \ # can also be `yarn-client` for client mode + --executor-memory 20G \ + --num-executors 50 +{% endhighlight %} + +### Loading Configurations from a File + +The `spark-submit` script can load default `SparkConf` values from a properties file and pass them +onto your application. By default it will read configuration options from +`conf/spark-defaults.properties`. Any values specified in the file will be passed on to the +application when run. They can obviate the need for certain flags to `spark-submit`: for +instance, if `spark.master` property is set, you can safely omit the +`--master` flag from `spark-submit`. In general, configuration values explicitly set on a +`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values +in the defaults file. + +If you are ever unclear where configuration options are coming from. fine-grained debugging +information can be printed by adding the `--verbose` option to `./spark-submit`. + +### Advanced Dependency Management +When using `./bin/spark-submit` jars will be automatically transferred to the cluster. For many +users this is sufficient. However, advanced users can add jars by calling `addFile` or `addJar` +on an existing SparkContext. This can be used to distribute JAR files (Java/Scala) or .egg and +.zip libraries (Python) to executors. Spark uses the following URL scheme to allow different +strategies for disseminating jars: + +- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and + every executor pulls the file from the driver HTTP server - **hdfs:**, **http:**, **https:**, **ftp:** - these pull down files and JARs from the URI as expected - **local:** - a URI starting with local:/ is expected to exist as a local file on each worker node. This means that no network IO will be incurred, and works well for large files/JARs that are pushed to each worker, diff --git a/docs/quick-start.md b/docs/quick-start.md index 5b7a6582cd320..6b4f4ba4254a2 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -104,10 +104,13 @@ that these same functions can be used on very large data sets, even when they ar tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). -# A Standalone App in Scala +# A Standalone Application Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. +
+
+ We'll create a very simple Spark application in Scala. So simple, in fact, that it's named `SimpleApp.scala`: @@ -154,14 +157,7 @@ libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSIO resolvers += "Akka Repository" at "http://repo.akka.io/releases/" {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on -`hadoop-client` for your version of HDFS: - -{% highlight scala %} -libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "" -{% endhighlight %} - -Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` +For sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use the `spark-submit` script to run our program. @@ -188,22 +184,23 @@ $ YOUR_SPARK_HOME/bin/spark-submit target/scala-2.10/simple-project_2.10-1.0.jar Lines with a: 46, Lines with b: 23 {% endhighlight %} -# A Standalone App in Java -Now say we wanted to write a standalone application using the Java API. This example will use -maven to compile an application jar, but any similar build system will work. +
+
+This example will use Maven to compile an application jar, but any similar build system will work. We'll create a very simple Spark application, `SimpleApp.java`: {% highlight java %} /*** SimpleApp.java ***/ import org.apache.spark.api.java.*; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.Function; public class SimpleApp { public static void main(String[] args) { - String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system - JavaSparkContext sc = new JavaSparkContext("local", "Simple App", - "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"}); + String logFile = "YOUR_SPARK_HOME/README.md"; // Should be some file on your system + SparkConf conf = new SparkConf().setAppName("Simple Application"); + JavaSparkContext sc = new JavaSparkContext(conf); JavaRDD logData = sc.textFile(logFile).cache(); long numAs = logData.filter(new Function() { @@ -219,9 +216,16 @@ public class SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail. +This program just counts the number of lines containing 'a' and the number containing 'b' in a text +file. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. +As with the Scala example, we initialize a SparkContext, though we use the special +`JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by +`JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes +that extend `spark.api.java.function.Function`. The +[Java programming guide](java-programming-guide.html) describes these differences in more detail. -To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version. +To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. +Note that Spark artifacts are tagged with a Scala version. {% highlight xml %} @@ -247,16 +251,6 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: - -{% highlight xml %} - - org.apache.hadoop - hadoop-client - ... - -{% endhighlight %} - We lay out these files according to the canonical Maven directory structure: {% highlight bash %} $ find . @@ -267,16 +261,25 @@ $ find . ./src/main/java/SimpleApp.java {% endhighlight %} -Now, we can execute the application using Maven: +Now, we can package the application using Maven and execute it with `./bin/spark-submit`. {% highlight bash %} +# Package a jar containing your application $ mvn package -$ mvn exec:java -Dexec.mainClass="SimpleApp" +... +[INFO] Building jar: {..}/{..}/target/simple-project-1.0.jar + +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit target/simple-project-1.0.jar \ + --class "SimpleApp" \ + --master local[4] ... Lines with a: 46, Lines with b: 23 {% endhighlight %} -# A Standalone App in Python +
+
+ Now we will show how to write a standalone application using the Python API (PySpark). As an example, we'll create a simple Spark application, `SimpleApp.py`: @@ -285,7 +288,7 @@ As an example, we'll create a simple Spark application, `SimpleApp.py`: """SimpleApp.py""" from pyspark import SparkContext -logFile = "$YOUR_SPARK_HOME/README.md" # Should be some file on your system +logFile = "YOUR_SPARK_HOME/README.md" # Should be some file on your system sc = SparkContext("local", "Simple App") logData = sc.textFile(logFile).cache() @@ -296,11 +299,15 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs) {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. -Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. +This program just counts the number of lines containing 'a' and the number containing 'b' in a +text file. +Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala and Java examples, we use a SparkContext to create RDDs. -We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. -For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html). +We can pass Python functions to Spark, which are automatically serialized along with any variables +that they reference. +For applications that use custom classes or third-party libraries, we can add those code +dependencies to SparkContext to ensure that they will be available on remote machines; this is +described in more detail in the [Python programming guide](python-programming-guide.html). `SimpleApp` is simple enough that we do not need to specify any code dependencies. We can run this application using the `bin/pyspark` script: @@ -312,57 +319,12 @@ $ ./bin/pyspark SimpleApp.py Lines with a: 46, Lines with b: 23 {% endhighlight python %} -# Running on a Cluster - -There are a few additional considerations when running applicaitons on a -[Spark](spark-standalone.html), [YARN](running-on-yarn.html), or -[Mesos](running-on-mesos.html) cluster. - -### Including Your Dependencies -If your code depends on other projects, you will need to ensure they are also -present on the slave nodes. A popular approach is to create an -assembly jar (or "uber" jar) containing your code and its dependencies. Both -[sbt](https://github.com/sbt/sbt-assembly) and -[Maven](http://maven.apache.org/plugins/maven-assembly-plugin/) -have assembly plugins. When creating assembly jars, list Spark -itself as a `provided` dependency; it need not be bundled since it is -already present on the slaves. Once you have an assembled jar, -add it to the SparkContext as shown here. It is also possible to add -your dependent jars one-by-one using the `addJar` method of `SparkContext`. - -For Python, you can use the `pyFiles` argument of SparkContext -or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. - -### Setting Configuration Options -Spark includes several [configuration options](configuration.html#spark-properties) -that influence the behavior of your application. -These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) -object and passing it to the SparkContext constructor. -For example, in Java and Scala, you can do: - -{% highlight scala %} -import org.apache.spark.{SparkConf, SparkContext} -val conf = new SparkConf() - .setMaster("local") - .setAppName("My application") - .set("spark.executor.memory", "1g") -val sc = new SparkContext(conf) -{% endhighlight %} - -Or in Python: - -{% highlight scala %} -from pyspark import SparkConf, SparkContext -conf = SparkConf() -conf.setMaster("local") -conf.setAppName("My application") -conf.set("spark.executor.memory", "1g")) -sc = SparkContext(conf = conf) -{% endhighlight %} +
+
-### Accessing Hadoop Filesystems +# Where to go from here +Congratulations on running your first Spark application! -The examples here access a local file. To read data from a distributed -filesystem, such as HDFS, include -[Hadoop version information](index.html#a-note-about-hadoop-versions) -in your build file. By default, Spark builds against HDFS 1.0.4. +* For an in-depth overview of the API see "Programming Guides" menu section. +* For running applications on a cluster head to the [deployment overview](cluster-overview.html). +* For configuration options available to Spark applications see the [configuration page](configuration.html). \ No newline at end of file diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 2b0a51e9dfc54..4431da0721ac7 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -25,36 +25,43 @@ To write a Spark application, you need to add a dependency on Spark. If you use artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} -In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS: +In addition, if you wish to access an HDFS cluster, you need to add a dependency on +`hadoop-client` for your version of HDFS. Some common HDFS version tags are listed on the +[third party distributions](hadoop-third-party-distributions.html) page. groupId = org.apache.hadoop artifactId = hadoop-client version = -For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). - Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines: {% highlight scala %} import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.SparkConf {% endhighlight %} # Initializing Spark -The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark how to access a cluster. -This is done through the following constructor: +The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a `SparkConf` object +that contains information about your application. {% highlight scala %} -new SparkContext(master, appName, [sparkHome], [jars]) +val conf = new SparkConf().setAppName().setMaster() +new SparkContext(conf) {% endhighlight %} -or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) -object for more advanced configuration. - -The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. +The `` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls) +to connect to, or a special "local" string to run in local mode, as described below. `` is +a name for your application, which will be shown in the cluster web UI. It's also possible to set +these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) +which avoids hard-coding the master name in your application. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the +variable called `sc`. Making your own SparkContext will not work. You can set which master the +context connects to using the `MASTER` environment variable, and you can add JARs to the classpath +with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use {% highlight bash %} $ MASTER=local[4] ./bin/spark-shell @@ -83,21 +90,16 @@ The master URL passed to Spark can be in one of the following formats: The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, which is 5050 by default. + yarn-client Connect to a YARN cluster in +client mode. The cluster location will be inferred based on the local Hadoop configuration. + + yarn-cluster Connect to a YARN cluster in +cluster mode. The cluster location will be inferred based on the local Hadoop configuration. + If no master URL is specified, the spark shell defaults to "local[*]". -For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. - -### Deploying Code on a Cluster - -If you want to run your application on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code: - -* `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them). -* `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies. - -If you run `bin/spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./bin/spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed. - # Resilient Distributed Datasets (RDDs) Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are currently two types of RDDs: *parallelized collections*, which take an existing Scala collection and run functions on it in parallel, and *Hadoop datasets*, which run functions on each record of a file in Hadoop distributed file system or any other storage system supported by Hadoop. Both types of RDDs can be operated on through the same methods. diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6f616fb7c2448..8e98cc0c80a34 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -69,7 +69,7 @@ import sqlContext._ -
+
The entry point into all relational functionality in Spark is the [JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one From 0086939e3a930c95bbb35019deea6911b38d5544 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 20 Apr 2014 20:50:41 -0700 Subject: [PATCH 28/31] Minor style fixes --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 1 - .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 3dea722a02da5..494b356aeaa99 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -107,7 +107,6 @@ object SparkSubmit { val sysProps = new HashMap[String, String]() var childMainClass = "" - if (clusterManager == MESOS && deployOnCluster) { printErrorAndExit("Mesos does not support running the driver on the cluster") } 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 eba6e981b51d3..0c869b98e9316 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -325,4 +325,4 @@ object SparkSubmitArguments { } properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) } -} \ No newline at end of file +} From b4b496c95b8c41dfc4c9c3a55551f189eb415a9f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 20 Apr 2014 20:57:04 -0700 Subject: [PATCH 29/31] spark-defaults.properties -> spark-defaults.conf --- .rat-excludes | 2 +- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 4 ++-- docs/cluster-overview.md | 2 +- docs/configuration.md | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.rat-excludes b/.rat-excludes index fe5a55f5e96b6..50766954ef070 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -11,7 +11,7 @@ RELEASE control docs fairscheduler.xml.template -spark-defaults.properties.template +spark-defaults.conf.template log4j.properties log4j.properties.template metrics.properties.template diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 20aa3d73dabda..bd21fdc5a18e4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -244,7 +244,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { |This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. | |Please instead use: - | - ./spark-submit with conf/spark-defaults.properties to set defaults for an application + | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application | - ./spark-submit with --driver-java-options to set -X options for a driver | - spark.executor.extraJavaOptions to set -X options for executors | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) 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 0c869b98e9316..fb1abc6e8490b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -82,7 +82,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (propertiesFile == null) { sys.env.get("SPARK_HOME").foreach { sparkHome => val sep = File.separator - val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.properties" + val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.conf" val file = new File(defaultPath) if (file.exists()) { propertiesFile = file.getAbsolutePath @@ -283,7 +283,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | --files FILES Comma separated list of files to be placed in the working dir | of each executor. | --properties-file FILE Path to a file from which to load extra properties. If not - | specified, this will look for conf/spark-defaults.properties. + | specified, this will look for conf/spark-defaults.conf. | | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). | --driver-java-options Extra Java options to pass to the driver diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index dea766ab30106..dcc063042628c 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -103,7 +103,7 @@ HADOOP_CONF_DIR=XX /bin/spark-submit my-app.jar \ The `spark-submit` script can load default `SparkConf` values from a properties file and pass them onto your application. By default it will read configuration options from -`conf/spark-defaults.properties`. Any values specified in the file will be passed on to the +`conf/spark-defaults.conf`. Any values specified in the file will be passed on to the application when run. They can obviate the need for certain flags to `spark-submit`: for instance, if `spark.master` property is set, you can safely omit the `--master` flag from `spark-submit`. In general, configuration values explicitly set on a diff --git a/docs/configuration.md b/docs/configuration.md index ce34a367a5bce..5a4abca2646b6 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -659,7 +659,7 @@ Apart from these, the following properties are also available, and may be useful A string of extra JVM options to pass to executors. For instance, GC settings or other logging. Note that it is illegal to set Spark properties or heap size settings with this option. Spark properties should be set using a SparkConf object or the - spark-defaults.properties file used with the spark-submit script. Heap size settings can be set + spark-defaults.conf file used with the spark-submit script. Heap size settings can be set with spark.executor.memory. From a0064640394a97a114cdcca4de00666c802605bb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 20 Apr 2014 21:26:53 -0700 Subject: [PATCH 30/31] Moving properties file template. --- ...-defaults.properties.template => spark-defaults.conf.template} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename conf/{spark-defaults.properties.template => spark-defaults.conf.template} (100%) diff --git a/conf/spark-defaults.properties.template b/conf/spark-defaults.conf.template similarity index 100% rename from conf/spark-defaults.properties.template rename to conf/spark-defaults.conf.template From 127f301104717761d0ba6e9ab1de7729380ab750 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 20 Apr 2014 23:22:44 -0700 Subject: [PATCH 31/31] Improvements to testing --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 6 +- .../spark/deploy/SparkSubmitArguments.scala | 2 +- .../spark/deploy/SparkSubmitSuite.scala | 103 ++++++++++++++---- 4 files changed, 90 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b9011d238b43c..9bebaef84e1c9 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -160,7 +160,7 @@ class SparkContext(config: SparkConf) extends Logging { throw new SparkException("A master URL must be set in your configuration") } if (!conf.contains("spark.app.name")) { - throw new SparkException("An application must be set in your configuration") + throw new SparkException("An application name must be set in your configuration") } if (conf.getBoolean("spark.logConf", false)) { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 494b356aeaa99..1b1e0fce0e0df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -145,11 +145,13 @@ object SparkSubmit { sysProp = "spark.cores.max"), new OptionAssigner(appArgs.files, YARN, false, sysProp = "spark.yarn.dist.files"), new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), - new OptionAssigner(appArgs.files, STANDALONE | MESOS, true, sysProp = "spark.files"), new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars"), - new OptionAssigner(appArgs.jars, STANDALONE | MESOS, false, sysProp = "spark.jars") + new OptionAssigner(appArgs.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), + new OptionAssigner(appArgs.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), + new OptionAssigner(appArgs.name, LOCAL | STANDALONE | MESOS, false, + sysProp = "spark.app.name") ) // For client mode make any added jars immediately visible on the classpath 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 fb1abc6e8490b..02502adfbd0c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -245,7 +245,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { val suggestion: Option[String] = value match { case v if v.startsWith("--") && v.contains("=") => val parts = v.split("=") - Some(s"Perhaps you meant '${parts(0)} ${parts(1)}'?") + Some(s"Perhaps you want '${parts(0)} ${parts(1)}'?") case _ => None } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 115b3b084bbe0..f82d717719b69 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -21,14 +21,11 @@ import java.io.{File, OutputStream, PrintStream} import scala.collection.mutable.ArrayBuffer -import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers - +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException, TestUtils} import org.apache.spark.deploy.SparkSubmit._ -import org.scalatest.prop.Tables.Table -import org.scalatest.prop.TableDrivenPropertyChecks._ import org.apache.spark.util.Utils - +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers class SparkSubmitSuite extends FunSuite with ShouldMatchers { @@ -45,7 +42,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } /** Returns true if the script exits and the given search string is printed. */ - def testPrematureExit(input: Array[String], searchString: String): Boolean = { + def testPrematureExit(input: Array[String], searchString: String) = { val printStream = new BufferPrintStream() SparkSubmit.printStream = printStream @@ -63,35 +60,38 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } thread.start() thread.join() - printStream.lineBuffer.find(s => s.contains(searchString)).size > 0 + val joined = printStream.lineBuffer.mkString("\n") + if (!joined.contains(searchString)) { + fail(s"Search string '$searchString' not found in $joined") + } } test("prints usage on empty input") { - testPrematureExit(Array[String](), "Usage: spark-submit") should be (true) + testPrematureExit(Array[String](), "Usage: spark-submit") } test("prints usage with only --help") { - testPrematureExit(Array("--help"), "Usage: spark-submit") should be (true) + testPrematureExit(Array("--help"), "Usage: spark-submit") } test("prints error with unrecognized option") { - testPrematureExit(Array("my.jar --blarg"), "Unrecognized option '--blarg'") should be (true) - testPrematureExit(Array("my.jar -bleg"), "Unrecognized option: '-bleg'") should be (true) - testPrematureExit(Array("my.jar --master=abc"), - "Unrecognized option: '--master=abc'. Perhaps you want '--master abc'?") should be (true) + testPrematureExit(Array("--blarg"), "Unrecognized option '--blarg'") + testPrematureExit(Array("-bleg"), "Unrecognized option '-bleg'") + testPrematureExit(Array("--master=abc"), + "Unrecognized option '--master=abc'. Perhaps you want '--master abc'?") } test("handles multiple binary definitions") { val adjacentJars = Array("foo.jar", "bar.jar") - testPrematureExit(adjacentJars, "error: Found two conflicting resources") should be (true) + testPrematureExit(adjacentJars, "error: Found two conflicting resources") val nonAdjacentJars = Array("foo.jar", "--master", "123", "--class", "abc", "bar.jar") - testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") should be (true) + testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") } test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "must specify a main class") + testPrematureExit(Array("foo.jar"), "Must specify a main class") } test("handles YARN cluster mode") { @@ -150,12 +150,11 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") - print("child args: " + childArgsStr) childArgsStr.startsWith("--memory 4g --cores 5 --supervise") should be (true) childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have length (0) - sysProps should have size (0) + sysProps should have size (1) // contains --jar entry } test("handles standalone client mode") { @@ -186,6 +185,27 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { sysProps("spark.cores.max") should be ("5") } + test("launch simple application with spark-submit") { + runSparkSubmit( + Seq("unUsed.jar", + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local")) + } + + test("spark submit includes jars passed in through --jar") { + val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) + val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) + val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") + runSparkSubmit( + Seq("unUsed.jar", + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--jars", jarsString)) + } + + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. def runSparkSubmit(args: Seq[String]): String = { val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get Utils.executeAndGetOutput( @@ -193,6 +213,51 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } +} + +object JarCreationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + val result = sc.makeRDD(1 to 100, 10).mapPartitions{ x => + var foundClasses = false + try { + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + foundClasses = true + } catch { + case _: Throwable => // catch all + } + Seq(foundClasses).iterator + }.collect() + if (result.contains(false)) { + throw new Exception("Could not load user defined classes inside of executors") + } + } +} + +object SimpleApplicationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + val configs = Seq("spark.master", "spark.app.name") + for (config <- configs) { + val masterValue = conf.get(config) + val executorValues = sc + .makeRDD(1 to 100, 10) + .map(x => SparkEnv.get.conf.get(config)) + .collect() + .distinct + if (executorValues.size != 1) { + throw new SparkException(s"Inconsistent values for $config: $executorValues") + } + val executorValue = executorValues(0) + if (executorValue != masterValue) { + throw new SparkException( + s"Master had $config=$masterValue but executor had $config=$executorValue") + } + } + } }