aboutsummaryrefslogtreecommitdiff
path: root/docs/spark-standalone.md
diff options
context:
space:
mode:
authorAaron Davidson <aaron@databricks.com>2013-10-10 14:45:12 -0700
committerAaron Davidson <aaron@databricks.com>2013-10-10 14:45:12 -0700
commit66c20635fa1fe18604bb4042ce31152180cb541d (patch)
treee8a075ca0c776bfac965c9cee8b3df574a60133a /docs/spark-standalone.md
parent42d8b8efe68f6f81af5ed31a046138f9dad5be4a (diff)
downloadspark-66c20635fa1fe18604bb4042ce31152180cb541d.tar.gz
spark-66c20635fa1fe18604bb4042ce31152180cb541d.tar.bz2
spark-66c20635fa1fe18604bb4042ce31152180cb541d.zip
Minor clarification and cleanup to spark-standalone.md
Diffstat (limited to 'docs/spark-standalone.md')
-rw-r--r--docs/spark-standalone.md43
1 files changed, 33 insertions, 10 deletions
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index daf04f145d..17066ef0dd 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -185,14 +185,27 @@ Utilizing ZooKeeper to provide leader election and some state storage, you can l
Learn more about getting started with ZooKeeper [here](http://zookeeper.apache.org/doc/trunk/zookeeperStarted.html).
-**Possible gotcha:** If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently).
-
**Configuration**
- # May be configured as SPARK_DAEMON_JAVA_OPTS in spark-env.sh
- spark.deploy.recoveryMode=ZOOKEEPER
- spark.deploy.zookeeper.url=ZK_URL1:ZK_PORT1,ZK_URL2:ZK_PORT2 # eg 192.168.1.100:2181,192.168.1.101:2181
- spark.deploy.zookeeper.dir=/spark # OPTIONAL! /spark is the default.
+In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration:
+
+<table class="table">
+ <tr><th style="width:21%">System property</th><th>Meaning</th></tr>
+ <tr>
+ <td><code>spark.deploy.recoveryMode</code></td>
+ <td>Set to ZOOKEEPER to enable standby Master recovery mode (default: NONE).</td>
+ </tr>
+ <tr>
+ <td><code>spark.deploy.zookeeper.url</code></td>
+ <td>The ZooKeeper cluster url (e.g., 192.168.1.100:2181,192.168.1.101:2181).</td>
+ </tr>
+ <tr>
+ <td><code>spark.deploy.zookeeper.dir</code></td>
+ <td>The directory in ZooKeeper to store recovery state (default: /spark).</td>
+ </tr>
+</table>
+
+Possible gotcha: If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently).
**Details**
@@ -212,12 +225,22 @@ ZooKeeper is the best way to go for production-level high availability, but if y
**Configuration**
- # May be configured as SPARK_DAEMON_JAVA_OPTS in spark-env.sh
- spark.deploy.recoveryMode=FILESYSTEM
- spark.deploy.recoveryDirectory=PATH_ACCESSIBLE_TO_MASTER
+In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration:
+
+<table class="table">
+ <tr><th style="width:21%">System property</th><th>Meaning</th></tr>
+ <tr>
+ <td><code>spark.deploy.recoveryMode</code></td>
+ <td>Set to FILESYSTEM to enable single-node recovery mode (default: NONE).</td>
+ </tr>
+ <tr>
+ <td><code>spark.deploy.recoveryDirectory</code></td>
+ <td>The directory in which Spark will store recovery state, accessible from the Master's perspective.</td>
+ </tr>
+</table>
**Details**
* This solution can be used in tandem with a process monitor/manager like [monit](http://mmonit.com/monit/), or just to enable manual recovery via restart.
* While filesystem recovery seems straightforwardly better than not doing any recovery at all, this mode may be suboptimal for certain development or experimental purposes. In particular, killing a master via stop-master.sh does not clean up its recovery state, so whenever you start a new Master, it will enter recovery mode. This could increase the startup time by up to 1 minute if it needs to wait for all previously-registered Workers/clients to timeout.
-* While it's not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/clients (equivalent to ZooKeeper recovery). Note, however, that you **cannot** have multiple Masters alive concurrently using this approach; you need to upgrade to ZooKeeper to provide leader election for that use-case.
+* While it's not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/applications (equivalent to ZooKeeper recovery). Future applications will have to be able to find the new Master, however, in order to register.