aboutsummaryrefslogtreecommitdiff
path: root/docs/spark-standalone.md
diff options
context:
space:
mode:
authorMatei Zaharia <matei@eecs.berkeley.edu>2013-08-31 14:21:10 -0700
committerMatei Zaharia <matei@eecs.berkeley.edu>2013-08-31 14:21:10 -0700
commit4819baa658a6c8a3e4c5c504af284ea6091e4c35 (patch)
tree00eda629ac7292487ef14f858d19297c38a19607 /docs/spark-standalone.md
parent4293533032bd5c354bb011f8d508b99615c6e0f0 (diff)
downloadspark-4819baa658a6c8a3e4c5c504af284ea6091e4c35.tar.gz
spark-4819baa658a6c8a3e4c5c504af284ea6091e4c35.tar.bz2
spark-4819baa658a6c8a3e4c5c504af284ea6091e4c35.zip
More updates, describing changes to recommended use of environment vars
and new Python stuff
Diffstat (limited to 'docs/spark-standalone.md')
-rw-r--r--docs/spark-standalone.md52
1 files changed, 25 insertions, 27 deletions
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index bb8be276c5..9ab6ba0830 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -3,18 +3,7 @@ layout: global
title: Spark Standalone Mode
---
-{% comment %}
-TODO(andyk):
- - Add a table of contents
- - Move configuration towards the end so that it doesn't come first
- - Say the scripts will guess the resource amounts (i.e. # cores) automatically
-{% endcomment %}
-
-In addition to running on top of [Mesos](https://github.com/mesos/mesos), Spark also supports a standalone mode, consisting of one Spark master and several Spark worker processes. You can run the Spark standalone mode either locally (for testing) or on a cluster. If you wish to run on a cluster, we have provided [a set of deploy scripts](#cluster-launch-scripts) to launch a whole cluster.
-
-# Getting Started
-
-Compile Spark with `sbt package` as described in the [Getting Started Guide](index.html). You do not need to install Mesos on your machine if you are using the standalone mode.
+In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [deploy scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing.
# Starting a Cluster Manually
@@ -22,8 +11,8 @@ You can start a standalone master server by executing:
./spark-class spark.deploy.master.Master
-Once started, the master will print out a `spark://IP:PORT` URL for itself, which you can use to connect workers to it,
-or pass as the "master" argument to `SparkContext` to connect a job to the cluster. You can also find this URL on
+Once started, the master will print out a `spark://HOST:PORT` URL for itself, which you can use to connect workers to it,
+or pass as the "master" argument to `SparkContext`. You can also find this URL on
the master's web UI, which is [http://localhost:8080](http://localhost:8080) by default.
Similarly, you can start one or more workers and connect them to the master via:
@@ -68,7 +57,7 @@ Finally, the following configuration options can be passed to the master and wor
To launch a Spark standalone cluster with the deploy scripts, you need to create a file called `conf/slaves` in your Spark directory, which should contain the hostnames of all the machines where you would like to start Spark workers, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing, you can just put `localhost` in this file.
-Once you've set up this fine, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
+Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
@@ -85,47 +74,56 @@ You can optionally configure the cluster further by setting environment variable
<tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
<tr>
<td><code>SPARK_MASTER_IP</code></td>
- <td>Bind the master to a specific IP address, for example a public one</td>
+ <td>Bind the master to a specific IP address, for example a public one.</td>
</tr>
<tr>
<td><code>SPARK_MASTER_PORT</code></td>
- <td>Start the master on a different port (default: 7077)</td>
+ <td>Start the master on a different port (default: 7077).</td>
</tr>
<tr>
<td><code>SPARK_MASTER_WEBUI_PORT</code></td>
- <td>Port for the master web UI (default: 8080)</td>
+ <td>Port for the master web UI (default: 8080).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_PORT</code></td>
- <td>Start the Spark worker on a specific port (default: random)</td>
+ <td>Start the Spark worker on a specific port (default: random).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_DIR</code></td>
- <td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work)</td>
+ <td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_CORES</code></td>
- <td>Total number of cores to allow Spark jobs to use on the machine (default: all available cores)</td>
+ <td>Total number of cores to allow Spark jobs to use on the machine (default: all available cores).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_MEMORY</code></td>
- <td>Total amount of memory to allow Spark jobs to use on the machine, e.g. 1000M, 2G (default: total memory minus 1 GB); note that each job's <i>individual</i> memory is configured using <code>SPARK_MEM</code></td>
+ <td>Total amount of memory to allow Spark jobs to use on the machine, e.g. <code>1000m</code>, <code>2g</code> (default: total memory minus 1 GB); note that each job's <i>individual</i> memory is configured using its <code>spark.executor.memory</code> property.</td>
</tr>
<tr>
<td><code>SPARK_WORKER_WEBUI_PORT</code></td>
- <td>Port for the worker web UI (default: 8081)</td>
+ <td>Port for the worker web UI (default: 8081).</td>
+ </tr>
+ <tr>
+ <td><code>SPARK_WORKER_INSTANCES</code></td>
+ <td>
+ Number of worker instances to run on each machine (default: 1). You can make this more than 1 if
+ you have have very large machines and would like multiple Spark worker processes. If you do set
+ this, make sure to also set <code>SPARK_WORKER_CORES</code> explicitly to limit the cores per worker,
+ or else each worker will try to use all the cores.
+ </td>
</tr>
<tr>
<td><code>SPARK_DAEMON_MEMORY</code></td>
- <td>Memory to allocate to the Spark master and worker daemons themselves (default: 512m)</td>
+ <td>Memory to allocate to the Spark master and worker daemons themselves (default: 512m).</td>
</tr>
<tr>
<td><code>SPARK_DAEMON_JAVA_OPTS</code></td>
- <td>JVM options for the Spark master and worker daemons themselves (default: none)</td>
+ <td>JVM options for the Spark master and worker daemons themselves (default: none).</td>
</tr>
</table>
-
+**Note:** The launch scripts do not currently support Windows. To run a Spark cluster on Windows, start the master and workers by hand.
# Connecting a Job to the Cluster
@@ -155,5 +153,5 @@ In addition, detailed log output for each job is also written to the work direct
# Running Alongside Hadoop
-You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://<namenode>:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on).
+You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the same machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://<namenode>:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on).