aboutsummaryrefslogtreecommitdiff
path: root/docs/scala-programming-guide.md
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/scala-programming-guide.md
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/scala-programming-guide.md')
-rw-r--r--docs/scala-programming-guide.md46
1 files changed, 24 insertions, 22 deletions
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.