spark-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From pwend...@apache.org
Subject [2/2] git commit: Clean up and simplify Spark configuration
Date Mon, 21 Apr 2014 17:26:43 GMT
Clean up and simplify Spark configuration

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 <pwendell@gmail.com>

Closes #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


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fb98488f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fb98488f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fb98488f

Branch: refs/heads/master
Commit: fb98488fc8e68cc84f6e0750fd4e9e29029879d2
Parents: 3a390bf
Author: Patrick Wendell <pwendell@gmail.com>
Authored: Mon Apr 21 10:26:33 2014 -0700
Committer: Patrick Wendell <pwendell@gmail.com>
Committed: Mon Apr 21 10:26:33 2014 -0700

----------------------------------------------------------------------
 .rat-excludes                                   |   1 +
 bin/run-example                                 |   1 -
 bin/spark-class                                 |   2 +-
 bin/spark-submit                                |   7 +-
 conf/spark-defaults.conf.template               |   7 +
 conf/spark-env.sh.template                      |  43 +++--
 .../main/scala/org/apache/spark/SparkConf.scala |  76 ++++++++
 .../scala/org/apache/spark/SparkContext.scala   |  37 ++--
 .../scala/org/apache/spark/deploy/Client.scala  |  15 +-
 .../scala/org/apache/spark/deploy/Command.scala |   5 +-
 .../org/apache/spark/deploy/SparkSubmit.scala   |  53 ++++--
 .../spark/deploy/SparkSubmitArguments.scala     | 190 +++++++++++++++----
 .../apache/spark/deploy/client/TestClient.scala |   4 +-
 .../spark/deploy/worker/CommandUtils.scala      |  19 +-
 .../spark/deploy/worker/DriverRunner.scala      |  16 +-
 .../spark/deploy/worker/ExecutorRunner.scala    |   4 +-
 .../org/apache/spark/executor/Executor.scala    |   5 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |  13 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |  23 ++-
 .../cluster/mesos/MesosSchedulerBackend.scala   |   2 +-
 .../apache/spark/deploy/JsonProtocolSuite.scala |  15 +-
 .../apache/spark/deploy/SparkSubmitSuite.scala  | 111 +++++++++--
 .../spark/deploy/worker/DriverRunnerTest.scala  |   2 +-
 .../deploy/worker/ExecutorRunnerTest.scala      |   3 +-
 dev/audit-release/audit_release.py              |   1 +
 .../sbt_app_core/src/main/scala/SparkApp.scala  |   7 +-
 .../src/main/scala/GraphxApp.scala              |  10 +-
 .../src/main/scala/StreamingApp.scala           |   8 +-
 docs/cluster-overview.md                        | 135 +++++++------
 docs/configuration.md                           |  50 +++--
 docs/quick-start.md                             | 178 ++++++++---------
 docs/scala-programming-guide.md                 |  46 ++---
 docs/sql-programming-guide.md                   |   2 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |   3 +
 .../org/apache/spark/deploy/yarn/Client.scala   |   4 +-
 .../spark/deploy/yarn/ExecutorLauncher.scala    |   6 +
 .../deploy/yarn/YarnAllocationHandler.scala     |   4 +
 .../apache/spark/deploy/yarn/ClientBase.scala   | 137 +++++++------
 .../deploy/yarn/ExecutorRunnableUtil.scala      |  21 +-
 .../cluster/YarnClientSchedulerBackend.scala    |   4 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |   3 +
 .../org/apache/spark/deploy/yarn/Client.scala   |   4 +-
 .../spark/deploy/yarn/ExecutorLauncher.scala    |   6 +
 .../deploy/yarn/YarnAllocationHandler.scala     |   4 +
 44 files changed, 886 insertions(+), 401 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/.rat-excludes
----------------------------------------------------------------------
diff --git a/.rat-excludes b/.rat-excludes
index 9e9abb3..5076695 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -11,6 +11,7 @@ RELEASE
 control
 docs
 fairscheduler.xml.template
+spark-defaults.conf.template
 log4j.properties
 log4j.properties.template
 metrics.properties.template

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/bin/run-example
----------------------------------------------------------------------
diff --git a/bin/run-example b/bin/run-example
index 5af95a0..b299919 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`"

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/bin/spark-class
----------------------------------------------------------------------
diff --git a/bin/spark-class b/bin/spark-class
index 1b0d309..6871e18 100755
--- a/bin/spark-class
+++ b/bin/spark-class
@@ -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

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/bin/spark-submit
----------------------------------------------------------------------
diff --git a/bin/spark-submit b/bin/spark-submit
index d92d55a..498d0b2 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
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/conf/spark-defaults.conf.template
----------------------------------------------------------------------
diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template
new file mode 100644
index 0000000..f840ff6
--- /dev/null
+++ b/conf/spark-defaults.conf.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

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/conf/spark-env.sh.template
----------------------------------------------------------------------
diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template
index 6432a56..177a21c 100755
--- a/conf/spark-env.sh.template
+++ b/conf/spark-env.sh.template
@@ -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

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index b947feb..bd21fdc 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -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.

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index d3ef75b..7933d68 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -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
@@ -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)) {
@@ -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")
@@ -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.")
@@ -247,22 +252,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]()
-  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 {
@@ -270,7 +273,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)

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/deploy/Client.scala
----------------------------------------------------------------------
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 8fd2c7e..7ead117 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,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,

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/deploy/Command.scala
----------------------------------------------------------------------
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 fa8af9a..32f3ba3 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],
+    extraJavaOptions: Option[String] = None) {
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
----------------------------------------------------------------------
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 e5d593c..1b1e0fc 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,12 @@
 
 package org.apache.spark.deploy
 
-import java.io.{PrintStream, File}
+import java.io.{File, PrintStream}
 import java.net.{URI, URL}
 
-import org.apache.spark.executor.ExecutorURLClassLoader
+import scala.collection.mutable.{ArrayBuffer, HashMap, Map}
 
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
+import org.apache.spark.executor.ExecutorURLClassLoader
 
 /**
  * Scala code behind the spark-submit script.  The script handles setting up the classpath with
@@ -63,7 +61,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) = {
@@ -123,6 +122,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.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"),
@@ -142,10 +147,14 @@ 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.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")
     )
 
-    // 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
@@ -163,6 +172,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"
@@ -173,7 +190,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
@@ -184,6 +201,10 @@ object SparkSubmit {
       }
     }
 
+    for ((k, v) <- appArgs.getDefaultSparkProperties) {
+      if (!sysProps.contains(k)) sysProps(k) = v
+    }
+
     (childArgs, childClasspath, sysProps, childMainClass)
   }
 
@@ -191,11 +212,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),
@@ -226,6 +247,10 @@ object SparkSubmit {
   }
 }
 
+/**
+ * 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,

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index 834b3df..02502ad 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -17,18 +17,28 @@
 
 package org.apache.spark.deploy
 
-import scala.collection.mutable.ArrayBuffer
+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
   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
@@ -42,42 +52,102 @@ 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
+  }
+
+  /** 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.conf"
+        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)
 
-  // 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")
+    // 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 =  {
     s"""Parsed arguments:
-    |  master             $master
-    |  deployMode         $deployMode
-    |  executorMemory     $executorMemory
-    |  executorCores      $executorCores
-    |  totalExecutorCores $totalExecutorCores
-    |  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
+    |
+    |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
@@ -122,6 +192,22 @@ private[spark] class SparkSubmitArguments(args: Array[String]) {
       driverCores = value
       parseOpts(tail)
 
+    case ("--driver-class-path") :: value :: tail =>
+      driverExtraClassPath = value
+      parseOpts(tail)
+
+    case ("--driver-java-options") :: 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)
+
     case ("--supervise") :: tail =>
       supervise = true
       parseOpts(tail)
@@ -154,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 want '${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."
@@ -178,11 +276,21 @@ 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.
+        |  --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.conf.
+        |
+        |  --driver-memory MEM         Memory for driver (e.g. 1000M, 2G) (Default: 512M).
+        |  --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
+        |
+        |  --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).
@@ -193,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)))
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
----------------------------------------------------------------------
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 63f166d..888dd45 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()), Some("dummy-spark-home"), "ignored")
     val listener = new TestListener
     val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf)
     client.start()

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
----------------------------------------------------------------------
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 0c761df..9103c88 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,21 +46,26 @@ 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")
+    // 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)
+        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) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts
+    Seq("-cp", classPathWithUser) ++ libraryOpts ++ extraOpts ++ memoryOpts
   }
 
   /** Spawn a thread that will redirect a given stream to a file */

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
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 b4df1a0..f918b42 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.extraJavaOptions)
           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)

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------
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 2edd921..f94cd68 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.extraJavaOptions)
     CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/executor/Executor.scala
----------------------------------------------------------------------
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 f89b2bf..2bfb9c3 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) {

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
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 936e9db..9544ca0 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,11 +42,20 @@ 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 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)
+    }
+
     val command = Command(
-      "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
+      "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))

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
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 06b041e..2cd9d6c 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,18 @@ 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 =>
+      environment.addVariables(
+        Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build())
+    }
+    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) =>
       environment.addVariables(Environment.Variable.newBuilder()
         .setName(key)
         .setValue(value)
@@ -123,20 +134,22 @@ private[spark] class CoarseMesosSchedulerBackend(
       conf.get("spark.driver.host"),
       conf.get("spark.driver.port"),
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
     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()

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
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 dfdcafe..c975f31 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
----------------------------------------------------------------------
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 9f2924c..bfae32d 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"), Some("-Dfoo")
   )
 
   def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3,
@@ -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", "<unknown>")).stripMargin
 
   val executorRunnerJsonStr =

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
----------------------------------------------------------------------
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 4e489cd..f82d717 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -17,16 +17,16 @@
 
 package org.apache.spark.deploy
 
-import java.io.{OutputStream, PrintStream}
+import java.io.{File, OutputStream, PrintStream}
 
 import scala.collection.mutable.ArrayBuffer
 
+import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException, TestUtils}
+import org.apache.spark.deploy.SparkSubmit._
+import org.apache.spark.util.Utils
 import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
 
-import org.apache.spark.deploy.SparkSubmit._
-
-
 class SparkSubmitSuite extends FunSuite with ShouldMatchers {
 
   val noOpOutputStream = new OutputStream {
@@ -42,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
 
@@ -60,28 +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("--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") {
@@ -140,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") {
@@ -175,4 +184,80 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers {
     sysProps("spark.executor.memory") should be ("5g")
     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(
+      Seq("./bin/spark-submit") ++ args,
+      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")
+      }
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
----------------------------------------------------------------------
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 a2c131b..4633bc3 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/")

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
----------------------------------------------------------------------
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 3cab8e7..8ae387f 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(".")),

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/dev/audit-release/audit_release.py
----------------------------------------------------------------------
diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py
index fa2f02d..4a816d4 100755
--- a/dev/audit-release/audit_release.py
+++ b/dev/audit-release/audit_release.py
@@ -114,6 +114,7 @@ os.chdir("blank_sbt_build")
 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
----------------------------------------------------------------------
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 53fe432..a89b0d7 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()

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala
----------------------------------------------------------------------
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 da08e01..24c7f8d 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")),

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala
----------------------------------------------------------------------
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 3d0722d..a1d8971 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/docs/cluster-overview.md
----------------------------------------------------------------------
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index 7f75ea4..dcc0630 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 `<app jar>` `<options>`
-
-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 <app jar> --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.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
+`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,
@@ -139,6 +156,14 @@ The following table summarizes terms you'll see used to refer to cluster concept
       <td>User program built on Spark. Consists of a <em>driver program</em> and <em>executors</em> on the cluster.</td>
     </tr>
     <tr>
+      <td>Application jar</td>
+      <td>
+        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.
+      </td>
+    </tr>
+    <tr>
       <td>Driver program</td>
       <td>The process running the main() function of the application and creating the SparkContext</td>
     </tr>

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index a302983..5a4abca 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.
   </td>
 </tr>
 <tr>
@@ -578,7 +581,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. <br/>
     <b>Note:</b> this setting needs to be configured in the standalone cluster master, not in individual
-    applications; you can set it through <code>SPARK_JAVA_OPTS</code> in <code>spark-env.sh</code>.
+    applications; you can set it through <code>SPARK_MASTER_OPTS</code> in <code>spark-env.sh</code>.
   </td>
 </tr>
 <tr>
@@ -591,7 +594,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. <br/>
     <b>Note:</b> this setting needs to be configured in the standalone cluster master, not in individual
-    applications; you can set it through <code>SPARK_JAVA_OPTS</code> in <code>spark-env.sh</code>.
+    applications; you can set it through <code>SPARK_MASTER_OPTS</code> in <code>spark-env.sh</code>.
   </td>
 </tr>
 <tr>
@@ -649,6 +652,34 @@ Apart from these, the following properties are also available, and may be useful
     Number of cores to allocate for each task.
   </td>
 </tr>
+<tr>
+  <td>spark.executor.extraJavaOptions</td>
+  <td>(none)</td>
+  <td>
+    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
+    with spark.executor.memory.
+  </td>
+</tr>
+<tr>
+  <td>spark.executor.extraClassPath</td>
+  <td>(none)</td>
+  <td>
+    Extra classpath entries to append to the classpath of executors. This exists primarily
+    for backwards-compatibility with older versions of Spark. Users typically should not need
+    to set this option.
+  </td>
+</tr>
+<tr>
+  <td>spark.executor.extraLibraryPath</td>
+  <td>(none)</td>
+  <td>
+    Set a special library path to use when launching executor JVM's.
+  </td>
+</tr>
+
 </table>
 
 ## Viewing Spark Properties
@@ -659,10 +690,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.
@@ -672,13 +702,7 @@ 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.
-* `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`.
+* `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.
 


Mime
View raw message