Skip to content

Commit

Permalink
Clean up and simplify Spark configuration
Browse files Browse the repository at this point in the history
Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements:

1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file.
2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath.
3. Adds ability to set these same variables for the driver using `spark-submit`.
4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`.
5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node.

Author: Patrick Wendell <[email protected]>

Closes apache#299 from pwendell/config-cleanup and squashes the following commits:

127f301 [Patrick Wendell] Improvements to testing
a006464 [Patrick Wendell] Moving properties file template.
b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf
0086939 [Patrick Wendell] Minor style fixes
af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs
b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide
af0adf7 [Patrick Wendell] Automatically add user jar
a56b125 [Patrick Wendell] Responses to Tom's review
d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup
a762901 [Patrick Wendell] Fixing test failures
ffa00fe [Patrick Wendell] Review feedback
fda0301 [Patrick Wendell] Note
308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN
e83cd8f [Patrick Wendell] Changes to allow re-use of test applications
be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set
c2a2909 [Patrick Wendell] Test compile fixes
4ee6f9d [Patrick Wendell] Making YARN doc changes consistent
afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors.
b08893b [Patrick Wendell] Additional improvements.
ace4ead [Patrick Wendell] Responses to review feedback.
b72d183 [Patrick Wendell] Review feedback for spark env file
46555c1 [Patrick Wendell] Review feedback and import clean-ups
437aed1 [Patrick Wendell] Small fix
761ebcd [Patrick Wendell] Library path and classpath for drivers
7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script
5b0ba8e [Patrick Wendell] Don't ship executor envs
84cc5e5 [Patrick Wendell] Small clean-up
1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings
4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH
6eaf7d0 [Patrick Wendell] executorJavaOpts
0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN
ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
  • Loading branch information
pwendell committed Apr 21, 2014
1 parent 3a390bf commit fb98488
Show file tree
Hide file tree
Showing 44 changed files with 886 additions and 401 deletions.
1 change: 1 addition & 0 deletions .rat-excludes
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ RELEASE
control
docs
fairscheduler.xml.template
spark-defaults.conf.template
log4j.properties
log4j.properties.template
metrics.properties.template
Expand Down
1 change: 0 additions & 1 deletion bin/run-example
Original file line number Diff line number Diff line change
Expand Up @@ -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`"
Expand Down
2 changes: 1 addition & 1 deletion bin/spark-class
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +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 -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
Expand Down
7 changes: 6 additions & 1 deletion bin/spark-submit
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
7 changes: 7 additions & 0 deletions conf/spark-defaults.conf.template
Original file line number Diff line number Diff line change
@@ -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
43 changes: 31 additions & 12 deletions conf/spark-env.sh.template
Original file line number Diff line number Diff line change
@@ -1,22 +1,41 @@
#!/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 programs.
# Copy it as spark-env.sh and edit that to configure Spark for your site.

# Options read when launching programs locally with
# ./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

# 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, 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
# - 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:

# Options read in YARN client mode
# - 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.
# - 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
# - 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 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
76 changes: 76 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,82 @@ 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. 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)'. " +
"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")) {
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 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)
""".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.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)
}
}
}
}

/**
* Return a string listing all keys and values, one per line. This is useful to print the
* configuration out for debugging.
Expand Down
37 changes: 20 additions & 17 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,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
Expand All @@ -159,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)) {
Expand All @@ -170,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")
Expand Down Expand Up @@ -235,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.")
Expand All @@ -247,30 +252,28 @@ 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]()
for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS");
value <- Option(System.getenv(key))) {
executorEnvs(key) = value
}
// 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 {
Option(System.getProperty("user.name")).getOrElse(System.getenv("SPARK_USER"))
}.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)
Expand Down
15 changes: 14 additions & 1 deletion core/src/main/scala/org/apache/spark/deploy/Client.scala
Original file line number Diff line number Diff line change
Expand Up @@ -54,8 +54,21 @@ 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 classPathConf = "spark.driver.extraClassPath"
val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp =>
cp.split(java.io.File.pathSeparator)
}

val libraryPathConf = "spark.driver.extraLibraryPath"
val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp =>
cp.split(java.io.File.pathSeparator)
}

val javaOptionsConf = "spark.driver.extraJavaOptions"
val javaOpts = sys.props.get(javaOptionsConf)
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,
Expand Down
5 changes: 4 additions & 1 deletion core/src/main/scala/org/apache/spark/deploy/Command.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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],
extraJavaOptions: Option[String] = None) {
}
Loading

0 comments on commit fb98488

Please sign in to comment.