From d5a96feccb15dd290b282af9e2f94479c8e4554e Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 17 Sep 2013 09:40:06 -0700 Subject: [PATCH 01/12] Standalone Scheduler fault recovery Implements a basic form of Standalone Scheduler fault recovery. In particular, this allows faults to be manually recovered from by means of restarting the Master process on the same machine. This is the majority of the code necessary for general fault tolerance, which will first elect a leader and then recover the Master state. In order to enable fault recovery, the Master will persist a small amount of state related to the registration of Workers and Applications to disk. If the Master is started and sees that this state is still around, it will enter Recovery mode, during which time it will not schedule any new Executors on Workers (but it does accept the registration of new Clients and Workers). At this point, the Master attempts to reconnect to all Workers and Client applications that were registered at the time of failure. After confirming either the existence or nonexistence of all such nodes (within a certain timeout), the Master will exit Recovery mode and resume normal scheduling. --- bin/stop-slaves.sh | 2 +- .../apache/spark/deploy/DeployMessage.scala | 12 ++ .../spark/deploy/ExecutorDescription.scala | 29 ++++ .../apache/spark/deploy/client/Client.scala | 17 +- .../spark/deploy/master/ApplicationInfo.scala | 65 +++++--- .../deploy/master/ApplicationState.scala | 4 +- .../spark/deploy/master/ExecutorInfo.scala | 7 +- .../master/FileSystemPersistenceEngine.scala | 90 ++++++++++ .../apache/spark/deploy/master/Master.scala | 154 +++++++++++++++--- .../spark/deploy/master/MasterState.scala | 26 +++ .../deploy/master/PersistenceEngine.scala | 50 ++++++ .../spark/deploy/master/WorkerInfo.scala | 28 +++- .../spark/deploy/master/WorkerState.scala | 6 +- .../spark/deploy/worker/ExecutorRunner.scala | 13 +- .../apache/spark/deploy/worker/Worker.scala | 29 ++-- .../cluster/SparkDeploySchedulerBackend.scala | 3 +- 16 files changed, 459 insertions(+), 76 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh index 03e416a13..abf1c7be6 100755 --- a/bin/stop-slaves.sh +++ b/bin/stop-slaves.sh @@ -17,7 +17,7 @@ # limitations under the License. # -# Starts the master on the machine this script is executed on. +# Starts workers 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/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 1cfff5e56..0d0745a48 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -52,6 +52,8 @@ 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 @@ -76,6 +78,8 @@ 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 @@ -94,6 +98,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 @@ -127,6 +135,10 @@ private[deploy] object DeployMessages { case object CheckForWorkerTimeOut + case class BeginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) + + case object EndRecoveryProcess + case object RequestWebUIPort case class WebUIPortResponse(webUIBoundPort: Int) 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 000000000..716ee483d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala @@ -0,0 +1,29 @@ +/* + * 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 + +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/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index a342dd724..28548a2ca 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 @@ -92,20 +92,25 @@ private[spark] class Client( listener.executorRemoved(fullId, message.getOrElse(""), exitStatus) } + case MasterChanged(materUrl, masterWebUiUrl) => + logInfo("Master has changed, new master is at " + masterUrl) + context.unwatch(master) + master = context.actorFor(Master.toAkkaUrl(masterUrl)) + masterAddress = master.path.address + sender ! MasterChangeAcknowledged(appId) + context.watch(master) + case Terminated(actor_) if actor_ == master => - logError("Connection to master failed; stopping client") + logError("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") + logError("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") + logError("Connection to master failed; waiting for master to reconnect...") markDisconnected() - context.stop(self) case StopClient => markDisconnected() 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 bd5327627..e437a0e7a 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 @@ -23,29 +23,52 @@ import akka.actor.ActorRef import scala.collection.mutable private[spark] class ApplicationInfo( - val startTime: Long, - val id: String, - val desc: ApplicationDescription, - 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 + val startTime: Long, + val id: String, + val desc: ApplicationDescription, + val submitDate: Date, + val driver: ActorRef, + val appUiUrl: String) + 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 7e804223c..fedf879ef 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 cf384a985..d235234c1 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 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 000000000..2fc13821b --- /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/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index bde59905b..c6e039eed 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,16 +27,17 @@ 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.MasterState.MasterState 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 { @@ -44,7 +45,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act 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", "") + var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] val idToWorker = new HashMap[String, WorkerInfo] @@ -74,6 +76,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (envVar != null) envVar else host } + var state: MasterState = _ + + var persistenceEngine: PersistenceEngine = _ + // 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. @@ -89,6 +95,23 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() applicationMetricsSystem.start() + + persistenceEngine = + if (RECOVERY_DIR.isEmpty()) { + new BlackHolePersistenceEngine() + } else { + logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) + new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system)) + } + + val (storedApps, storedWorkers) = persistenceEngine.readPersistedData() + state = + if (storedApps.isEmpty && storedWorkers.isEmpty) { + MasterState.ALIVE + } else { + self ! BeginRecovery(storedApps, storedWorkers) + MasterState.RECOVERING + } } override def postStop() { @@ -98,14 +121,16 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } override def receive = { - case RegisterWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) => { + 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)) { 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 + persistenceEngine.addWorker(worker) sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort.get) schedule() } @@ -113,10 +138,11 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case RegisterApplication(description) => { logInfo("Registering app " + description.name) - val app = addApplication(description, sender) + val app = createApplication(description, sender) + registerApplication(app) 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 + persistenceEngine.addApplication(app) sender ! RegisteredApplication(app.id) schedule() } @@ -158,23 +184,78 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } } + case BeginRecovery(storedApps, storedWorkers) => { + context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis, self, EndRecoveryProcess) + + val masterUrl = "spark://" + host + ":" + port + val masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get + for (app <- storedApps) { + registerApplication(app) + app.state = ApplicationState.UNKNOWN + app.driver ! MasterChanged(masterUrl, masterWebUiUrl) + } + for (worker <- storedWorkers) { + registerWorker(worker) + worker.state = WorkerState.UNKNOWN + worker.actor ! MasterChanged(masterUrl, masterWebUiUrl) + } + } + + case MasterChangeAcknowledged(appId) => { + val appOption = idToApp.get(appId) + appOption match { + case Some(app) => + 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) => + worker.state = WorkerState.ALIVE + + val validExecutors = executors.filter(exec => idToApp.get(exec.appId) != None) + 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 EndRecoveryProcess => { + 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 == MasterState.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 == MasterState.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 == MasterState.RECOVERING && canCompleteRecovery) { completeRecovery() } } case RequestMasterState => { @@ -190,6 +271,25 @@ 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 completeRecovery() { + synchronized { + if (state != MasterState.RECOVERING) { return } + state = MasterState.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 = MasterState.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 +304,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 != MasterState.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) { @@ -257,8 +358,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act 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 +366,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 + addressToWorker(workerAddress) = worker } def removeWorker(worker: WorkerInfo) { @@ -286,25 +391,35 @@ 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) } 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) { @@ -336,6 +451,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act 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/MasterState.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala new file mode 100644 index 000000000..9ea5e9752 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterState.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 MasterState + extends Enumeration("ALIVE", "RECOVERING", "COMPLETING_RECOVERY") { + + type MasterState = Value + + val ALIVE, RECOVERING, COMPLETING_RECOVERY = Value +} 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 000000000..07d23c6bf --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -0,0 +1,50 @@ +/* + * 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. + */ +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 order of creation). + */ + def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) +} + +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/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 6219f11f2..2ab7bb233 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 @@ -29,21 +29,37 @@ private[spark] class WorkerInfo( val memory: Int, val actor: ActorRef, val webUiPort: Int, - val publicAddress: String) { + 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[String, ExecutorInfo] + 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 b5ee6dca7..c8d34f25e 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/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index e3dc30eef..8fabc9566 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 09530beb3..46455aa5a 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 @@ -27,8 +27,8 @@ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} 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.{SparkEnv, 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 @@ -42,7 +42,7 @@ private[spark] class Worker( webUiPort: Int, cores: Int, memory: Int, - masterUrl: String, + var masterUrl: String, workDirPath: String = null) extends Actor with Logging { @@ -125,19 +125,30 @@ private[spark] class Worker( master ! Heartbeat(workerId) } + case MasterChanged(url, uiUrl) => + logInfo("Master has changed, new master is at " + url) + masterUrl = url + masterWebUiUrl = uiUrl + context.unwatch(master) + master = context.actorFor(Master.toAkkaUrl(masterUrl)) + context.watch(master) // Doesn't work with remote actors, but useful for testing + 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) + 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_ - master ! ExecutorStateChanged(appId, execId, ExecutorState.RUNNING, None, None) + master ! ExecutorStateChanged(appId, execId, manager.state, None, None) case ExecutorStateChanged(appId, execId, state, message, exitStatus) => master ! ExecutorStateChanged(appId, execId, state, message, exitStatus) @@ -174,11 +185,7 @@ private[spark] class Worker( } 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...") } def generateWorkerId(): String = { 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 9c49768c0..c173cdf44 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 @@ -71,8 +71,7 @@ private[spark] class SparkDeploySchedulerBackend( override def disconnected() { if (!stopping) { - logError("Disconnected from Spark cluster!") - scheduler.error("Disconnected from Spark cluster") + logError("Disconnected from Spark cluster! Waiting for reconnection...") } } From f549ea33d3d5a584f5d9965bb8e56462a1d6528e Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 19 Sep 2013 14:40:14 -0700 Subject: [PATCH 02/12] Standalone Scheduler fault tolerance using ZooKeeper This patch implements full distributed fault tolerance for standalone scheduler Masters. There is only one master Leader at a time, which is actively serving scheduling requests. If this Leader crashes, another master will eventually be elected, reconstruct the state from the first Master, and continue serving scheduling requests. Leader election is performed using the ZooKeeper leader election pattern. We try to minimize the use of ZooKeeper and the assumptions about ZooKeeper's behavior, so there is a layer of retries and session monitoring on top of the ZooKeeper client. Master failover follows directly from the single-node Master recovery via the file system (patch 194ba4b8), save that the Master state is stored in ZooKeeper instead. Configuration: By default, no recovery mechanism is enabled (spark.deploy.recoveryMode = NONE). By setting spark.deploy.recoveryMode to ZOOKEEPER and setting spark.deploy.zookeeper.url to an appropriate ZooKeeper URL, ZooKeeper recovery mode is enabled. By setting spark.deploy.recoveryMode to FILESYSTEM and setting spark.deploy.recoveryDirectory to an appropriate directory accessible by the Master, we will keep the behavior of from 194ba4b8. Additionally, places where a Master could be specificied by a spark:// url can now take comma-delimited lists to specify backup masters. Note that this is only used for registration of NEW Workers and application Clients. Once a Worker or Client has registered with the Master Leader, it is "in the system" and will never need to register again. Forthcoming: Documentation, tests (! - only ad hoc testing has been performed so far) I do not intend for this commit to be merged until tests are added, but this patch should still be mostly reviewable until then. --- .../scala/org/apache/spark/SparkContext.scala | 7 +- .../apache/spark/deploy/DeployMessage.scala | 20 +- .../spark/deploy/ExecutorDescription.scala | 5 + .../spark/deploy/LocalSparkCluster.scala | 7 +- .../apache/spark/deploy/client/Client.scala | 58 ++++-- .../spark/deploy/client/ClientListener.scala | 4 + .../spark/deploy/client/TestClient.scala | 7 +- .../spark/deploy/master/ApplicationInfo.scala | 18 +- .../spark/deploy/master/ExecutorInfo.scala | 2 +- .../master/FileSystemPersistenceEngine.scala | 14 +- .../deploy/master/LeaderElectionAgent.scala | 28 +++ .../apache/spark/deploy/master/Master.scala | 145 ++++++++------ .../spark/deploy/master/MasterMessages.scala | 29 +++ .../spark/deploy/master/MasterState.scala | 4 +- .../deploy/master/PersistenceEngine.scala | 4 +- .../deploy/master/SparkZooKeeperSession.scala | 183 ++++++++++++++++++ .../spark/deploy/master/WorkerInfo.scala | 24 +-- .../master/ZooKeeperLeaderElectionAgent.scala | 109 +++++++++++ .../master/ZooKeeperPersistenceEngine.scala | 64 ++++++ .../apache/spark/deploy/worker/Worker.scala | 125 ++++++++---- .../spark/deploy/worker/WorkerArguments.scala | 8 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 11 +- pom.xml | 11 ++ project/SparkBuild.scala | 1 + 25 files changed, 720 insertions(+), 170 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 912ce752f..531884727 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -169,7 +169,8 @@ class SparkContext( case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) - val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) + val masterUrls = sparkUrl.split(",") + val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName) scheduler.initialize(backend) scheduler @@ -185,8 +186,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() 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 0d0745a48..31d190927 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -27,6 +27,7 @@ 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 @@ -58,13 +59,14 @@ private[deploy] object DeployMessages { // 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, @@ -82,7 +84,7 @@ private[deploy] object DeployMessages { // 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) { @@ -131,16 +133,4 @@ private[deploy] object DeployMessages { assert (port > 0) } - // Actor System to Master - - case object CheckForWorkerTimeOut - - case class BeginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) - - case object EndRecoveryProcess - - case object RequestWebUIPort - - case class WebUIPortResponse(webUIBoundPort: Int) - } diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala index 716ee483d..2abf0b69d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala @@ -17,6 +17,11 @@ 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, 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 10161c820..308a2bfa2 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 28548a2ca..aa2a10a8a 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 @@ -40,27 +41,27 @@ import org.apache.spark.deploy.master.Master */ private[spark] class Client( actorSystem: ActorSystem, - masterUrl: String, + masterUrls: Array[String], appDescription: ApplicationDescription, listener: ClientListener) extends Logging { + val REGISTRATION_TIMEOUT = 60 * 1000 + 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 + connectToMaster() } catch { case e: Exception => logError("Failed to connect to master", e) @@ -69,9 +70,34 @@ private[spark] class Client( } } + def connectToMaster() { + for (masterUrl <- masterUrls) { + logInfo("Connecting to master " + masterUrl + "...") + val actor = context.actorFor(Master.toAkkaUrl(masterUrl)) + actor ! RegisterApplication(appDescription) + } + + context.system.scheduler.scheduleOnce(REGISTRATION_TIMEOUT millis) { + if (!registered) { + logError("All masters are unresponsive! Giving up.") + markDead() + } + } + } + + 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,13 +118,12 @@ private[spark] class Client( listener.executorRemoved(fullId, message.getOrElse(""), exitStatus) } - case MasterChanged(materUrl, masterWebUiUrl) => + case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) context.unwatch(master) - master = context.actorFor(Master.toAkkaUrl(masterUrl)) - masterAddress = master.path.address + changeMaster(masterUrl) + alreadyDisconnected = false sender ! MasterChangeAcknowledged(appId) - context.watch(master) case Terminated(actor_) if actor_ == master => logError("Connection to master failed; waiting for master to reconnect...") @@ -113,7 +138,7 @@ private[spark] class Client( markDisconnected() case StopClient => - markDisconnected() + markDead() sender ! true context.stop(self) } @@ -127,6 +152,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 4605368c1..be7a11bd1 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 d5e9a0e09..5b62d3ba6 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 e437a0e7a..8291e29ec 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 @@ -23,12 +23,12 @@ import akka.actor.ActorRef import scala.collection.mutable private[spark] class ApplicationInfo( - val startTime: Long, - val id: String, - val desc: ApplicationDescription, - val submitDate: Date, - val driver: ActorRef, - val appUiUrl: String) + val startTime: Long, + val id: String, + val desc: ApplicationDescription, + val submitDate: Date, + val driver: ActorRef, + val appUiUrl: String) extends Serializable { @transient var state: ApplicationState.Value = _ @@ -39,14 +39,14 @@ private[spark] class ApplicationInfo( @transient private var nextExecutorId: Int = _ - init() + init private def readObject(in: java.io.ObjectInputStream) : Unit = { in.defaultReadObject() - init() + init } - private def init() { + private def init = { state = ApplicationState.WAITING executors = new mutable.HashMap[Int, ExecutorInfo] coresGranted = 0 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 d235234c1..76db61dd6 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 @@ -28,7 +28,7 @@ private[spark] class ExecutorInfo( var state = ExecutorState.LAUNCHING - /** Copy all state variables from the given on-the-wire ExecutorDescription. */ + /** Copy all state (non-val) variables from the given on-the-wire ExecutorDescription. */ def copyState(execDesc: ExecutorDescription) { state = execDesc.state } 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 index 2fc13821b..c0849ef32 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -32,8 +32,8 @@ import org.apache.spark.Logging * @param serialization Used to serialize our objects. */ private[spark] class FileSystemPersistenceEngine( - val dir: String, - val serialization: Serialization) + val dir: String, + val serialization: Serialization) extends PersistenceEngine with Logging { new File(dir).mkdir() @@ -57,11 +57,11 @@ private[spark] class FileSystemPersistenceEngine( } 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](_)) + 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) } 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 000000000..c44a23f8c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -0,0 +1,28 @@ +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]] + */ +trait LeaderElectionAgent extends Actor { + val masterActor: ActorRef +} + +/** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */ +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 c6e039eed..e13a8cba4 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 @@ -34,18 +34,18 @@ 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.MasterState.MasterState +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.{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] @@ -76,75 +76,115 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (envVar != null) envVar else host } - var state: MasterState = _ + val masterUrl = "spark://" + host + ":" + port + var masterWebUiUrl: String = _ + + var state = MasterState.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 = - if (RECOVERY_DIR.isEmpty()) { - new BlackHolePersistenceEngine() - } else { + 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() + } - val (storedApps, storedWorkers) = persistenceEngine.readPersistedData() - state = - if (storedApps.isEmpty && storedWorkers.isEmpty) { - MasterState.ALIVE - } else { - self ! BeginRecovery(storedApps, storedWorkers) - MasterState.RECOVERING - } + 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]) { + 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 ElectedLeader => { + val (storedApps, storedWorkers) = persistenceEngine.readPersistedData() + state = if (storedApps.isEmpty && storedWorkers.isEmpty) + MasterState.ALIVE + else + MasterState.RECOVERING + + logInfo("I have been elected leader! New state: " + state) + + if (state == MasterState.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 == MasterState.STANDBY) { + // ignore, don't send response + } else if (idToWorker.contains(id)) { sender ! RegisterWorkerFailed("Duplicate worker ID") } else { 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 persistenceEngine.addWorker(worker) - sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort.get) + sender ! RegisteredWorker(masterUrl, masterWebUiUrl) schedule() } } case RegisterApplication(description) => { - 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) - schedule() + if (state == MasterState.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) => { @@ -184,27 +224,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } } - case BeginRecovery(storedApps, storedWorkers) => { - context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis, self, EndRecoveryProcess) - - val masterUrl = "spark://" + host + ":" + port - val masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get - for (app <- storedApps) { - registerApplication(app) - app.state = ApplicationState.UNKNOWN - app.driver ! MasterChanged(masterUrl, masterWebUiUrl) - } - for (worker <- storedWorkers) { - registerWorker(worker) - worker.state = WorkerState.UNKNOWN - worker.actor ! MasterChanged(masterUrl, masterWebUiUrl) - } - } - case MasterChangeAcknowledged(appId) => { - val appOption = idToApp.get(appId) - appOption match { + 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) @@ -216,9 +239,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act 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) != None) + 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)) @@ -232,10 +256,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (canCompleteRecovery) { completeRecovery() } } - case EndRecoveryProcess => { - 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 @@ -275,15 +295,29 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act workers.count(_.state == WorkerState.UNKNOWN) == 0 && apps.count(_.state == ApplicationState.UNKNOWN) == 0 + def beginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) { + for (app <- storedApps) { + registerApplication(app) + app.state = ApplicationState.UNKNOWN + app.driver ! MasterChanged(masterUrl, masterWebUiUrl) + } + for (worker <- storedWorkers) { + registerWorker(worker) + worker.state = WorkerState.UNKNOWN + worker.actor ! MasterChanged(masterUrl, masterWebUiUrl) + } + } + def completeRecovery() { + // Ensure "only-once" recovery semantics using a short synchronization period. synchronized { if (state != MasterState.RECOVERING) { return } state = MasterState.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(_)) + workers.filter(_.state == WorkerState.UNKNOWN).foreach(removeWorker) + apps.filter(_.state == ApplicationState.UNKNOWN).foreach(finishApplication) state = MasterState.ALIVE schedule() @@ -352,7 +386,7 @@ 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) @@ -415,6 +449,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act 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 >= app.desc.memoryPerSlave)) { logWarning("Could not find any workers with enough memory for " + firstApp.get.id) @@ -444,7 +479,7 @@ 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) 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 000000000..6e31d40b4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -0,0 +1,29 @@ +package org.apache.spark.deploy.master + +import org.apache.spark.util.Utils + +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 RequestWebUIPort + + case class WebUIPortResponse(webUIBoundPort: Int) +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala index 9ea5e9752..eec3df3b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala @@ -18,9 +18,9 @@ package org.apache.spark.deploy.master private[spark] object MasterState - extends Enumeration("ALIVE", "RECOVERING", "COMPLETING_RECOVERY") { + extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") { type MasterState = Value - val ALIVE, RECOVERING, COMPLETING_RECOVERY = Value + val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value } 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 index 07d23c6bf..8c4878bd3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -36,9 +36,11 @@ trait PersistenceEngine { /** * Returns the persisted data sorted by their respective ids (which implies that they're - * sorted by time order of creation). + * sorted by time of creation). */ def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) + + def close() {} } class BlackHolePersistenceEngine extends PersistenceEngine { 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 000000000..5492a3a98 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -0,0 +1,183 @@ +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"). + */ +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() + spawn(sessionMonitorThread) + } + + def sessionMonitorThread = { + 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)(implicit 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 2ab7bb233..26090c6a9 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,14 +22,14 @@ 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") @@ -42,18 +42,18 @@ private[spark] class WorkerInfo( @transient var lastHeartbeat: Long = _ - init() + init def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed private def readObject(in: java.io.ObjectInputStream) : Unit = { in.defaultReadObject() - init() + init } - private def init() { - executors = new mutable.HashMap[String, ExecutorInfo] + private def init = { + executors = new mutable.HashMap state = WorkerState.ALIVE coresUsed = 0 memoryUsed = 0 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 000000000..4ca59e5b2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -0,0 +1,109 @@ +package org.apache.spark.deploy.master + +import scala.collection.JavaConversions._ + +import org.apache.spark.deploy.master.MasterMessages.{CheckLeader, ElectedLeader, RevokedLeadership} +import org.apache.spark.Logging +import org.apache.zookeeper._ +import org.apache.zookeeper.Watcher.Event.EventType + +import akka.actor.{Cancellable, ActorRef} +import akka.util.duration._ + +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() { + zk.mkdirRecursive(WORKING_DIR) + myLeaderFile = + zk.create(WORKING_DIR + "/master_", masterUrl.getBytes, CreateMode.EPHEMERAL_SEQUENTIAL) + self ! CheckLeader + } + + 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.get(0) + 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 + } + + 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. + 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 000000000..f45b62cbd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -0,0 +1,64 @@ +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/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 46455aa5a..73fb0c8bd 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,11 +23,11 @@ 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.{SparkEnv, Logging} +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 @@ -35,14 +35,13 @@ import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} - private[spark] class Worker( host: String, port: Int, webUiPort: Int, cores: Int, memory: Int, - var masterUrl: String, + masterUrls: Array[String], workDirPath: String = null) extends Actor with Logging { @@ -54,8 +53,16 @@ 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 + var master: ActorRef = null - var masterWebUiUrl : String = "" + var activeMasterUrl: String = "" + var activeMasterWebUiUrl : String = "" + var registered = false val workerId = generateWorkerId() var sparkHome: File = null var workDir: File = null @@ -103,35 +110,62 @@ 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) + def changeMaster(url: String, uiUrl: String) { + 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 } + 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") + case RegisteredWorker(masterUrl, masterWebUiUrl) => + logInfo("Successfully registered with master " + masterUrl) + registered = true + changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) { master ! Heartbeat(workerId) } - case MasterChanged(url, uiUrl) => - logInfo("Master has changed, new master is at " + url) - masterUrl = url - masterWebUiUrl = uiUrl + case MasterChanged(masterUrl, masterWebUiUrl) => + logInfo("Master has changed, new master is at " + masterUrl) context.unwatch(master) - master = context.actorFor(Master.toAkkaUrl(masterUrl)) - context.watch(master) // Doesn't work with remote actors, but useful for testing + changeMaster(masterUrl, masterWebUiUrl) + val execs = executors.values. map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state)) sender ! WorkerSchedulerStateResponse(workerId, execs.toList) @@ -140,15 +174,19 @@ private[spark] class Worker( 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, ExecutorState.RUNNING) - executors(appId + "/" + execId) = manager - manager.start() - coresUsed += cores_ - memoryUsed += memory_ - master ! ExecutorStateChanged(appId, execId, manager.state, None, None) + 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_ + master ! ExecutorStateChanged(appId, execId, manager.state, None, None) + } case ExecutorStateChanged(appId, execId, state, message, exitStatus) => master ! ExecutorStateChanged(appId, execId, state, message, exitStatus) @@ -164,14 +202,18 @@ 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(_, _) => @@ -179,8 +221,8 @@ private[spark] class Worker( 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) } } @@ -203,17 +245,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 0ae89a864..16d868649 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.split(",") 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 95d6007f3..800f1cafc 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 = {Utils.offsetBytes(path, startByte, endByte)} - val linkToMaster =

Back to Master

+ val linkToMaster =

Back to Master

val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} 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 c173cdf44..8a3017e96 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() } @@ -75,6 +75,13 @@ private[spark] class SparkDeploySchedulerBackend( } } + override def dead() { + if (!stopping) { + logError("Spark cluster looks dead, giving up.") + scheduler.error("Spark cluster looks down") + } + } + override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) { logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format( fullId, hostPort, cores, Utils.megabytesToString(memory))) diff --git a/pom.xml b/pom.xml index d74d45adf..f8ea2b892 100644 --- a/pom.xml +++ b/pom.xml @@ -344,6 +344,17 @@ 0.9 test + + org.apache.zookeeper + zookeeper + 3.4.5 + + + org.jboss.netty + netty + + + org.apache.hadoop hadoop-client diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 99cdadb9e..156f501a0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -211,6 +211,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", From 42d72308fb772bf5dc579c9da174e6057ee86171 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 26 Sep 2013 15:45:20 -0700 Subject: [PATCH 03/12] Add license notices --- .../deploy/master/LeaderElectionAgent.scala | 17 +++++++++++++++++ .../spark/deploy/master/MasterMessages.scala | 19 +++++++++++++++++-- .../deploy/master/SparkZooKeeperSession.scala | 17 +++++++++++++++++ .../master/ZooKeeperLeaderElectionAgent.scala | 17 +++++++++++++++++ .../master/ZooKeeperPersistenceEngine.scala | 17 +++++++++++++++++ .../spark/deploy/JsonProtocolSuite.scala | 2 +- 6 files changed, 86 insertions(+), 3 deletions(-) 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 index c44a23f8c..fc8255fa6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -1,3 +1,20 @@ +/* + * 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} 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 index 6e31d40b4..08fe5334c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -1,6 +1,21 @@ -package org.apache.spark.deploy.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. + */ -import org.apache.spark.util.Utils +package org.apache.spark.deploy.master sealed trait MasterMessages extends Serializable 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 index 5492a3a98..f43f9f6ed 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -1,3 +1,20 @@ +/* + * 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._ 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 index 4ca59e5b2..f8e86d633 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -1,3 +1,20 @@ +/* + * 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._ 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 index f45b62cbd..13a6b627b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -1,3 +1,20 @@ +/* + * 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 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 05f8545c7..4346571c4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -79,7 +79,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) { From db6f1549406be22f0b7c8ab4425af30602e52283 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 4 Oct 2013 19:48:47 -0700 Subject: [PATCH 04/12] Fix race conditions during recovery One major change was the use of messages instead of raw functions as the parameter of Akka scheduled timers. Since messages are serialized, unlike raw functions, the behavior is easier to think about and doesn't cause race conditions when exceptions are thrown. Another change is to avoid using global pointers that might change without a lock. --- .../apache/spark/deploy/DeployMessage.scala | 7 ++- .../apache/spark/deploy/JsonProtocol.scala | 3 +- .../apache/spark/deploy/client/Client.scala | 29 ++++++++--- .../apache/spark/deploy/master/Master.scala | 29 ++++++++--- .../spark/deploy/master/MasterMessages.scala | 2 + .../master/ZooKeeperLeaderElectionAgent.scala | 51 ++++++++++++------- .../apache/spark/deploy/worker/Worker.scala | 48 ++++++++++++----- .../spark/deploy/JsonProtocolSuite.scala | 5 +- 8 files changed, 122 insertions(+), 52 deletions(-) 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 31d190927..979e65ac6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -21,6 +21,7 @@ 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.MasterState.MasterState import org.apache.spark.deploy.worker.ExecutorRunner import org.apache.spark.util.Utils @@ -111,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) @@ -133,4 +135,7 @@ private[deploy] object DeployMessages { assert (port > 0) } + // Actor System to Worker + + case object SendHeartbeat } 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 87a703427..f87b88528 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -71,7 +71,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/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index aa2a10a8a..198d5cee7 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 @@ -46,7 +46,8 @@ private[spark] class Client( listener: ClientListener) extends Logging { - val REGISTRATION_TIMEOUT = 60 * 1000 + val REGISTRATION_TIMEOUT = 20.seconds + val REGISTRATION_RETRIES = 3 var actor: ActorRef = null var appId: String = null @@ -61,7 +62,7 @@ private[spark] class Client( override def preStart() { try { - connectToMaster() + registerWithMaster() } catch { case e: Exception => logError("Failed to connect to master", e) @@ -70,19 +71,31 @@ private[spark] class Client( } } - def connectToMaster() { + def tryRegisterAllMasters() { for (masterUrl <- masterUrls) { logInfo("Connecting to master " + masterUrl + "...") val actor = context.actorFor(Master.toAkkaUrl(masterUrl)) actor ! RegisterApplication(appDescription) } + } - context.system.scheduler.scheduleOnce(REGISTRATION_TIMEOUT millis) { - if (!registered) { - logError("All masters are unresponsive! Giving up.") - markDead() + 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) { 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 e13a8cba4..093ce09b1 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 @@ -123,6 +123,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } override def preRestart(reason: Throwable, message: Option[Any]) { + super.preRestart(reason, message) // calls postStop()! logError("Master actor restarted due to exception", reason) } @@ -279,7 +280,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } 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 => { @@ -297,14 +299,25 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def beginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) { for (app <- storedApps) { - registerApplication(app) - app.state = ApplicationState.UNKNOWN - app.driver ! MasterChanged(masterUrl, masterWebUiUrl) + 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) { - registerWorker(worker) - worker.state = WorkerState.UNKNOWN - worker.actor ! MasterChanged(masterUrl, masterWebUiUrl) + 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") + } } } @@ -409,7 +422,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act workers += worker idToWorker(worker.id) = worker - actorToWorker(sender) = worker + actorToWorker(worker.actor) = worker addressToWorker(workerAddress) = worker } 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 index 08fe5334c..74a9f8cd8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -38,6 +38,8 @@ private[master] object MasterMessages { 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/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index f8e86d633..065635af8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -44,10 +44,18 @@ class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) } override def zkSessionCreated() { - zk.mkdirRecursive(WORKING_DIR) - myLeaderFile = - zk.create(WORKING_DIR + "/master_", masterUrl.getBytes, CreateMode.EPHEMERAL_SEQUENTIAL) - self ! CheckLeader + 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() { @@ -75,7 +83,7 @@ class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) /** Uses ZK leader election. Navigates several ZK potholes along the way. */ def checkLeader() { val masters = zk.getChildren(WORKING_DIR).toList - val leader = masters.sorted.get(0) + val leader = masters.sorted.head val leaderFile = WORKING_DIR + "/" + leader // Setup a watch for the current leader. @@ -92,20 +100,25 @@ class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) return } - 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. - logWarning("Cleaning up old ZK master election file that points to this master.") - zk.delete(leaderFile) - } else { - updateLeadershipStatus(isLeader) + // 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) + } } } 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 73fb0c8bd..25ba75619 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 @@ -59,10 +59,12 @@ private[spark] class Worker( // Index into masterUrls that we're currently trying to register with. var masterIndex = 0 + val masterLock: Object = new Object() var master: ActorRef = null var activeMasterUrl: String = "" var activeMasterWebUiUrl : String = "" - var registered = false + @volatile var registered = false + @volatile var connected = false val workerId = generateWorkerId() var sparkHome: File = null var workDir: File = null @@ -102,6 +104,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(".")) @@ -117,11 +120,14 @@ private[spark] class Worker( } def changeMaster(url: String, uiUrl: String) { - 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 + 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() { @@ -157,8 +163,11 @@ private[spark] class Worker( logInfo("Successfully registered with master " + masterUrl) registered = true changeMaster(masterUrl, masterWebUiUrl) - context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) { - master ! Heartbeat(workerId) + context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) + + case SendHeartbeat => + masterLock.synchronized { + if (connected) { master ! Heartbeat(workerId) } } case MasterChanged(masterUrl, masterWebUiUrl) => @@ -171,8 +180,10 @@ private[spark] class Worker( sender ! WorkerSchedulerStateResponse(workerId, execs.toList) case RegisterWorkerFailed(message) => - logError("Worker registration failed: " + message) - System.exit(1) + if (!registered) { + logError("Worker registration failed: " + message) + System.exit(1) + } case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_, execSparkHome_) => if (masterUrl != activeMasterUrl) { @@ -185,11 +196,15 @@ private[spark] class Worker( manager.start() coresUsed += cores_ memoryUsed += memory_ - master ! ExecutorStateChanged(appId, execId, manager.state, None, None) + 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) @@ -216,7 +231,13 @@ private[spark] class Worker( } } - 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 => { @@ -228,6 +249,7 @@ private[spark] class Worker( def masterDisconnected() { logError("Connection to master failed! Waiting for master to reconnect...") + connected = false } def generateWorkerId(): String = { 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 4346571c4..c59e1f4de 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, MasterState, 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, + MasterState.ALIVE) val output = JsonProtocol.writeMasterState(stateResponse) assertValidJson(output) } From 0f070279e7cd224de48333b572d3080b742a82d7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sat, 5 Oct 2013 15:15:29 -0700 Subject: [PATCH 05/12] Address Matei's comments --- bin/stop-slaves.sh | 2 -- .../apache/spark/deploy/DeployMessage.scala | 2 +- .../spark/deploy/master/ApplicationInfo.scala | 6 ++--- .../deploy/master/LeaderElectionAgent.scala | 4 +-- .../apache/spark/deploy/master/Master.scala | 26 +++++++++---------- .../deploy/master/PersistenceEngine.scala | 7 ++--- ...{MasterState.scala => RecoveryState.scala} | 2 +- .../deploy/master/SparkZooKeeperSession.scala | 13 ++++++---- .../spark/deploy/master/WorkerInfo.scala | 6 ++--- .../master/ZooKeeperLeaderElectionAgent.scala | 11 +++----- .../spark/deploy/JsonProtocolSuite.scala | 4 +-- 11 files changed, 41 insertions(+), 42 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/master/{MasterState.scala => RecoveryState.scala} (96%) diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh index abf1c7be6..fcb8555d4 100755 --- a/bin/stop-slaves.sh +++ b/bin/stop-slaves.sh @@ -17,8 +17,6 @@ # limitations under the License. # -# Starts workers 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/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 979e65ac6..275331724 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -21,7 +21,7 @@ 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.MasterState.MasterState +import org.apache.spark.deploy.master.RecoveryState.MasterState import org.apache.spark.deploy.worker.ExecutorRunner import org.apache.spark.util.Utils 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 8291e29ec..5150b7c7d 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 @@ -39,14 +39,14 @@ private[spark] class ApplicationInfo( @transient private var nextExecutorId: Int = _ - init + init() private def readObject(in: java.io.ObjectInputStream) : Unit = { in.defaultReadObject() - init + init() } - private def init = { + private def init() { state = ApplicationState.WAITING executors = new mutable.HashMap[Int, ExecutorInfo] coresGranted = 0 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 index fc8255fa6..f25a1ad3b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -29,12 +29,12 @@ import org.apache.spark.deploy.master.MasterMessages.ElectedLeader * [[org.apache.spark.deploy.master.MasterMessages.ElectedLeader ElectedLeader]] * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]] */ -trait LeaderElectionAgent extends Actor { +private[spark] trait LeaderElectionAgent extends Actor { val masterActor: ActorRef } /** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */ -class MonarchyLeaderAgent(val masterActor: ActorRef) extends LeaderElectionAgent { +private[spark] class MonarchyLeaderAgent(val masterActor: ActorRef) extends LeaderElectionAgent { override def preStart() { masterActor ! ElectedLeader } 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 093ce09b1..cd916672a 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 @@ -79,7 +79,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val masterUrl = "spark://" + host + ":" + port var masterWebUiUrl: String = _ - var state = MasterState.STANDBY + var state = RecoveryState.STANDBY var persistenceEngine: PersistenceEngine = _ @@ -139,13 +139,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case ElectedLeader => { val (storedApps, storedWorkers) = persistenceEngine.readPersistedData() state = if (storedApps.isEmpty && storedWorkers.isEmpty) - MasterState.ALIVE + RecoveryState.ALIVE else - MasterState.RECOVERING + RecoveryState.RECOVERING logInfo("I have been elected leader! New state: " + state) - if (state == MasterState.RECOVERING) { + if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedWorkers) context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() } } @@ -159,7 +159,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act 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 (state == MasterState.STANDBY) { + if (state == RecoveryState.STANDBY) { // ignore, don't send response } else if (idToWorker.contains(id)) { sender ! RegisterWorkerFailed("Duplicate worker ID") @@ -174,7 +174,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } case RegisterApplication(description) => { - if (state == MasterState.STANDBY) { + if (state == RecoveryState.STANDBY) { // ignore, don't send response } else { logInfo("Registering app " + description.name) @@ -262,21 +262,21 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // those we have an entry for in the corresponding actor hashmap actorToWorker.get(actor).foreach(removeWorker) actorToApp.get(actor).foreach(finishApplication) - if (state == MasterState.RECOVERING && canCompleteRecovery) { completeRecovery() } + 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 == MasterState.RECOVERING && canCompleteRecovery) { completeRecovery() } + 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 == MasterState.RECOVERING && canCompleteRecovery) { completeRecovery() } + if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() } } case RequestMasterState => { @@ -324,15 +324,15 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def completeRecovery() { // Ensure "only-once" recovery semantics using a short synchronization period. synchronized { - if (state != MasterState.RECOVERING) { return } - state = MasterState.COMPLETING_RECOVERY + 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 = MasterState.ALIVE + state = RecoveryState.ALIVE schedule() logInfo("Recovery complete - resuming operations!") } @@ -351,7 +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 != MasterState.ALIVE) { return } + 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) { 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 index 8c4878bd3..94b986caf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -23,9 +23,10 @@ package org.apache.spark.deploy.master * - 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. + * we might not have yet deleted apps or workers that finished (so their liveness must be verified + * during recovery). */ -trait PersistenceEngine { +private[spark] trait PersistenceEngine { def addApplication(app: ApplicationInfo) def removeApplication(app: ApplicationInfo) @@ -43,7 +44,7 @@ trait PersistenceEngine { def close() {} } -class BlackHolePersistenceEngine extends PersistenceEngine { +private[spark] class BlackHolePersistenceEngine extends PersistenceEngine { override def addApplication(app: ApplicationInfo) {} override def removeApplication(app: ApplicationInfo) {} override def addWorker(worker: WorkerInfo) {} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala rename to core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala index eec3df3b7..b91be821f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object MasterState +private[spark] object RecoveryState extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") { type MasterState = 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 index f43f9f6ed..81e15c534 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -35,7 +35,7 @@ import org.apache.zookeeper.Watcher.Event.KeeperState * 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"). */ -class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { +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 @@ -53,10 +53,13 @@ class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { /** Connect to ZooKeeper to start the session. Must be called before anything else. */ def connect() { connectToZooKeeper() - spawn(sessionMonitorThread) + + new Thread() { + override def run() = sessionMonitorThread() + }.start() } - def sessionMonitorThread = { + def sessionMonitorThread(): Unit = { while (!closed) { Thread.sleep(ZK_CHECK_PERIOD_MILLIS) if (zk.getState != ZooKeeper.States.CONNECTED) { @@ -170,7 +173,7 @@ class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { * * @param fn Block to execute, possibly multiple times. */ - def retry[T](fn: => T)(implicit n: Int = MAX_RECONNECT_ATTEMPTS): T = { + def retry[T](fn: => T, n: Int = MAX_RECONNECT_ATTEMPTS): T = { try { fn } catch { @@ -179,7 +182,7 @@ class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { case e if n > 0 => logError("ZooKeeper exception, " + n + " more retries...", e) Thread.sleep(RETRY_WAIT_MILLIS) - retry(fn)(n-1) + retry(fn, n-1) } } } 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 26090c6a9..e05f587b5 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 @@ -42,17 +42,17 @@ private[spark] class WorkerInfo( @transient var lastHeartbeat: Long = _ - init + init() def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed private def readObject(in: java.io.ObjectInputStream) : Unit = { in.defaultReadObject() - init + init() } - private def init = { + private def init() { executors = new mutable.HashMap state = WorkerState.ALIVE coresUsed = 0 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 index 065635af8..7809013e8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -17,17 +17,14 @@ package org.apache.spark.deploy.master -import scala.collection.JavaConversions._ - -import org.apache.spark.deploy.master.MasterMessages.{CheckLeader, ElectedLeader, RevokedLeadership} -import org.apache.spark.Logging +import akka.actor.ActorRef import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.EventType -import akka.actor.{Cancellable, ActorRef} -import akka.util.duration._ +import org.apache.spark.deploy.master.MasterMessages._ +import org.apache.spark.Logging -class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) +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" 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 c59e1f4de..0b38e239f 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, MasterState, WorkerInfo} +import org.apache.spark.deploy.master.{ApplicationInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.ExecutorRunner class JsonProtocolSuite extends FunSuite { @@ -54,7 +54,7 @@ class JsonProtocolSuite extends FunSuite { val activeApps = Array[ApplicationInfo](createAppInfo()) val completedApps = Array[ApplicationInfo]() val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps, - MasterState.ALIVE) + RecoveryState.ALIVE) val output = JsonProtocol.writeMasterState(stateResponse) assertValidJson(output) } From e1190229e13453cdb1e7c28fdf300d1f8dd717c2 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sat, 5 Oct 2013 23:07:26 -0700 Subject: [PATCH 06/12] Add end-to-end test for standalone scheduler fault tolerance Docker files drawn mostly from Matt Masse. Some updates from Andre Schumacher. --- .../spark/deploy/FaultToleranceTest.scala | 413 ++++++++++++++++++ .../apache/spark/ui/UIWorkloadGenerator.scala | 2 +- docker/README.md | 5 + docker/build | 5 + docker/spark-test/README.md | 10 + docker/spark-test/base/Dockerfile | 23 + docker/spark-test/build | 5 + docker/spark-test/master/Dockerfile | 4 + docker/spark-test/master/default_cmd | 4 + docker/spark-test/worker/Dockerfile | 5 + docker/spark-test/worker/default_cmd | 4 + spark-class | 2 +- 12 files changed, 480 insertions(+), 2 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala create mode 100644 docker/README.md create mode 100755 docker/build create mode 100644 docker/spark-test/README.md create mode 100644 docker/spark-test/base/Dockerfile create mode 100755 docker/spark-test/build create mode 100644 docker/spark-test/master/Dockerfile create mode 100755 docker/spark-test/master/default_cmd create mode 100644 docker/spark-test/worker/Dockerfile create mode 100755 docker/spark-test/worker/default_cmd 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 000000000..f9e40187c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -0,0 +1,413 @@ +/* + * + * * 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. + * Unfortunately, this dependency means that the 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 dockerMountString = "%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(sparkHome) } + } + + def addWorkers(num: Int) { + val masterUrls = getMasterUrls(masters) + (1 to num).foreach { _ => workers += SparkDocker.startWorker(sparkHome, 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 = { + masters.map(master => "spark://" + 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/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 453394dfd..fcd1b518d 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/docker/README.md b/docker/README.md new file mode 100644 index 000000000..bf59e77d1 --- /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 000000000..bb2d972cd --- /dev/null +++ b/docker/build @@ -0,0 +1,5 @@ +#!/bin/bash + +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 000000000..addea277a --- /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 + +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 000000000..f9e7217af --- /dev/null +++ b/docker/spark-test/base/Dockerfile @@ -0,0 +1,23 @@ +# Spark 0.8.1 +# +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 000000000..3835fcf27 --- /dev/null +++ b/docker/spark-test/build @@ -0,0 +1,5 @@ +#!/bin/bash + +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 000000000..9e6ddb1ab --- /dev/null +++ b/docker/spark-test/master/Dockerfile @@ -0,0 +1,4 @@ +# Spark Master +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 000000000..92910068a --- /dev/null +++ b/docker/spark-test/master/default_cmd @@ -0,0 +1,4 @@ +#!/bin/bash +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 000000000..569067805 --- /dev/null +++ b/docker/spark-test/worker/Dockerfile @@ -0,0 +1,5 @@ +# Spark Worker +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 000000000..360993f17 --- /dev/null +++ b/docker/spark-test/worker/default_cmd @@ -0,0 +1,4 @@ +#!/bin/bash +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/spark-class b/spark-class index e111ef6da..f678d5e6c 100755 --- a/spark-class +++ b/spark-class @@ -41,7 +41,7 @@ if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark. SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" # Do not overwrite SPARK_JAVA_OPTS environment variable in this script - OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS" # Empty by default + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_DAEMON_JAVA_OPTS" # Empty by default else OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" fi From 718e8c20526847657a58ab7ea5e4c86c367ae6d9 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 6 Oct 2013 00:02:08 -0700 Subject: [PATCH 07/12] Change url format to spark://host1:port1,host2:port2 This replaces the format of spark://host1:port1,spark://host2:port2 and is more consistent with ZooKeeper's zk:// urls. --- .../src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- .../org/apache/spark/deploy/FaultToleranceTest.scala | 10 ++++++---- .../scala/org/apache/spark/deploy/client/Client.scala | 2 ++ .../scala/org/apache/spark/deploy/worker/Worker.scala | 3 +++ .../apache/spark/deploy/worker/WorkerArguments.scala | 2 +- 5 files changed, 14 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 531884727..b2643879a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -153,7 +153,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 @@ -169,7 +169,7 @@ class SparkContext( case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) - val masterUrls = sparkUrl.split(",") + val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName) scheduler.initialize(backend) scheduler diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index f9e40187c..8bac62b86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -36,6 +36,8 @@ import org.apache.spark.deploy.master.RecoveryState /** * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master. + * Execute using + * ./spark-class org.apache.spark.deploy.FaultToleranceTest * * In order to mimic a real distributed cluster more closely, Docker is used. * Unfortunately, this dependency means that the suite cannot be run automatically without a @@ -56,7 +58,7 @@ private[spark] object FaultToleranceTest extends App with Logging { assertTrue(sparkHome != null, "Run with a valid SPARK_HOME") val containerSparkHome = "/opt/spark" - val dockerMountString = "%s:%s".format(sparkHome, containerSparkHome) + val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip @@ -172,12 +174,12 @@ private[spark] object FaultToleranceTest extends App with Logging { } def addMasters(num: Int) { - (1 to num).foreach { _ => masters += SparkDocker.startMaster(sparkHome) } + (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) } } def addWorkers(num: Int) { val masterUrls = getMasterUrls(masters) - (1 to num).foreach { _ => workers += SparkDocker.startWorker(sparkHome, masterUrls) } + (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) } } /** Creates a SparkContext, which constructs a Client to interact with our cluster. */ @@ -190,7 +192,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } def getMasterUrls(masters: Seq[TestMasterInfo]): String = { - masters.map(master => "spark://" + master.ip + ":7077").mkString(",") + "spark://" + masters.map(master => master.ip + ":7077").mkString(",") } def getLeader: TestMasterInfo = { 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 198d5cee7..0d4682fcc 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 @@ -38,6 +38,8 @@ 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, 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 25ba75619..216d9d44a 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 @@ -35,6 +35,9 @@ 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, 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 16d868649..3ed528e6b 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 @@ -89,7 +89,7 @@ private[spark] class WorkerArguments(args: Array[String]) { if (masters != null) { // Two positional arguments were given printUsageAndExit(1) } - masters = value.split(",") + masters = value.stripPrefix("spark://").split(",").map("spark://" + _) parse(tail) case Nil => From 1cd57cd4d34d5e0603c25a5ad7d28501fe9c94fd Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 8 Oct 2013 11:41:12 -0700 Subject: [PATCH 08/12] Add license agreements to dockerfiles --- docker/build | 17 +++++++++++++++++ docker/spark-test/base/Dockerfile | 17 ++++++++++++++++- docker/spark-test/build | 17 +++++++++++++++++ docker/spark-test/master/Dockerfile | 17 +++++++++++++++++ docker/spark-test/master/default_cmd | 18 ++++++++++++++++++ docker/spark-test/worker/Dockerfile | 17 +++++++++++++++++ docker/spark-test/worker/default_cmd | 18 ++++++++++++++++++ 7 files changed, 120 insertions(+), 1 deletion(-) diff --git a/docker/build b/docker/build index bb2d972cd..253a2fc8d 100755 --- a/docker/build +++ b/docker/build @@ -1,5 +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/base/Dockerfile b/docker/spark-test/base/Dockerfile index f9e7217af..60962776d 100644 --- a/docker/spark-test/base/Dockerfile +++ b/docker/spark-test/base/Dockerfile @@ -1,5 +1,20 @@ -# Spark 0.8.1 # +# 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 diff --git a/docker/spark-test/build b/docker/spark-test/build index 3835fcf27..6f9e19743 100755 --- a/docker/spark-test/build +++ b/docker/spark-test/build @@ -1,5 +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 index 9e6ddb1ab..f729534ab 100644 --- a/docker/spark-test/master/Dockerfile +++ b/docker/spark-test/master/Dockerfile @@ -1,4 +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 index 92910068a..a5b1303c2 100755 --- a/docker/spark-test/master/default_cmd +++ b/docker/spark-test/master/default_cmd @@ -1,4 +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 index 569067805..890febe7b 100644 --- a/docker/spark-test/worker/Dockerfile +++ b/docker/spark-test/worker/Dockerfile @@ -1,4 +1,21 @@ # 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/ diff --git a/docker/spark-test/worker/default_cmd b/docker/spark-test/worker/default_cmd index 360993f17..ab6336f70 100755 --- a/docker/spark-test/worker/default_cmd +++ b/docker/spark-test/worker/default_cmd @@ -1,4 +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 From 749233b869da188920d8d72af7b82e586993d17c Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 8 Oct 2013 11:41:52 -0700 Subject: [PATCH 09/12] Revert change to spark-class Also adds comment about how to configure for FaultToleranceTest. --- .../org/apache/spark/deploy/FaultToleranceTest.scala | 9 +++++++-- spark-class | 2 +- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 8bac62b86..668032a3a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -36,11 +36,16 @@ 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 * - * In order to mimic a real distributed cluster more closely, Docker is used. - * Unfortunately, this dependency means that the suite cannot be run automatically without a + * 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 diff --git a/spark-class b/spark-class index f678d5e6c..e111ef6da 100755 --- a/spark-class +++ b/spark-class @@ -41,7 +41,7 @@ if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark. SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" # Do not overwrite SPARK_JAVA_OPTS environment variable in this script - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_DAEMON_JAVA_OPTS" # Empty by default + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS" # Empty by default else OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" fi From 4ea8ee468fb1f50fce56853a5127a89efc45b706 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 8 Oct 2013 14:18:31 -0700 Subject: [PATCH 10/12] Add docs for standalone scheduler fault tolerance Also fix a couple HTML/Markdown issues in other files. --- docs/spark-standalone.md | 45 +++++++++++++++++++++++++++++ docs/streaming-programming-guide.md | 5 ++-- docs/tuning.md | 2 +- 3 files changed, 48 insertions(+), 4 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 81cdbefd0..5707e1927 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,45 @@ 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://:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on). + +# Fault tolerance + +By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new jobs can be created. In order to circumvent this, we have two fault tolerance schemes, detailed below. + +**Possible gotcha:** Production-level fault tolerance is enabled by having multiple Master nodes tied together with ZooKeeper. If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will start scheduling independently). + +## Standby masters with ZooKeeper + +**Configuration** + + # May be configured as SPARK_DAEMON_JAVA_OPTS in spark-env.sh + spark.deploy.recoveryMode=ZOOKEEPER + spark.deploy.zookeeper.url=ZK_URL:ZK_PORT # eg 192.168.1.100:2181 + spark.deploy.zookeeper.dir=/spark # OPTIONAL! /spark is the default. + +**Details** + +Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ jobs -- jobs that were already running during Master failover are unaffected. + +In order to schedule new jobs or add Workers to the cluster, they need to know the IP address of the current leader. This can be accomplished by simply passing in a list of Masters where you used to pass in a single one. For example, you might start your SparkContext pointing to ``spark://host1:port1,host2:port2``. This would cause your SparkContext to try registering with both Masters -- if host1 goes down, this configuration would still be correct as we'd find the new leader, host2. + +There's an important distinction to be made between "registering with a Master" and normal operation. When starting up, a SparkContext or Worker needs to be able to find and register with the current lead Master. Once it successfully registers, though, it is "in the system" (i.e., stored in ZooKeeper). If failover occurs, the new leader will contact all previously registered SparkContexts and Workers to inform them of the change in leadership, so they need not have even known of the existence of the new Master at startup. +
This means that new Masters can be created at any time, and the only thing you need to worry about is that _new_ SparkContexts and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of. + +## Single-node recovery with local file system + +**Configuration** + + # May be configured as SPARK_DAEMON_JAVA_OPTS in spark-env.sh + spark.deploy.recoveryMode=FILESYSTEM + spark.deploy.recoveryDirectory=PATH_ACCESSIBLE_TO_MASTER + +**Details** + +ZooKeeper is the best way to go for production-level fault tolerance, but if you just want to be able to restart the Master if it goes down, FILESYSTEM mode can take care of it. When SparkContexts and Workers register, they have enough state written to the provided directory so that they can be recovered upon a restart of the Master process. + +**Notes:** + +* This solution can be used in tandem with a process monitor/manager like [monit](http://mmonit.com/monit/), or just to enable manual recovery via restart. +* While filesystem recovery seems straightforwardly better than not doing any recovery at all, this mode may be suboptimal for certain development or experimental purposes. In particular, killing a master via stop-master.sh does not clean up its recovery state, so whenever you start a new Master, it will enter recovery mode. This could increase the startup time by up to 1 minute if it needs to wait for all previously-registered Workers/clients to timeout. +* While it's not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/clients (equivalent to ZooKeeper recovery). Note, however, that you **cannot** have multiple Masters alive concurrently using this approach; you need to upgrade to ZooKeeper to provide leader election for that use-case. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index c7df17202..835b25723 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 - + - + @@ -161,7 +161,6 @@ Spark Streaming features windowed computations, which allow you to apply transfo windowDuration and slideDuration are exactly as defined in window(). -
TransformationMeaning
window(windowDuration, slideDuration) window(windowDuration, slideDuration) Return a new DStream which is computed based on windowed batches of the source DStream. windowDuration is the width of the window and slideTime is the frequency during which the window is calculated. Both times must be multiples of the batch interval.
countByWindow(windowDuration, slideDuration) countByWindow(windowDuration, slideDuration) Return a sliding count of elements in the stream. windowDuration and slideDuration are exactly as defined in window().
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 28d88a265..f491ae9b9 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 From 42d8b8efe68f6f81af5ed31a046138f9dad5be4a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 10 Oct 2013 00:32:58 -0700 Subject: [PATCH 11/12] Address Matei's comments on documentation Updates to the documentation and changing some logError()s to logWarning()s. --- .../apache/spark/deploy/client/Client.scala | 8 ++--- .../master/ZooKeeperPersistenceEngine.scala | 6 +++- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- docs/spark-standalone.md | 35 +++++++++++-------- 4 files changed, 31 insertions(+), 20 deletions(-) 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 0d4682fcc..77422f61e 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 @@ -67,7 +67,7 @@ private[spark] class Client( registerWithMaster() } catch { case e: Exception => - logError("Failed to connect to master", e) + logWarning("Failed to connect to master", e) markDisconnected() context.stop(self) } @@ -141,15 +141,15 @@ private[spark] class Client( sender ! MasterChangeAcknowledged(appId) case Terminated(actor_) if actor_ == master => - logError("Connection to master failed; waiting for master to reconnect...") + logWarning("Connection to master failed; waiting for master to reconnect...") markDisconnected() case RemoteClientDisconnected(transport, address) if address == masterAddress => - logError("Connection to master failed; waiting for master to reconnect...") + logWarning("Connection to master failed; waiting for master to reconnect...") markDisconnected() case RemoteClientShutdown(transport, address) if address == masterAddress => - logError("Connection to master failed; waiting for master to reconnect...") + logWarning("Connection to master failed; waiting for master to reconnect...") markDisconnected() case StopClient => 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 index 13a6b627b..a0233a727 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -22,7 +22,11 @@ import org.apache.zookeeper._ import akka.serialization.Serialization -class ZooKeeperPersistenceEngine(serialization: Serialization) extends PersistenceEngine with SparkZooKeeperWatcher with Logging { +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) 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 8a3017e96..cb88159b8 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 @@ -71,7 +71,7 @@ private[spark] class SparkDeploySchedulerBackend( override def disconnected() { if (!stopping) { - logError("Disconnected from Spark cluster! Waiting for reconnection...") + logWarning("Disconnected from Spark cluster! Waiting for reconnection...") } } diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 5707e1927..daf04f145 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -173,31 +173,42 @@ In addition, detailed log output for each job is also written to the work direct You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the same machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on). -# Fault tolerance +# High Availability -By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new jobs can be created. In order to circumvent this, we have two fault tolerance schemes, detailed below. +By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below. -**Possible gotcha:** Production-level fault tolerance is enabled by having multiple Master nodes tied together with ZooKeeper. If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will start scheduling independently). +## Standby Masters with ZooKeeper -## Standby masters with ZooKeeper +**Overview** + +Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ applications -- applications that were already running during Master failover are unaffected. + +Learn more about getting started with ZooKeeper [here](http://zookeeper.apache.org/doc/trunk/zookeeperStarted.html). + +**Possible gotcha:** If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently). **Configuration** # May be configured as SPARK_DAEMON_JAVA_OPTS in spark-env.sh spark.deploy.recoveryMode=ZOOKEEPER - spark.deploy.zookeeper.url=ZK_URL:ZK_PORT # eg 192.168.1.100:2181 + spark.deploy.zookeeper.url=ZK_URL1:ZK_PORT1,ZK_URL2:ZK_PORT2 # eg 192.168.1.100:2181,192.168.1.101:2181 spark.deploy.zookeeper.dir=/spark # OPTIONAL! /spark is the default. **Details** -Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ jobs -- jobs that were already running during Master failover are unaffected. +After you have a ZooKeeper cluster set up, enabling high availability is straightforward. Simply start multiple Master processes on different nodes with the same ZooKeeper configuration (ZooKeeper URL and directory). Masters can be added and removed at any time. + +In order to schedule new applications or add Workers to the cluster, they need to know the IP address of the current leader. This can be accomplished by simply passing in a list of Masters where you used to pass in a single one. For example, you might start your SparkContext pointing to ``spark://host1:port1,host2:port2``. This would cause your SparkContext to try registering with both Masters -- if ``host1`` goes down, this configuration would still be correct as we'd find the new leader, ``host2``. -In order to schedule new jobs or add Workers to the cluster, they need to know the IP address of the current leader. This can be accomplished by simply passing in a list of Masters where you used to pass in a single one. For example, you might start your SparkContext pointing to ``spark://host1:port1,host2:port2``. This would cause your SparkContext to try registering with both Masters -- if host1 goes down, this configuration would still be correct as we'd find the new leader, host2. +There's an important distinction to be made between "registering with a Master" and normal operation. When starting up, an application or Worker needs to be able to find and register with the current lead Master. Once it successfully registers, though, it is "in the system" (i.e., stored in ZooKeeper). If failover occurs, the new leader will contact all previously registered applications and Workers to inform them of the change in leadership, so they need not even have known of the existence of the new Master at startup. -There's an important distinction to be made between "registering with a Master" and normal operation. When starting up, a SparkContext or Worker needs to be able to find and register with the current lead Master. Once it successfully registers, though, it is "in the system" (i.e., stored in ZooKeeper). If failover occurs, the new leader will contact all previously registered SparkContexts and Workers to inform them of the change in leadership, so they need not have even known of the existence of the new Master at startup. -
This means that new Masters can be created at any time, and the only thing you need to worry about is that _new_ SparkContexts and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of. +Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of. -## Single-node recovery with local file system +## Single-Node Recovery with Local File System + +**Overview** + +ZooKeeper is the best way to go for production-level high availability, but if you just want to be able to restart the Master if it goes down, FILESYSTEM mode can take care of it. When applications and Workers register, they have enough state written to the provided directory so that they can be recovered upon a restart of the Master process. **Configuration** @@ -207,10 +218,6 @@ There's an important distinction to be made between "registering with a Master" **Details** -ZooKeeper is the best way to go for production-level fault tolerance, but if you just want to be able to restart the Master if it goes down, FILESYSTEM mode can take care of it. When SparkContexts and Workers register, they have enough state written to the provided directory so that they can be recovered upon a restart of the Master process. - -**Notes:** - * This solution can be used in tandem with a process monitor/manager like [monit](http://mmonit.com/monit/), or just to enable manual recovery via restart. * While filesystem recovery seems straightforwardly better than not doing any recovery at all, this mode may be suboptimal for certain development or experimental purposes. In particular, killing a master via stop-master.sh does not clean up its recovery state, so whenever you start a new Master, it will enter recovery mode. This could increase the startup time by up to 1 minute if it needs to wait for all previously-registered Workers/clients to timeout. * While it's not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/clients (equivalent to ZooKeeper recovery). Note, however, that you **cannot** have multiple Masters alive concurrently using this approach; you need to upgrade to ZooKeeper to provide leader election for that use-case. From 66c20635fa1fe18604bb4042ce31152180cb541d Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 10 Oct 2013 14:45:12 -0700 Subject: [PATCH 12/12] Minor clarification and cleanup to spark-standalone.md --- docs/spark-standalone.md | 43 ++++++++++++++++++++++++++++++---------- 1 file changed, 33 insertions(+), 10 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index daf04f145..17066ef0d 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -185,14 +185,27 @@ Utilizing ZooKeeper to provide leader election and some state storage, you can l Learn more about getting started with ZooKeeper [here](http://zookeeper.apache.org/doc/trunk/zookeeperStarted.html). -**Possible gotcha:** If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently). - **Configuration** - # May be configured as SPARK_DAEMON_JAVA_OPTS in spark-env.sh - spark.deploy.recoveryMode=ZOOKEEPER - spark.deploy.zookeeper.url=ZK_URL1:ZK_PORT1,ZK_URL2:ZK_PORT2 # eg 192.168.1.100:2181,192.168.1.101:2181 - spark.deploy.zookeeper.dir=/spark # OPTIONAL! /spark is the default. +In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration: + + + + + + + + + + + + + + + +
System propertyMeaning
spark.deploy.recoveryModeSet to ZOOKEEPER to enable standby Master recovery mode (default: NONE).
spark.deploy.zookeeper.urlThe ZooKeeper cluster url (e.g., 192.168.1.100:2181,192.168.1.101:2181).
spark.deploy.zookeeper.dirThe directory in ZooKeeper to store recovery state (default: /spark).
+ +Possible gotcha: If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently). **Details** @@ -212,12 +225,22 @@ ZooKeeper is the best way to go for production-level high availability, but if y **Configuration** - # May be configured as SPARK_DAEMON_JAVA_OPTS in spark-env.sh - spark.deploy.recoveryMode=FILESYSTEM - spark.deploy.recoveryDirectory=PATH_ACCESSIBLE_TO_MASTER +In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration: + + + + + + + + + + + +
System propertyMeaning
spark.deploy.recoveryModeSet to FILESYSTEM to enable single-node recovery mode (default: NONE).
spark.deploy.recoveryDirectoryThe directory in which Spark will store recovery state, accessible from the Master's perspective.
**Details** * This solution can be used in tandem with a process monitor/manager like [monit](http://mmonit.com/monit/), or just to enable manual recovery via restart. * While filesystem recovery seems straightforwardly better than not doing any recovery at all, this mode may be suboptimal for certain development or experimental purposes. In particular, killing a master via stop-master.sh does not clean up its recovery state, so whenever you start a new Master, it will enter recovery mode. This could increase the startup time by up to 1 minute if it needs to wait for all previously-registered Workers/clients to timeout. -* While it's not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/clients (equivalent to ZooKeeper recovery). Note, however, that you **cannot** have multiple Masters alive concurrently using this approach; you need to upgrade to ZooKeeper to provide leader election for that use-case. +* While it's not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/applications (equivalent to ZooKeeper recovery). Future applications will have to be able to find the new Master, however, in order to register.