Skip to content

Commit

Permalink
Detail how to set configurations + remove legacy instructions
Browse files Browse the repository at this point in the history
This commit removes the section on using org.apache.spark.deploy.Client
to launch an application. This is subsumed by the Spark submit section
immediately preceding it.

This commit also clarifies how we set the Spark configuration properties
in the 1.0 world. Previously it was pretty unclear, and the necessary
details were in the wrong page (cluster-overview.html) instead of where
it is supposed to be (configuration.html).
  • Loading branch information
andrewor14 committed May 13, 2014
1 parent 5b7140a commit 3cc0649
Show file tree
Hide file tree
Showing 4 changed files with 58 additions and 70 deletions.
1 change: 1 addition & 0 deletions conf/spark-defaults.conf.template
Original file line number Diff line number Diff line change
Expand Up @@ -5,3 +5,4 @@
# spark.master spark://master:7077
# spark.eventLog.enabled true
# spark.eventLog.dir hdfs://namenode:8021/directory
# spark.serializer org.apache.spark.serializer.KryoSerializer
30 changes: 13 additions & 17 deletions docs/cluster-overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -105,23 +105,19 @@ HADOOP_CONF_DIR=XX ./bin/spark-submit \

### Loading Configurations from a File

The `spark-submit` script can load default [Spark configuration values](configuration.html) from
a properties file and pass them on to your application. By default it will read configuration
options from `conf/spark-defaults.conf`, in which each line consists of a key and a value separated
by whitespace. For example,

spark.master spark://5.6.7.8:7077
spark.executor.memory 512m
spark.eventLog.enabled true

Any values specified in the file will be passed on to the application. Loading default Spark
configurations this way 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 running `spark-submit` with the `--verbose` option.
The `spark-submit` script can load default [Spark configuration values](configuration.html) from a
properties file and pass them on to your application. By default it will read configuration options
from `conf/spark-defaults.conf`. For more detail, see the section on
[loading default configurations](configuration.html#loading-default-configurations).

Loading default Spark configurations this way can obviate the need for certain flags to
`spark-submit`. For instance, if the `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, you can print out fine-grained
debugging information by running `spark-submit` with the `--verbose` option.

### Advanced Dependency Management
When using `spark-submit`, the application jar along with any jars included with the `--jars` option
Expand Down
62 changes: 40 additions & 22 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -5,35 +5,51 @@ title: Spark Configuration

Spark provides three locations to configure the system:

* [Spark properties](#spark-properties) control most application parameters and can be set by passing
a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java
system properties.
* [Spark properties](#spark-properties) control most application parameters and can be set by
passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext,
or through the `conf/spark-defaults.conf` properties file.
* [Environment variables](#environment-variables) can be used to set per-machine settings, such as
the IP address, through the `conf/spark-env.sh` script on each node.
* [Logging](#configuring-logging) can be configured through `log4j.properties`.


# Spark Properties

Spark properties control most application settings and are configured separately for each application.
The preferred way to set them is by passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf)
class to your SparkContext constructor.
Alternatively, Spark will also load them from Java system properties, for compatibility with old versions
of Spark.

SparkConf lets you configure most of the common properties to initialize a cluster (e.g., master URL and
application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could
initialize an application as follows:
Spark properties control most application settings and are configured separately for each
application. The preferred way is to set them through
[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) and passing it as an argument to your
SparkContext. SparkConf lets you configure most of the common properties to initialize a cluster
(e.g., master URL and application name), as well as arbitrary key-value pairs through the `set()`
method. For example, we could initialize an application as follows:

{% highlight scala %}
val conf = new SparkConf().
setMaster("local").
setAppName("My application").
set("spark.executor.memory", "1g")
val conf = new SparkConf
.setMaster("local")
.setAppName("CountingSheep")
.set("spark.executor.memory", "1g")
val sc = new SparkContext(conf)
{% endhighlight %}

Most of the properties control internal settings that have reasonable default values. However,
## Loading Default Configurations

In the case of `spark-shell`, a SparkContext has already been created for you, so you cannot control
the configuration properties through SparkConf. However, you can still set configuration properties
through a default configuration file. By default, `spark-shell` (and more generally `spark-submit`)
will read configuration options from `conf/spark-defaults.conf`, in which each line consists of a
key and a value separated by whitespace. For example,

spark.master spark://5.6.7.8:7077
spark.executor.memory 512m
spark.eventLog.enabled true
spark.serializer org.apache.spark.serializer.KryoSerializer

Any values specified in the file will be passed on to the application, and merged with those
specified through SparkConf. If the same configuration property exists in both `spark-defaults.conf`
and SparkConf, then the latter will take precedence as it is most application-specific.

## All Configuration Properties

Most of the properties that control internal settings have reasonable default values. However,
there are at least five properties that you will commonly want to control:

<table class="table">
Expand Down Expand Up @@ -101,9 +117,9 @@ Apart from these, the following properties are also available, and may be useful
<td>spark.default.parallelism</td>
<td>
<ul>
<li>Local mode: number of cores on the local machine</li>
<li>Mesos fine grained mode: 8</li>
<li>Local mode: core number of the local machine</li>
<li>Others: total core number of all executor nodes or 2, whichever is larger</li>
<li>Others: total number of cores on all executor nodes or 2, whichever is larger</li>
</ul>
</td>
<td>
Expand Down Expand Up @@ -696,7 +712,9 @@ Apart from these, the following properties are also available, and may be useful
## Viewing Spark Properties

The application web UI at `http://<driver>:4040` lists Spark properties in the "Environment" tab.
This is a useful place to check to make sure that your properties have been set correctly.
This is a useful place to check to make sure that your properties have been set correctly. Note
that only values explicitly specified through either `spark-defaults.conf` or SparkConf will
appear. For all other configuration properties, you can assume the default value is used.

# Environment Variables

Expand All @@ -714,8 +732,8 @@ The following variables can be set in `spark-env.sh`:
* `PYSPARK_PYTHON`, the Python binary to use for PySpark
* `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to.
* `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.
* 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.

Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might
compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface.
Expand Down
35 changes: 4 additions & 31 deletions docs/spark-standalone.md
Original file line number Diff line number Diff line change
Expand Up @@ -178,37 +178,10 @@ The spark-submit script provides the most straightforward way to submit a compil
application-jar: Path to a bundled jar including your application and all dependencies. The URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes.
application-arguments: Arguments passed to the main method of <main-class>

Behind the scenes, this invokes the standalone Client to launch your application, which is also the legacy way to launch your application before Spark 1.0.

./bin/spark-class org.apache.spark.deploy.Client launch
[client-options] \
<master-url> <application-jar> <main-class> \
[application-arguments]

client-options:
--memory <count> (amount of memory, in MB, allocated for your driver program)
--cores <count> (number of cores allocated for your driver program)
--supervise (whether to automatically restart your driver on application or node failure)
--verbose (prints increased logging output)

Keep in mind that your driver program will be executed on a remote worker machine. You can control the execution environment in the following ways:

* __Environment variables__: These are captured from the environment within which the client
is launched and applied when launching the driver program. These environment variables should be
exported in `conf/spark-env.sh`.
* __Java options__: You can add java options by setting `SPARK_JAVA_OPTS` in the environment in
which you launch the submission client. (_Note_: as of Spark 1.0, application specific
[Spark configuration properties](configuration.html#spark-properties) should be specified through
`conf/spark-defaults.conf` loaded by `spark-submit`.)
* __Dependencies__: If your application is launched through `spark-submit`, then the application
jar is automatically distributed to all worker nodes. Otherwise, you'll need to explicitly add the
jar through `sc.addJars`.

Once you submit a driver program, it will appear in the cluster management UI at port 8080 and
be assigned an identifier. If you'd like to prematurely terminate the program, you can do so as
follows:

./bin/spark-class org.apache.spark.deploy.Client kill <driverId>
If your application is launched through `spark-submit`, then the application jar is automatically
distributed to all worker nodes. Otherwise, you'll need to explicitly add the jar through
`sc.addJars`. To control the application's configuration or execution environment, see
[Spark Configuration](configuration.html).

# Resource Scheduling

Expand Down

0 comments on commit 3cc0649

Please sign in to comment.