aboutsummaryrefslogtreecommitdiff
path: root/docs
diff options
context:
space:
mode:
authorPatrick Wendell <pwendell@gmail.com>2014-04-21 10:26:33 -0700
committerPatrick Wendell <pwendell@gmail.com>2014-04-21 10:26:33 -0700
commitfb98488fc8e68cc84f6e0750fd4e9e29029879d2 (patch)
treeeba99b56bea8ec2e357020a413bf9cf04a4e3308 /docs
parent3a390bfd80f80739b9d847780eccc443fc2dc0ea (diff)
downloadspark-fb98488fc8e68cc84f6e0750fd4e9e29029879d2.tar.gz
spark-fb98488fc8e68cc84f6e0750fd4e9e29029879d2.tar.bz2
spark-fb98488fc8e68cc84f6e0750fd4e9e29029879d2.zip
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
Diffstat (limited to 'docs')
-rw-r--r--docs/cluster-overview.md135
-rw-r--r--docs/configuration.md50
-rw-r--r--docs/quick-start.md178
-rw-r--r--docs/scala-programming-guide.md46
-rw-r--r--docs/sql-programming-guide.md2
5 files changed, 225 insertions, 186 deletions
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index 7f75ea44e4..dcc0630426 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>
diff --git a/docs/configuration.md b/docs/configuration.md
index a3029837ff..5a4abca264 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.
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 60e8b1ba0e..6b4f4ba425 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -99,23 +99,32 @@ scala> linesWithSpark.count()
res9: Long = 15
{% endhighlight %}
-It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
+It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is
+that these same functions can be used on very large data sets, even when they are striped across
+tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to
+a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
-# A Standalone App in Scala
-Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
+# A Standalone Application
+Now say we wanted to write a standalone application using the Spark API. We will walk through a
+simple application in both Scala (with SBT), Java (with Maven), and Python.
-We'll create a very simple Spark application in Scala. So simple, in fact, that it's named `SimpleApp.scala`:
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+
+We'll create a very simple Spark application in Scala. So simple, in fact, that it's
+named `SimpleApp.scala`:
{% highlight scala %}
/*** SimpleApp.scala ***/
import org.apache.spark.SparkContext
import org.apache.spark.SparkContext._
+import org.apache.spark.SparkConf
object SimpleApp {
def main(args: Array[String]) {
- val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system
- val sc = new SparkContext("local", "Simple App", "YOUR_SPARK_HOME",
- List("target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar"))
+ val logFile = "YOUR_SPARK_HOME/README.md" // Should be some file on your system
+ val conf = new SparkConf().setAppName("Simple Application")
+ val sc = new SparkContext(conf)
val logData = sc.textFile(logFile, 2).cache()
val numAs = logData.filter(line => line.contains("a")).count()
val numBs = logData.filter(line => line.contains("b")).count()
@@ -124,9 +133,17 @@ object SimpleApp {
}
{% endhighlight %}
-This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
+This program just counts the number of lines containing 'a' and the number containing 'b' in the
+Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is
+installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext,
+we initialize a SparkContext as part of the program.
+
+We pass the SparkContext constructor a SparkConf object which contains information about our
+application. We also call sc.addJar to make sure that when our application is launched in cluster
+mode, the jar file containing it will be shipped automatically to worker nodes.
-This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on:
+This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt`
+which explains that Spark is a dependency. This file also adds a repository that Spark depends on:
{% highlight scala %}
name := "Simple Project"
@@ -140,15 +157,12 @@ libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSIO
resolvers += "Akka Repository" at "http://repo.akka.io/releases/"
{% endhighlight %}
-If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS:
-
-{% highlight scala %}
-libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "<your-hdfs-version>"
-{% endhighlight %}
-
-Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt/sbt run` to execute our program.
+For sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt`
+according to the typical directory structure. Once that is in place, we can create a JAR package
+containing the application's code, then use the `spark-submit` script to run our program.
{% highlight bash %}
+# Your directory layout should look like this
$ find .
.
./simple.sbt
@@ -157,27 +171,36 @@ $ find .
./src/main/scala
./src/main/scala/SimpleApp.scala
-$ sbt/sbt package
-$ sbt/sbt run
+# Package a jar containing your application
+$ sbt package
+...
+[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar
+
+# Use spark-submit to run your application
+$ YOUR_SPARK_HOME/bin/spark-submit target/scala-2.10/simple-project_2.10-1.0.jar \
+ --class "SimpleApp" \
+ --master local[4]
...
Lines with a: 46, Lines with b: 23
{% endhighlight %}
-# A Standalone App in Java
-Now say we wanted to write a standalone application using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
+</div>
+<div data-lang="java" markdown="1">
+This example will use Maven to compile an application jar, but any similar build system will work.
We'll create a very simple Spark application, `SimpleApp.java`:
{% highlight java %}
/*** SimpleApp.java ***/
import org.apache.spark.api.java.*;
+import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.Function;
public class SimpleApp {
public static void main(String[] args) {
- String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system
- JavaSparkContext sc = new JavaSparkContext("local", "Simple App",
- "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"});
+ String logFile = "YOUR_SPARK_HOME/README.md"; // Should be some file on your system
+ SparkConf conf = new SparkConf().setAppName("Simple Application");
+ JavaSparkContext sc = new JavaSparkContext(conf);
JavaRDD<String> logData = sc.textFile(logFile).cache();
long numAs = logData.filter(new Function<String, Boolean>() {
@@ -193,9 +216,16 @@ public class SimpleApp {
}
{% endhighlight %}
-This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
+This program just counts the number of lines containing 'a' and the number containing 'b' in a text
+file. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed.
+As with the Scala example, we initialize a SparkContext, though we use the special
+`JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by
+`JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes
+that extend `spark.api.java.function.Function`. The
+[Java programming guide](java-programming-guide.html) describes these differences in more detail.
-To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
+To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency.
+Note that Spark artifacts are tagged with a Scala version.
{% highlight xml %}
<project>
@@ -221,16 +251,6 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a
</project>
{% endhighlight %}
-If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS:
-
-{% highlight xml %}
-<dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-client</artifactId>
- <version>...</version>
-</dependency>
-{% endhighlight %}
-
We lay out these files according to the canonical Maven directory structure:
{% highlight bash %}
$ find .
@@ -241,16 +261,25 @@ $ find .
./src/main/java/SimpleApp.java
{% endhighlight %}
-Now, we can execute the application using Maven:
+Now, we can package the application using Maven and execute it with `./bin/spark-submit`.
{% highlight bash %}
+# Package a jar containing your application
$ mvn package
-$ mvn exec:java -Dexec.mainClass="SimpleApp"
+...
+[INFO] Building jar: {..}/{..}/target/simple-project-1.0.jar
+
+# Use spark-submit to run your application
+$ YOUR_SPARK_HOME/bin/spark-submit target/simple-project-1.0.jar \
+ --class "SimpleApp" \
+ --master local[4]
...
Lines with a: 46, Lines with b: 23
{% endhighlight %}
-# A Standalone App in Python
+</div>
+<div data-lang="python" markdown="1">
+
Now we will show how to write a standalone application using the Python API (PySpark).
As an example, we'll create a simple Spark application, `SimpleApp.py`:
@@ -259,7 +288,7 @@ As an example, we'll create a simple Spark application, `SimpleApp.py`:
"""SimpleApp.py"""
from pyspark import SparkContext
-logFile = "$YOUR_SPARK_HOME/README.md" # Should be some file on your system
+logFile = "YOUR_SPARK_HOME/README.md" # Should be some file on your system
sc = SparkContext("local", "Simple App")
logData = sc.textFile(logFile).cache()
@@ -270,11 +299,15 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs)
{% endhighlight %}
-This program just counts the number of lines containing 'a' and the number containing 'b' in a text file.
-Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed.
+This program just counts the number of lines containing 'a' and the number containing 'b' in a
+text file.
+Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed.
As with the Scala and Java examples, we use a SparkContext to create RDDs.
-We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference.
-For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html).
+We can pass Python functions to Spark, which are automatically serialized along with any variables
+that they reference.
+For applications that use custom classes or third-party libraries, we can add those code
+dependencies to SparkContext to ensure that they will be available on remote machines; this is
+described in more detail in the [Python programming guide](python-programming-guide.html).
`SimpleApp` is simple enough that we do not need to specify any code dependencies.
We can run this application using the `bin/pyspark` script:
@@ -286,57 +319,12 @@ $ ./bin/pyspark SimpleApp.py
Lines with a: 46, Lines with b: 23
{% endhighlight python %}
-# Running on a Cluster
-
-There are a few additional considerations when running applicaitons on a
-[Spark](spark-standalone.html), [YARN](running-on-yarn.html), or
-[Mesos](running-on-mesos.html) cluster.
-
-### Including Your Dependencies
-If your code depends on other projects, you will need to ensure they are also
-present on the slave nodes. A popular approach is to create an
-assembly jar (or "uber" jar) containing your code and its dependencies. Both
-[sbt](https://github.com/sbt/sbt-assembly) and
-[Maven](http://maven.apache.org/plugins/maven-assembly-plugin/)
-have assembly plugins. When creating assembly jars, list Spark
-itself as a `provided` dependency; it need not be bundled since it is
-already present on the slaves. Once you have an assembled jar,
-add it to the SparkContext as shown here. It is also possible to add
-your dependent jars one-by-one using the `addJar` method of `SparkContext`.
-
-For Python, you can use the `pyFiles` argument of SparkContext
-or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed.
-
-### Setting Configuration Options
-Spark includes several [configuration options](configuration.html#spark-properties)
-that influence the behavior of your application.
-These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
-object and passing it to the SparkContext constructor.
-For example, in Java and Scala, you can do:
-
-{% highlight scala %}
-import org.apache.spark.{SparkConf, SparkContext}
-val conf = new SparkConf()
- .setMaster("local")
- .setAppName("My application")
- .set("spark.executor.memory", "1g")
-val sc = new SparkContext(conf)
-{% endhighlight %}
-
-Or in Python:
-
-{% highlight scala %}
-from pyspark import SparkConf, SparkContext
-conf = SparkConf()
-conf.setMaster("local")
-conf.setAppName("My application")
-conf.set("spark.executor.memory", "1g"))
-sc = SparkContext(conf = conf)
-{% endhighlight %}
+</div>
+</div>
-### Accessing Hadoop Filesystems
+# Where to go from here
+Congratulations on running your first Spark application!
-The examples here access a local file. To read data from a distributed
-filesystem, such as HDFS, include
-[Hadoop version information](index.html#a-note-about-hadoop-versions)
-in your build file. By default, Spark builds against HDFS 1.0.4.
+* For an in-depth overview of the API see "Programming Guides" menu section.
+* For running applications on a cluster head to the [deployment overview](cluster-overview.html).
+* For configuration options available to Spark applications see the [configuration page](configuration.html). \ No newline at end of file
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 2b0a51e9df..4431da0721 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -25,36 +25,43 @@ To write a Spark application, you need to add a dependency on Spark. If you use
artifactId = spark-core_{{site.SCALA_BINARY_VERSION}}
version = {{site.SPARK_VERSION}}
-In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS:
+In addition, if you wish to access an HDFS cluster, you need to add a dependency on
+`hadoop-client` for your version of HDFS. Some common HDFS version tags are listed on the
+[third party distributions](hadoop-third-party-distributions.html) page.
groupId = org.apache.hadoop
artifactId = hadoop-client
version = <your-hdfs-version>
-For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions).
-
Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines:
{% highlight scala %}
import org.apache.spark.SparkContext
import org.apache.spark.SparkContext._
+import org.apache.spark.SparkConf
{% endhighlight %}
# Initializing Spark
-The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark how to access a cluster.
-This is done through the following constructor:
+The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark
+how to access a cluster. To create a `SparkContext` you first need to build a `SparkConf` object
+that contains information about your application.
{% highlight scala %}
-new SparkContext(master, appName, [sparkHome], [jars])
+val conf = new SparkConf().setAppName(<app name>).setMaster(<master>)
+new SparkContext(conf)
{% endhighlight %}
-or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
-object for more advanced configuration.
-
-The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later.
+The `<master>` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls)
+to connect to, or a special "local" string to run in local mode, as described below. `<app name>` is
+a name for your application, which will be shown in the cluster web UI. It's also possible to set
+these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file)
+which avoids hard-coding the master name in your application.
-In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use
+In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the
+variable called `sc`. Making your own SparkContext will not work. You can set which master the
+context connects to using the `MASTER` environment variable, and you can add JARs to the classpath
+with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use
{% highlight bash %}
$ MASTER=local[4] ./bin/spark-shell
@@ -83,21 +90,16 @@ The master URL passed to Spark can be in one of the following formats:
The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use,
which is 5050 by default.
</td></tr>
+<tr><td> yarn-client </td><td> Connect to a <a href="running-on-yarn.html"> YARN </a> cluster in
+client mode. The cluster location will be inferred based on the local Hadoop configuration.
+</td></tr>
+<tr><td> yarn-cluster </td><td> Connect to a <a href="running-on-yarn.html"> YARN </a> cluster in
+cluster mode. The cluster location will be inferred based on the local Hadoop configuration.
+</td></tr>
</table>
If no master URL is specified, the spark shell defaults to "local[*]".
-For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details.
-
-### Deploying Code on a Cluster
-
-If you want to run your application on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code:
-
-* `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them).
-* `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies.
-
-If you run `bin/spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./bin/spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed.
-
# Resilient Distributed Datasets (RDDs)
Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are currently two types of RDDs: *parallelized collections*, which take an existing Scala collection and run functions on it in parallel, and *Hadoop datasets*, which run functions on each record of a file in Hadoop distributed file system or any other storage system supported by Hadoop. Both types of RDDs can be operated on through the same methods.
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 6f616fb7c2..8e98cc0c80 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -69,7 +69,7 @@ import sqlContext._
</div>
-<div data-lang="java" markdown="1">
+<div data-lang="java" markdown="1">
The entry point into all relational functionality in Spark is the
[JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one