aboutsummaryrefslogtreecommitdiff
path: root/docs/spark-standalone.md
diff options
context:
space:
mode:
authorAaron Davidson <aaron@databricks.com>2013-10-10 00:32:58 -0700
committerAaron Davidson <aaron@databricks.com>2013-10-10 00:33:47 -0700
commit42d8b8efe68f6f81af5ed31a046138f9dad5be4a (patch)
treeb9d60a0c8de72c47e0f81ab91350072502ac3ce2 /docs/spark-standalone.md
parent4ea8ee468fb1f50fce56853a5127a89efc45b706 (diff)
downloadspark-42d8b8efe68f6f81af5ed31a046138f9dad5be4a.tar.gz
spark-42d8b8efe68f6f81af5ed31a046138f9dad5be4a.tar.bz2
spark-42d8b8efe68f6f81af5ed31a046138f9dad5be4a.zip
Address Matei's comments on documentation
Updates to the documentation and changing some logError()s to logWarning()s.
Diffstat (limited to 'docs/spark-standalone.md')
-rw-r--r--docs/spark-standalone.md35
1 files changed, 21 insertions, 14 deletions
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 5707e19277..daf04f145d 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -173,31 +173,42 @@ In addition, detailed log output for each job is also written to the work direct
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).
-# Fault tolerance
+# High Availability
-By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new jobs can be created. In order to circumvent this, we have two fault tolerance schemes, detailed below.
+By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below.
-**Possible gotcha:** Production-level fault tolerance is enabled by having multiple Master nodes tied together with ZooKeeper. 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 start scheduling independently).
+## Standby Masters with ZooKeeper
-## Standby masters with ZooKeeper
+**Overview**
+
+Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ applications -- applications that were already running during Master failover are unaffected.
+
+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_URL:ZK_PORT # eg 192.168.1.100:2181
+ 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.
**Details**
-Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ jobs -- jobs that were already running during Master failover are unaffected.
+After you have a ZooKeeper cluster set up, enabling high availability is straightforward. Simply start multiple Master processes on different nodes with the same ZooKeeper configuration (ZooKeeper URL and directory). Masters can be added and removed at any time.
+
+In order to schedule new applications or add Workers to the cluster, they need to know the IP address of the current leader. This can be accomplished by simply passing in a list of Masters where you used to pass in a single one. For example, you might start your SparkContext pointing to ``spark://host1:port1,host2:port2``. This would cause your SparkContext to try registering with both Masters -- if ``host1`` goes down, this configuration would still be correct as we'd find the new leader, ``host2``.
-In order to schedule new jobs or add Workers to the cluster, they need to know the IP address of the current leader. This can be accomplished by simply passing in a list of Masters where you used to pass in a single one. For example, you might start your SparkContext pointing to ``spark://host1:port1,host2:port2``. This would cause your SparkContext to try registering with both Masters -- if host1 goes down, this configuration would still be correct as we'd find the new leader, host2.
+There's an important distinction to be made between "registering with a Master" and normal operation. When starting up, an application or Worker needs to be able to find and register with the current lead Master. Once it successfully registers, though, it is "in the system" (i.e., stored in ZooKeeper). If failover occurs, the new leader will contact all previously registered applications and Workers to inform them of the change in leadership, so they need not even have known of the existence of the new Master at startup.
-There's an important distinction to be made between "registering with a Master" and normal operation. When starting up, a SparkContext or Worker needs to be able to find and register with the current lead Master. Once it successfully registers, though, it is "in the system" (i.e., stored in ZooKeeper). If failover occurs, the new leader will contact all previously registered SparkContexts and Workers to inform them of the change in leadership, so they need not have even known of the existence of the new Master at startup.
-<br />This means that new Masters can be created at any time, and the only thing you need to worry about is that _new_ SparkContexts and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of.
+Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of.
-## Single-node recovery with local file system
+## Single-Node Recovery with Local File System
+
+**Overview**
+
+ZooKeeper is the best way to go for production-level high availability, but if you just want to be able to restart the Master if it goes down, FILESYSTEM mode can take care of it. When applications and Workers register, they have enough state written to the provided directory so that they can be recovered upon a restart of the Master process.
**Configuration**
@@ -207,10 +218,6 @@ There's an important distinction to be made between "registering with a Master"
**Details**
-ZooKeeper is the best way to go for production-level fault tolerance, but if you just want to be able to restart the Master if it goes down, FILESYSTEM mode can take care of it. When SparkContexts and Workers register, they have enough state written to the provided directory so that they can be recovered upon a restart of the Master process.
-
-**Notes:**
-
* 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.