diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh
index 03e416a13274d5c94caa381257332ec42bb31fd1..fcb8555d4e5706a6f4f2ec9670525437a3a2acae 100755
--- a/bin/stop-slaves.sh
+++ b/bin/stop-slaves.sh
@@ -17,8 +17,6 @@
 # limitations under the License.
 #
 
-# Starts the master on the machine this script is executed on.
-
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 
diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index 3ef402926e8985e30744da8f0e67a30715b21c4e..1a2ec55876c35089d3b448a078d2212cafda08f4 100644
--- a/core/src/main/scala/org/apache/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -17,43 +17,42 @@
 
 package org.apache.spark
 
-import java.util.{HashMap => JHashMap}
+import org.apache.spark.util.AppendOnlyMap
 
-import scala.collection.JavaConversions._
-
-/** A set of functions used to aggregate data.
-  * 
-  * @param createCombiner function to create the initial value of the aggregation.
-  * @param mergeValue function to merge a new value into the aggregation result.
-  * @param mergeCombiners function to merge outputs from multiple mergeValue function.
-  */
+/**
+ * A set of functions used to aggregate data.
+ *
+ * @param createCombiner function to create the initial value of the aggregation.
+ * @param mergeValue function to merge a new value into the aggregation result.
+ * @param mergeCombiners function to merge outputs from multiple mergeValue function.
+ */
 case class Aggregator[K, V, C] (
     createCombiner: V => C,
     mergeValue: (C, V) => C,
     mergeCombiners: (C, C) => C) {
 
   def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = {
-    val combiners = new JHashMap[K, C]
-    for (kv <- iter) {
-      val oldC = combiners.get(kv._1)
-      if (oldC == null) {
-        combiners.put(kv._1, createCombiner(kv._2))
-      } else {
-        combiners.put(kv._1, mergeValue(oldC, kv._2))
-      }
+    val combiners = new AppendOnlyMap[K, C]
+    var kv: Product2[K, V] = null
+    val update = (hadValue: Boolean, oldValue: C) => {
+      if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
+    }
+    while (iter.hasNext) {
+      kv = iter.next()
+      combiners.changeValue(kv._1, update)
     }
     combiners.iterator
   }
 
   def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = {
-    val combiners = new JHashMap[K, C]
-    iter.foreach { case(k, c) =>
-      val oldC = combiners.get(k)
-      if (oldC == null) {
-        combiners.put(k, c)
-      } else {
-        combiners.put(k, mergeCombiners(oldC, c))
-      }
+    val combiners = new AppendOnlyMap[K, C]
+    var kc: (K, C) = null
+    val update = (hadValue: Boolean, oldValue: C) => {
+      if (hadValue) mergeCombiners(oldValue, kc._2) else kc._2
+    }
+    while (iter.hasNext) {
+      kc = iter.next()
+      combiners.changeValue(kc._1, update)
     }
     combiners.iterator
   }
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 2a82c5c5d7042def58fdcaec2f347c808b5e26aa..17247be78add2f860a015749603e33ca1a4d8115 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -156,7 +156,7 @@ class SparkContext(
     // Regular expression for simulating a Spark cluster of [N, cores, memory] locally
     val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r
     // Regular expression for connecting to Spark deploy clusters
-    val SPARK_REGEX = """(spark://.*)""".r
+    val SPARK_REGEX = """spark://(.*)""".r
     //Regular expression for connection to Mesos cluster
     val MESOS_REGEX = """(mesos://.*)""".r
 
@@ -172,7 +172,8 @@ class SparkContext(
 
       case SPARK_REGEX(sparkUrl) =>
         val scheduler = new ClusterScheduler(this)
-        val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
+        val masterUrls = sparkUrl.split(",").map("spark://" + _)
+        val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName)
         scheduler.initialize(backend)
         scheduler
 
@@ -188,8 +189,8 @@ class SparkContext(
         val scheduler = new ClusterScheduler(this)
         val localCluster = new LocalSparkCluster(
           numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
-        val sparkUrl = localCluster.start()
-        val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
+        val masterUrls = localCluster.start()
+        val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName)
         scheduler.initialize(backend)
         backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
           localCluster.stop()
@@ -671,10 +672,21 @@ class SparkContext(
         key = uri.getScheme match {
           case null | "file" =>
             if (env.hadoop.isYarnMode()) {
-              logWarning("local jar specified as parameter to addJar under Yarn mode")
-              return
+              // In order for this to work on yarn the user must specify the --addjars option to
+              // the client to upload the file into the distributed cache to make it show up in the
+              // current working directory.
+              val fileName = new Path(uri.getPath).getName()
+              try {
+                env.httpFileServer.addJar(new File(fileName))
+              } catch {
+                case e: Exception => {
+                  logError("Error adding jar (" + e + "), was the --addJars option used?")
+                  throw e
+                }
+              }
+            } else {
+              env.httpFileServer.addJar(new File(uri.getPath))
             }
-            env.httpFileServer.addJar(new File(uri.getPath))
           case _ =>
             path
         }
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 1cfff5e565d173c730e65ff301b26b59d7c31085..275331724afba010988baf082c6364fbcce9b5bf 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -21,12 +21,14 @@ import scala.collection.immutable.List
 
 import org.apache.spark.deploy.ExecutorState.ExecutorState
 import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo}
+import org.apache.spark.deploy.master.RecoveryState.MasterState
 import org.apache.spark.deploy.worker.ExecutorRunner
 import org.apache.spark.util.Utils
 
 
 private[deploy] sealed trait DeployMessage extends Serializable
 
+/** Contains messages sent between Scheduler actor nodes. */
 private[deploy] object DeployMessages {
 
   // Worker to Master
@@ -52,17 +54,20 @@ private[deploy] object DeployMessages {
       exitStatus: Option[Int])
     extends DeployMessage
 
+  case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription])
+
   case class Heartbeat(workerId: String) extends DeployMessage
 
   // Master to Worker
 
-  case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage
+  case class RegisteredWorker(masterUrl: String, masterWebUiUrl: String) extends DeployMessage
 
   case class RegisterWorkerFailed(message: String) extends DeployMessage
 
-  case class KillExecutor(appId: String, execId: Int) extends DeployMessage
+  case class KillExecutor(masterUrl: String, appId: String, execId: Int) extends DeployMessage
 
   case class LaunchExecutor(
+      masterUrl: String,
       appId: String,
       execId: Int,
       appDesc: ApplicationDescription,
@@ -76,9 +81,11 @@ private[deploy] object DeployMessages {
   case class RegisterApplication(appDescription: ApplicationDescription)
     extends DeployMessage
 
+  case class MasterChangeAcknowledged(appId: String)
+
   // Master to Client
 
-  case class RegisteredApplication(appId: String) extends DeployMessage
+  case class RegisteredApplication(appId: String, masterUrl: String) extends DeployMessage
 
   // TODO(matei): replace hostPort with host
   case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) {
@@ -94,6 +101,10 @@ private[deploy] object DeployMessages {
 
   case object StopClient
 
+  // Master to Worker & Client
+
+  case class MasterChanged(masterUrl: String, masterWebUiUrl: String)
+
   // MasterWebUI To Master
 
   case object RequestMasterState
@@ -101,7 +112,8 @@ private[deploy] object DeployMessages {
   // Master to MasterWebUI
 
   case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo],
-    activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) {
+    activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo],
+    status: MasterState) {
 
     Utils.checkHost(host, "Required hostname")
     assert (port > 0)
@@ -123,12 +135,7 @@ private[deploy] object DeployMessages {
     assert (port > 0)
   }
 
-  // Actor System to Master
-
-  case object CheckForWorkerTimeOut
-
-  case object RequestWebUIPort
-
-  case class WebUIPortResponse(webUIBoundPort: Int)
+  // Actor System to Worker
 
+  case object SendHeartbeat
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2abf0b69dddb329140b622be0278630834ff6092
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy
+
+/**
+ * Used to send state on-the-wire about Executors from Worker to Master.
+ * This state is sufficient for the Master to reconstruct its internal data structures during
+ * failover.
+ */
+private[spark] class ExecutorDescription(
+    val appId: String,
+    val execId: Int,
+    val cores: Int,
+    val state: ExecutorState.Value)
+  extends Serializable {
+
+  override def toString: String =
+    "ExecutorState(appId=%s, execId=%d, cores=%d, state=%s)".format(appId, execId, cores, state)
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
new file mode 100644
index 0000000000000000000000000000000000000000..668032a3a2680aac0085cb6e17c1bb01653ca4e5
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
@@ -0,0 +1,420 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *    http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.spark.deploy
+
+import java.io._
+import java.net.URL
+import java.util.concurrent.TimeoutException
+
+import scala.concurrent.{Await, future, promise}
+import scala.concurrent.duration._
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.collection.mutable.ListBuffer
+import scala.sys.process._
+
+import net.liftweb.json.JsonParser
+
+import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.deploy.master.RecoveryState
+
+/**
+ * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master.
+ * In order to mimic a real distributed cluster more closely, Docker is used.
+ * Execute using
+ * ./spark-class org.apache.spark.deploy.FaultToleranceTest
+ *
+ * Make sure that that the environment includes the following properties in SPARK_DAEMON_JAVA_OPTS:
+ *   - spark.deploy.recoveryMode=ZOOKEEPER
+ *   - spark.deploy.zookeeper.url=172.17.42.1:2181
+ * Note that 172.17.42.1 is the default docker ip for the host and 2181 is the default ZK port.
+ *
+ * Unfortunately, due to the Docker dependency this suite cannot be run automatically without a
+ * working installation of Docker. In addition to having Docker, the following are assumed:
+ *   - Docker can run without sudo (see http://docs.docker.io/en/latest/use/basics/)
+ *   - The docker images tagged spark-test-master and spark-test-worker are built from the
+ *     docker/ directory. Run 'docker/spark-test/build' to generate these.
+ */
+private[spark] object FaultToleranceTest extends App with Logging {
+  val masters = ListBuffer[TestMasterInfo]()
+  val workers = ListBuffer[TestWorkerInfo]()
+  var sc: SparkContext = _
+
+  var numPassed = 0
+  var numFailed = 0
+
+  val sparkHome = System.getenv("SPARK_HOME")
+  assertTrue(sparkHome != null, "Run with a valid SPARK_HOME")
+
+  val containerSparkHome = "/opt/spark"
+  val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome)
+
+  System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip
+
+  def afterEach() {
+    if (sc != null) {
+      sc.stop()
+      sc = null
+    }
+    terminateCluster()
+  }
+
+  test("sanity-basic") {
+    addMasters(1)
+    addWorkers(1)
+    createClient()
+    assertValidClusterState()
+  }
+
+  test("sanity-many-masters") {
+    addMasters(3)
+    addWorkers(3)
+    createClient()
+    assertValidClusterState()
+  }
+
+  test("single-master-halt") {
+    addMasters(3)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+
+    killLeader()
+    delay(30 seconds)
+    assertValidClusterState()
+    createClient()
+    assertValidClusterState()
+  }
+
+  test("single-master-restart") {
+    addMasters(1)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+
+    killLeader()
+    addMasters(1)
+    delay(30 seconds)
+    assertValidClusterState()
+
+    killLeader()
+    addMasters(1)
+    delay(30 seconds)
+    assertValidClusterState()
+  }
+
+  test("cluster-failure") {
+    addMasters(2)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+
+    terminateCluster()
+    addMasters(2)
+    addWorkers(2)
+    assertValidClusterState()
+  }
+
+  test("all-but-standby-failure") {
+    addMasters(2)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+
+    killLeader()
+    workers.foreach(_.kill())
+    workers.clear()
+    delay(30 seconds)
+    addWorkers(2)
+    assertValidClusterState()
+  }
+
+  test("rolling-outage") {
+    addMasters(1)
+    delay()
+    addMasters(1)
+    delay()
+    addMasters(1)
+    addWorkers(2)
+    createClient()
+    assertValidClusterState()
+    assertTrue(getLeader == masters.head)
+
+    (1 to 3).foreach { _ =>
+      killLeader()
+      delay(30 seconds)
+      assertValidClusterState()
+      assertTrue(getLeader == masters.head)
+      addMasters(1)
+    }
+  }
+
+  def test(name: String)(fn: => Unit) {
+    try {
+      fn
+      numPassed += 1
+      logInfo("Passed: " + name)
+    } catch {
+      case e: Exception =>
+        numFailed += 1
+        logError("FAILED: " + name, e)
+    }
+    afterEach()
+  }
+
+  def addMasters(num: Int) {
+    (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) }
+  }
+
+  def addWorkers(num: Int) {
+    val masterUrls = getMasterUrls(masters)
+    (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) }
+  }
+
+  /** Creates a SparkContext, which constructs a Client to interact with our cluster. */
+  def createClient() = {
+    if (sc != null) { sc.stop() }
+    // Counter-hack: Because of a hack in SparkEnv#createFromSystemProperties() that changes this
+    // property, we need to reset it.
+    System.setProperty("spark.driver.port", "0")
+    sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome)
+  }
+
+  def getMasterUrls(masters: Seq[TestMasterInfo]): String = {
+    "spark://" + masters.map(master => master.ip + ":7077").mkString(",")
+  }
+
+  def getLeader: TestMasterInfo = {
+    val leaders = masters.filter(_.state == RecoveryState.ALIVE)
+    assertTrue(leaders.size == 1)
+    leaders(0)
+  }
+
+  def killLeader(): Unit = {
+    masters.foreach(_.readState())
+    val leader = getLeader
+    masters -= leader
+    leader.kill()
+  }
+
+  def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis)
+
+  def terminateCluster() {
+    masters.foreach(_.kill())
+    workers.foreach(_.kill())
+    masters.clear()
+    workers.clear()
+  }
+
+  /** This includes Client retry logic, so it may take a while if the cluster is recovering. */
+  def assertUsable() = {
+    val f = future {
+      try {
+        val res = sc.parallelize(0 until 10).collect()
+        assertTrue(res.toList == (0 until 10))
+        true
+      } catch {
+        case e: Exception =>
+          logError("assertUsable() had exception", e)
+          e.printStackTrace()
+          false
+      }
+    }
+
+    // Avoid waiting indefinitely (e.g., we could register but get no executors).
+    assertTrue(Await.result(f, 120 seconds))
+  }
+
+  /**
+   * Asserts that the cluster is usable and that the expected masters and workers
+   * are all alive in a proper configuration (e.g., only one leader).
+   */
+  def assertValidClusterState() = {
+    assertUsable()
+    var numAlive = 0
+    var numStandby = 0
+    var numLiveApps = 0
+    var liveWorkerIPs: Seq[String] = List()
+
+    def stateValid(): Boolean = {
+      (workers.map(_.ip) -- liveWorkerIPs).isEmpty &&
+        numAlive == 1 && numStandby == masters.size - 1 && numLiveApps >= 1
+    }
+
+    val f = future {
+      try {
+        while (!stateValid()) {
+          Thread.sleep(1000)
+
+          numAlive = 0
+          numStandby = 0
+          numLiveApps = 0
+
+          masters.foreach(_.readState())
+
+          for (master <- masters) {
+            master.state match {
+              case RecoveryState.ALIVE =>
+                numAlive += 1
+                liveWorkerIPs = master.liveWorkerIPs
+              case RecoveryState.STANDBY =>
+                numStandby += 1
+              case _ => // ignore
+            }
+
+            numLiveApps += master.numLiveApps
+          }
+        }
+        true
+      } catch {
+        case e: Exception =>
+          logError("assertValidClusterState() had exception", e)
+          false
+      }
+    }
+
+    try {
+      assertTrue(Await.result(f, 120 seconds))
+    } catch {
+      case e: TimeoutException =>
+        logError("Master states: " + masters.map(_.state))
+        logError("Num apps: " + numLiveApps)
+        logError("IPs expected: " + workers.map(_.ip) + " / found: " + liveWorkerIPs)
+        throw new RuntimeException("Failed to get into acceptable cluster state after 2 min.", e)
+    }
+  }
+
+  def assertTrue(bool: Boolean, message: String = "") {
+    if (!bool) {
+      throw new IllegalStateException("Assertion failed: " + message)
+    }
+  }
+
+  logInfo("Ran %s tests, %s passed and %s failed".format(numPassed+numFailed, numPassed, numFailed))
+}
+
+private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File)
+  extends Logging  {
+
+  implicit val formats = net.liftweb.json.DefaultFormats
+  var state: RecoveryState.Value = _
+  var liveWorkerIPs: List[String] = _
+  var numLiveApps = 0
+
+  logDebug("Created master: " + this)
+
+  def readState() {
+    try {
+      val masterStream = new InputStreamReader(new URL("http://%s:8080/json".format(ip)).openStream)
+      val json = JsonParser.parse(masterStream, closeAutomatically = true)
+
+      val workers = json \ "workers"
+      val liveWorkers = workers.children.filter(w => (w \ "state").extract[String] == "ALIVE")
+      liveWorkerIPs = liveWorkers.map(w => (w \ "host").extract[String])
+
+      numLiveApps = (json \ "activeapps").children.size
+
+      val status = json \\ "status"
+      val stateString = status.extract[String]
+      state = RecoveryState.values.filter(state => state.toString == stateString).head
+    } catch {
+      case e: Exception =>
+        // ignore, no state update
+        logWarning("Exception", e)
+    }
+  }
+
+  def kill() { Docker.kill(dockerId) }
+
+  override def toString: String =
+    "[ip=%s, id=%s, logFile=%s, state=%s]".
+      format(ip, dockerId.id, logFile.getAbsolutePath, state)
+}
+
+private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File)
+  extends Logging {
+
+  implicit val formats = net.liftweb.json.DefaultFormats
+
+  logDebug("Created worker: " + this)
+
+  def kill() { Docker.kill(dockerId) }
+
+  override def toString: String =
+    "[ip=%s, id=%s, logFile=%s]".format(ip, dockerId, logFile.getAbsolutePath)
+}
+
+private[spark] object SparkDocker {
+  def startMaster(mountDir: String): TestMasterInfo = {
+    val cmd = Docker.makeRunCmd("spark-test-master", mountDir = mountDir)
+    val (ip, id, outFile) = startNode(cmd)
+    new TestMasterInfo(ip, id, outFile)
+  }
+
+  def startWorker(mountDir: String, masters: String): TestWorkerInfo = {
+    val cmd = Docker.makeRunCmd("spark-test-worker", args = masters, mountDir = mountDir)
+    val (ip, id, outFile) = startNode(cmd)
+    new TestWorkerInfo(ip, id, outFile)
+  }
+
+  private def startNode(dockerCmd: ProcessBuilder) : (String, DockerId, File) = {
+    val ipPromise = promise[String]()
+    val outFile = File.createTempFile("fault-tolerance-test", "")
+    outFile.deleteOnExit()
+    val outStream: FileWriter = new FileWriter(outFile)
+    def findIpAndLog(line: String): Unit = {
+      if (line.startsWith("CONTAINER_IP=")) {
+        val ip = line.split("=")(1)
+        ipPromise.success(ip)
+      }
+
+      outStream.write(line + "\n")
+      outStream.flush()
+    }
+
+    dockerCmd.run(ProcessLogger(findIpAndLog _))
+    val ip = Await.result(ipPromise.future, 30 seconds)
+    val dockerId = Docker.getLastProcessId
+    (ip, dockerId, outFile)
+  }
+}
+
+private[spark] class DockerId(val id: String) {
+  override def toString = id
+}
+
+private[spark] object Docker extends Logging {
+  def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = {
+    val mountCmd = if (mountDir != "") { " -v " + mountDir } else ""
+
+    val cmd = "docker run %s %s %s".format(mountCmd, imageTag, args)
+    logDebug("Run command: " + cmd)
+    cmd
+  }
+
+  def kill(dockerId: DockerId) : Unit = {
+    "docker kill %s".format(dockerId.id).!
+  }
+
+  def getLastProcessId: DockerId = {
+    var id: String = null
+    "docker ps -l -q".!(ProcessLogger(line => id = line))
+    new DockerId(id)
+  }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
index 04d01c169d3e5d36d303599bf5cb150235546549..e607b8c6f4a361d5590dd11adc3519bed45518fc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
@@ -72,7 +72,8 @@ private[spark] object JsonProtocol {
     ("memory" -> obj.workers.map(_.memory).sum) ~
     ("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~
     ("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~
-    ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo))
+    ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) ~
+    ("status" -> obj.status.toString)
   }
 
   def writeWorkerState(obj: WorkerStateResponse) = {
diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 10161c82041b91c622f6dc3d5feb4ded72a96352..308a2bfa222c35531ef024f194a39e4efeec0de4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -39,22 +39,23 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
   private val masterActorSystems = ArrayBuffer[ActorSystem]()
   private val workerActorSystems = ArrayBuffer[ActorSystem]()
   
-  def start(): String = {
+  def start(): Array[String] = {
     logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
 
     /* Start the Master */
     val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0)
     masterActorSystems += masterSystem
     val masterUrl = "spark://" + localHostname + ":" + masterPort
+    val masters = Array(masterUrl)
 
     /* Start the Workers */
     for (workerNum <- 1 to numWorkers) {
       val (workerSystem, _) = Worker.startSystemAndActor(localHostname, 0, 0, coresPerWorker,
-        memoryPerWorker, masterUrl, null, Some(workerNum))
+        memoryPerWorker, masters, null, Some(workerNum))
       workerActorSystems += workerSystem
     }
 
-    return masterUrl
+    return masters
   }
 
   def stop() {
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
index a342dd724a8ed74bbc1b80bc5d2eb309ded1e7c6..77422f61ec9010cc58f29f5c3c7b5c8870f555a4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
@@ -23,6 +23,7 @@ import akka.actor._
 import akka.actor.Terminated
 import akka.pattern.ask
 import akka.util.Duration
+import akka.util.duration._
 import akka.remote.RemoteClientDisconnected
 import akka.remote.RemoteClientLifeCycleEvent
 import akka.remote.RemoteClientShutdown
@@ -37,41 +38,81 @@ import org.apache.spark.deploy.master.Master
 /**
  * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
  * and a listener for cluster events, and calls back the listener when various events occur.
+ *
+ * @param masterUrls Each url should look like spark://host:port.
  */
 private[spark] class Client(
     actorSystem: ActorSystem,
-    masterUrl: String,
+    masterUrls: Array[String],
     appDescription: ApplicationDescription,
     listener: ClientListener)
   extends Logging {
 
+  val REGISTRATION_TIMEOUT = 20.seconds
+  val REGISTRATION_RETRIES = 3
+
   var actor: ActorRef = null
   var appId: String = null
+  var registered = false
+  var activeMasterUrl: String = null
 
   class ClientActor extends Actor with Logging {
     var master: ActorRef = null
     var masterAddress: Address = null
     var alreadyDisconnected = false  // To avoid calling listener.disconnected() multiple times
+    var alreadyDead = false  // To avoid calling listener.dead() multiple times
 
     override def preStart() {
-      logInfo("Connecting to master " + masterUrl)
       try {
-        master = context.actorFor(Master.toAkkaUrl(masterUrl))
-        masterAddress = master.path.address
-        master ! RegisterApplication(appDescription)
-        context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
-        context.watch(master)  // Doesn't work with remote actors, but useful for testing
+        registerWithMaster()
       } catch {
         case e: Exception =>
-          logError("Failed to connect to master", e)
+          logWarning("Failed to connect to master", e)
           markDisconnected()
           context.stop(self)
       }
     }
 
+    def tryRegisterAllMasters() {
+      for (masterUrl <- masterUrls) {
+        logInfo("Connecting to master " + masterUrl + "...")
+        val actor = context.actorFor(Master.toAkkaUrl(masterUrl))
+        actor ! RegisterApplication(appDescription)
+      }
+    }
+
+    def registerWithMaster() {
+      tryRegisterAllMasters()
+
+      var retries = 0
+      lazy val retryTimer: Cancellable =
+        context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
+          retries += 1
+          if (registered) {
+            retryTimer.cancel()
+          } else if (retries >= REGISTRATION_RETRIES) {
+            logError("All masters are unresponsive! Giving up.")
+            markDead()
+          } else {
+            tryRegisterAllMasters()
+          }
+        }
+      retryTimer // start timer
+    }
+
+    def changeMaster(url: String) {
+      activeMasterUrl = url
+      master = context.actorFor(Master.toAkkaUrl(url))
+      masterAddress = master.path.address
+      context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+      context.watch(master)  // Doesn't work with remote actors, but useful for testing
+    }
+
     override def receive = {
-      case RegisteredApplication(appId_) =>
+      case RegisteredApplication(appId_, masterUrl) =>
         appId = appId_
+        registered = true
+        changeMaster(masterUrl)
         listener.connected(appId)
 
       case ApplicationRemoved(message) =>
@@ -92,23 +133,27 @@ private[spark] class Client(
           listener.executorRemoved(fullId, message.getOrElse(""), exitStatus)
         }
 
+      case MasterChanged(masterUrl, masterWebUiUrl) =>
+        logInfo("Master has changed, new master is at " + masterUrl)
+        context.unwatch(master)
+        changeMaster(masterUrl)
+        alreadyDisconnected = false
+        sender ! MasterChangeAcknowledged(appId)
+
       case Terminated(actor_) if actor_ == master =>
-        logError("Connection to master failed; stopping client")
+        logWarning("Connection to master failed; waiting for master to reconnect...")
         markDisconnected()
-        context.stop(self)
 
       case RemoteClientDisconnected(transport, address) if address == masterAddress =>
-        logError("Connection to master failed; stopping client")
+        logWarning("Connection to master failed; waiting for master to reconnect...")
         markDisconnected()
-        context.stop(self)
 
       case RemoteClientShutdown(transport, address) if address == masterAddress =>
-        logError("Connection to master failed; stopping client")
+        logWarning("Connection to master failed; waiting for master to reconnect...")
         markDisconnected()
-        context.stop(self)
 
       case StopClient =>
-        markDisconnected()
+        markDead()
         sender ! true
         context.stop(self)
     }
@@ -122,6 +167,13 @@ private[spark] class Client(
         alreadyDisconnected = true
       }
     }
+
+    def markDead() {
+      if (!alreadyDead) {
+        listener.dead()
+        alreadyDead = true
+      }
+    }
   }
 
   def start() {
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
index 4605368c1177f56639356639f3aae9951aaee06e..be7a11bd1553724376d334245bf14a30d6a06a35 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
@@ -27,8 +27,12 @@ package org.apache.spark.deploy.client
 private[spark] trait ClientListener {
   def connected(appId: String): Unit
 
+  /** Disconnection may be a temporary state, as we fail over to a new Master. */
   def disconnected(): Unit
 
+  /** Dead means that we couldn't find any Masters to connect to, and have given up. */
+  def dead(): Unit
+
   def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit
 
   def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index d5e9a0e09575844dc9138f1dd43dc507275529ad..5b62d3ba6c0e0ac8bb456ea8a8d39d69f3117165 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@ -33,6 +33,11 @@ private[spark] object TestClient {
       System.exit(0)
     }
 
+    def dead() {
+      logInfo("Could not connect to master")
+      System.exit(0)
+    }
+
     def executorAdded(id: String, workerId: String, hostPort: String, cores: Int, memory: Int) {}
 
     def executorRemoved(id: String, message: String, exitStatus: Option[Int]) {}
@@ -44,7 +49,7 @@ private[spark] object TestClient {
     val desc = new ApplicationDescription(
       "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
     val listener = new TestListener
-    val client = new Client(actorSystem, url, desc, listener)
+    val client = new Client(actorSystem, Array(url), desc, listener)
     client.start()
     actorSystem.awaitTermination()
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
index bd5327627a832fc3b28202920f889bbd421d7ec5..5150b7c7dec6ab135ba65b1d7c992901e164f5e0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
@@ -29,23 +29,46 @@ private[spark] class ApplicationInfo(
     val submitDate: Date,
     val driver: ActorRef,
     val appUiUrl: String)
-{
-  var state = ApplicationState.WAITING
-  var executors = new mutable.HashMap[Int, ExecutorInfo]
-  var coresGranted = 0
-  var endTime = -1L
-  val appSource = new ApplicationSource(this)
-
-  private var nextExecutorId = 0
-
-  def newExecutorId(): Int = {
-    val id = nextExecutorId
-    nextExecutorId += 1
-    id
+  extends Serializable {
+
+  @transient var state: ApplicationState.Value = _
+  @transient var executors: mutable.HashMap[Int, ExecutorInfo] = _
+  @transient var coresGranted: Int = _
+  @transient var endTime: Long = _
+  @transient var appSource: ApplicationSource = _
+
+  @transient private var nextExecutorId: Int = _
+
+  init()
+
+  private def readObject(in: java.io.ObjectInputStream) : Unit = {
+    in.defaultReadObject()
+    init()
+  }
+
+  private def init() {
+    state = ApplicationState.WAITING
+    executors = new mutable.HashMap[Int, ExecutorInfo]
+    coresGranted = 0
+    endTime = -1L
+    appSource = new ApplicationSource(this)
+    nextExecutorId = 0
+  }
+
+  private def newExecutorId(useID: Option[Int] = None): Int = {
+    useID match {
+      case Some(id) =>
+        nextExecutorId = math.max(nextExecutorId, id + 1)
+        id
+      case None =>
+        val id = nextExecutorId
+        nextExecutorId += 1
+        id
+    }
   }
 
-  def addExecutor(worker: WorkerInfo, cores: Int): ExecutorInfo = {
-    val exec = new ExecutorInfo(newExecutorId(), this, worker, cores, desc.memoryPerSlave)
+  def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorInfo = {
+    val exec = new ExecutorInfo(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave)
     executors(exec.id) = exec
     coresGranted += cores
     exec
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
index 7e804223cf48a6459a7c60caa50cff33e7675d89..fedf879eff73bf1abaae089a84c1a8a5efe00c89 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
@@ -18,11 +18,11 @@
 package org.apache.spark.deploy.master
 
 private[spark] object ApplicationState
-  extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
+  extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED", "UNKNOWN") {
 
   type ApplicationState = Value
 
-  val WAITING, RUNNING, FINISHED, FAILED = Value
+  val WAITING, RUNNING, FINISHED, FAILED, UNKNOWN = Value
 
   val MAX_NUM_RETRY = 10
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
index cf384a985e90ede61afa0eb7267d636af0eada5e..76db61dd619c698089832d788ce8b8a1438866d1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.deploy.master
 
-import org.apache.spark.deploy.ExecutorState
+import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
 
 private[spark] class ExecutorInfo(
     val id: Int,
@@ -28,5 +28,10 @@ private[spark] class ExecutorInfo(
 
   var state = ExecutorState.LAUNCHING
 
+  /** Copy all state (non-val) variables from the given on-the-wire ExecutorDescription. */
+  def copyState(execDesc: ExecutorDescription) {
+    state = execDesc.state
+  }
+
   def fullId: String = application.id + "/" + id
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c0849ef324273e7da6ef527b6ce196ba9f415fda
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.master
+
+import java.io._
+
+import scala.Serializable
+
+import akka.serialization.Serialization
+import org.apache.spark.Logging
+
+/**
+ * Stores data in a single on-disk directory with one file per application and worker.
+ * Files are deleted when applications and workers are removed.
+ *
+ * @param dir Directory to store files. Created if non-existent (but not recursively).
+ * @param serialization Used to serialize our objects.
+ */
+private[spark] class FileSystemPersistenceEngine(
+    val dir: String,
+    val serialization: Serialization)
+  extends PersistenceEngine with Logging {
+
+  new File(dir).mkdir()
+
+  override def addApplication(app: ApplicationInfo) {
+    val appFile = new File(dir + File.separator + "app_" + app.id)
+    serializeIntoFile(appFile, app)
+  }
+
+  override def removeApplication(app: ApplicationInfo) {
+    new File(dir + File.separator + "app_" + app.id).delete()
+  }
+
+  override def addWorker(worker: WorkerInfo) {
+    val workerFile = new File(dir + File.separator + "worker_" + worker.id)
+    serializeIntoFile(workerFile, worker)
+  }
+
+  override def removeWorker(worker: WorkerInfo) {
+    new File(dir + File.separator + "worker_" + worker.id).delete()
+  }
+
+  override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
+    val sortedFiles = new File(dir).listFiles().sortBy(_.getName)
+    val appFiles = sortedFiles.filter(_.getName.startsWith("app_"))
+    val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
+    val workerFiles = sortedFiles.filter(_.getName.startsWith("worker_"))
+    val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
+    (apps, workers)
+  }
+
+  private def serializeIntoFile(file: File, value: Serializable) {
+    val created = file.createNewFile()
+    if (!created) { throw new IllegalStateException("Could not create file: " + file) }
+
+    val serializer = serialization.findSerializerFor(value)
+    val serialized = serializer.toBinary(value)
+
+    val out = new FileOutputStream(file)
+    out.write(serialized)
+    out.close()
+  }
+
+  def deserializeFromFile[T <: Serializable](file: File)(implicit m: Manifest[T]): T = {
+    val fileData = new Array[Byte](file.length().asInstanceOf[Int])
+    val dis = new DataInputStream(new FileInputStream(file))
+    dis.readFully(fileData)
+    dis.close()
+
+    val clazz = m.erasure.asInstanceOf[Class[T]]
+    val serializer = serialization.serializerFor(clazz)
+    serializer.fromBinary(fileData).asInstanceOf[T]
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f25a1ad3bf92afbd22539ba552b76bd60382ec78
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.master
+
+import akka.actor.{Actor, ActorRef}
+
+import org.apache.spark.deploy.master.MasterMessages.ElectedLeader
+
+/**
+ * A LeaderElectionAgent keeps track of whether the current Master is the leader, meaning it
+ * is the only Master serving requests.
+ * In addition to the API provided, the LeaderElectionAgent will use of the following messages
+ * to inform the Master of leader changes:
+ * [[org.apache.spark.deploy.master.MasterMessages.ElectedLeader ElectedLeader]]
+ * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]]
+ */
+private[spark] trait LeaderElectionAgent extends Actor {
+  val masterActor: ActorRef
+}
+
+/** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */
+private[spark] class MonarchyLeaderAgent(val masterActor: ActorRef) extends LeaderElectionAgent {
+  override def preStart() {
+    masterActor ! ElectedLeader
+  }
+
+  override def receive = {
+    case _ =>
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index bde59905bc7646ab20170500c331cf77ecdf294a..cd916672aceffbc09048e24d3e1babdad3e4044d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -27,24 +27,26 @@ import akka.actor.Terminated
 import akka.dispatch.Await
 import akka.pattern.ask
 import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
+import akka.serialization.SerializationExtension
 import akka.util.duration._
-import akka.util.Timeout
+import akka.util.{Duration, Timeout}
 
 import org.apache.spark.{Logging, SparkException}
 import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.MasterMessages._
 import org.apache.spark.deploy.master.ui.MasterWebUI
 import org.apache.spark.metrics.MetricsSystem
-import org.apache.spark.util.{Utils, AkkaUtils}
-import akka.util.{Duration, Timeout}
-
+import org.apache.spark.util.{AkkaUtils, Utils}
 
 private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
   val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
   val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt
   val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt
- 
+  val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "")
+  val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE")
+
   var nextAppNumber = 0
   val workers = new HashSet[WorkerInfo]
   val idToWorker = new HashMap[String, WorkerInfo]
@@ -74,51 +76,116 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     if (envVar != null) envVar else host
   }
 
+  val masterUrl = "spark://" + host + ":" + port
+  var masterWebUiUrl: String = _
+
+  var state = RecoveryState.STANDBY
+
+  var persistenceEngine: PersistenceEngine = _
+
+  var leaderElectionAgent: ActorRef = _
+
   // As a temporary workaround before better ways of configuring memory, we allow users to set
   // a flag that will perform round-robin scheduling across the nodes (spreading out each app
   // among all the nodes) instead of trying to consolidate each app onto a small # of nodes.
   val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean
 
   override def preStart() {
-    logInfo("Starting Spark master at spark://" + host + ":" + port)
+    logInfo("Starting Spark master at " + masterUrl)
     // Listen for remote client disconnection events, since they don't go through Akka's watch()
     context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
     webUi.start()
+    masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get
     context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut)
 
     masterMetricsSystem.registerSource(masterSource)
     masterMetricsSystem.start()
     applicationMetricsSystem.start()
+
+    persistenceEngine = RECOVERY_MODE match {
+      case "ZOOKEEPER" =>
+        logInfo("Persisting recovery state to ZooKeeper")
+        new ZooKeeperPersistenceEngine(SerializationExtension(context.system))
+      case "FILESYSTEM" =>
+        logInfo("Persisting recovery state to directory: " + RECOVERY_DIR)
+        new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system))
+      case _ =>
+        new BlackHolePersistenceEngine()
+    }
+
+    leaderElectionAgent = context.actorOf(Props(
+      RECOVERY_MODE match {
+        case "ZOOKEEPER" =>
+          new ZooKeeperLeaderElectionAgent(self, masterUrl)
+        case _ =>
+          new MonarchyLeaderAgent(self)
+      }))
+  }
+
+  override def preRestart(reason: Throwable, message: Option[Any]) {
+    super.preRestart(reason, message) // calls postStop()!
+    logError("Master actor restarted due to exception", reason)
   }
 
   override def postStop() {
     webUi.stop()
     masterMetricsSystem.stop()
     applicationMetricsSystem.stop()
+    persistenceEngine.close()
+    context.stop(leaderElectionAgent)
   }
 
   override def receive = {
-    case RegisterWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) => {
+    case ElectedLeader => {
+      val (storedApps, storedWorkers) = persistenceEngine.readPersistedData()
+      state = if (storedApps.isEmpty && storedWorkers.isEmpty)
+        RecoveryState.ALIVE
+      else
+        RecoveryState.RECOVERING
+
+      logInfo("I have been elected leader! New state: " + state)
+
+      if (state == RecoveryState.RECOVERING) {
+        beginRecovery(storedApps, storedWorkers)
+        context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() }
+      }
+    }
+
+    case RevokedLeadership => {
+      logError("Leadership has been revoked -- master shutting down.")
+      System.exit(0)
+    }
+
+    case RegisterWorker(id, host, workerPort, cores, memory, webUiPort, publicAddress) => {
       logInfo("Registering worker %s:%d with %d cores, %s RAM".format(
         host, workerPort, cores, Utils.megabytesToString(memory)))
-      if (idToWorker.contains(id)) {
+      if (state == RecoveryState.STANDBY) {
+        // ignore, don't send response
+      } else if (idToWorker.contains(id)) {
         sender ! RegisterWorkerFailed("Duplicate worker ID")
       } else {
-        addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress)
+        val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
+        registerWorker(worker)
         context.watch(sender)  // This doesn't work with remote actors but helps for testing
-        sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort.get)
+        persistenceEngine.addWorker(worker)
+        sender ! RegisteredWorker(masterUrl, masterWebUiUrl)
         schedule()
       }
     }
 
     case RegisterApplication(description) => {
-      logInfo("Registering app " + description.name)
-      val app = addApplication(description, sender)
-      logInfo("Registered app " + description.name + " with ID " + app.id)
-      waitingApps += app
-      context.watch(sender)  // This doesn't work with remote actors but helps for testing
-      sender ! RegisteredApplication(app.id)
-      schedule()
+      if (state == RecoveryState.STANDBY) {
+        // ignore, don't send response
+      } else {
+        logInfo("Registering app " + description.name)
+        val app = createApplication(description, sender)
+        registerApplication(app)
+        logInfo("Registered app " + description.name + " with ID " + app.id)
+        context.watch(sender)  // This doesn't work with remote actors but helps for testing
+        persistenceEngine.addApplication(app)
+        sender ! RegisteredApplication(app.id, masterUrl)
+        schedule()
+      }
     }
 
     case ExecutorStateChanged(appId, execId, state, message, exitStatus) => {
@@ -158,27 +225,63 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       }
     }
 
+    case MasterChangeAcknowledged(appId) => {
+      idToApp.get(appId) match {
+        case Some(app) =>
+          logInfo("Application has been re-registered: " + appId)
+          app.state = ApplicationState.WAITING
+        case None =>
+          logWarning("Master change ack from unknown app: " + appId)
+      }
+
+      if (canCompleteRecovery) { completeRecovery() }
+    }
+
+    case WorkerSchedulerStateResponse(workerId, executors) => {
+      idToWorker.get(workerId) match {
+        case Some(worker) =>
+          logInfo("Worker has been re-registered: " + workerId)
+          worker.state = WorkerState.ALIVE
+
+          val validExecutors = executors.filter(exec => idToApp.get(exec.appId).isDefined)
+          for (exec <- validExecutors) {
+            val app = idToApp.get(exec.appId).get
+            val execInfo = app.addExecutor(worker, exec.cores, Some(exec.execId))
+            worker.addExecutor(execInfo)
+            execInfo.copyState(exec)
+          }
+        case None =>
+          logWarning("Scheduler state from unknown worker: " + workerId)
+      }
+
+      if (canCompleteRecovery) { completeRecovery() }
+    }
+
     case Terminated(actor) => {
       // The disconnected actor could've been either a worker or an app; remove whichever of
       // those we have an entry for in the corresponding actor hashmap
       actorToWorker.get(actor).foreach(removeWorker)
       actorToApp.get(actor).foreach(finishApplication)
+      if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
     }
 
     case RemoteClientDisconnected(transport, address) => {
       // The disconnected client could've been either a worker or an app; remove whichever it was
       addressToWorker.get(address).foreach(removeWorker)
       addressToApp.get(address).foreach(finishApplication)
+      if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
     }
 
     case RemoteClientShutdown(transport, address) => {
       // The disconnected client could've been either a worker or an app; remove whichever it was
       addressToWorker.get(address).foreach(removeWorker)
       addressToApp.get(address).foreach(finishApplication)
+      if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
     }
 
     case RequestMasterState => {
-      sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray)
+      sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray,
+        state)
     }
 
     case CheckForWorkerTimeOut => {
@@ -190,6 +293,50 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }
   }
 
+  def canCompleteRecovery =
+    workers.count(_.state == WorkerState.UNKNOWN) == 0 &&
+      apps.count(_.state == ApplicationState.UNKNOWN) == 0
+
+  def beginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) {
+    for (app <- storedApps) {
+      logInfo("Trying to recover app: " + app.id)
+      try {
+        registerApplication(app)
+        app.state = ApplicationState.UNKNOWN
+        app.driver ! MasterChanged(masterUrl, masterWebUiUrl)
+      } catch {
+        case e: Exception => logInfo("App " + app.id + " had exception on reconnect")
+      }
+    }
+
+    for (worker <- storedWorkers) {
+      logInfo("Trying to recover worker: " + worker.id)
+      try {
+        registerWorker(worker)
+        worker.state = WorkerState.UNKNOWN
+        worker.actor ! MasterChanged(masterUrl, masterWebUiUrl)
+      } catch {
+        case e: Exception => logInfo("Worker " + worker.id + " had exception on reconnect")
+      }
+    }
+  }
+
+  def completeRecovery() {
+    // Ensure "only-once" recovery semantics using a short synchronization period.
+    synchronized {
+      if (state != RecoveryState.RECOVERING) { return }
+      state = RecoveryState.COMPLETING_RECOVERY
+    }
+
+    // Kill off any workers and apps that didn't respond to us.
+    workers.filter(_.state == WorkerState.UNKNOWN).foreach(removeWorker)
+    apps.filter(_.state == ApplicationState.UNKNOWN).foreach(finishApplication)
+
+    state = RecoveryState.ALIVE
+    schedule()
+    logInfo("Recovery complete - resuming operations!")
+  }
+
   /**
    * Can an app use the given worker? True if the worker has enough memory and we haven't already
    * launched an executor for the app on it (right now the standalone backend doesn't like having
@@ -204,6 +351,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
    * every time a new app joins or resource availability changes.
    */
   def schedule() {
+    if (state != RecoveryState.ALIVE) { return }
     // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
     // in the queue, then the second app, etc.
     if (spreadOutApps) {
@@ -251,14 +399,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
   def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) {
     logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
     worker.addExecutor(exec)
-    worker.actor ! LaunchExecutor(
+    worker.actor ! LaunchExecutor(masterUrl,
       exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome)
     exec.application.driver ! ExecutorAdded(
       exec.id, worker.id, worker.hostPort, exec.cores, exec.memory)
   }
 
-  def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int,
-    publicAddress: String): WorkerInfo = {
+  def registerWorker(worker: WorkerInfo): Unit = {
     // There may be one or more refs to dead workers on this same node (w/ different ID's),
     // remove them.
     workers.filter { w =>
@@ -266,12 +413,17 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }.foreach { w =>
       workers -= w
     }
-    val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
+
+    val workerAddress = worker.actor.path.address
+    if (addressToWorker.contains(workerAddress)) {
+      logInfo("Attempted to re-register worker at same address: " + workerAddress)
+      return
+    }
+
     workers += worker
     idToWorker(worker.id) = worker
-    actorToWorker(sender) = worker
-    addressToWorker(sender.path.address) = worker
-    worker
+    actorToWorker(worker.actor) = worker
+    addressToWorker(workerAddress) = worker
   }
 
   def removeWorker(worker: WorkerInfo) {
@@ -286,25 +438,36 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         exec.id, ExecutorState.LOST, Some("worker lost"), None)
       exec.application.removeExecutor(exec)
     }
+    persistenceEngine.removeWorker(worker)
   }
 
-  def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
+  def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
     val now = System.currentTimeMillis()
     val date = new Date(now)
-    val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
+    new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
+  }
+
+  def registerApplication(app: ApplicationInfo): Unit = {
+    val appAddress = app.driver.path.address
+    if (addressToWorker.contains(appAddress)) {
+      logInfo("Attempted to re-register application at same address: " + appAddress)
+      return
+    }
+
     applicationMetricsSystem.registerSource(app.appSource)
     apps += app
     idToApp(app.id) = app
-    actorToApp(driver) = app
-    addressToApp(driver.path.address) = app
+    actorToApp(app.driver) = app
+    addressToApp(appAddress) = app
     if (firstApp == None) {
       firstApp = Some(app)
     }
+    // TODO: What is firstApp?? Can we remove it?
     val workersAlive = workers.filter(_.state == WorkerState.ALIVE).toArray
-    if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) {
+    if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= app.desc.memoryPerSlave)) {
       logWarning("Could not find any workers with enough memory for " + firstApp.get.id)
     }
-    app
+    waitingApps += app
   }
 
   def finishApplication(app: ApplicationInfo) {
@@ -329,13 +492,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       waitingApps -= app
       for (exec <- app.executors.values) {
         exec.worker.removeExecutor(exec)
-        exec.worker.actor ! KillExecutor(exec.application.id, exec.id)
+        exec.worker.actor ! KillExecutor(masterUrl, exec.application.id, exec.id)
         exec.state = ExecutorState.KILLED
       }
       app.markFinished(state)
       if (state != ApplicationState.FINISHED) {
         app.driver ! ApplicationRemoved(state.toString)
       }
+      persistenceEngine.removeApplication(app)
       schedule()
     }
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
new file mode 100644
index 0000000000000000000000000000000000000000..74a9f8cd824fb7bda55d4e97ada511c9aeb821fd
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.master
+
+sealed trait MasterMessages extends Serializable
+
+/** Contains messages seen only by the Master and its associated entities. */
+private[master] object MasterMessages {
+
+  // LeaderElectionAgent to Master
+
+  case object ElectedLeader
+
+  case object RevokedLeadership
+
+  // Actor System to LeaderElectionAgent
+
+  case object CheckLeader
+
+  // Actor System to Master
+
+  case object CheckForWorkerTimeOut
+
+  case class BeginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo])
+
+  case object CompleteRecovery
+
+  case object RequestWebUIPort
+
+  case class WebUIPortResponse(webUIBoundPort: Int)
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
new file mode 100644
index 0000000000000000000000000000000000000000..94b986caf283518e2e809d5b6742541bc481ead7
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.master
+
+/**
+ * Allows Master to persist any state that is necessary in order to recover from a failure.
+ * The following semantics are required:
+ *   - addApplication and addWorker are called before completing registration of a new app/worker.
+ *   - removeApplication and removeWorker are called at any time.
+ * Given these two requirements, we will have all apps and workers persisted, but
+ * we might not have yet deleted apps or workers that finished (so their liveness must be verified
+ * during recovery).
+ */
+private[spark] trait PersistenceEngine {
+  def addApplication(app: ApplicationInfo)
+
+  def removeApplication(app: ApplicationInfo)
+
+  def addWorker(worker: WorkerInfo)
+
+  def removeWorker(worker: WorkerInfo)
+
+  /**
+   * Returns the persisted data sorted by their respective ids (which implies that they're
+   * sorted by time of creation).
+   */
+  def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo])
+
+  def close() {}
+}
+
+private[spark] class BlackHolePersistenceEngine extends PersistenceEngine {
+  override def addApplication(app: ApplicationInfo) {}
+  override def removeApplication(app: ApplicationInfo) {}
+  override def addWorker(worker: WorkerInfo) {}
+  override def removeWorker(worker: WorkerInfo) {}
+  override def readPersistedData() = (Nil, Nil)
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b91be821f016c834f3eff3aa00334a8154c6afe9
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.master
+
+private[spark] object RecoveryState
+  extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") {
+
+  type MasterState = Value
+
+  val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
new file mode 100644
index 0000000000000000000000000000000000000000..81e15c534fc31e1e8b8b4914ff9c24a1149014a2
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.master
+
+import scala.collection.JavaConversions._
+import scala.concurrent.ops._
+
+import org.apache.spark.Logging
+import org.apache.zookeeper._
+import org.apache.zookeeper.data.Stat
+import org.apache.zookeeper.Watcher.Event.KeeperState
+
+/**
+ * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry
+ * logic. If the ZooKeeper session expires or otherwise dies, a new ZooKeeper session will be
+ * created. If ZooKeeper remains down after several retries, the given
+ * [[org.apache.spark.deploy.master.SparkZooKeeperWatcher SparkZooKeeperWatcher]] will be
+ * informed via zkDown().
+ *
+ * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many
+ * times or a semantic exception is thrown (e.g.., "node already exists").
+ */
+private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging {
+  val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "")
+
+  val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE
+  val ZK_TIMEOUT_MILLIS = 30000
+  val RETRY_WAIT_MILLIS = 5000
+  val ZK_CHECK_PERIOD_MILLIS = 10000
+  val MAX_RECONNECT_ATTEMPTS = 3
+
+  private var zk: ZooKeeper = _
+
+  private val watcher = new ZooKeeperWatcher()
+  private var reconnectAttempts = 0
+  private var closed = false
+
+  /** Connect to ZooKeeper to start the session. Must be called before anything else. */
+  def connect() {
+    connectToZooKeeper()
+
+    new Thread() {
+      override def run() = sessionMonitorThread()
+    }.start()
+  }
+
+  def sessionMonitorThread(): Unit = {
+    while (!closed) {
+      Thread.sleep(ZK_CHECK_PERIOD_MILLIS)
+      if (zk.getState != ZooKeeper.States.CONNECTED) {
+        reconnectAttempts += 1
+        val attemptsLeft = MAX_RECONNECT_ATTEMPTS - reconnectAttempts
+        if (attemptsLeft <= 0) {
+          logError("Could not connect to ZooKeeper: system failure")
+          zkWatcher.zkDown()
+          close()
+        } else {
+          logWarning("ZooKeeper connection failed, retrying " + attemptsLeft + " more times...")
+          connectToZooKeeper()
+        }
+      }
+    }
+  }
+
+  def close() {
+    if (!closed && zk != null) { zk.close() }
+    closed = true
+  }
+
+  private def connectToZooKeeper() {
+    if (zk != null) zk.close()
+    zk = new ZooKeeper(ZK_URL, ZK_TIMEOUT_MILLIS, watcher)
+  }
+
+  /**
+   * Attempts to maintain a live ZooKeeper exception despite (very) transient failures.
+   * Mainly useful for handling the natural ZooKeeper session expiration.
+   */
+  private class ZooKeeperWatcher extends Watcher {
+    def process(event: WatchedEvent) {
+      if (closed) { return }
+
+      event.getState match {
+        case KeeperState.SyncConnected =>
+          reconnectAttempts = 0
+          zkWatcher.zkSessionCreated()
+        case KeeperState.Expired =>
+          connectToZooKeeper()
+        case KeeperState.Disconnected =>
+          logWarning("ZooKeeper disconnected, will retry...")
+      }
+    }
+  }
+
+  def create(path: String, bytes: Array[Byte], createMode: CreateMode): String = {
+    retry {
+      zk.create(path, bytes, ZK_ACL, createMode)
+    }
+  }
+
+  def exists(path: String, watcher: Watcher = null): Stat = {
+    retry {
+      zk.exists(path, watcher)
+    }
+  }
+
+  def getChildren(path: String, watcher: Watcher = null): List[String] = {
+    retry {
+      zk.getChildren(path, watcher).toList
+    }
+  }
+
+  def getData(path: String): Array[Byte] = {
+    retry {
+      zk.getData(path, false, null)
+    }
+  }
+
+  def delete(path: String, version: Int = -1): Unit = {
+    retry {
+      zk.delete(path, version)
+    }
+  }
+
+  /**
+   * Creates the given directory (non-recursively) if it doesn't exist.
+   * All znodes are created in PERSISTENT mode with no data.
+   */
+  def mkdir(path: String) {
+    if (exists(path) == null) {
+      try {
+        create(path, "".getBytes, CreateMode.PERSISTENT)
+      } catch {
+        case e: Exception =>
+          // If the exception caused the directory not to be created, bubble it up,
+          // otherwise ignore it.
+          if (exists(path) == null) { throw e }
+      }
+    }
+  }
+
+  /**
+   * Recursively creates all directories up to the given one.
+   * All znodes are created in PERSISTENT mode with no data.
+   */
+  def mkdirRecursive(path: String) {
+    var fullDir = ""
+    for (dentry <- path.split("/").tail) {
+      fullDir += "/" + dentry
+      mkdir(fullDir)
+    }
+  }
+
+  /**
+   * Retries the given function up to 3 times. The assumption is that failure is transient,
+   * UNLESS it is a semantic exception (i.e., trying to get data from a node that doesn't exist),
+   * in which case the exception will be thrown without retries.
+   *
+   * @param fn Block to execute, possibly multiple times.
+   */
+  def retry[T](fn: => T, n: Int = MAX_RECONNECT_ATTEMPTS): T = {
+    try {
+      fn
+    } catch {
+      case e: KeeperException.NoNodeException => throw e
+      case e: KeeperException.NodeExistsException => throw e
+      case e if n > 0 =>
+        logError("ZooKeeper exception, " + n + " more retries...", e)
+        Thread.sleep(RETRY_WAIT_MILLIS)
+        retry(fn, n-1)
+    }
+  }
+}
+
+trait SparkZooKeeperWatcher {
+  /**
+   * Called whenever a ZK session is created --
+   * this will occur when we create our first session as well as each time
+   * the session expires or errors out.
+   */
+  def zkSessionCreated()
+
+  /**
+   * Called if ZK appears to be completely down (i.e., not just a transient error).
+   * We will no longer attempt to reconnect to ZK, and the SparkZooKeeperSession is considered dead.
+   */
+  def zkDown()
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index 6219f11f2a2b2c3b7b02bc7a56b7643ca8b1020f..e05f587b58c6437ce869fd26d7021c8dc20338b0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -22,28 +22,44 @@ import scala.collection.mutable
 import org.apache.spark.util.Utils
 
 private[spark] class WorkerInfo(
-  val id: String,
-  val host: String,
-  val port: Int,
-  val cores: Int,
-  val memory: Int,
-  val actor: ActorRef,
-  val webUiPort: Int,
-  val publicAddress: String) {
+    val id: String,
+    val host: String,
+    val port: Int,
+    val cores: Int,
+    val memory: Int,
+    val actor: ActorRef,
+    val webUiPort: Int,
+    val publicAddress: String)
+  extends Serializable {
 
   Utils.checkHost(host, "Expected hostname")
   assert (port > 0)
 
-  var executors = new mutable.HashMap[String, ExecutorInfo]  // fullId => info
-  var state: WorkerState.Value = WorkerState.ALIVE
-  var coresUsed = 0
-  var memoryUsed = 0
+  @transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // fullId => info
+  @transient var state: WorkerState.Value = _
+  @transient var coresUsed: Int = _
+  @transient var memoryUsed: Int = _
 
-  var lastHeartbeat = System.currentTimeMillis()
+  @transient var lastHeartbeat: Long = _
+
+  init()
 
   def coresFree: Int = cores - coresUsed
   def memoryFree: Int = memory - memoryUsed
 
+  private def readObject(in: java.io.ObjectInputStream) : Unit = {
+    in.defaultReadObject()
+    init()
+  }
+
+  private def init() {
+    executors = new mutable.HashMap
+    state = WorkerState.ALIVE
+    coresUsed = 0
+    memoryUsed = 0
+    lastHeartbeat = System.currentTimeMillis()
+  }
+
   def hostPort: String = {
     assert (port > 0)
     host + ":" + port
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
index b5ee6dca79fab36aeace009d436d7fd1ea69e481..c8d34f25e25e053cb950323d78003680eb46f71c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
@@ -17,8 +17,10 @@
 
 package org.apache.spark.deploy.master
 
-private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") {
+private[spark] object WorkerState
+  extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED", "UNKNOWN") {
+
   type WorkerState = Value
 
-  val ALIVE, DEAD, DECOMMISSIONED = Value
+  val ALIVE, DEAD, DECOMMISSIONED, UNKNOWN = Value
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7809013e8383b2b65532d7cb1c1df738ac68c9a8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.master
+
+import akka.actor.ActorRef
+import org.apache.zookeeper._
+import org.apache.zookeeper.Watcher.Event.EventType
+
+import org.apache.spark.deploy.master.MasterMessages._
+import org.apache.spark.Logging
+
+private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String)
+  extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging  {
+
+  val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
+
+  private val watcher = new ZooKeeperWatcher()
+  private val zk = new SparkZooKeeperSession(this)
+  private var status = LeadershipStatus.NOT_LEADER
+  private var myLeaderFile: String = _
+  private var leaderUrl: String = _
+
+  override def preStart() {
+    logInfo("Starting ZooKeeper LeaderElection agent")
+    zk.connect()
+  }
+
+  override def zkSessionCreated() {
+    synchronized {
+      zk.mkdirRecursive(WORKING_DIR)
+      myLeaderFile =
+        zk.create(WORKING_DIR + "/master_", masterUrl.getBytes, CreateMode.EPHEMERAL_SEQUENTIAL)
+      self ! CheckLeader
+    }
+  }
+
+  override def preRestart(reason: scala.Throwable, message: scala.Option[scala.Any]) {
+    logError("LeaderElectionAgent failed, waiting " + zk.ZK_TIMEOUT_MILLIS + "...", reason)
+    Thread.sleep(zk.ZK_TIMEOUT_MILLIS)
+    super.preRestart(reason, message)
+  }
+
+  override def zkDown() {
+    logError("ZooKeeper down! LeaderElectionAgent shutting down Master.")
+    System.exit(1)
+  }
+
+  override def postStop() {
+    zk.close()
+  }
+
+  override def receive = {
+    case CheckLeader => checkLeader()
+  }
+
+  private class ZooKeeperWatcher extends Watcher {
+    def process(event: WatchedEvent) {
+      if (event.getType == EventType.NodeDeleted) {
+        logInfo("Leader file disappeared, a master is down!")
+        self ! CheckLeader
+      }
+    }
+  }
+
+  /** Uses ZK leader election. Navigates several ZK potholes along the way. */
+  def checkLeader() {
+    val masters = zk.getChildren(WORKING_DIR).toList
+    val leader = masters.sorted.head
+    val leaderFile = WORKING_DIR + "/" + leader
+
+    // Setup a watch for the current leader.
+    zk.exists(leaderFile, watcher)
+
+    try {
+      leaderUrl = new String(zk.getData(leaderFile))
+    } catch {
+      // A NoNodeException may be thrown if old leader died since the start of this method call.
+      // This is fine -- just check again, since we're guaranteed to see the new values.
+      case e: KeeperException.NoNodeException =>
+        logInfo("Leader disappeared while reading it -- finding next leader")
+        checkLeader()
+        return
+    }
+
+    // Synchronization used to ensure no interleaving between the creation of a new session and the
+    // checking of a leader, which could cause us to delete our real leader file erroneously.
+    synchronized {
+      val isLeader = myLeaderFile == leaderFile
+      if (!isLeader && leaderUrl == masterUrl) {
+        // We found a different master file pointing to this process.
+        // This can happen in the following two cases:
+        // (1) The master process was restarted on the same node.
+        // (2) The ZK server died between creating the node and returning the name of the node.
+        //     For this case, we will end up creating a second file, and MUST explicitly delete the
+        //     first one, since our ZK session is still open.
+        // Note that this deletion will cause a NodeDeleted event to be fired so we check again for
+        // leader changes.
+        assert(leaderFile < myLeaderFile)
+        logWarning("Cleaning up old ZK master election file that points to this master.")
+        zk.delete(leaderFile)
+      } else {
+        updateLeadershipStatus(isLeader)
+      }
+    }
+  }
+
+  def updateLeadershipStatus(isLeader: Boolean) {
+    if (isLeader && status == LeadershipStatus.NOT_LEADER) {
+      status = LeadershipStatus.LEADER
+      masterActor ! ElectedLeader
+    } else if (!isLeader && status == LeadershipStatus.LEADER) {
+      status = LeadershipStatus.NOT_LEADER
+      masterActor ! RevokedLeadership
+    }
+  }
+
+  private object LeadershipStatus extends Enumeration {
+    type LeadershipStatus = Value
+    val LEADER, NOT_LEADER = Value
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a0233a727118255041cd53416aff3d4d55bdfc99
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.master
+
+import org.apache.spark.Logging
+import org.apache.zookeeper._
+
+import akka.serialization.Serialization
+
+class ZooKeeperPersistenceEngine(serialization: Serialization)
+  extends PersistenceEngine
+  with SparkZooKeeperWatcher
+  with Logging
+{
+  val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
+
+  val zk = new SparkZooKeeperSession(this)
+
+  zk.connect()
+
+  override def zkSessionCreated() {
+    zk.mkdirRecursive(WORKING_DIR)
+  }
+
+  override def zkDown() {
+    logError("PersistenceEngine disconnected from ZooKeeper -- ZK looks down.")
+  }
+
+  override def addApplication(app: ApplicationInfo) {
+    serializeIntoFile(WORKING_DIR + "/app_" + app.id, app)
+  }
+
+  override def removeApplication(app: ApplicationInfo) {
+    zk.delete(WORKING_DIR + "/app_" + app.id)
+  }
+
+  override def addWorker(worker: WorkerInfo) {
+    serializeIntoFile(WORKING_DIR + "/worker_" + worker.id, worker)
+  }
+
+  override def removeWorker(worker: WorkerInfo) {
+    zk.delete(WORKING_DIR + "/worker_" + worker.id)
+  }
+
+  override def close() {
+    zk.close()
+  }
+
+  override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
+    val sortedFiles = zk.getChildren(WORKING_DIR).toList.sorted
+    val appFiles = sortedFiles.filter(_.startsWith("app_"))
+    val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
+    val workerFiles = sortedFiles.filter(_.startsWith("worker_"))
+    val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
+    (apps, workers)
+  }
+
+  private def serializeIntoFile(path: String, value: Serializable) {
+    val serializer = serialization.findSerializerFor(value)
+    val serialized = serializer.toBinary(value)
+    zk.create(path, serialized, CreateMode.PERSISTENT)
+  }
+
+  def deserializeFromFile[T <: Serializable](filename: String)(implicit m: Manifest[T]): T = {
+    val fileData = zk.getData("/spark/master_status/" + filename)
+    val clazz = m.erasure.asInstanceOf[Class[T]]
+    val serializer = serialization.serializerFor(clazz)
+    serializer.fromBinary(fileData).asInstanceOf[T]
+  }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index e3dc30eefc56f951e69aa186982c0ddfe03573d2..8fabc956659015c4df1796a918779a954b81705d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -43,7 +43,8 @@ private[spark] class ExecutorRunner(
     val workerId: String,
     val host: String,
     val sparkHome: File,
-    val workDir: File)
+    val workDir: File,
+    var state: ExecutorState.Value)
   extends Logging {
 
   val fullId = appId + "/" + execId
@@ -83,7 +84,8 @@ private[spark] class ExecutorRunner(
         process.destroy()
         process.waitFor()
       }
-      worker ! ExecutorStateChanged(appId, execId, ExecutorState.KILLED, None, None)
+      state = ExecutorState.KILLED
+      worker ! ExecutorStateChanged(appId, execId, state, None, None)
       Runtime.getRuntime.removeShutdownHook(shutdownHook)
     }
   }
@@ -180,9 +182,9 @@ private[spark] class ExecutorRunner(
       // long-lived processes only. However, in the future, we might restart the executor a few
       // times on the same machine.
       val exitCode = process.waitFor()
+      state = ExecutorState.FAILED
       val message = "Command exited with code " + exitCode
-      worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message),
-                                    Some(exitCode))
+      worker ! ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode))
     } catch {
       case interrupted: InterruptedException =>
         logInfo("Runner thread for executor " + fullId + " interrupted")
@@ -192,8 +194,9 @@ private[spark] class ExecutorRunner(
         if (process != null) {
           process.destroy()
         }
+        state = ExecutorState.FAILED
         val message = e.getClass + ": " + e.getMessage
-        worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message), None)
+        worker ! ExecutorStateChanged(appId, execId, state, Some(message), None)
       }
     }
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 09530beb3bec05ca7e37b1f006ae7d14d1748257..216d9d44ac619ab046d15211071b5b65fffa0aa6 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -23,26 +23,28 @@ import java.io.File
 
 import scala.collection.mutable.HashMap
 
-import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
+import akka.actor._
 import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
 import akka.util.duration._
 
-import org.apache.spark.{Logging}
-import org.apache.spark.deploy.ExecutorState
+import org.apache.spark.Logging
+import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.deploy.worker.ui.WorkerWebUI
 import org.apache.spark.metrics.MetricsSystem
 import org.apache.spark.util.{Utils, AkkaUtils}
 
-
+/**
+  * @param masterUrls Each url should look like spark://host:port.
+  */
 private[spark] class Worker(
     host: String,
     port: Int,
     webUiPort: Int,
     cores: Int,
     memory: Int,
-    masterUrl: String,
+    masterUrls: Array[String],
     workDirPath: String = null)
   extends Actor with Logging {
 
@@ -54,8 +56,18 @@ private[spark] class Worker(
   // Send a heartbeat every (heartbeat timeout) / 4 milliseconds
   val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4
 
+  val REGISTRATION_TIMEOUT = 20.seconds
+  val REGISTRATION_RETRIES = 3
+
+  // Index into masterUrls that we're currently trying to register with.
+  var masterIndex = 0
+
+  val masterLock: Object = new Object()
   var master: ActorRef = null
-  var masterWebUiUrl : String = ""
+  var activeMasterUrl: String = ""
+  var activeMasterWebUiUrl : String = ""
+  @volatile var registered = false
+  @volatile var connected = false
   val workerId = generateWorkerId()
   var sparkHome: File = null
   var workDir: File = null
@@ -95,6 +107,7 @@ private[spark] class Worker(
   }
 
   override def preStart() {
+    assert(!registered)
     logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format(
       host, port, cores, Utils.megabytesToString(memory)))
     sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse("."))
@@ -103,44 +116,98 @@ private[spark] class Worker(
     webUi = new WorkerWebUI(this, workDir, Some(webUiPort))
 
     webUi.start()
-    connectToMaster()
+    registerWithMaster()
 
     metricsSystem.registerSource(workerSource)
     metricsSystem.start()
   }
 
-  def connectToMaster() {
-    logInfo("Connecting to master " + masterUrl)
-    master = context.actorFor(Master.toAkkaUrl(masterUrl))
-    master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress)
-    context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
-    context.watch(master) // Doesn't work with remote actors, but useful for testing
+  def changeMaster(url: String, uiUrl: String) {
+    masterLock.synchronized {
+      activeMasterUrl = url
+      activeMasterWebUiUrl = uiUrl
+      master = context.actorFor(Master.toAkkaUrl(activeMasterUrl))
+      context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+      context.watch(master) // Doesn't work with remote actors, but useful for testing
+      connected = true
+    }
+  }
+
+  def tryRegisterAllMasters() {
+    for (masterUrl <- masterUrls) {
+      logInfo("Connecting to master " + masterUrl + "...")
+      val actor = context.actorFor(Master.toAkkaUrl(masterUrl))
+      actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get,
+        publicAddress)
+    }
+  }
+
+  def registerWithMaster() {
+    tryRegisterAllMasters()
+
+    var retries = 0
+    lazy val retryTimer: Cancellable =
+      context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
+        retries += 1
+        if (registered) {
+          retryTimer.cancel()
+        } else if (retries >= REGISTRATION_RETRIES) {
+          logError("All masters are unresponsive! Giving up.")
+          System.exit(1)
+        } else {
+          tryRegisterAllMasters()
+        }
+      }
+    retryTimer // start timer
   }
 
   override def receive = {
-    case RegisteredWorker(url) =>
-      masterWebUiUrl = url
-      logInfo("Successfully registered with master")
-      context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) {
-        master ! Heartbeat(workerId)
+    case RegisteredWorker(masterUrl, masterWebUiUrl) =>
+      logInfo("Successfully registered with master " + masterUrl)
+      registered = true
+      changeMaster(masterUrl, masterWebUiUrl)
+      context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat)
+
+    case SendHeartbeat =>
+      masterLock.synchronized {
+        if (connected) { master ! Heartbeat(workerId) }
       }
 
+    case MasterChanged(masterUrl, masterWebUiUrl) =>
+      logInfo("Master has changed, new master is at " + masterUrl)
+      context.unwatch(master)
+      changeMaster(masterUrl, masterWebUiUrl)
+
+      val execs = executors.values.
+        map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state))
+      sender ! WorkerSchedulerStateResponse(workerId, execs.toList)
+
     case RegisterWorkerFailed(message) =>
-      logError("Worker registration failed: " + message)
-      System.exit(1)
-
-    case LaunchExecutor(appId, execId, appDesc, cores_, memory_, execSparkHome_) =>
-      logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
-      val manager = new ExecutorRunner(
-        appId, execId, appDesc, cores_, memory_, self, workerId, host, new File(execSparkHome_), workDir)
-      executors(appId + "/" + execId) = manager
-      manager.start()
-      coresUsed += cores_
-      memoryUsed += memory_
-      master ! ExecutorStateChanged(appId, execId, ExecutorState.RUNNING, None, None)
+      if (!registered) {
+        logError("Worker registration failed: " + message)
+        System.exit(1)
+      }
+
+    case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_, execSparkHome_) =>
+      if (masterUrl != activeMasterUrl) {
+        logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.")
+      } else {
+        logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
+        val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_,
+          self, workerId, host, new File(execSparkHome_), workDir, ExecutorState.RUNNING)
+        executors(appId + "/" + execId) = manager
+        manager.start()
+        coresUsed += cores_
+        memoryUsed += memory_
+        masterLock.synchronized {
+          master ! ExecutorStateChanged(appId, execId, manager.state, None, None)
+        }
+      }
 
     case ExecutorStateChanged(appId, execId, state, message, exitStatus) =>
-      master ! ExecutorStateChanged(appId, execId, state, message, exitStatus)
+      masterLock.synchronized {
+        master ! ExecutorStateChanged(appId, execId, state, message, exitStatus)
+      }
       val fullId = appId + "/" + execId
       if (ExecutorState.isFinished(state)) {
         val executor = executors(fullId)
@@ -153,32 +220,39 @@ private[spark] class Worker(
         memoryUsed -= executor.memory
       }
 
-    case KillExecutor(appId, execId) =>
-      val fullId = appId + "/" + execId
-      executors.get(fullId) match {
-        case Some(executor) =>
-          logInfo("Asked to kill executor " + fullId)
-          executor.kill()
-        case None =>
-          logInfo("Asked to kill unknown executor " + fullId)
+    case KillExecutor(masterUrl, appId, execId) =>
+      if (masterUrl != activeMasterUrl) {
+        logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor " + execId)
+      } else {
+        val fullId = appId + "/" + execId
+        executors.get(fullId) match {
+          case Some(executor) =>
+            logInfo("Asked to kill executor " + fullId)
+            executor.kill()
+          case None =>
+            logInfo("Asked to kill unknown executor " + fullId)
+        }
       }
 
-    case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
+    case Terminated(actor_) if actor_ == master =>
+      masterDisconnected()
+
+    case RemoteClientDisconnected(transport, address) if address == master.path.address =>
+      masterDisconnected()
+
+    case RemoteClientShutdown(transport, address) if address == master.path.address =>
       masterDisconnected()
 
     case RequestWorkerState => {
       sender ! WorkerStateResponse(host, port, workerId, executors.values.toList,
-        finishedExecutors.values.toList, masterUrl, cores, memory,
-        coresUsed, memoryUsed, masterWebUiUrl)
+        finishedExecutors.values.toList, activeMasterUrl, cores, memory,
+        coresUsed, memoryUsed, activeMasterWebUiUrl)
     }
   }
 
   def masterDisconnected() {
-    // TODO: It would be nice to try to reconnect to the master, but just shut down for now.
-    // (Note that if reconnecting we would also need to assign IDs differently.)
-    logError("Connection to master failed! Shutting down.")
-    executors.values.foreach(_.kill())
-    System.exit(1)
+    logError("Connection to master failed! Waiting for master to reconnect...")
+    connected = false
   }
 
   def generateWorkerId(): String = {
@@ -196,17 +270,18 @@ private[spark] object Worker {
   def main(argStrings: Array[String]) {
     val args = new WorkerArguments(argStrings)
     val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores,
-      args.memory, args.master, args.workDir)
+      args.memory, args.masters, args.workDir)
     actorSystem.awaitTermination()
   }
 
   def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int,
-    masterUrl: String, workDir: String, workerNumber: Option[Int] = None): (ActorSystem, Int) = {
+    masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
+    : (ActorSystem, Int) = {
     // The LocalSparkCluster runs multiple local sparkWorkerX actor systems
     val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
     val actor = actorSystem.actorOf(Props(new Worker(host, boundPort, webUiPort, cores, memory,
-      masterUrl, workDir)), name = "Worker")
+      masterUrls, workDir)), name = "Worker")
     (actorSystem, boundPort)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index 0ae89a864fd57a48694137e2bd6554c0a16f16fc..3ed528e6b3773cf7bc3bf903d24adc7970f13752 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -29,7 +29,7 @@ private[spark] class WorkerArguments(args: Array[String]) {
   var webUiPort = 8081
   var cores = inferDefaultCores()
   var memory = inferDefaultMemory()
-  var master: String = null
+  var masters: Array[String] = null
   var workDir: String = null
   
   // Check for settings in environment variables 
@@ -86,14 +86,14 @@ private[spark] class WorkerArguments(args: Array[String]) {
       printUsageAndExit(0)
 
     case value :: tail =>
-      if (master != null) {  // Two positional arguments were given
+      if (masters != null) {  // Two positional arguments were given
         printUsageAndExit(1)
       }
-      master = value
+      masters = value.stripPrefix("spark://").split(",").map("spark://" + _)
       parse(tail)
 
     case Nil =>
-      if (master == null) {  // No positional argument was given
+      if (masters == null) {  // No positional argument was given
         printUsageAndExit(1)
       }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index 95d6007f3b35e1be4384c605927f4f3c87c32ce4..800f1cafccaa54b2f54adcb9fd277d7eecba7808 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -105,7 +105,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
 
     val logText = <node>{Utils.offsetBytes(path, startByte, endByte)}</node>
 
-    val linkToMaster = <p><a href={worker.masterWebUiUrl}>Back to Master</a></p>
+    val linkToMaster = <p><a href={worker.activeMasterWebUiUrl}>Back to Master</a></p>
 
     val range = <span>Bytes {startByte.toString} - {endByte.toString} of {logLength}</span>
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 2015c336a3858768999556b6b5f1530a983c8900..911a002884d65db61efb9ebc040ec6324e66ee3e 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -18,13 +18,12 @@
 package org.apache.spark.rdd
 
 import java.io.{ObjectOutputStream, IOException}
-import java.util.{HashMap => JHashMap}
 
-import scala.collection.JavaConversions
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext}
 import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency}
+import org.apache.spark.util.AppendOnlyMap
 
 
 private[spark] sealed trait CoGroupSplitDep extends Serializable
@@ -105,17 +104,14 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
     val split = s.asInstanceOf[CoGroupPartition]
     val numRdds = split.deps.size
     // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs)
-    val map = new JHashMap[K, Seq[ArrayBuffer[Any]]]
+    val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]]
 
-    def getSeq(k: K): Seq[ArrayBuffer[Any]] = {
-      val seq = map.get(k)
-      if (seq != null) {
-        seq
-      } else {
-        val seq = Array.fill(numRdds)(new ArrayBuffer[Any])
-        map.put(k, seq)
-        seq
-      }
+    val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => {
+      if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any])
+    }
+
+    val getSeq = (k: K) => {
+      map.changeValue(k, update)
     }
 
     val ser = SparkEnv.get.serializerManager.get(serializerClass)
@@ -134,7 +130,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
         }
       }
     }
-    new InterruptibleIterator(context, JavaConversions.mapAsScalaMap(map).iterator)
+    new InterruptibleIterator(context, map.iterator)
   }
 
   override def clearDependencies() {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 9c49768c0caaa1211fb08457c356951bb68e6252..cb88159b8d3910bc688af411c8f4acb9b3bf97fa 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -26,7 +26,7 @@ import org.apache.spark.util.Utils
 private[spark] class SparkDeploySchedulerBackend(
     scheduler: ClusterScheduler,
     sc: SparkContext,
-    master: String,
+    masters: Array[String],
     appName: String)
   extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
   with ClientListener
@@ -52,7 +52,7 @@ private[spark] class SparkDeploySchedulerBackend(
     val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
         "http://" + sc.ui.appUIAddress)
 
-    client = new Client(sc.env.actorSystem, master, appDesc, this)
+    client = new Client(sc.env.actorSystem, masters, appDesc, this)
     client.start()
   }
 
@@ -71,8 +71,14 @@ private[spark] class SparkDeploySchedulerBackend(
 
   override def disconnected() {
     if (!stopping) {
-      logError("Disconnected from Spark cluster!")
-      scheduler.error("Disconnected from Spark cluster")
+      logWarning("Disconnected from Spark cluster! Waiting for reconnection...")
+    }
+  }
+
+  override def dead() {
+    if (!stopping) {
+      logError("Spark cluster looks dead, giving up.")
+      scheduler.error("Spark cluster looks down")
     }
   }
 
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index 453394dfda1ba0a5f317d81896fd2663fffa9e59..fcd1b518d099b366e2532f700282fd1c658aee1a 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
-      println("usage: ./spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
+      println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
       System.exit(1)
     }
     val master = args(0)
diff --git a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f60deafc6f32386bd1640fefaee233942936cf76
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+/**
+ * A simple open hash table optimized for the append-only use case, where keys
+ * are never removed, but the value for each key may be changed.
+ *
+ * This implementation uses quadratic probing with a power-of-2 hash table
+ * size, which is guaranteed to explore all spaces for each key (see
+ * http://en.wikipedia.org/wiki/Quadratic_probing).
+ *
+ * TODO: Cache the hash values of each key? java.util.HashMap does that.
+ */
+private[spark]
+class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable {
+  require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements")
+  require(initialCapacity >= 1, "Invalid initial capacity")
+
+  private var capacity = nextPowerOf2(initialCapacity)
+  private var mask = capacity - 1
+  private var curSize = 0
+
+  // Holds keys and values in the same array for memory locality; specifically, the order of
+  // elements is key0, value0, key1, value1, key2, value2, etc.
+  private var data = new Array[AnyRef](2 * capacity)
+
+  // Treat the null key differently so we can use nulls in "data" to represent empty items.
+  private var haveNullValue = false
+  private var nullValue: V = null.asInstanceOf[V]
+
+  private val LOAD_FACTOR = 0.7
+
+  /** Get the value for a given key */
+  def apply(key: K): V = {
+    val k = key.asInstanceOf[AnyRef]
+    if (k.eq(null)) {
+      return nullValue
+    }
+    var pos = rehash(k.hashCode) & mask
+    var i = 1
+    while (true) {
+      val curKey = data(2 * pos)
+      if (k.eq(curKey) || k == curKey) {
+        return data(2 * pos + 1).asInstanceOf[V]
+      } else if (curKey.eq(null)) {
+        return null.asInstanceOf[V]
+      } else {
+        val delta = i
+        pos = (pos + delta) & mask
+        i += 1
+      }
+    }
+    return null.asInstanceOf[V]
+  }
+
+  /** Set the value for a key */
+  def update(key: K, value: V): Unit = {
+    val k = key.asInstanceOf[AnyRef]
+    if (k.eq(null)) {
+      if (!haveNullValue) {
+        incrementSize()
+      }
+      nullValue = value
+      haveNullValue = true
+      return
+    }
+    val isNewEntry = putInto(data, k, value.asInstanceOf[AnyRef])
+    if (isNewEntry) {
+      incrementSize()
+    }
+  }
+
+  /**
+   * Set the value for key to updateFunc(hadValue, oldValue), where oldValue will be the old value
+   * for key, if any, or null otherwise. Returns the newly updated value.
+   */
+  def changeValue(key: K, updateFunc: (Boolean, V) => V): V = {
+    val k = key.asInstanceOf[AnyRef]
+    if (k.eq(null)) {
+      if (!haveNullValue) {
+        incrementSize()
+      }
+      nullValue = updateFunc(haveNullValue, nullValue)
+      haveNullValue = true
+      return nullValue
+    }
+    var pos = rehash(k.hashCode) & mask
+    var i = 1
+    while (true) {
+      val curKey = data(2 * pos)
+      if (k.eq(curKey) || k == curKey) {
+        val newValue = updateFunc(true, data(2 * pos + 1).asInstanceOf[V])
+        data(2 * pos + 1) = newValue.asInstanceOf[AnyRef]
+        return newValue
+      } else if (curKey.eq(null)) {
+        val newValue = updateFunc(false, null.asInstanceOf[V])
+        data(2 * pos) = k
+        data(2 * pos + 1) = newValue.asInstanceOf[AnyRef]
+        incrementSize()
+        return newValue
+      } else {
+        val delta = i
+        pos = (pos + delta) & mask
+        i += 1
+      }
+    }
+    null.asInstanceOf[V] // Never reached but needed to keep compiler happy
+  }
+
+  /** Iterator method from Iterable */
+  override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] {
+    var pos = -1
+
+    /** Get the next value we should return from next(), or null if we're finished iterating */
+    def nextValue(): (K, V) = {
+      if (pos == -1) {    // Treat position -1 as looking at the null value
+        if (haveNullValue) {
+          return (null.asInstanceOf[K], nullValue)
+        }
+        pos += 1
+      }
+      while (pos < capacity) {
+        if (!data(2 * pos).eq(null)) {
+          return (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V])
+        }
+        pos += 1
+      }
+      null
+    }
+
+    override def hasNext: Boolean = nextValue() != null
+
+    override def next(): (K, V) = {
+      val value = nextValue()
+      if (value == null) {
+        throw new NoSuchElementException("End of iterator")
+      }
+      pos += 1
+      value
+    }
+  }
+
+  override def size: Int = curSize
+
+  /** Increase table size by 1, rehashing if necessary */
+  private def incrementSize() {
+    curSize += 1
+    if (curSize > LOAD_FACTOR * capacity) {
+      growTable()
+    }
+  }
+
+  /**
+   * Re-hash a value to deal better with hash functions that don't differ
+   * in the lower bits, similar to java.util.HashMap
+   */
+  private def rehash(h: Int): Int = {
+    val r = h ^ (h >>> 20) ^ (h >>> 12)
+    r ^ (r >>> 7) ^ (r >>> 4)
+  }
+
+  /**
+   * Put an entry into a table represented by data, returning true if
+   * this increases the size of the table or false otherwise. Assumes
+   * that "data" has at least one empty slot.
+   */
+  private def putInto(data: Array[AnyRef], key: AnyRef, value: AnyRef): Boolean = {
+    val mask = (data.length / 2) - 1
+    var pos = rehash(key.hashCode) & mask
+    var i = 1
+    while (true) {
+      val curKey = data(2 * pos)
+      if (curKey.eq(null)) {
+        data(2 * pos) = key
+        data(2 * pos + 1) = value.asInstanceOf[AnyRef]
+        return true
+      } else if (curKey.eq(key) || curKey == key) {
+        data(2 * pos + 1) = value.asInstanceOf[AnyRef]
+        return false
+      } else {
+        val delta = i
+        pos = (pos + delta) & mask
+        i += 1
+      }
+    }
+    return false  // Never reached but needed to keep compiler happy
+  }
+
+  /** Double the table's size and re-hash everything */
+  private def growTable() {
+    val newCapacity = capacity * 2
+    if (newCapacity >= (1 << 30)) {
+      // We can't make the table this big because we want an array of 2x
+      // that size for our data, but array sizes are at most Int.MaxValue
+      throw new Exception("Can't make capacity bigger than 2^29 elements")
+    }
+    val newData = new Array[AnyRef](2 * newCapacity)
+    var pos = 0
+    while (pos < capacity) {
+      if (!data(2 * pos).eq(null)) {
+        putInto(newData, data(2 * pos), data(2 * pos + 1))
+      }
+      pos += 1
+    }
+    data = newData
+    capacity = newCapacity
+    mask = newCapacity - 1
+  }
+
+  private def nextPowerOf2(n: Int): Int = {
+    val highBit = Integer.highestOneBit(n)
+    if (highBit == n) n else highBit << 1
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 05f8545c7b82837695769383c5bcd1ceeb49b4e0..0b38e239f9b024f787c4346d4b9adb076633b190 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -25,7 +25,7 @@ import net.liftweb.json.JsonAST.JValue
 import org.scalatest.FunSuite
 
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
-import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo}
+import org.apache.spark.deploy.master.{ApplicationInfo, RecoveryState, WorkerInfo}
 import org.apache.spark.deploy.worker.ExecutorRunner
 
 class JsonProtocolSuite extends FunSuite {
@@ -53,7 +53,8 @@ class JsonProtocolSuite extends FunSuite {
     val workers = Array[WorkerInfo](createWorkerInfo(), createWorkerInfo())
     val activeApps = Array[ApplicationInfo](createAppInfo())
     val completedApps = Array[ApplicationInfo]()
-    val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps)
+    val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps,
+      RecoveryState.ALIVE)
     val output = JsonProtocol.writeMasterState(stateResponse)
     assertValidJson(output)
   }
@@ -79,7 +80,7 @@ class JsonProtocolSuite extends FunSuite {
   }
   def createExecutorRunner() : ExecutorRunner = {
     new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host",
-      new File("sparkHome"), new File("workDir"))
+      new File("sparkHome"), new File("workDir"), ExecutorState.RUNNING)
   }
 
   def assertValidJson(json: JValue) {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index 0d8742cb81b0bd3e2f9f4cc2f430a3e320bfd7a5..a549417a47a44472ec3838d3349fb0394150194e 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -38,7 +38,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
     }
 
     val d = sc.parallelize(1 to 1e4.toInt, 64).map{i => w(i)}
-    d.count
+    d.count()
     val WAIT_TIMEOUT_MILLIS = 10000
     assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
     listener.stageInfos.size should be (1)
@@ -50,7 +50,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
     val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => w(k) -> (v1.size, v2.size)}
     d4.setName("A Cogroup")
 
-    d4.collectAsMap
+    d4.collectAsMap()
 
     assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
     listener.stageInfos.size should be (4)
diff --git a/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7177919a58157c89400ae2fe88eaa256b08b945e
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import scala.collection.mutable.HashSet
+
+import org.scalatest.FunSuite
+
+class AppendOnlyMapSuite extends FunSuite {
+  test("initialization") {
+    val goodMap1 = new AppendOnlyMap[Int, Int](1)
+    assert(goodMap1.size === 0)
+    val goodMap2 = new AppendOnlyMap[Int, Int](255)
+    assert(goodMap2.size === 0)
+    val goodMap3 = new AppendOnlyMap[Int, Int](256)
+    assert(goodMap3.size === 0)
+    intercept[IllegalArgumentException] {
+      new AppendOnlyMap[Int, Int](1 << 30) // Invalid map size: bigger than 2^29
+    }
+    intercept[IllegalArgumentException] {
+      new AppendOnlyMap[Int, Int](-1)
+    }
+    intercept[IllegalArgumentException] {
+      new AppendOnlyMap[Int, Int](0)
+    }
+  }
+
+  test("object keys and values") {
+    val map = new AppendOnlyMap[String, String]()
+    for (i <- 1 to 100) {
+      map("" + i) = "" + i
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      assert(map("" + i) === "" + i)
+    }
+    assert(map("0") === null)
+    assert(map("101") === null)
+    assert(map(null) === null)
+    val set = new HashSet[(String, String)]
+    for ((k, v) <- map) {   // Test the foreach method
+      set += ((k, v))
+    }
+    assert(set === (1 to 100).map(_.toString).map(x => (x, x)).toSet)
+  }
+
+  test("primitive keys and values") {
+    val map = new AppendOnlyMap[Int, Int]()
+    for (i <- 1 to 100) {
+      map(i) = i
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      assert(map(i) === i)
+    }
+    assert(map(0) === null)
+    assert(map(101) === null)
+    val set = new HashSet[(Int, Int)]
+    for ((k, v) <- map) {   // Test the foreach method
+      set += ((k, v))
+    }
+    assert(set === (1 to 100).map(x => (x, x)).toSet)
+  }
+
+  test("null keys") {
+    val map = new AppendOnlyMap[String, String]()
+    for (i <- 1 to 100) {
+      map("" + i) = "" + i
+    }
+    assert(map.size === 100)
+    assert(map(null) === null)
+    map(null) = "hello"
+    assert(map.size === 101)
+    assert(map(null) === "hello")
+  }
+
+  test("null values") {
+    val map = new AppendOnlyMap[String, String]()
+    for (i <- 1 to 100) {
+      map("" + i) = null
+    }
+    assert(map.size === 100)
+    assert(map("1") === null)
+    assert(map(null) === null)
+    assert(map.size === 100)
+    map(null) = null
+    assert(map.size === 101)
+    assert(map(null) === null)
+  }
+
+  test("changeValue") {
+    val map = new AppendOnlyMap[String, String]()
+    for (i <- 1 to 100) {
+      map("" + i) = "" + i
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      val res = map.changeValue("" + i, (hadValue, oldValue) => {
+        assert(hadValue === true)
+        assert(oldValue === "" + i)
+        oldValue + "!"
+      })
+      assert(res === i + "!")
+    }
+    // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a
+    // bug where changeValue would return the wrong result when the map grew on that insert
+    for (i <- 101 to 400) {
+      val res = map.changeValue("" + i, (hadValue, oldValue) => {
+        assert(hadValue === false)
+        i + "!"
+      })
+      assert(res === i + "!")
+    }
+    assert(map.size === 400)
+    assert(map(null) === null)
+    map.changeValue(null, (hadValue, oldValue) => {
+      assert(hadValue === false)
+      "null!"
+    })
+    assert(map.size === 401)
+    map.changeValue(null, (hadValue, oldValue) => {
+      assert(hadValue === true)
+      assert(oldValue === "null!")
+      "null!!"
+    })
+    assert(map.size === 401)
+  }
+
+  test("inserting in capacity-1 map") {
+    val map = new AppendOnlyMap[String, String](1)
+    for (i <- 1 to 100) {
+      map("" + i) = "" + i
+    }
+    assert(map.size === 100)
+    for (i <- 1 to 100) {
+      assert(map("" + i) === "" + i)
+    }
+  }
+}
diff --git a/docker/README.md b/docker/README.md
new file mode 100644
index 0000000000000000000000000000000000000000..bf59e77d111f9c121ea64b949dba3a557a38622d
--- /dev/null
+++ b/docker/README.md
@@ -0,0 +1,5 @@
+Spark docker files
+===========
+
+Drawn from Matt Massie's docker files (https://github.com/massie/dockerfiles),
+as well as some updates from Andre Schumacher (https://github.com/AndreSchumacher/docker).
\ No newline at end of file
diff --git a/docker/build b/docker/build
new file mode 100755
index 0000000000000000000000000000000000000000..253a2fc8dd8e7a0a9dce47930874f000cfa18293
--- /dev/null
+++ b/docker/build
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+docker images > /dev/null || { echo Please install docker in non-sudo mode. ; exit; }
+
+./spark-test/build
\ No newline at end of file
diff --git a/docker/spark-test/README.md b/docker/spark-test/README.md
new file mode 100644
index 0000000000000000000000000000000000000000..addea277aa6c41015f37e585c89bffc35009e104
--- /dev/null
+++ b/docker/spark-test/README.md
@@ -0,0 +1,10 @@
+Spark Docker files usable for testing and development purposes.
+
+These images are intended to be run like so:
+docker run -v $SPARK_HOME:/opt/spark spark-test-master
+docker run -v $SPARK_HOME:/opt/spark spark-test-worker <master_ip>
+
+Using this configuration, the containers will have their Spark directories
+mounted to your actual SPARK_HOME, allowing you to modify and recompile
+your Spark source and have them immediately usable in the docker images
+(without rebuilding them).
diff --git a/docker/spark-test/base/Dockerfile b/docker/spark-test/base/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..60962776dda57d6b6432eaba24b9a82ae5daca3f
--- /dev/null
+++ b/docker/spark-test/base/Dockerfile
@@ -0,0 +1,38 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM ubuntu:precise
+
+RUN echo "deb http://archive.ubuntu.com/ubuntu precise main universe" > /etc/apt/sources.list
+
+# Upgrade package index
+RUN apt-get update
+
+# install a few other useful packages plus Open Jdk 7
+RUN apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server
+
+ENV SCALA_VERSION 2.9.3
+ENV SPARK_VERSION 0.8.1
+ENV CDH_VERSION cdh4
+ENV SCALA_HOME /opt/scala-$SCALA_VERSION
+ENV SPARK_HOME /opt/spark
+ENV PATH $SPARK_HOME:$SCALA_HOME/bin:$PATH
+
+# Install Scala
+ADD http://www.scala-lang.org/files/archive/scala-$SCALA_VERSION.tgz /
+RUN (cd / && gunzip < scala-$SCALA_VERSION.tgz)|(cd /opt && tar -xvf -)
+RUN rm /scala-$SCALA_VERSION.tgz
diff --git a/docker/spark-test/build b/docker/spark-test/build
new file mode 100755
index 0000000000000000000000000000000000000000..6f9e19743370b85c4dbeadf89ded369a1a3b75f4
--- /dev/null
+++ b/docker/spark-test/build
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+docker build -t spark-test-base spark-test/base/
+docker build -t spark-test-master spark-test/master/
+docker build -t spark-test-worker spark-test/worker/
diff --git a/docker/spark-test/master/Dockerfile b/docker/spark-test/master/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..f729534ab63ed429fd6aba90739645974d56170a
--- /dev/null
+++ b/docker/spark-test/master/Dockerfile
@@ -0,0 +1,21 @@
+# Spark Master
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-test-base
+ADD default_cmd /root/
+CMD ["/root/default_cmd"]
diff --git a/docker/spark-test/master/default_cmd b/docker/spark-test/master/default_cmd
new file mode 100755
index 0000000000000000000000000000000000000000..a5b1303c2ebdb957d3f2249fd13a13f537582599
--- /dev/null
+++ b/docker/spark-test/master/default_cmd
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+IP=$(ip -o -4 addr list eth0 | perl -n -e 'if (m{inet\s([\d\.]+)\/\d+\s}xms) { print $1 }')
+echo "CONTAINER_IP=$IP"
+/opt/spark/spark-class org.apache.spark.deploy.master.Master -i $IP
diff --git a/docker/spark-test/worker/Dockerfile b/docker/spark-test/worker/Dockerfile
new file mode 100644
index 0000000000000000000000000000000000000000..890febe7b68e29a55639ae5a8d21d283cd5d33a7
--- /dev/null
+++ b/docker/spark-test/worker/Dockerfile
@@ -0,0 +1,22 @@
+# Spark Worker
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-test-base
+ENV SPARK_WORKER_PORT 8888
+ADD default_cmd /root/
+ENTRYPOINT ["/root/default_cmd"]
diff --git a/docker/spark-test/worker/default_cmd b/docker/spark-test/worker/default_cmd
new file mode 100755
index 0000000000000000000000000000000000000000..ab6336f70c1c6a3913a8f217d0619514ac287752
--- /dev/null
+++ b/docker/spark-test/worker/default_cmd
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+IP=$(ip -o -4 addr list eth0 | perl -n -e 'if (m{inet\s([\d\.]+)\/\d+\s}xms) { print $1 }')
+echo "CONTAINER_IP=$IP"
+/opt/spark/spark-class org.apache.spark.deploy.worker.Worker $1
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 30128ec45db04d39e3537303f8bfc39d0e50c5cd..2898af0bed8c0d6d6de9be6a2c10472994e2a222 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -34,6 +34,8 @@ Environment variables:
 
 System Properties:
 * 'spark.yarn.applicationMaster.waitTries', property to set the number of times the ApplicationMaster waits for the the spark master and then also the number of tries it waits for the Spark Context to be intialized. Default is 10.
+* 'spark.yarn.submit.file.replication', the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives.
+* 'spark.yarn.preserve.staging.files', set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them.
 
 # Launching Spark on YARN
 
@@ -51,7 +53,10 @@ The command to launch the YARN Client is as follows:
       --worker-memory <MEMORY_PER_WORKER> \
       --worker-cores <CORES_PER_WORKER> \
       --name <application_name> \
-      --queue <queue_name>
+      --queue <queue_name> \
+      --addJars <any_local_files_used_in_SparkContext.addJar> \
+      --files <files_for_distributed_cache> \
+      --archives <archives_for_distributed_cache>
 
 For example:
 
@@ -84,3 +89,5 @@ The above starts a YARN Client programs which periodically polls the Application
 - When your application instantiates a Spark context it must use a special "yarn-standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "yarn-standalone" as an argument to your program, as shown in the example above.
 - We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed.
 - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored.
+- The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN.
+- The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files.
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 81cdbefd0c7ae83a3de3369da54b47bb3d9591cd..17066ef0dde5e884b91f5b88c18a3f3150fee593 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -3,6 +3,9 @@ layout: global
 title: Spark Standalone Mode
 ---
 
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
 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 [launch scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing.
 
 # Installing Spark Standalone to a Cluster
@@ -169,3 +172,75 @@ 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).
 
+
+# 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 applications can be created. In order to circumvent this, we have two high availability schemes, detailed below.
+
+## 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).
+
+**Configuration**
+
+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**
+
+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``.
+
+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.
+
+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
+
+**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**
+
+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/applications (equivalent to ZooKeeper recovery). Future applications will have to be able to find the new Master, however, in order to register.
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index c7df1720249481eed6a4b4268ac9c8796d0587b6..835b257238e4bf04d2e0024a37f93ffaec13f0b5 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -122,12 +122,12 @@ Spark Streaming features windowed computations, which allow you to apply transfo
 <table class="table">
 <tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
 <tr>
-  <td> <b>window</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
+  <td> <b>window</b>(<i>windowDuration</i>, <i>slideDuration</i>) </td>
   <td> Return a new DStream which is computed based on windowed batches of the source DStream. <i>windowDuration</i> is the width of the window and <i>slideTime</i> is the frequency during which the window is calculated. Both times must be multiples of the batch interval.
   </td>
 </tr>
 <tr>
-  <td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
+  <td> <b>countByWindow</b>(<i>windowDuration</i>, <i>slideDuration</i>) </td>
   <td> Return a sliding count of elements in the stream. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
   </td>
 </tr>
@@ -161,7 +161,6 @@ Spark Streaming features windowed computations, which allow you to apply transfo
  <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
 </td>
 </tr>
-
 </table>
 
 A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#org.apache.spark.streaming.DStream) and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.PairDStreamFunctions).
diff --git a/docs/tuning.md b/docs/tuning.md
index 28d88a26592725ff750d09b2c924fe3dd72a8ce2..f491ae9b95c08b757bbd3630575c5eee1a89e40e 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -175,7 +175,7 @@ To further tune garbage collection, we first need to understand some basic infor
 * Java Heap space is divided in to two regions Young and Old. The Young generation is meant to hold short-lived objects
   while the Old generation is intended for objects with longer lifetimes.
 
-* The Young generation is further divided into three regions [Eden, Survivor1, Survivor2].
+* The Young generation is further divided into three regions \[Eden, Survivor1, Survivor2\].
 
 * A simplified description of the garbage collection procedure: When Eden is full, a minor GC is run on Eden and objects
   that are alive from Eden and Survivor1 are copied to Survivor2. The Survivor regions are swapped. If an object is old
diff --git a/pom.xml b/pom.xml
index 48bf38d2005918f3310824b667f55086d6429a62..18df1bf8269e43c3ca88e81dee237f9fb0ec3012 100644
--- a/pom.xml
+++ b/pom.xml
@@ -345,6 +345,17 @@
         <version>0.9</version>
         <scope>test</scope>
       </dependency>
+      <dependency>
+        <groupId>org.apache.zookeeper</groupId>
+        <artifactId>zookeeper</artifactId>
+        <version>3.4.5</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-client</artifactId>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index eb4b96eb472efe14c4602dcdabd7a92ae7e8f85e..973f1e2f11d8d40d9e8f3bb8936124d5e5b1639f 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -216,6 +216,7 @@ object SparkBuild extends Build {
       "net.java.dev.jets3t" % "jets3t" % "0.7.1",
       "org.apache.avro" % "avro" % "1.7.4",
       "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty),
+      "org.apache.zookeeper" % "zookeeper" % "3.4.5" excludeAll(excludeNetty),
       "com.codahale.metrics" % "metrics-core" % "3.0.0",
       "com.codahale.metrics" % "metrics-jvm" % "3.0.0",
       "com.codahale.metrics" % "metrics-json" % "3.0.0",
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 858b58d338d71491b21d6098f84be8d34d0491f5..c1a87d33738bede69e4d85cfac13075351898ece 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -17,22 +17,25 @@
 
 package org.apache.spark.deploy.yarn
 
+import java.io.IOException;
 import java.net.Socket
+import java.security.PrivilegedExceptionAction
 import java.util.concurrent.CopyOnWriteArrayList
 import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.util.ShutdownHookManager
 import org.apache.hadoop.yarn.api._
 import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.api.protocolrecords._
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
-import scala.collection.JavaConversions._
 import org.apache.spark.{SparkContext, Logging}
 import org.apache.spark.util.Utils
 import org.apache.hadoop.security.UserGroupInformation
-import java.security.PrivilegedExceptionAction
+import scala.collection.JavaConversions._
 
 class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
 
@@ -43,18 +46,26 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   private var appAttemptId: ApplicationAttemptId = null
   private var userThread: Thread = null
   private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+  private val fs = FileSystem.get(yarnConf)
 
   private var yarnAllocator: YarnAllocationHandler = null
   private var isFinished:Boolean = false
   private var uiAddress: String = ""
+  private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
+    YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES)
+  private var isLastAMRetry: Boolean = true
 
 
   def run() {
     // setup the directories so things go to yarn approved directories rather
     // then user specified and /tmp
     System.setProperty("spark.local.dir", getLocalDirs())
+
+    // use priority 30 as its higher then HDFS. Its same priority as MapReduce is using
+    ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
     
     appAttemptId = getApplicationAttemptId()
+    isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts;
     resourceManager = registerWithResourceManager()
 
     // Workaround until hadoop moves to something which has
@@ -183,6 +194,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
           // It need shutdown hook to set SUCCEEDED
           successed = true
         } finally {
+          logDebug("finishing main")
+          isLastAMRetry = true;
           if (successed) {
             ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           } else {
@@ -229,8 +242,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     }
   }
 
-
-
   private def allocateWorkers() {
     try {
       logInfo("Allocating " + args.numWorkers + " workers.")
@@ -329,6 +340,40 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     resourceManager.finishApplicationMaster(finishReq)
 
   }
+
+  /**
+   * clean up the staging directory. 
+   */
+  private def cleanupStagingDir() { 
+    var stagingDirPath: Path = null
+    try {
+      val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean
+      if (!preserveFiles) {
+        stagingDirPath = new Path(System.getenv("SPARK_YARN_JAR_PATH")).getParent()
+        if (stagingDirPath == null) {
+          logError("Staging directory is null")
+          return
+        }
+        logInfo("Deleting staging directory " + stagingDirPath)
+        fs.delete(stagingDirPath, true)
+      }
+    } catch {
+      case e: IOException =>
+        logError("Failed to cleanup staging dir " + stagingDirPath, e)
+    }
+  }
+
+  // The shutdown hook that runs when a signal is received AND during normal
+  // close of the JVM. 
+  class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
+
+    def run() {
+      logInfo("AppMaster received a signal.")
+      // we need to clean up staging dir before HDFS is shut down
+      // make sure we don't delete it until this is the last AM
+      if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
+    }
+  }
  
 }
 
@@ -368,6 +413,8 @@ object ApplicationMaster {
     // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do System.exit
     // Should not really have to do this, but it helps yarn to evict resources earlier.
     // not to mention, prevent Client declaring failure even though we exit'ed properly.
+    // Note that this will unfortunately not properly clean up the staging files because it gets called to 
+    // late and the filesystem is already shutdown.
     if (modified) {
       Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
         // This is not just to log, but also to ensure that log system is initialized for this instance when we actually are 'run'
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 076dd3c9b023695e3d22ff9daa890e3c843accd0..8afb3e39cb8a6a0e7df4c4e41e03083867601b71 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -45,7 +45,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   
   var rpc: YarnRPC = YarnRPC.create(conf)
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
-  val credentials = UserGroupInformation.getCurrentUser().getCredentials();
+  val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+  private var distFiles = None: Option[String]
+  private var distFilesTimeStamps = None: Option[String]
+  private var distFilesFileSizes = None: Option[String]
+  private var distArchives = None: Option[String]
+  private var distArchivesTimeStamps = None: Option[String]
+  private var distArchivesFileSizes = None: Option[String]
   
   def run() {
     init(yarnConf)
@@ -57,7 +63,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     verifyClusterResources(newApp)
     val appContext = createApplicationSubmissionContext(appId)
-    val localResources = prepareLocalResources(appId, "spark")
+    val localResources = prepareLocalResources(appId, ".sparkStaging")
     val env = setupLaunchEnv(localResources)
     val amContainer = createContainerLaunchContext(newApp, localResources, env)
 
@@ -109,10 +115,73 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     appContext.setApplicationName(args.appName)
     return appContext
   }
-  
-  def prepareLocalResources(appId: ApplicationId, appName: String): HashMap[String, LocalResource] = {
+
+  /**
+   * Copy the local file into HDFS and configure to be distributed with the
+   * job via the distributed cache.
+   * If a fragment is specified the file will be referenced as that fragment.
+   */
+  private def copyLocalFile(
+      dstDir: Path,
+      resourceType: LocalResourceType,
+      originalPath: Path,
+      replication: Short,
+      localResources: HashMap[String,LocalResource],
+      fragment: String,
+      appMasterOnly: Boolean = false): Unit = {
+    val fs = FileSystem.get(conf)
+    val newPath = new Path(dstDir, originalPath.getName())
+    logInfo("Uploading " + originalPath + " to " + newPath)
+    fs.copyFromLocalFile(false, true, originalPath, newPath)
+    fs.setReplication(newPath, replication);
+    val destStatus = fs.getFileStatus(newPath)
+
+    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+    amJarRsrc.setType(resourceType)
+    amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION)
+    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(newPath))
+    amJarRsrc.setTimestamp(destStatus.getModificationTime())
+    amJarRsrc.setSize(destStatus.getLen())
+    var pathURI: URI = new URI(newPath.toString() + "#" + originalPath.getName());
+    if ((fragment == null) || (fragment.isEmpty())){
+      localResources(originalPath.getName()) = amJarRsrc
+    } else {
+      localResources(fragment) = amJarRsrc
+      pathURI = new URI(newPath.toString() + "#" + fragment);
+    }
+    val distPath = pathURI.toString()
+    if (appMasterOnly == true) return
+    if (resourceType == LocalResourceType.FILE) {
+      distFiles match {
+        case Some(path) =>
+          distFilesFileSizes = Some(distFilesFileSizes.get + "," + 
+            destStatus.getLen().toString())
+          distFilesTimeStamps = Some(distFilesTimeStamps.get + "," + 
+            destStatus.getModificationTime().toString())
+          distFiles = Some(path + "," + distPath)
+        case _ => 
+          distFilesFileSizes = Some(destStatus.getLen().toString())
+          distFilesTimeStamps = Some(destStatus.getModificationTime().toString())
+          distFiles = Some(distPath)
+      }
+    } else {
+      distArchives match {
+        case Some(path) =>
+          distArchivesTimeStamps = Some(distArchivesTimeStamps.get + "," +
+            destStatus.getModificationTime().toString())
+          distArchivesFileSizes = Some(distArchivesFileSizes.get + "," + 
+            destStatus.getLen().toString())
+          distArchives = Some(path + "," + distPath)
+        case _ => 
+          distArchivesTimeStamps = Some(destStatus.getModificationTime().toString())
+          distArchivesFileSizes = Some(destStatus.getLen().toString())
+          distArchives = Some(distPath)
+      }
+    }
+  }
+
+  def prepareLocalResources(appId: ApplicationId, sparkStagingDir: String): HashMap[String, LocalResource] = {
     logInfo("Preparing Local resources")
-    val locaResources = HashMap[String, LocalResource]()
     // Upload Spark and the application JAR to the remote file system
     // Add them as local resources to the AM
     val fs = FileSystem.get(conf)
@@ -125,33 +194,69 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       }
     }
 
+    val pathSuffix = sparkStagingDir + "/" + appId.toString() + "/"
+    val dst = new Path(fs.getHomeDirectory(), pathSuffix)
+    val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      val dstFs = dst.getFileSystem(conf)
+      dstFs.addDelegationTokens(delegTokenRenewer, credentials);
+    }
+    val localResources = HashMap[String, LocalResource]()
+
     Map("spark.jar" -> System.getenv("SPARK_JAR"), "app.jar" -> args.userJar, "log4j.properties" -> System.getenv("SPARK_LOG4J_CONF"))
     .foreach { case(destName, _localPath) =>
       val localPath: String = if (_localPath != null) _localPath.trim() else ""
       if (! localPath.isEmpty()) {
         val src = new Path(localPath)
-        val pathSuffix = appName + "/" + appId.getId() + destName
-        val dst = new Path(fs.getHomeDirectory(), pathSuffix)
-        logInfo("Uploading " + src + " to " + dst)
-        fs.copyFromLocalFile(false, true, src, dst)
-        val destStatus = fs.getFileStatus(dst)
-
-        // get tokens for anything we upload to hdfs
-        if (UserGroupInformation.isSecurityEnabled()) {
-          fs.addDelegationTokens(delegTokenRenewer, credentials);
-        }
+        val newPath = new Path(dst, destName)
+        logInfo("Uploading " + src + " to " + newPath)
+        fs.copyFromLocalFile(false, true, src, newPath)
+        fs.setReplication(newPath, replication);
+        val destStatus = fs.getFileStatus(newPath)
 
         val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
         amJarRsrc.setType(LocalResourceType.FILE)
         amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION)
-        amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(dst))
+        amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(newPath))
         amJarRsrc.setTimestamp(destStatus.getModificationTime())
         amJarRsrc.setSize(destStatus.getLen())
-        locaResources(destName) = amJarRsrc
+        localResources(destName) = amJarRsrc
       }
     }
+
+    // handle any add jars
+    if ((args.addJars != null) && (!args.addJars.isEmpty())){
+      args.addJars.split(',').foreach { case file: String =>
+        val tmpURI = new URI(file)
+        val tmp = new Path(tmpURI)
+        copyLocalFile(dst, LocalResourceType.FILE, tmp, replication, localResources,
+          tmpURI.getFragment(), true)
+      }
+    }
+
+    // handle any distributed cache files
+    if ((args.files != null) && (!args.files.isEmpty())){
+      args.files.split(',').foreach { case file: String =>
+        val tmpURI = new URI(file)
+        val tmp = new Path(tmpURI)
+        copyLocalFile(dst, LocalResourceType.FILE, tmp, replication, localResources,
+          tmpURI.getFragment())
+      }
+    }
+
+    // handle any distributed cache archives
+    if ((args.archives != null) && (!args.archives.isEmpty())) {
+      args.archives.split(',').foreach { case file:String =>
+        val tmpURI = new URI(file)
+        val tmp = new Path(tmpURI)
+        copyLocalFile(dst, LocalResourceType.ARCHIVE, tmp, replication, 
+          localResources, tmpURI.getFragment())
+      }
+    }
+
     UserGroupInformation.getCurrentUser().addCredentials(credentials);
-    return locaResources
+    return localResources
   }
   
   def setupLaunchEnv(localResources: HashMap[String, LocalResource]): HashMap[String, String] = {
@@ -160,11 +265,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val env = new HashMap[String, String]()
 
-    // If log4j present, ensure ours overrides all others
-    if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./")
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name,
+      Environment.PWD.$() + Path.SEPARATOR + "*")
 
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*")
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH")
     Client.populateHadoopClasspath(yarnConf, env)
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_JAR_PATH") = 
@@ -186,6 +290,18 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       env("SPARK_YARN_LOG4J_SIZE") =  log4jConfLocalRes.getSize().toString()
     }
 
+    // set the environment variables to be passed on to the Workers
+    if (distFiles != None) {
+      env("SPARK_YARN_CACHE_FILES") = distFiles.get
+      env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = distFilesTimeStamps.get
+      env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = distFilesFileSizes.get
+    }
+    if (distArchives != None) {
+      env("SPARK_YARN_CACHE_ARCHIVES") = distArchives.get
+      env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = distArchivesTimeStamps.get
+      env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") = distArchivesFileSizes.get
+    }
+
     // allow users to specify some environment variables
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
 
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index c56dbd99ba4d9767151843159d5d2ca3e273ac3b..852dbd7dabf66e391c4e64584670cdf0a74a853f 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -24,6 +24,9 @@ import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
 
 // TODO: Add code and support for ensuring that yarn resource 'asks' are location aware !
 class ClientArguments(val args: Array[String]) {
+  var addJars: String = null
+  var files: String = null
+  var archives: String = null
   var userJar: String = null
   var userClass: String = null
   var userArgs: Seq[String] = Seq[String]()
@@ -81,6 +84,17 @@ class ClientArguments(val args: Array[String]) {
 
         case ("--name") :: value :: tail =>
           appName = value
+
+        case ("--addJars") :: value :: tail =>
+          addJars = value
+          args = tail
+
+        case ("--files") :: value :: tail =>
+          files = value
+          args = tail
+
+        case ("--archives") :: value :: tail =>
+          archives = value
           args = tail
 
         case Nil =>
@@ -97,7 +111,7 @@ class ClientArguments(val args: Array[String]) {
     inputFormatInfo = inputFormatMap.values.toList
   }
 
-  
+
   def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
     if (unknownParam != null) {
       System.err.println("Unknown/unsupported param " + unknownParam)
@@ -113,10 +127,13 @@ class ClientArguments(val args: Array[String]) {
       "  --worker-cores NUM   Number of cores for the workers (Default: 1). This is unsused right now.\n" +
       "  --master-memory MEM  Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
       "  --worker-memory MEM  Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" +
-      "  --name NAME          The name of your application (Default: Spark)\n" + 
-      "  --queue QUEUE        The hadoop queue to use for allocation requests (Default: 'default')"
+      "  --name NAME          The name of your application (Default: Spark)\n" +
+      "  --queue QUEUE        The hadoop queue to use for allocation requests (Default: 'default')\n" +
+      "  --addJars jars       Comma separated list of local jars that want SparkContext.addJar to work with.\n" +
+      "  --files files        Comma separated list of files to be distributed with the job.\n" +
+      "  --archives archives  Comma separated list of archives to be distributed with the job."
       )
     System.exit(exitCode)
   }
-  
+
 }
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index a60e8a300786e31df2d7408d34a833b1deb54f28..8dac9e02ac0b712fcf712c60575b43b3d2729f9c 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -137,11 +137,26 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     startReq.setContainerLaunchContext(ctx)
     cm.startContainer(startReq)
   }
+
+  private def setupDistributedCache(file: String,
+      rtype: LocalResourceType,
+      localResources: HashMap[String, LocalResource],
+      timestamp: String,
+      size: String) = {
+    val uri = new URI(file)
+    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+    amJarRsrc.setType(rtype)
+    amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION)
+    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
+    amJarRsrc.setTimestamp(timestamp.toLong)
+    amJarRsrc.setSize(size.toLong)
+    localResources(uri.getFragment()) = amJarRsrc
+  }
   
   
   def prepareLocalResources: HashMap[String, LocalResource] = {
     logInfo("Preparing Local resources")
-    val locaResources = HashMap[String, LocalResource]()
+    val localResources = HashMap[String, LocalResource]()
     
     // Spark JAR
     val sparkJarResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
@@ -151,7 +166,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
       new URI(System.getenv("SPARK_YARN_JAR_PATH"))))
     sparkJarResource.setTimestamp(System.getenv("SPARK_YARN_JAR_TIMESTAMP").toLong)
     sparkJarResource.setSize(System.getenv("SPARK_YARN_JAR_SIZE").toLong)
-    locaResources("spark.jar") = sparkJarResource
+    localResources("spark.jar") = sparkJarResource
     // User JAR
     val userJarResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
     userJarResource.setType(LocalResourceType.FILE)
@@ -160,7 +175,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
       new URI(System.getenv("SPARK_YARN_USERJAR_PATH"))))
     userJarResource.setTimestamp(System.getenv("SPARK_YARN_USERJAR_TIMESTAMP").toLong)
     userJarResource.setSize(System.getenv("SPARK_YARN_USERJAR_SIZE").toLong)
-    locaResources("app.jar") = userJarResource
+    localResources("app.jar") = userJarResource
 
     // Log4j conf - if available
     if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) {
@@ -171,26 +186,39 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
         new URI(System.getenv("SPARK_YARN_LOG4J_PATH"))))
       log4jConfResource.setTimestamp(System.getenv("SPARK_YARN_LOG4J_TIMESTAMP").toLong)
       log4jConfResource.setSize(System.getenv("SPARK_YARN_LOG4J_SIZE").toLong)
-      locaResources("log4j.properties") = log4jConfResource
+      localResources("log4j.properties") = log4jConfResource
+    }
+
+    if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
+      val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
+      val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
+      val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',')
+      for( i <- 0 to distFiles.length - 1) {
+        setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
+          fileSizes(i))
+      }
     }
 
+    if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) {
+      val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',')
+      val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',')
+      val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',')
+      for( i <- 0 to distArchives.length - 1) {
+        setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, 
+          timeStamps(i), fileSizes(i))
+      }
+    }
     
-    logInfo("Prepared Local resources " + locaResources)
-    return locaResources
+    logInfo("Prepared Local resources " + localResources)
+    return localResources
   }
   
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
-    // If log4j present, ensure ours overrides all others
-    if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) {
-      // Which is correct ?
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./log4j.properties")
-      Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./")
-    }
-
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*")
-    Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH")
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, 
+      Environment.PWD.$() + Path.SEPARATOR + "*")
     Client.populateHadoopClasspath(yarnConf, env)
 
     // allow users to specify some environment variables