From e3facddac50f06be40e4321d59821cf54a4ff809 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Fri, 20 Feb 2015 00:17:11 -0800 Subject: [PATCH 01/36] Add Mesos Cluster dispatcher --- .../deploy/mesos/MesosClusterDispatcher.scala | 256 ++++++++++++++ .../spark/deploy/rest/MesosRestServer.scala | 179 ++++++++++ .../apache/spark/deploy/rest/RestServer.scala | 325 +++++++++++++++++ .../deploy/rest/StandaloneRestServer.scala | 331 +++--------------- .../rest/StandaloneRestSubmitSuite.scala | 22 +- 5 files changed, 816 insertions(+), 297 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala new file mode 100644 index 0000000000000..8cf8a70b0b96b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -0,0 +1,256 @@ +/* + * 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.mesos + +import akka.actor.{Props, ActorSystem, Actor} + +import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.Some + +import org.apache.spark.deploy.rest.{ErrorResponse, SubmitRestProtocolResponse, MesosRestServer} +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, IntParam, Utils} + +import org.apache.spark.deploy.DeployMessages._ + +import org.apache.spark.deploy.DriverDescription +import org.apache.spark.deploy.master.DriverInfo +import org.apache.spark.deploy.master.DriverState.DriverState +import org.apache.spark.deploy.master.DriverState +import org.apache.spark.deploy.worker.DriverRunner + +import java.io.File +import java.util.Date +import java.text.SimpleDateFormat + + /* + * A dispatcher actor that is responsible for managing drivers, that is intended to + * used for Mesos cluster mode. + * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as + * a daemon to launch drivers as Mesos frameworks upon request. + */ +class MesosClusterDispatcher( + host: String, + serverPort: Int, + actorPort: Int, + systemName: String, + actorName: String, + conf: SparkConf, + masterUrl: String, + workDirPath: Option[String] = None) extends Actor with ActorLogReceive with Logging { + val server = new MesosRestServer(host, serverPort, self, conf, masterUrl) + + val runners = new HashMap[String, DriverRunner] + val drivers = new HashMap[String, DriverInfo] + val completedDrivers = new ArrayBuffer[DriverInfo] + val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) + var nextDriverNumber = 0 + + var workDir: File = null + + def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + + def createWorkDir() { + workDir = workDirPath.map(new File(_)).getOrElse(new File(sparkHome, "work")) + try { + // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() + // So attempting to create and then check if directory was created or not. + workDir.mkdirs() + if (!workDir.exists() || !workDir.isDirectory) { + logError("Failed to create work directory " + workDir) + System.exit(1) + } + assert (workDir.isDirectory) + } catch { + case e: Exception => + logError("Failed to create work directory " + workDir, e) + System.exit(1) + } + } + + val sparkHome = + new File(sys.env.get("SPARK_HOME").getOrElse(".")) + + val akkaUrl = AkkaUtils.address( + AkkaUtils.protocol(context.system), + systemName, + host, + actorPort, + actorName) + + def newDriverId(submitDate: Date): String = { + val appId = "driver-%s-%04d".format(createDateFormat.format(submitDate), nextDriverNumber) + nextDriverNumber += 1 + appId + } + + def createDriver(desc: DriverDescription): DriverInfo = { + val now = System.currentTimeMillis() + val date = new Date(now) + new DriverInfo(now, newDriverId(date), desc, date) + } + + override def preStart() { + createWorkDir() + server.start() + } + + override def postStop() { + server.stop() + } + + override def receiveWithLogging = { + case RequestSubmitDriver(driverDescription) => { + val driverInfo = createDriver(driverDescription) + val runner = new DriverRunner(conf, driverInfo.id, workDir, + sparkHome, driverDescription, self, akkaUrl) + runners(driverInfo.id) = runner + drivers(driverInfo.id) = driverInfo + runner.start() + sender ! SubmitDriverResponse(true, Option(driverInfo.id), "") + } + + case RequestKillDriver(driverId) => { + if (!drivers.contains(driverId)) { + if (completedDrivers.exists(_.id == driverId)) { + sender ! KillDriverResponse(driverId, false, "Driver already completed") + } else { + sender ! KillDriverResponse(driverId, false, "Unknown driver") + } + } else { + runners(driverId).kill() + sender ! KillDriverResponse(driverId, true, "") + } + } + + case RequestDriverStatus(driverId) => { + drivers.get(driverId).orElse(completedDrivers.find(_.id == driverId)) match { + case Some(driver) => + sender ! DriverStatusResponse(found = true, Some(driver.state), + None, None, driver.exception) + case None => + sender ! DriverStatusResponse(found = false, None, None, None, None) + } + } + + case DriverStateChanged(driverId, state, exception) => { + state match { + case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED => + removeDriver(driverId, state, exception) + case _ => + throw new Exception(s"Received unexpected state update for driver $driverId: $state") + } + } + } + + def removeDriver(driverId: String, state: DriverState, exception: Option[Exception]) { + if (completedDrivers.size >= RETAINED_DRIVERS) { + val toRemove = math.max(RETAINED_DRIVERS / 10, 1) + completedDrivers.trimStart(toRemove) + } + val driverInfo = drivers.remove(driverId).get + driverInfo.exception = exception + driverInfo.state = state + completedDrivers += driverInfo + } +} + +object MesosClusterDispatcher { + def main(args: Array[String]) { + val conf = new SparkConf + val clusterArgs = new ClusterDispatcherArguments(args, conf) + val actorSystem = startSystemAndActor(clusterArgs) + actorSystem.awaitTermination() + } + + def startSystemAndActor( + args: ClusterDispatcherArguments): ActorSystem = { + // The LocalSparkCluster runs multiple local sparkWorkerX actor systems + val conf = new SparkConf + val systemName = "spark-mesos-cluster" + val actorName = "MesosClusterDispatcher" + val securityMgr = new SecurityManager(conf) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + systemName, args.host, 0, conf, securityMgr) + actorSystem.actorOf( + Props(classOf[MesosClusterDispatcher], + args.host, + args.port, + boundPort, + systemName, + actorName, + conf, + args.masterUrl), + name = actorName) + actorSystem + } + + class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { + var host = Utils.localHostName() + var port = 7077 + var masterUrl: String = null + + parse(args.toList) + + def parse(args: List[String]): Unit = args match { + case ("--host" | "-h") :: value :: tail => + Utils.checkHost(value, "Please use hostname " + value) + host = value + parse(tail) + + case ("--port" | "-p") :: IntParam(value) :: tail => + port = value + parse(tail) + + case ("--master" | "-m") :: value :: tail => + if (!value.startsWith("mesos://")) { + System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") + System.exit(1) + } + masterUrl = value + parse(tail) + + case ("--help") :: tail => + printUsageAndExit(0) + + case Nil => { + if (masterUrl == null) { + System.err.println("--master is required") + System.exit(1) + } + } + + case _ => + printUsageAndExit(1) + } + + /** + * Print usage and exit JVM with the given exit code. + */ + def printUsageAndExit(exitCode: Int) { + System.err.println( + "Usage: MesosClusterDispatcher [options]\n" + + "\n" + + "Options:\n" + + " -h HOST, --host HOST Hostname to listen on\n" + + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + + " -m --master MASTER URI for connecting to Mesos master\n") + System.exit(exitCode) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala new file mode 100644 index 0000000000000..9e9e17d750cdf --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala @@ -0,0 +1,179 @@ +/* + * 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.rest + +import akka.actor.ActorRef + +import java.io.File +import javax.servlet.http.HttpServletResponse + +import org.apache.spark.deploy.{DriverDescription, DeployMessages} +import org.apache.spark.deploy.ClientArguments._ +import org.apache.spark.deploy.Command + +import org.apache.spark.{SparkConf, SPARK_VERSION => sparkVersion} +import org.apache.spark.util.{Utils, AkkaUtils} + +/** + * A server that responds to requests submitted by the [[StandaloneRestClient]]. + * This is intended to be used in Mesos cluster mode only. + * + * This server responds with different HTTP codes depending on the situation: + * 200 OK - Request was processed successfully + * 400 BAD REQUEST - Request was malformed, not successfully validated, or of unexpected type + * 468 UNKNOWN PROTOCOL VERSION - Request specified a protocol this server does not understand + * 500 INTERNAL SERVER ERROR - Server throws an exception internally while processing the request + * + * The server always includes a JSON representation of the relevant [[SubmitRestProtocolResponse]] + * in the HTTP body. If an error occurs, however, the server will include an [[ErrorResponse]] + * instead of the one expected by the client. If the construction of this error response itself + * fails, the response will consist of an empty body with a response code that indicates internal + * server error. + * + * @param host the address this server should bind to + * @param requestedPort the port this server will attempt to bind to + * @param dispatcherActor reference to the Dispatcher actor to which requests can be sent + * @param masterUrl the URL of the Master new drivers will attempt to connect to + * @param masterConf the conf used by the Master + */ +private [spark] class MesosRestServer( + host: String, + requestedPort: Int, + dispatcherActor: ActorRef, + masterConf: SparkConf, + masterUrl: String) + extends RestServer( + host, + requestedPort, + masterConf, + new MesosSubmitRequestServlet(dispatcherActor, masterUrl, masterConf), + new MesosKillRequestServlet(dispatcherActor, masterConf), + new MesosStatusRequestServlet(dispatcherActor, masterConf)) {} + +class MesosSubmitRequestServlet( + dispatcherActor: ActorRef, + masterUrl: String, + conf: SparkConf) + extends SubmitRequestServlet { + + /** + * Build a driver description from the fields specified in the submit request. + * + * This involves constructing a command that launches a mesos framework for the job. + * This does not currently consider fields used by python applications since python + * is not supported in mesos cluster mode yet. + */ + private def buildDriverDescription(request: CreateSubmissionRequest): DriverDescription = { + // Required fields, including the main class because python is not yet supported + val appResource = Option(request.appResource).getOrElse { + throw new SubmitRestMissingFieldException("Application jar is missing.") + } + val mainClass = Option(request.mainClass).getOrElse { + throw new SubmitRestMissingFieldException("Main class is missing.") + } + + // Optional fields + val sparkProperties = request.sparkProperties + val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") + val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") + val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") + val superviseDriver = sparkProperties.get("spark.driver.supervise") + val appArgs = request.appArgs + val environmentVariables = request.environmentVariables + + // Construct driver description + val conf = new SparkConf(false) + .setAll(sparkProperties) + .set("spark.master", masterUrl) + val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) + val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) + val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) + val sparkJavaOpts = Utils.sparkJavaOpts(conf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts + val command = new Command( + "org.apache.spark.deploy.SparkSubmit", + Seq("--master", masterUrl, mainClass) ++ appArgs, // args to the DriverWrapper + environmentVariables, extraClassPath, extraLibraryPath, javaOpts) + val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) + + // Cores and memory are ignored in Mesos mode. + new DriverDescription( + appResource, 0, 0, actualSuperviseDriver, command) + } + + protected override def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + requestMessage match { + case submitRequest: CreateSubmissionRequest => + val askTimeout = AkkaUtils.askTimeout(conf) + val driverDescription = buildDriverDescription(submitRequest) + val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( + DeployMessages.RequestSubmitDriver(driverDescription), dispatcherActor, askTimeout) + val submitResponse = new CreateSubmissionResponse + submitResponse.serverSparkVersion = sparkVersion + submitResponse.message = response.message + submitResponse.success = response.success + submitResponse.submissionId = response.driverId.orNull + val unknownFields = findUnknownFields(requestMessageJson, requestMessage) + if (unknownFields.nonEmpty) { + // If there are fields that the server does not know about, warn the client + submitResponse.unknownFields = unknownFields + } + submitResponse + case unexpected => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError(s"Received message of unexpected type ${unexpected.messageType}.") + } + } +} + +class MesosKillRequestServlet( + dispatcherActor: ActorRef, + conf: SparkConf) extends KillRequestServlet { + protected override def handleKill(submissionId: String): KillSubmissionResponse = { + val askTimeout = AkkaUtils.askTimeout(conf) + val response = AkkaUtils.askWithReply[DeployMessages.KillDriverResponse]( + DeployMessages.RequestKillDriver(submissionId), dispatcherActor, askTimeout) + val k = new KillSubmissionResponse + k.serverSparkVersion = sparkVersion + k.message = response.message + k.submissionId = submissionId + k.success = response.success + k + } +} + +class MesosStatusRequestServlet( + dispatcherActor: ActorRef, + conf: SparkConf) extends StatusRequestServlet { + protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { + val askTimeout = AkkaUtils.askTimeout(conf) + val response = AkkaUtils.askWithReply[DeployMessages.DriverStatusResponse]( + DeployMessages.RequestDriverStatus(submissionId), dispatcherActor, askTimeout) + val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } + val d = new SubmissionStatusResponse + d.serverSparkVersion = sparkVersion + d.submissionId = submissionId + d.success = response.found + d.driverState = response.state.map(_.toString).orNull + d.message = message.orNull + d + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala new file mode 100644 index 0000000000000..3c0137c442894 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala @@ -0,0 +1,325 @@ +/* + * 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.rest + +import java.net.InetSocketAddress +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} + +import scala.io.Source +import com.fasterxml.jackson.core.JsonProcessingException +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.servlet.{ServletHolder, ServletContextHandler} +import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} +import org.apache.spark.util.Utils + +/** + * A server that responds to requests submitted by the [[StandaloneRestClient]]. + * + * This server responds with different HTTP codes depending on the situation: + * 200 OK - Request was processed successfully + * 400 BAD REQUEST - Request was malformed, not successfully validated, or of unexpected type + * 468 UNKNOWN PROTOCOL VERSION - Request specified a protocol this server does not understand + * 500 INTERNAL SERVER ERROR - Server throws an exception internally while processing the request + * + * The server always includes a JSON representation of the relevant [[SubmitRestProtocolResponse]] + * in the HTTP body. If an error occurs, however, the server will include an [[ErrorResponse]] + * instead of the one expected by the client. If the construction of this error response itself + * fails, the response will consist of an empty body with a response code that indicates internal + * server error. + * + * @param host the address this server should bind to + * @param requestedPort the port this server will attempt to bind to + */ +private[spark] abstract class RestServer( + host: String, + requestedPort: Int, + masterConf: SparkConf, + submitRequestServlet: SubmitRequestServlet, + killRequestServlet: KillRequestServlet, + statusRequestServlet: StatusRequestServlet) + extends Logging { + + import RestServer._ + + private var _server: Option[Server] = None + + // A mapping from URL prefixes to servlets that serve them. Exposed for testing. + protected val baseContext = s"/$PROTOCOL_VERSION/submissions" + protected val contextToServlet = Map[String, RestServlet]( + s"$baseContext/create/*" -> submitRequestServlet, + s"$baseContext/kill/*" -> killRequestServlet, + s"$baseContext/status/*" -> statusRequestServlet, + "/*" -> new ErrorServlet // default handler + ) + + /** Start the server and return the bound port. */ + def start(): Int = { + val (server, boundPort) = Utils.startServiceOnPort[Server](requestedPort, doStart, masterConf) + _server = Some(server) + logInfo(s"Started REST server for submitting applications on port $boundPort") + boundPort + } + + /** + * Map the servlets to their corresponding contexts and attach them to a server. + * Return a 2-tuple of the started server and the bound port. + */ + private def doStart(startPort: Int): (Server, Int) = { + val server = new Server(new InetSocketAddress(host, startPort)) + val threadPool = new QueuedThreadPool + threadPool.setDaemon(true) + server.setThreadPool(threadPool) + val mainHandler = new ServletContextHandler + mainHandler.setContextPath("/") + contextToServlet.foreach { case (prefix, servlet) => + mainHandler.addServlet(new ServletHolder(servlet), prefix) + } + server.setHandler(mainHandler) + server.start() + val boundPort = server.getConnectors()(0).getLocalPort + (server, boundPort) + } + + def stop(): Unit = { + _server.foreach(_.stop()) + } +} + +private[rest] object RestServer { + val PROTOCOL_VERSION = StandaloneRestClient.PROTOCOL_VERSION + val SC_UNKNOWN_PROTOCOL_VERSION = 468 +} + +/** + * An abstract servlet for handling requests passed to the [[StandaloneRestServer]]. + */ +private[rest] abstract class RestServlet extends HttpServlet with Logging { + + /** + * Serialize the given response message to JSON and send it through the response servlet. + * This validates the response before sending it to ensure it is properly constructed. + */ + protected def sendResponse( + responseMessage: SubmitRestProtocolResponse, + responseServlet: HttpServletResponse): Unit = { + val message = validateResponse(responseMessage, responseServlet) + responseServlet.setContentType("application/json") + responseServlet.setCharacterEncoding("utf-8") + responseServlet.getWriter.write(message.toJson) + } + + /** + * Return any fields in the client request message that the server does not know about. + * + * The mechanism for this is to reconstruct the JSON on the server side and compare the + * diff between this JSON and the one generated on the client side. Any fields that are + * only in the client JSON are treated as unexpected. + */ + protected def findUnknownFields( + requestJson: String, + requestMessage: SubmitRestProtocolMessage): Array[String] = { + val clientSideJson = parse(requestJson) + val serverSideJson = parse(requestMessage.toJson) + val Diff(_, _, unknown) = clientSideJson.diff(serverSideJson) + unknown match { + case j: JObject => j.obj.map { case (k, _) => k }.toArray + case _ => Array.empty[String] // No difference + } + } + + /** Return a human readable String representation of the exception. */ + protected def formatException(e: Throwable): String = { + val stackTraceString = e.getStackTrace.map { "\t" + _ }.mkString("\n") + s"$e\n$stackTraceString" + } + + /** Construct an error message to signal the fact that an exception has been thrown. */ + protected def handleError(message: String): ErrorResponse = { + val e = new ErrorResponse + e.serverSparkVersion = sparkVersion + e.message = message + e + } + + /** + * Parse a submission ID from the relative path, assuming it is the first part of the path. + * For instance, we expect the path to take the form /[submission ID]/maybe/something/else. + * The returned submission ID cannot be empty. If the path is unexpected, return None. + */ + protected def parseSubmissionId(path: String): Option[String] = { + if (path == null || path.isEmpty) { + None + } else { + path.stripPrefix("/").split("/").headOption.filter(_.nonEmpty) + } + } + + /** + * Validate the response to ensure that it is correctly constructed. + * + * If it is, simply return the message as is. Otherwise, return an error response instead + * to propagate the exception back to the client and set the appropriate error code. + */ + private def validateResponse( + responseMessage: SubmitRestProtocolResponse, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + try { + responseMessage.validate() + responseMessage + } catch { + case e: Exception => + responseServlet.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + handleError("Internal server error: " + formatException(e)) + } + } +} + +/** + * A servlet for handling kill requests passed to the [[RestServer]]. + */ +private[rest] abstract class KillRequestServlet + extends RestServlet { + + /** + * If a submission ID is specified in the URL, have the Master kill the corresponding + * driver and return an appropriate response to the client. Otherwise, return error. + */ + protected override def doPost( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + val submissionId = parseSubmissionId(request.getPathInfo) + val responseMessage = submissionId.map(handleKill).getOrElse { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Submission ID is missing in kill request.") + } + sendResponse(responseMessage, response) + } + + protected def handleKill(submissionId: String): KillSubmissionResponse +} + +/** + * A servlet for handling status requests passed to the [[RestServer]]. + */ +private[rest] abstract class StatusRequestServlet extends RestServlet { + + /** + * If a submission ID is specified in the URL, request the status of the corresponding + * driver from the Master and include it in the response. Otherwise, return error. + */ + protected override def doGet( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + val submissionId = parseSubmissionId(request.getPathInfo) + val responseMessage = submissionId.map(handleStatus).getOrElse { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Submission ID is missing in status request.") + } + sendResponse(responseMessage, response) + } + + protected def handleStatus(submissionId: String): SubmissionStatusResponse +} + +/** + * A servlet for handling submit requests passed to the [[RestServer]]. + */ +private[rest] abstract class SubmitRequestServlet extends RestServlet { + + /** + * Submit an application to the Master with parameters specified in the request. + * + * The request is assumed to be a [[SubmitRestProtocolRequest]] in the form of JSON. + * If the request is successfully processed, return an appropriate response to the + * client indicating so. Otherwise, return error instead. + */ + protected override def doPost( + requestServlet: HttpServletRequest, + responseServlet: HttpServletResponse): Unit = { + val responseMessage = + try { + val requestMessageJson = Source.fromInputStream(requestServlet.getInputStream).mkString + val requestMessage = SubmitRestProtocolMessage.fromJson(requestMessageJson) + // The response should have already been validated on the client. + // In case this is not true, validate it ourselves to avoid potential NPEs. + requestMessage.validate() + handleSubmit(requestMessageJson, requestMessage, responseServlet) + } catch { + // The client failed to provide a valid JSON, so this is not our fault + case e @ (_: JsonProcessingException | _: SubmitRestProtocolException) => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Malformed request: " + formatException(e)) + } + sendResponse(responseMessage, responseServlet) + } + + protected def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse +} + +/** + * A default servlet that handles error cases that are not captured by other servlets. + */ +private class ErrorServlet extends RestServlet { + private val serverVersion = RestServer.PROTOCOL_VERSION + + /** Service a faulty request by returning an appropriate error message to the client. */ + protected override def service( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + val path = request.getPathInfo + val parts = path.stripPrefix("/").split("/").filter(_.nonEmpty).toList + var versionMismatch = false + var msg = + parts match { + case Nil => + // http://host:port/ + "Missing protocol version." + case `serverVersion` :: Nil => + // http://host:port/correct-version + "Missing the /submissions prefix." + case `serverVersion` :: "submissions" :: tail => + // http://host:port/correct-version/submissions/* + "Missing an action: please specify one of /create, /kill, or /status." + case unknownVersion :: tail => + // http://host:port/unknown-version/* + versionMismatch = true + s"Unknown protocol version '$unknownVersion'." + case _ => + // never reached + s"Malformed path $path." + } + msg += s" Please submit requests through http://[host]:[port]/$serverVersion/submissions/..." + val error = handleError(msg) + // If there is a version mismatch, include the highest protocol version that + // this server supports in case the client wants to retry with our version + if (versionMismatch) { + error.highestProtocolVersion = serverVersion + response.setStatus(RestServer.SC_UNKNOWN_PROTOCOL_VERSION) + } else { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + } + sendResponse(error, response) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 2d6b8d4204795..948719fbaa582 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -18,18 +18,9 @@ package org.apache.spark.deploy.rest import java.io.File -import java.net.InetSocketAddress -import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} - -import scala.io.Source +import javax.servlet.http.HttpServletResponse import akka.actor.ActorRef -import com.fasterxml.jackson.core.JsonProcessingException -import org.eclipse.jetty.server.Server -import org.eclipse.jetty.servlet.{ServletHolder, ServletContextHandler} -import org.eclipse.jetty.util.thread.QueuedThreadPool -import org.json4s._ -import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} import org.apache.spark.util.{AkkaUtils, RpcUtils, Utils} @@ -64,163 +55,19 @@ private[deploy] class StandaloneRestServer( masterActor: ActorRef, masterUrl: String, masterConf: SparkConf) - extends Logging { - - import StandaloneRestServer._ - - private var _server: Option[Server] = None - - // A mapping from URL prefixes to servlets that serve them. Exposed for testing. - protected val baseContext = s"/$PROTOCOL_VERSION/submissions" - protected val contextToServlet = Map[String, StandaloneRestServlet]( - s"$baseContext/create/*" -> new SubmitRequestServlet(masterActor, masterUrl, masterConf), - s"$baseContext/kill/*" -> new KillRequestServlet(masterActor, masterConf), - s"$baseContext/status/*" -> new StatusRequestServlet(masterActor, masterConf), - "/*" -> new ErrorServlet // default handler - ) - - /** Start the server and return the bound port. */ - def start(): Int = { - val (server, boundPort) = Utils.startServiceOnPort[Server](requestedPort, doStart, masterConf) - _server = Some(server) - logInfo(s"Started REST server for submitting applications on port $boundPort") - boundPort - } - - /** - * Map the servlets to their corresponding contexts and attach them to a server. - * Return a 2-tuple of the started server and the bound port. - */ - private def doStart(startPort: Int): (Server, Int) = { - val server = new Server(new InetSocketAddress(host, startPort)) - val threadPool = new QueuedThreadPool - threadPool.setDaemon(true) - server.setThreadPool(threadPool) - val mainHandler = new ServletContextHandler - mainHandler.setContextPath("/") - contextToServlet.foreach { case (prefix, servlet) => - mainHandler.addServlet(new ServletHolder(servlet), prefix) - } - server.setHandler(mainHandler) - server.start() - val boundPort = server.getConnectors()(0).getLocalPort - (server, boundPort) - } - - def stop(): Unit = { - _server.foreach(_.stop()) - } -} - -private[rest] object StandaloneRestServer { - val PROTOCOL_VERSION = StandaloneRestClient.PROTOCOL_VERSION - val SC_UNKNOWN_PROTOCOL_VERSION = 468 -} - -/** - * An abstract servlet for handling requests passed to the [[StandaloneRestServer]]. - */ -private[rest] abstract class StandaloneRestServlet extends HttpServlet with Logging { - - /** - * Serialize the given response message to JSON and send it through the response servlet. - * This validates the response before sending it to ensure it is properly constructed. - */ - protected def sendResponse( - responseMessage: SubmitRestProtocolResponse, - responseServlet: HttpServletResponse): Unit = { - val message = validateResponse(responseMessage, responseServlet) - responseServlet.setContentType("application/json") - responseServlet.setCharacterEncoding("utf-8") - responseServlet.getWriter.write(message.toJson) - } - - /** - * Return any fields in the client request message that the server does not know about. - * - * The mechanism for this is to reconstruct the JSON on the server side and compare the - * diff between this JSON and the one generated on the client side. Any fields that are - * only in the client JSON are treated as unexpected. - */ - protected def findUnknownFields( - requestJson: String, - requestMessage: SubmitRestProtocolMessage): Array[String] = { - val clientSideJson = parse(requestJson) - val serverSideJson = parse(requestMessage.toJson) - val Diff(_, _, unknown) = clientSideJson.diff(serverSideJson) - unknown match { - case j: JObject => j.obj.map { case (k, _) => k }.toArray - case _ => Array.empty[String] // No difference - } - } - - /** Return a human readable String representation of the exception. */ - protected def formatException(e: Throwable): String = { - val stackTraceString = e.getStackTrace.map { "\t" + _ }.mkString("\n") - s"$e\n$stackTraceString" - } - - /** Construct an error message to signal the fact that an exception has been thrown. */ - protected def handleError(message: String): ErrorResponse = { - val e = new ErrorResponse - e.serverSparkVersion = sparkVersion - e.message = message - e - } - - /** - * Parse a submission ID from the relative path, assuming it is the first part of the path. - * For instance, we expect the path to take the form /[submission ID]/maybe/something/else. - * The returned submission ID cannot be empty. If the path is unexpected, return None. - */ - protected def parseSubmissionId(path: String): Option[String] = { - if (path == null || path.isEmpty) { - None - } else { - path.stripPrefix("/").split("/").headOption.filter(_.nonEmpty) - } - } - - /** - * Validate the response to ensure that it is correctly constructed. - * - * If it is, simply return the message as is. Otherwise, return an error response instead - * to propagate the exception back to the client and set the appropriate error code. - */ - private def validateResponse( - responseMessage: SubmitRestProtocolResponse, - responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { - try { - responseMessage.validate() - responseMessage - } catch { - case e: Exception => - responseServlet.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR) - handleError("Internal server error: " + formatException(e)) - } - } -} + extends RestServer( + host, + requestedPort, + masterConf, + new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf), + new StandaloneKillRequestServlet(masterActor, masterConf), + new StandaloneStatusRequestServlet(masterActor, masterConf)) {} /** * A servlet for handling kill requests passed to the [[StandaloneRestServer]]. */ -private[rest] class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) - extends StandaloneRestServlet { - - /** - * If a submission ID is specified in the URL, have the Master kill the corresponding - * driver and return an appropriate response to the client. Otherwise, return error. - */ - protected override def doPost( - request: HttpServletRequest, - response: HttpServletResponse): Unit = { - val submissionId = parseSubmissionId(request.getPathInfo) - val responseMessage = submissionId.map(handleKill).getOrElse { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError("Submission ID is missing in kill request.") - } - sendResponse(responseMessage, response) - } +private[rest] class StandaloneKillRequestServlet(masterActor: ActorRef, conf: SparkConf) + extends KillRequestServlet { protected def handleKill(submissionId: String): KillSubmissionResponse = { val askTimeout = RpcUtils.askTimeout(conf) @@ -238,23 +85,8 @@ private[rest] class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) /** * A servlet for handling status requests passed to the [[StandaloneRestServer]]. */ -private[rest] class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) - extends StandaloneRestServlet { - - /** - * If a submission ID is specified in the URL, request the status of the corresponding - * driver from the Master and include it in the response. Otherwise, return error. - */ - protected override def doGet( - request: HttpServletRequest, - response: HttpServletResponse): Unit = { - val submissionId = parseSubmissionId(request.getPathInfo) - val responseMessage = submissionId.map(handleStatus).getOrElse { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError("Submission ID is missing in status request.") - } - sendResponse(responseMessage, response) - } +private[rest] class StandaloneStatusRequestServlet(masterActor: ActorRef, conf: SparkConf) + extends StatusRequestServlet { protected def handleStatus(submissionId: String): SubmissionStatusResponse = { val askTimeout = RpcUtils.askTimeout(conf) @@ -276,71 +108,11 @@ private[rest] class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) /** * A servlet for handling submit requests passed to the [[StandaloneRestServer]]. */ -private[rest] class SubmitRequestServlet( +private[rest] class StandaloneSubmitRequestServlet( masterActor: ActorRef, masterUrl: String, conf: SparkConf) - extends StandaloneRestServlet { - - /** - * Submit an application to the Master with parameters specified in the request. - * - * The request is assumed to be a [[SubmitRestProtocolRequest]] in the form of JSON. - * If the request is successfully processed, return an appropriate response to the - * client indicating so. Otherwise, return error instead. - */ - protected override def doPost( - requestServlet: HttpServletRequest, - responseServlet: HttpServletResponse): Unit = { - val responseMessage = - try { - val requestMessageJson = Source.fromInputStream(requestServlet.getInputStream).mkString - val requestMessage = SubmitRestProtocolMessage.fromJson(requestMessageJson) - // The response should have already been validated on the client. - // In case this is not true, validate it ourselves to avoid potential NPEs. - requestMessage.validate() - handleSubmit(requestMessageJson, requestMessage, responseServlet) - } catch { - // The client failed to provide a valid JSON, so this is not our fault - case e @ (_: JsonProcessingException | _: SubmitRestProtocolException) => - responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError("Malformed request: " + formatException(e)) - } - sendResponse(responseMessage, responseServlet) - } - - /** - * Handle the submit request and construct an appropriate response to return to the client. - * - * This assumes that the request message is already successfully validated. - * If the request message is not of the expected type, return error to the client. - */ - private def handleSubmit( - requestMessageJson: String, - requestMessage: SubmitRestProtocolMessage, - responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { - requestMessage match { - case submitRequest: CreateSubmissionRequest => - val askTimeout = RpcUtils.askTimeout(conf) - val driverDescription = buildDriverDescription(submitRequest) - val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( - DeployMessages.RequestSubmitDriver(driverDescription), masterActor, askTimeout) - val submitResponse = new CreateSubmissionResponse - submitResponse.serverSparkVersion = sparkVersion - submitResponse.message = response.message - submitResponse.success = response.success - submitResponse.submissionId = response.driverId.orNull - val unknownFields = findUnknownFields(requestMessageJson, requestMessage) - if (unknownFields.nonEmpty) { - // If there are fields that the server does not know about, warn the client - submitResponse.unknownFields = unknownFields - } - submitResponse - case unexpected => - responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError(s"Received message of unexpected type ${unexpected.messageType}.") - } - } + extends SubmitRequestServlet { /** * Build a driver description from the fields specified in the submit request. @@ -389,50 +161,37 @@ private[rest] class SubmitRequestServlet( new DriverDescription( appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command) } -} - -/** - * A default servlet that handles error cases that are not captured by other servlets. - */ -private class ErrorServlet extends StandaloneRestServlet { - private val serverVersion = StandaloneRestServer.PROTOCOL_VERSION - /** Service a faulty request by returning an appropriate error message to the client. */ - protected override def service( - request: HttpServletRequest, - response: HttpServletResponse): Unit = { - val path = request.getPathInfo - val parts = path.stripPrefix("/").split("/").filter(_.nonEmpty).toList - var versionMismatch = false - var msg = - parts match { - case Nil => - // http://host:port/ - "Missing protocol version." - case `serverVersion` :: Nil => - // http://host:port/correct-version - "Missing the /submissions prefix." - case `serverVersion` :: "submissions" :: tail => - // http://host:port/correct-version/submissions/* - "Missing an action: please specify one of /create, /kill, or /status." - case unknownVersion :: tail => - // http://host:port/unknown-version/* - versionMismatch = true - s"Unknown protocol version '$unknownVersion'." - case _ => - // never reached - s"Malformed path $path." - } - msg += s" Please submit requests through http://[host]:[port]/$serverVersion/submissions/..." - val error = handleError(msg) - // If there is a version mismatch, include the highest protocol version that - // this server supports in case the client wants to retry with our version - if (versionMismatch) { - error.highestProtocolVersion = serverVersion - response.setStatus(StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION) - } else { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + /** + * Handle the submit request and construct an appropriate response to return to the client. + * + * This assumes that the request message is already successfully validated. + * If the request message is not of the expected type, return error to the client. + */ + protected override def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + requestMessage match { + case submitRequest: CreateSubmissionRequest => + val askTimeout = AkkaUtils.askTimeout(conf) + val driverDescription = buildDriverDescription(submitRequest) + val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( + DeployMessages.RequestSubmitDriver(driverDescription), masterActor, askTimeout) + val submitResponse = new CreateSubmissionResponse + submitResponse.serverSparkVersion = sparkVersion + submitResponse.message = response.message + submitResponse.success = response.success + submitResponse.submissionId = response.driverId.orNull + val unknownFields = findUnknownFields(requestMessageJson, requestMessage) + if (unknownFields.nonEmpty) { + // If there are fields that the server does not know about, warn the client + submitResponse.unknownFields = unknownFields + } + submitResponse + case unexpected => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError(s"Received message of unexpected type ${unexpected.messageType}.") } - sendResponse(error, response) } -} +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 8e09976636386..c93d307262ae0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -208,7 +208,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("good request paths") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestServer.PROTOCOL_VERSION val json = constructSubmitRequest(masterUrl).toJson val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill" @@ -238,7 +238,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("good request paths, bad requests") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill" val statusRequestPath = s"$httpUrl/$v/submissions/status" @@ -276,7 +276,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("bad request paths") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestServer.PROTOCOL_VERSION val (response1, code1) = sendHttpRequestWithResponse(httpUrl, "GET") val (response2, code2) = sendHttpRequestWithResponse(s"$httpUrl/", "GET") val (response3, code3) = sendHttpRequestWithResponse(s"$httpUrl/$v", "GET") @@ -292,7 +292,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { assert(code5 === HttpServletResponse.SC_BAD_REQUEST) assert(code6 === HttpServletResponse.SC_BAD_REQUEST) assert(code7 === HttpServletResponse.SC_BAD_REQUEST) - assert(code8 === StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION) + assert(code8 === RestServer.SC_UNKNOWN_PROTOCOL_VERSION) // all responses should be error responses val errorResponse1 = getErrorResponse(response1) val errorResponse2 = getErrorResponse(response2) @@ -310,13 +310,13 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { assert(errorResponse5.highestProtocolVersion === null) assert(errorResponse6.highestProtocolVersion === null) assert(errorResponse7.highestProtocolVersion === null) - assert(errorResponse8.highestProtocolVersion === StandaloneRestServer.PROTOCOL_VERSION) + assert(errorResponse8.highestProtocolVersion === RestServer.PROTOCOL_VERSION) } test("server returns unknown fields") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val oldJson = constructSubmitRequest(masterUrl).toJson val oldFields = parse(oldJson).asInstanceOf[JObject].obj @@ -340,7 +340,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("client handles faulty server") { val masterUrl = startFaultyServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill/anything" val statusRequestPath = s"$httpUrl/$v/submissions/status/anything" @@ -568,7 +568,7 @@ private class FaultyStandaloneRestServer( masterConf: SparkConf) extends StandaloneRestServer(host, requestedPort, masterActor, masterUrl, masterConf) { - protected override val contextToServlet = Map[String, StandaloneRestServlet]( + protected override val contextToServlet = Map[String, RestServlet]( s"$baseContext/create/*" -> new MalformedSubmitServlet, s"$baseContext/kill/*" -> new InvalidKillServlet, s"$baseContext/status/*" -> new ExplodingStatusServlet, @@ -576,7 +576,7 @@ private class FaultyStandaloneRestServer( ) /** A faulty servlet that produces malformed responses. */ - class MalformedSubmitServlet extends SubmitRequestServlet(masterActor, masterUrl, masterConf) { + class MalformedSubmitServlet extends StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) { protected override def sendResponse( responseMessage: SubmitRestProtocolResponse, responseServlet: HttpServletResponse): Unit = { @@ -586,7 +586,7 @@ private class FaultyStandaloneRestServer( } /** A faulty servlet that produces invalid responses. */ - class InvalidKillServlet extends KillRequestServlet(masterActor, masterConf) { + class InvalidKillServlet extends StandaloneKillRequestServlet(masterActor, masterConf) { protected override def handleKill(submissionId: String): KillSubmissionResponse = { val k = super.handleKill(submissionId) k.submissionId = null @@ -595,7 +595,7 @@ private class FaultyStandaloneRestServer( } /** A faulty status servlet that explodes. */ - class ExplodingStatusServlet extends StatusRequestServlet(masterActor, masterConf) { + class ExplodingStatusServlet extends StandaloneStatusRequestServlet(masterActor, masterConf) { private def explode: Int = 1 / 0 protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { val s = super.handleStatus(submissionId) From 67cbc18cba9ddde141078af19af5392c1ef35fe9 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 24 Feb 2015 00:46:21 -0800 Subject: [PATCH 02/36] Rename StandaloneRestClient to RestClient and add sbin scripts --- .../org/apache/spark/deploy/SparkSubmit.scala | 21 +++++++--- .../spark/deploy/SparkSubmitArguments.scala | 11 ++++-- .../deploy/mesos/MesosClusterDispatcher.scala | 5 ++- .../spark/deploy/rest/MesosRestServer.scala | 11 ++++-- ...aloneRestClient.scala => RestClient.scala} | 32 ++++++++++------ .../apache/spark/deploy/rest/RestServer.scala | 4 +- .../deploy/rest/StandaloneRestServer.scala | 4 +- .../spark/deploy/SparkSubmitSuite.scala | 2 +- .../rest/StandaloneRestSubmitSuite.scala | 4 +- sbin/start-mesos-dispatcher.sh | 38 +++++++++++++++++++ sbin/stop-mesos-dispatcher.sh | 28 ++++++++++++++ 11 files changed, 126 insertions(+), 34 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/rest/{StandaloneRestClient.scala => RestClient.scala} (93%) create mode 100755 sbin/start-mesos-dispatcher.sh create mode 100755 sbin/stop-mesos-dispatcher.sh diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 296a0764b8baf..afce8ccf4a610 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -116,7 +116,7 @@ object SparkSubmit { /** Kill an existing submission using the REST protocol. Standalone cluster mode only. */ private def kill(args: SparkSubmitArguments): Unit = { - new StandaloneRestClient() + new RestClient() .killSubmission(args.master, args.submissionToKill) } @@ -125,7 +125,7 @@ object SparkSubmit { * Standalone cluster mode only. */ private def requestStatus(args: SparkSubmitArguments): Unit = { - new StandaloneRestClient() + new RestClient() .requestSubmissionStatus(args.master, args.submissionToRequestStatusFor) } @@ -252,6 +252,7 @@ object SparkSubmit { } val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER + val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files // too for packages that include Python code @@ -294,8 +295,9 @@ object SparkSubmit { // The following modes are not supported or applicable (clusterManager, deployMode) match { - case (MESOS, CLUSTER) => - printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.") + case (MESOS, CLUSTER) if args.isPython => + printErrorAndExit("Cluster deploy mode is currently not supported for python " + + "on Mesos clusters.") case (STANDALONE, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + "applications on standalone clusters.") @@ -452,7 +454,7 @@ object SparkSubmit { // All Spark parameters are expected to be passed to the client through system properties. if (args.isStandaloneCluster) { if (args.useRest) { - childMainClass = "org.apache.spark.deploy.rest.StandaloneRestClient" + childMainClass = "org.apache.spark.deploy.rest.RestClient" childArgs += (args.primaryResource, args.mainClass) } else { // In legacy standalone cluster mode, use Client as a wrapper around the user class @@ -496,6 +498,15 @@ object SparkSubmit { } } + if (isMesosCluster) { + // Mesos cluster dispatcher only supports the REST interface + childMainClass = "org.apache.spark.deploy.rest.RestClient" + childArgs += (args.primaryResource, args.mainClass) + if (args.childArgs != null) { + childArgs ++= args.childArgs + } + } + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sysProps.getOrElseUpdate(k, v) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index faa8780288ea3..2e73704f2c1f0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -231,8 +231,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateKillArguments(): Unit = { - if (!master.startsWith("spark://")) { - SparkSubmit.printErrorAndExit("Killing submissions is only supported in standalone mode!") + if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { + SparkSubmit.printErrorAndExit( + "Killing submissions is only supported in standalone or Mesos mode!") } if (submissionToKill == null) { SparkSubmit.printErrorAndExit("Please specify a submission to kill.") @@ -240,9 +241,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateStatusRequestArguments(): Unit = { - if (!master.startsWith("spark://")) { + if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { SparkSubmit.printErrorAndExit( - "Requesting submission statuses is only supported in standalone mode!") + "Requesting submission statuses is only supported in standalone or Mesos mode!") } if (submissionToRequestStatusFor == null) { SparkSubmit.printErrorAndExit("Please specify a submission to request status for.") @@ -475,6 +476,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). + | + | Spark standalone or Mesos with cluster deploy mode only: | --supervise If given, restarts the driver on failure. | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 8cf8a70b0b96b..0a8c7cb86b650 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -22,7 +22,7 @@ import akka.actor.{Props, ActorSystem, Actor} import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.Some -import org.apache.spark.deploy.rest.{ErrorResponse, SubmitRestProtocolResponse, MesosRestServer} +import org.apache.spark.deploy.rest.MesosRestServer import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{ActorLogReceive, AkkaUtils, IntParam, Utils} @@ -195,7 +195,8 @@ object MesosClusterDispatcher { systemName, actorName, conf, - args.masterUrl), + args.masterUrl, + None), name = actorName) actorSystem } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala index 9e9e17d750cdf..5d7b9c9443c12 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala @@ -30,7 +30,7 @@ import org.apache.spark.{SparkConf, SPARK_VERSION => sparkVersion} import org.apache.spark.util.{Utils, AkkaUtils} /** - * A server that responds to requests submitted by the [[StandaloneRestClient]]. + * A server that responds to requests submitted by the [[RestClient]]. * This is intended to be used in Mesos cluster mode only. * * This server responds with different HTTP codes depending on the situation: @@ -93,6 +93,7 @@ class MesosSubmitRequestServlet( val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") val superviseDriver = sparkProperties.get("spark.driver.supervise") + val driverMemory = sparkProperties.get("spark.driver.memory") val appArgs = request.appArgs val environmentVariables = request.environmentVariables @@ -107,13 +108,15 @@ class MesosSubmitRequestServlet( val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = new Command( "org.apache.spark.deploy.SparkSubmit", - Seq("--master", masterUrl, mainClass) ++ appArgs, // args to the DriverWrapper + Seq("--master", masterUrl, "--class", mainClass, appResource) ++ appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) + val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) - // Cores and memory are ignored in Mesos mode. + // Cores and memory are ignored in Mesos mode. We override the memory to be driver memory + // as the memory is used for JVM heap size of the driver later. new DriverDescription( - appResource, 0, 0, actualSuperviseDriver, command) + appResource, actualDriverMemory, 0, actualSuperviseDriver, command) } protected override def handleSubmit( diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestClient.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala rename to core/src/main/scala/org/apache/spark/deploy/rest/RestClient.scala index b8fd406fb6f9a..8b9239c3b093f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestClient.scala @@ -30,8 +30,10 @@ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} import org.apache.spark.util.Utils /** - * A client that submits applications to the standalone Master using a REST protocol. - * This client is intended to communicate with the [[StandaloneRestServer]] and is + * A client that submits applications to the standalone Master and Mesos dispatcher + * using a REST protocol. + * This client is intended to communicate with the [[StandaloneRestServer]] or + * [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]] and is * currently used for cluster mode only. * * In protocol version v1, the REST URL takes the form http://[host:port]/v1/submissions/[action], @@ -53,8 +55,8 @@ import org.apache.spark.util.Utils * implementation of this client can use that information to retry using the version specified * by the server. */ -private[deploy] class StandaloneRestClient extends Logging { - import StandaloneRestClient._ +private[spark] class RestClient extends Logging { + import RestClient._ /** * Submit an application specified by the parameters in the provided request. @@ -62,7 +64,7 @@ private[deploy] class StandaloneRestClient extends Logging { * If the submission was successful, poll the status of the submission and report * it to the user. Otherwise, report the error message provided by the server. */ - private[rest] def createSubmission( + def createSubmission( master: String, request: CreateSubmissionRequest): SubmitRestProtocolResponse = { logInfo(s"Submitting a request to launch an application in $master.") @@ -107,7 +109,7 @@ private[deploy] class StandaloneRestClient extends Logging { } /** Construct a message that captures the specified parameters for submitting an application. */ - private[rest] def constructSubmitRequest( + def constructSubmitRequest( appResource: String, mainClass: String, appArgs: Array[String], @@ -219,14 +221,20 @@ private[deploy] class StandaloneRestClient extends Logging { /** Return the base URL for communicating with the server, including the protocol version. */ private def getBaseUrl(master: String): String = { - val masterUrl = master.stripPrefix("spark://").stripSuffix("/") + val masterUrl = if (master.startsWith("spark://")) { + master.stripPrefix("spark://").stripSuffix("/") + } else { + master.stripPrefix("mesos://").stripSuffix("/") + } + s"http://$masterUrl/$PROTOCOL_VERSION/submissions" } /** Throw an exception if this is not standalone mode. */ private def validateMaster(master: String): Unit = { - if (!master.startsWith("spark://")) { - throw new IllegalArgumentException("This REST client is only supported in standalone mode.") + if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { + throw new IllegalArgumentException( + "This REST client is only supported in standalone or Mesos mode.") } } @@ -295,7 +303,7 @@ private[deploy] class StandaloneRestClient extends Logging { } } -private[rest] object StandaloneRestClient { +private[spark] object RestClient { private val REPORT_DRIVER_STATUS_INTERVAL = 1000 private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" @@ -315,7 +323,7 @@ private[rest] object StandaloneRestClient { } val sparkProperties = conf.getAll.toMap val environmentVariables = env.filter { case (k, _) => k.startsWith("SPARK_") } - val client = new StandaloneRestClient + val client = new RestClient val submitRequest = client.constructSubmitRequest( appResource, mainClass, appArgs, sparkProperties, environmentVariables) client.createSubmission(master, submitRequest) @@ -323,7 +331,7 @@ private[rest] object StandaloneRestClient { def main(args: Array[String]): Unit = { if (args.size < 2) { - sys.error("Usage: StandaloneRestClient [app resource] [main class] [app args*]") + sys.error("Usage: RestClient [app resource] [main class] [app args*]") sys.exit(1) } val appResource = args(0) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala index 3c0137c442894..513a0cfa1b993 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala @@ -32,7 +32,7 @@ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} import org.apache.spark.util.Utils /** - * A server that responds to requests submitted by the [[StandaloneRestClient]]. + * A server that responds to requests submitted by the [[RestClient]]. * * This server responds with different HTTP codes depending on the situation: * 200 OK - Request was processed successfully @@ -105,7 +105,7 @@ private[spark] abstract class RestServer( } private[rest] object RestServer { - val PROTOCOL_VERSION = StandaloneRestClient.PROTOCOL_VERSION + val PROTOCOL_VERSION = RestClient.PROTOCOL_VERSION val SC_UNKNOWN_PROTOCOL_VERSION = 468 } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 948719fbaa582..1f3c3e97e9818 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ /** - * A server that responds to requests submitted by the [[StandaloneRestClient]]. + * A server that responds to requests submitted by the [[RestClient]]. * This is intended to be embedded in the standalone Master and used in cluster mode only. * * This server responds with different HTTP codes depending on the situation: @@ -194,4 +194,4 @@ private[rest] class StandaloneSubmitRequestServlet( handleError(s"Received message of unexpected type ${unexpected.messageType}.") } } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 4561e5b8e9663..52808cd713b4d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -231,7 +231,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties val childArgsStr = childArgs.mkString(" ") if (useRest) { childArgsStr should endWith ("thejar.jar org.SomeClass arg1 arg2") - mainClass should be ("org.apache.spark.deploy.rest.StandaloneRestClient") + mainClass should be ("org.apache.spark.deploy.rest.RestClient") } else { childArgsStr should startWith ("--supervise --memory 4g --cores 5") childArgsStr should include regex "launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2" diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index c93d307262ae0..b58fe426c705b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.deploy.master.DriverState._ * Tests for the REST application submission protocol used in standalone cluster mode. */ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { - private val client = new StandaloneRestClient + private val client = new RestClient private var actorSystem: Option[ActorSystem] = None private var server: Option[StandaloneRestServer] = None @@ -89,7 +89,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { conf.set("spark.app.name", "dreamer") val appArgs = Array("one", "two", "six") // main method calls this - val response = StandaloneRestClient.run("app-resource", "main-class", appArgs, conf) + val response = RestClient.run("app-resource", "main-class", appArgs, conf) val submitResponse = getSubmitResponse(response) assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) assert(submitResponse.serverSparkVersion === SPARK_VERSION) diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh new file mode 100755 index 0000000000000..22c002f716f41 --- /dev/null +++ b/sbin/start-mesos-dispatcher.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env 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. +# + +# Starts the Mesos Cluster Dispatcher on the machine this script is executed on. + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +if [ "$SPARK_MESOS_DISPATCHER_PORT" = "" ]; then + SPARK_MESOS_DISPATCHER_PORT=8077 +fi + +if [ "$SPARK_MESOS_DISPATCHER_HOST" = "" ]; then + SPARK_MESOS_DISPATCHER_HOST=`hostname` +fi + + +"$sbin"/spark-daemon.sh start org.apache.spark.deploy.mesos.MesosClusterDispatcher 1 --host $SPARK_MESOS_DISPATCHER_HOST --port $SPARK_MESOS_DISPATCHER_PORT "$@" diff --git a/sbin/stop-mesos-dispatcher.sh b/sbin/stop-mesos-dispatcher.sh new file mode 100755 index 0000000000000..d9adfe120d269 --- /dev/null +++ b/sbin/stop-mesos-dispatcher.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env 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. +# + +# Starts the master on the machine this script is executed on. + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +. "$sbin/spark-config.sh" + +"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.mesos.MesosClusterDispatcher 1 + From 99867310b0f151c36375ec6ac135c8702906ecab Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 24 Feb 2015 14:50:25 -0800 Subject: [PATCH 03/36] Kill drivers when shutdown --- .../deploy/mesos/MesosClusterDispatcher.scala | 47 ++++++++++++++++--- 1 file changed, 40 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 0a8c7cb86b650..b61574c73fed2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -34,16 +34,17 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.DriverRunner -import java.io.File +import java.io.{IOException, File} import java.util.Date import java.text.SimpleDateFormat +import scala.collection.mutable - /* - * A dispatcher actor that is responsible for managing drivers, that is intended to - * used for Mesos cluster mode. - * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as - * a daemon to launch drivers as Mesos frameworks upon request. - */ +/* + * A dispatcher actor that is responsible for managing drivers, that is intended to + * used for Mesos cluster mode. + * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as + * a daemon to launch drivers as Mesos frameworks upon request. + */ class MesosClusterDispatcher( host: String, serverPort: Int, @@ -67,6 +68,21 @@ class MesosClusterDispatcher( def createWorkDir() { workDir = workDirPath.map(new File(_)).getOrElse(new File(sparkHome, "work")) + + // Attempt to remove the work directory if it exists on startup. + // This is to avoid unbounded growing the work directory as drivers + // are only deleted when it is over the retained count while it's running. + // We don't fail startup if we are not able to remove, as this is + // a short-term solution. + try { + if (workDir.exists()) { + workDir.delete() + } + } catch { + case e: IOException => + logError("Unable to remove work directory " + workDir, e) + } + try { // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() // So attempting to create and then check if directory was created or not. @@ -112,6 +128,7 @@ class MesosClusterDispatcher( override def postStop() { server.stop() + runners.values.foreach(_.kill()) } override def receiveWithLogging = { @@ -161,6 +178,15 @@ class MesosClusterDispatcher( def removeDriver(driverId: String, state: DriverState, exception: Option[Exception]) { if (completedDrivers.size >= RETAINED_DRIVERS) { val toRemove = math.max(RETAINED_DRIVERS / 10, 1) + for (i <- 0 to (toRemove - 1)) { + val driverId = completedDrivers(i).id + try { + new File(workDir, driverId).delete() + } catch { + case e: Exception => + logWarning("Unable to remove work dir for completed driver " + driverId, e) + } + } completedDrivers.trimStart(toRemove) } val driverInfo = drivers.remove(driverId).get @@ -175,6 +201,13 @@ object MesosClusterDispatcher { val conf = new SparkConf val clusterArgs = new ClusterDispatcherArguments(args, conf) val actorSystem = startSystemAndActor(clusterArgs) + Runtime.getRuntime().addShutdownHook(new Thread("MesosClusterDispatcherShutdownHook") { + override def run() = { + // Makes sure we shut down the actor, which will kill all the drivers. + actorSystem.shutdown() + actorSystem.awaitTermination() + } + }) actorSystem.awaitTermination() } From 880bc276c9deecff60ef1049f6f53ebc523c96e6 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Thu, 26 Feb 2015 00:20:16 -0800 Subject: [PATCH 04/36] Add Mesos Cluster UI to display driver results --- .../deploy/mesos/MesosClusterDispatcher.scala | 72 ++++++++++++------ .../deploy/mesos/ui/DriverOutputPage.scala | 75 +++++++++++++++++++ .../deploy/mesos/ui/MesosClusterUI.scala | 56 ++++++++++++++ .../apache/spark/deploy/worker/Worker.scala | 10 +-- .../spark/deploy/worker/ui/LogPage.scala | 6 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- 6 files changed, 193 insertions(+), 28 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index b61574c73fed2..c791d13b58265 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -37,7 +37,17 @@ import org.apache.spark.deploy.worker.DriverRunner import java.io.{IOException, File} import java.util.Date import java.text.SimpleDateFormat -import scala.collection.mutable +import org.apache.spark.deploy.mesos.ui.MesosClusterUI +import org.apache.spark.deploy.mesos.Messages.{DispatcherStateResponse, RequestDispatcherState} + +private [deploy] object Messages { + case object RequestDispatcherState + + case class DispatcherStateResponse( + activeDrivers: Iterable[DriverInfo], + completedDrivers: Iterable[DriverInfo]) { + } +} /* * A dispatcher actor that is responsible for managing drivers, that is intended to @@ -45,10 +55,11 @@ import scala.collection.mutable * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as * a daemon to launch drivers as Mesos frameworks upon request. */ -class MesosClusterDispatcher( +private [spark] class MesosClusterDispatcher( host: String, serverPort: Int, actorPort: Int, + webUiPort: Int, systemName: String, actorName: String, conf: SparkConf, @@ -56,18 +67,13 @@ class MesosClusterDispatcher( workDirPath: Option[String] = None) extends Actor with ActorLogReceive with Logging { val server = new MesosRestServer(host, serverPort, self, conf, masterUrl) - val runners = new HashMap[String, DriverRunner] - val drivers = new HashMap[String, DriverInfo] - val completedDrivers = new ArrayBuffer[DriverInfo] - val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) - var nextDriverNumber = 0 - - var workDir: File = null - - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + val dispatcherPublicAddress = { + val envVar = System.getenv("SPARK_PUBLIC_DNS") + if (envVar != null) envVar else host + } - def createWorkDir() { - workDir = workDirPath.map(new File(_)).getOrElse(new File(sparkHome, "work")) + lazy val workDir: File = { + val dir = workDirPath.map(new File(_)).getOrElse(new File(sparkHome, "work")) // Attempt to remove the work directory if it exists on startup. // This is to avoid unbounded growing the work directory as drivers @@ -75,8 +81,8 @@ class MesosClusterDispatcher( // We don't fail startup if we are not able to remove, as this is // a short-term solution. try { - if (workDir.exists()) { - workDir.delete() + if (dir.exists()) { + dir.delete() } } catch { case e: IOException => @@ -86,19 +92,31 @@ class MesosClusterDispatcher( try { // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() // So attempting to create and then check if directory was created or not. - workDir.mkdirs() - if (!workDir.exists() || !workDir.isDirectory) { - logError("Failed to create work directory " + workDir) + dir.mkdirs() + if (!dir.exists() || !dir.isDirectory) { + logError("Failed to create work directory " + dir) System.exit(1) } - assert (workDir.isDirectory) + assert (dir.isDirectory) } catch { case e: Exception => - logError("Failed to create work directory " + workDir, e) + logError("Failed to create work directory " + dir, e) System.exit(1) } + dir } + val webUi = new MesosClusterUI( + self, new SecurityManager(conf), webUiPort, conf, workDir, dispatcherPublicAddress) + + val runners = new HashMap[String, DriverRunner] + val drivers = new HashMap[String, DriverInfo] + val completedDrivers = new ArrayBuffer[DriverInfo] + val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) + var nextDriverNumber = 0 + + def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + val sparkHome = new File(sys.env.get("SPARK_HOME").getOrElse(".")) @@ -122,11 +140,12 @@ class MesosClusterDispatcher( } override def preStart() { - createWorkDir() server.start() + webUi.bind() } override def postStop() { + webUi.stop() server.stop() runners.values.foreach(_.kill()) } @@ -173,6 +192,10 @@ class MesosClusterDispatcher( throw new Exception(s"Received unexpected state update for driver $driverId: $state") } } + + case RequestDispatcherState => { + sender ! DispatcherStateResponse(drivers.values, completedDrivers) + } } def removeDriver(driverId: String, state: DriverState, exception: Option[Exception]) { @@ -225,6 +248,7 @@ object MesosClusterDispatcher { args.host, args.port, boundPort, + args.webUiPort, systemName, actorName, conf, @@ -237,6 +261,7 @@ object MesosClusterDispatcher { class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 + var webUiPort = 8081 var masterUrl: String = null parse(args.toList) @@ -251,6 +276,10 @@ object MesosClusterDispatcher { port = value parse(tail) + case ("--webui-port" | "-p") :: IntParam(value) :: tail => + webUiPort = value + parse(tail) + case ("--master" | "-m") :: value :: tail => if (!value.startsWith("mesos://")) { System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") @@ -283,6 +312,7 @@ object MesosClusterDispatcher { "Options:\n" + " -h HOST, --host HOST Hostname to listen on\n" + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + + " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + " -m --master MASTER URI for connecting to Mesos master\n") System.exit(exitCode) } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala new file mode 100644 index 0000000000000..a29c571be5a13 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala @@ -0,0 +1,75 @@ +/* + * 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.mesos.ui + +import org.apache.spark.ui.{UIUtils, WebUIPage} +import javax.servlet.http.HttpServletRequest +import scala.xml.Node +import org.apache.spark.deploy.mesos.Messages.{DispatcherStateResponse, RequestDispatcherState} +import scala.concurrent.Await +import org.apache.spark.deploy.master.DriverInfo + +class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { + private val dispatcher = parent.dispatcherActorRef + private val timeout = parent.timeout + + def render(request: HttpServletRequest): Seq[Node] = { + val stateFuture = (dispatcher ? RequestDispatcherState)(timeout).mapTo[DispatcherStateResponse] + val state = Await.result(stateFuture, timeout) + val content = +
+
+

Active drivers

+ {state.activeDrivers.map(d => driverContent(d)).flatten} +

Completed drivers

+ {state.completedDrivers.map(d => completedDriverContent(d)).flatten} +
+
; + UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster") + } + + def driverContent(info: DriverInfo): Seq[Node] = { +
    +
  • ID: {info.id}
  • +
  • Submit Date: {info.submitDate}
  • +
  • Start Date: {info.startTime}
  • +
  • Output: + stdout + stderr +
  • +
+ } + + def completedDriverContent(info: DriverInfo): Seq[Node] = { +
    +
  • ID: {info.id}
  • +
  • Submit Date: {info.submitDate}
  • +
  • Start Date: {info.startTime}
  • +
  • Output: + stdout + stderr +
  • +
  • State:{info.state}
  • +
  • Exception:{info.exception}
  • +
+ } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala new file mode 100644 index 0000000000000..3d2f62d3539cb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala @@ -0,0 +1,56 @@ +/* + * 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.mesos.ui + +import java.io.File +import org.apache.spark.ui.{SparkUI, WebUI} +import org.apache.spark.SparkConf +import org.apache.spark.SecurityManager +import akka.pattern.AskableActorRef +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.AkkaUtils +import org.apache.spark.deploy.worker.ui.{ActiveWebUiUrlAccessor, LogPage} + +/** + * UI that displays driver results from the [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]] + */ +private [spark] class MesosClusterUI( + val dispatcherActorRef: AskableActorRef, + securityManager: SecurityManager, + port: Int, + conf: SparkConf, + workDir: File, + dispatcherPublicAddress: String) + extends WebUI(securityManager, port, conf) with ActiveWebUiUrlAccessor { + + val timeout = AkkaUtils.askTimeout(conf) + + initialize() + + def activeWebUiUrl: String = "http://" + dispatcherPublicAddress + ":" + boundPort + + override def initialize() { + attachPage(new DriverOutputPage(this)) + attachPage(new LogPage(this, workDir)) + attachHandler(createStaticHandler(MesosClusterUI.STATIC_RESOURCE_DIR, "/static")) + } +} + +private[spark] object MesosClusterUI { + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR +} 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 3ee2eb69e8a4e..1c9802ca81678 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -61,7 +61,7 @@ private[worker] class Worker( assert (port > 0) // For worker and executor IDs - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // Send a heartbeat every (heartbeat timeout) / 4 milliseconds private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 @@ -85,10 +85,10 @@ private[worker] class Worker( private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders - private val CLEANUP_INTERVAL_MILLIS = + private val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - private val APP_DATA_RETENTION_SECS = + private val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) private val testing: Boolean = sys.props.contains("spark.testing") @@ -112,7 +112,7 @@ private[worker] class Worker( } else { new File(sys.env.get("SPARK_HOME").getOrElse(".")) } - + var workDir: File = null val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] @@ -134,7 +134,7 @@ private[worker] class Worker( private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) private val workerSource = new WorkerSource(this) - + private var registrationRetryTimer: Option[Cancellable] = None var coresUsed = 0 diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 88170d4df3053..839a38a33194e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.worker.ui +import java.io.File import javax.servlet.http.HttpServletRequest import scala.xml.Node @@ -30,6 +31,9 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with private val worker = parent.worker private val workDir = parent.workDir +private[spark] class LogPage( + urlAccessor: ActiveWebUiUrlAccessor, + workDir: File) extends WebUIPage("logPage") with Logging { def renderLog(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 @@ -73,7 +77,7 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with } val (logText, startByte, endByte, logLength) = getLog(logDir, logType, offset, byteLength) - val linkToMaster =

Back to Master

+ val linkToMaster =

Back to Master

val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} val backButton = 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 b3bb5f911dbd7..8eb5fe0376598 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 @@ -44,7 +44,7 @@ class WorkerWebUI( /** Initialize all components of the server. */ def initialize() { - val logPage = new LogPage(this) + val logPage = new LogPage(worker, workDir) attachPage(logPage) attachPage(new WorkerPage(this)) attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static")) From 7179495feed093ccf27e4a38cf8d8db14df7e1c9 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Thu, 26 Feb 2015 20:48:01 -0800 Subject: [PATCH 05/36] Change Driver page output and add logging --- .../deploy/mesos/MesosClusterDispatcher.scala | 16 ++++++ .../deploy/mesos/ui/DriverOutputPage.scala | 54 ++++++++++--------- 2 files changed, 46 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index c791d13b58265..0a18eea9c175a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -185,6 +185,7 @@ private [spark] class MesosClusterDispatcher( } case DriverStateChanged(driverId, state, exception) => { + logDriverChange(driverId, state, exception) state match { case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED => removeDriver(driverId, state, exception) @@ -198,6 +199,21 @@ private [spark] class MesosClusterDispatcher( } } + def logDriverChange(driverId: String, state: DriverState, exception: Option[Exception]) { + state match { + case DriverState.ERROR => + logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}") + case DriverState.FAILED => + logWarning(s"Driver $driverId exited with failure") + case DriverState.FINISHED => + logInfo(s"Driver $driverId exited successfully") + case DriverState.KILLED => + logInfo(s"Driver $driverId was killed by user") + case _ => + logDebug(s"Driver $driverId changed state to $state") + } + } + def removeDriver(driverId: String, state: DriverState, exception: Option[Exception]) { if (completedDrivers.size >= RETAINED_DRIVERS) { val toRemove = math.max(RETAINED_DRIVERS / 10, 1) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala index a29c571be5a13..ab8ae20ef2b90 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala @@ -31,45 +31,51 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (dispatcher ? RequestDispatcherState)(timeout).mapTo[DispatcherStateResponse] val state = Await.result(stateFuture, timeout) + + val driverHeaders = Seq("DriverID", "Submit Date", "Start Date", "Logs") + val completedDriverHeaders = driverHeaders ++ Seq("State", "Exception") + val driverTable = UIUtils.listingTable(driverHeaders, driverRow, state.activeDrivers) + val completedDriverTable = + UIUtils.listingTable(completedDriverHeaders, completedDriverRow, state.completedDrivers) val content =
-

Active drivers

- {state.activeDrivers.map(d => driverContent(d)).flatten} -

Completed drivers

- {state.completedDrivers.map(d => completedDriverContent(d)).flatten} +

Running Drivers

+ {driverTable} +

Finished Drivers

+ {completedDriverTable}
; UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster") } - def driverContent(info: DriverInfo): Seq[Node] = { -
    -
  • ID: {info.id}
  • -
  • Submit Date: {info.submitDate}
  • -
  • Start Date: {info.startTime}
  • -
  • Output: + def driverRow(info: DriverInfo): Seq[Node] = { + + {info.id} + {info.submitDate} + {info.startTime} + stdout + .format(info.id)}>stdout, stderr -
  • -
+ + } - def completedDriverContent(info: DriverInfo): Seq[Node] = { -
    -
  • ID: {info.id}
  • -
  • Submit Date: {info.submitDate}
  • -
  • Start Date: {info.startTime}
  • -
  • Output: + def completedDriverRow(info: DriverInfo): Seq[Node] = { + + {info.id} + {info.submitDate} + {info.startTime} + stdout + .format(info.id)}>stdout, stderr -
  • -
  • State:{info.state}
  • -
  • Exception:{info.exception}
  • -
+ + {info.state} + {info.exception} + } } From e775001fddee58a3cb7f38847e0be54a56d8475f Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Fri, 27 Feb 2015 10:22:17 -0800 Subject: [PATCH 06/36] Support fetching remote uris in driver runner. --- .../org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala | 3 ++- core/src/main/scala/org/apache/spark/util/Utils.scala | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 0a18eea9c175a..b8ea7c05c5ac5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -114,6 +114,7 @@ private [spark] class MesosClusterDispatcher( val completedDrivers = new ArrayBuffer[DriverInfo] val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) var nextDriverNumber = 0 + val securityManager = new SecurityManager(conf) def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") @@ -154,7 +155,7 @@ private [spark] class MesosClusterDispatcher( case RequestSubmitDriver(driverDescription) => { val driverInfo = createDriver(driverDescription) val runner = new DriverRunner(conf, driverInfo.id, workDir, - sparkHome, driverDescription, self, akkaUrl) + sparkHome, driverDescription, self, akkaUrl, securityManager) runners(driverInfo.id) = runner drivers(driverInfo.id) = driverInfo runner.start() diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 1029b0f9fce1e..496d170e1e5c4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -471,6 +471,7 @@ private[spark] object Utils extends Logging { in: InputStream, destFile: File, fileOverwrite: Boolean): Unit = { + logDebug(s"Creating temp file in ${destFile.getParentFile.getAbsolutePath}") val tempFile = File.createTempFile("fetchFileTemp", null, new File(destFile.getParentFile.getAbsolutePath)) logInfo(s"Fetching $url to $tempFile") From 4b2f5ef88b92c31b3aeea1ff2afca4cd003d98a1 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Sat, 28 Feb 2015 01:09:29 -0800 Subject: [PATCH 07/36] Specify user jar in command to be replaced with local. --- .../scala/org/apache/spark/deploy/rest/MesosRestServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala index 5d7b9c9443c12..fbc7bb1f0c5aa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala @@ -108,7 +108,7 @@ class MesosSubmitRequestServlet( val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = new Command( "org.apache.spark.deploy.SparkSubmit", - Seq("--master", masterUrl, "--class", mainClass, appResource) ++ appArgs, + Seq("--master", masterUrl, "--class", mainClass, "{{USER_JAR}}") ++ appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) From 5b7a12b5634a0639b456945e12d542d940461657 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Mon, 9 Mar 2015 17:11:01 -0700 Subject: [PATCH 08/36] WIP: Making a cluster mode a mesos framework. --- .../deploy/mesos/MesosClusterDispatcher.scala | 237 +----------- .../deploy/mesos/ui/DriverOutputPage.scala | 66 ++-- .../deploy/mesos/ui/MesosClusterUI.scala | 14 +- .../spark/deploy/rest/MesosRestServer.scala | 72 ++-- .../mesos/CoarseMesosSchedulerBackend.scala | 52 +-- .../cluster/mesos/MesosClusterScheduler.scala | 343 ++++++++++++++++++ .../cluster/mesos/MesosSchedulerBackend.scala | 53 +-- .../cluster/mesos/MesosSchedulerHelper.scala | 84 +++++ .../mesos/MesosClusterSchedulerSuite.scala | 65 ++++ 9 files changed, 584 insertions(+), 402 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerHelper.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index b8ea7c05c5ac5..c592c3f200d08 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -17,37 +17,13 @@ package org.apache.spark.deploy.mesos -import akka.actor.{Props, ActorSystem, Actor} - -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.Some - -import org.apache.spark.deploy.rest.MesosRestServer import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, IntParam, Utils} - -import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.util.{IntParam, Utils} -import org.apache.spark.deploy.DriverDescription -import org.apache.spark.deploy.master.DriverInfo -import org.apache.spark.deploy.master.DriverState.DriverState -import org.apache.spark.deploy.master.DriverState -import org.apache.spark.deploy.worker.DriverRunner - -import java.io.{IOException, File} -import java.util.Date -import java.text.SimpleDateFormat +import java.io.File import org.apache.spark.deploy.mesos.ui.MesosClusterUI -import org.apache.spark.deploy.mesos.Messages.{DispatcherStateResponse, RequestDispatcherState} - -private [deploy] object Messages { - case object RequestDispatcherState - - case class DispatcherStateResponse( - activeDrivers: Iterable[DriverInfo], - completedDrivers: Iterable[DriverInfo]) { - } -} +import org.apache.spark.deploy.rest.MesosRestServer +import org.apache.spark.scheduler.cluster.mesos.{ClusterScheduler, MesosClusterScheduler} /* * A dispatcher actor that is responsible for managing drivers, that is intended to @@ -58,221 +34,46 @@ private [deploy] object Messages { private [spark] class MesosClusterDispatcher( host: String, serverPort: Int, - actorPort: Int, webUiPort: Int, - systemName: String, - actorName: String, conf: SparkConf, - masterUrl: String, - workDirPath: Option[String] = None) extends Actor with ActorLogReceive with Logging { - val server = new MesosRestServer(host, serverPort, self, conf, masterUrl) + scheduler: ClusterScheduler) extends Logging { + + val server = new MesosRestServer(host, serverPort, conf, scheduler) val dispatcherPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - lazy val workDir: File = { - val dir = workDirPath.map(new File(_)).getOrElse(new File(sparkHome, "work")) - - // Attempt to remove the work directory if it exists on startup. - // This is to avoid unbounded growing the work directory as drivers - // are only deleted when it is over the retained count while it's running. - // We don't fail startup if we are not able to remove, as this is - // a short-term solution. - try { - if (dir.exists()) { - dir.delete() - } - } catch { - case e: IOException => - logError("Unable to remove work directory " + workDir, e) - } - - try { - // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() - // So attempting to create and then check if directory was created or not. - dir.mkdirs() - if (!dir.exists() || !dir.isDirectory) { - logError("Failed to create work directory " + dir) - System.exit(1) - } - assert (dir.isDirectory) - } catch { - case e: Exception => - logError("Failed to create work directory " + dir, e) - System.exit(1) - } - dir - } - val webUi = new MesosClusterUI( - self, new SecurityManager(conf), webUiPort, conf, workDir, dispatcherPublicAddress) - - val runners = new HashMap[String, DriverRunner] - val drivers = new HashMap[String, DriverInfo] - val completedDrivers = new ArrayBuffer[DriverInfo] - val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) - var nextDriverNumber = 0 - val securityManager = new SecurityManager(conf) - - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + new SecurityManager(conf), webUiPort, conf, dispatcherPublicAddress, scheduler) val sparkHome = new File(sys.env.get("SPARK_HOME").getOrElse(".")) - val akkaUrl = AkkaUtils.address( - AkkaUtils.protocol(context.system), - systemName, - host, - actorPort, - actorName) - - def newDriverId(submitDate: Date): String = { - val appId = "driver-%s-%04d".format(createDateFormat.format(submitDate), nextDriverNumber) - nextDriverNumber += 1 - appId - } - - def createDriver(desc: DriverDescription): DriverInfo = { - val now = System.currentTimeMillis() - val date = new Date(now) - new DriverInfo(now, newDriverId(date), desc, date) - } - - override def preStart() { + def start() { server.start() webUi.bind() } - override def postStop() { + def stop() { webUi.stop() server.stop() - runners.values.foreach(_.kill()) - } - - override def receiveWithLogging = { - case RequestSubmitDriver(driverDescription) => { - val driverInfo = createDriver(driverDescription) - val runner = new DriverRunner(conf, driverInfo.id, workDir, - sparkHome, driverDescription, self, akkaUrl, securityManager) - runners(driverInfo.id) = runner - drivers(driverInfo.id) = driverInfo - runner.start() - sender ! SubmitDriverResponse(true, Option(driverInfo.id), "") - } - - case RequestKillDriver(driverId) => { - if (!drivers.contains(driverId)) { - if (completedDrivers.exists(_.id == driverId)) { - sender ! KillDriverResponse(driverId, false, "Driver already completed") - } else { - sender ! KillDriverResponse(driverId, false, "Unknown driver") - } - } else { - runners(driverId).kill() - sender ! KillDriverResponse(driverId, true, "") - } - } - - case RequestDriverStatus(driverId) => { - drivers.get(driverId).orElse(completedDrivers.find(_.id == driverId)) match { - case Some(driver) => - sender ! DriverStatusResponse(found = true, Some(driver.state), - None, None, driver.exception) - case None => - sender ! DriverStatusResponse(found = false, None, None, None, None) - } - } - - case DriverStateChanged(driverId, state, exception) => { - logDriverChange(driverId, state, exception) - state match { - case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED => - removeDriver(driverId, state, exception) - case _ => - throw new Exception(s"Received unexpected state update for driver $driverId: $state") - } - } - - case RequestDispatcherState => { - sender ! DispatcherStateResponse(drivers.values, completedDrivers) - } - } - - def logDriverChange(driverId: String, state: DriverState, exception: Option[Exception]) { - state match { - case DriverState.ERROR => - logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}") - case DriverState.FAILED => - logWarning(s"Driver $driverId exited with failure") - case DriverState.FINISHED => - logInfo(s"Driver $driverId exited successfully") - case DriverState.KILLED => - logInfo(s"Driver $driverId was killed by user") - case _ => - logDebug(s"Driver $driverId changed state to $state") - } - } - - def removeDriver(driverId: String, state: DriverState, exception: Option[Exception]) { - if (completedDrivers.size >= RETAINED_DRIVERS) { - val toRemove = math.max(RETAINED_DRIVERS / 10, 1) - for (i <- 0 to (toRemove - 1)) { - val driverId = completedDrivers(i).id - try { - new File(workDir, driverId).delete() - } catch { - case e: Exception => - logWarning("Unable to remove work dir for completed driver " + driverId, e) - } - } - completedDrivers.trimStart(toRemove) - } - val driverInfo = drivers.remove(driverId).get - driverInfo.exception = exception - driverInfo.state = state - completedDrivers += driverInfo } } object MesosClusterDispatcher { def main(args: Array[String]) { val conf = new SparkConf - val clusterArgs = new ClusterDispatcherArguments(args, conf) - val actorSystem = startSystemAndActor(clusterArgs) - Runtime.getRuntime().addShutdownHook(new Thread("MesosClusterDispatcherShutdownHook") { - override def run() = { - // Makes sure we shut down the actor, which will kill all the drivers. - actorSystem.shutdown() - actorSystem.awaitTermination() - } - }) - actorSystem.awaitTermination() - } - - def startSystemAndActor( - args: ClusterDispatcherArguments): ActorSystem = { - // The LocalSparkCluster runs multiple local sparkWorkerX actor systems - val conf = new SparkConf - val systemName = "spark-mesos-cluster" - val actorName = "MesosClusterDispatcher" - val securityMgr = new SecurityManager(conf) - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - systemName, args.host, 0, conf, securityMgr) - actorSystem.actorOf( - Props(classOf[MesosClusterDispatcher], - args.host, - args.port, - boundPort, - args.webUiPort, - systemName, - actorName, - conf, - args.masterUrl, - None), - name = actorName) - actorSystem + val dispatcherArgs = new ClusterDispatcherArguments(args, conf) + val scheduler = new MesosClusterScheduler(conf) + scheduler.start() + new MesosClusterDispatcher( + dispatcherArgs.host, + dispatcherArgs.port, + dispatcherArgs.webUiPort, + conf, + scheduler).start() } class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala index ab8ae20ef2b90..cbd7384aed769 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala @@ -20,62 +20,58 @@ package org.apache.spark.deploy.mesos.ui import org.apache.spark.ui.{UIUtils, WebUIPage} import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.deploy.mesos.Messages.{DispatcherStateResponse, RequestDispatcherState} -import scala.concurrent.Await import org.apache.spark.deploy.master.DriverInfo +import org.apache.spark.scheduler.cluster.mesos.{ClusterTaskState, DriverSubmission, ClusterScheduler} class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { - private val dispatcher = parent.dispatcherActorRef - private val timeout = parent.timeout def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (dispatcher ? RequestDispatcherState)(timeout).mapTo[DispatcherStateResponse] - val state = Await.result(stateFuture, timeout) + val state = parent.scheduler.getState() - val driverHeaders = Seq("DriverID", "Submit Date", "Start Date", "Logs") - val completedDriverHeaders = driverHeaders ++ Seq("State", "Exception") - val driverTable = UIUtils.listingTable(driverHeaders, driverRow, state.activeDrivers) - val completedDriverTable = - UIUtils.listingTable(completedDriverHeaders, completedDriverRow, state.completedDrivers) + val queuedHeaders = Seq("DriverID", "Submit Date", "Description") + val driverHeaders = queuedHeaders ++ + Seq("Start Date", "Mesos Slave ID", "Mesos Task ID", "State", "Logs") + + val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) + val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) + val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers) val content =
-

Running Drivers

- {driverTable} -

Finished Drivers

- {completedDriverTable} +

Queued Drivers:

+ {queuedTable} +

Launched Drivers:

+ {launchedTable} +

Finished Drivers:

+ {finishedTable}
; UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster") + + null } - def driverRow(info: DriverInfo): Seq[Node] = { + def queuedRow(submission: DriverSubmission): Seq[Node] = { - {info.id} - {info.submitDate} - {info.startTime} - - stdout, - stderr - + {submission.submissionId} + {submission.submitDate} + {submission.desc.command.mainClass} } - def completedDriverRow(info: DriverInfo): Seq[Node] = { + def driverRow(state: ClusterTaskState): Seq[Node] = { - {info.id} - {info.submitDate} - {info.startTime} + {state.submission.submissionId} + {state.submission.submitDate} + {state.submission.desc.command.mainClass} + {state.startDate} + {state.slaveId} + {state.taskId} + {state.taskState} - stdout, - stderr + stdout, + stderr - {info.state} - {info.exception} } } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala index 3d2f62d3539cb..ff1427fe9aad7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala @@ -17,36 +17,30 @@ package org.apache.spark.deploy.mesos.ui -import java.io.File import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.SparkConf import org.apache.spark.SecurityManager -import akka.pattern.AskableActorRef import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.AkkaUtils -import org.apache.spark.deploy.worker.ui.{ActiveWebUiUrlAccessor, LogPage} +import org.apache.spark.deploy.worker.ui.ActiveWebUiUrlAccessor +import org.apache.spark.scheduler.cluster.mesos.ClusterScheduler /** * UI that displays driver results from the [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]] */ private [spark] class MesosClusterUI( - val dispatcherActorRef: AskableActorRef, securityManager: SecurityManager, port: Int, conf: SparkConf, - workDir: File, - dispatcherPublicAddress: String) + dispatcherPublicAddress: String, + val scheduler: ClusterScheduler) extends WebUI(securityManager, port, conf) with ActiveWebUiUrlAccessor { - val timeout = AkkaUtils.askTimeout(conf) - initialize() def activeWebUiUrl: String = "http://" + dispatcherPublicAddress + ":" + boundPort override def initialize() { attachPage(new DriverOutputPage(this)) - attachPage(new LogPage(this, workDir)) attachHandler(createStaticHandler(MesosClusterUI.STATIC_RESOURCE_DIR, "/static")) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala index fbc7bb1f0c5aa..fcd86b3c73ee2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala @@ -17,17 +17,16 @@ package org.apache.spark.deploy.rest -import akka.actor.ActorRef - import java.io.File import javax.servlet.http.HttpServletResponse -import org.apache.spark.deploy.{DriverDescription, DeployMessages} +import org.apache.spark.deploy.DriverDescription import org.apache.spark.deploy.ClientArguments._ import org.apache.spark.deploy.Command import org.apache.spark.{SparkConf, SPARK_VERSION => sparkVersion} -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.util.Utils +import org.apache.spark.scheduler.cluster.mesos.ClusterScheduler /** * A server that responds to requests submitted by the [[RestClient]]. @@ -47,27 +46,24 @@ import org.apache.spark.util.{Utils, AkkaUtils} * * @param host the address this server should bind to * @param requestedPort the port this server will attempt to bind to - * @param dispatcherActor reference to the Dispatcher actor to which requests can be sent - * @param masterUrl the URL of the Master new drivers will attempt to connect to * @param masterConf the conf used by the Master + * @param scheduler the scheduler that handles driver requests */ private [spark] class MesosRestServer( host: String, requestedPort: Int, - dispatcherActor: ActorRef, masterConf: SparkConf, - masterUrl: String) + scheduler: ClusterScheduler) extends RestServer( host, requestedPort, masterConf, - new MesosSubmitRequestServlet(dispatcherActor, masterUrl, masterConf), - new MesosKillRequestServlet(dispatcherActor, masterConf), - new MesosStatusRequestServlet(dispatcherActor, masterConf)) {} + new MesosSubmitRequestServlet(scheduler, masterConf), + new MesosKillRequestServlet(scheduler, masterConf), + new MesosStatusRequestServlet(scheduler, masterConf)) {} class MesosSubmitRequestServlet( - dispatcherActor: ActorRef, - masterUrl: String, + scheduler: ClusterScheduler, conf: SparkConf) extends SubmitRequestServlet { @@ -94,29 +90,27 @@ class MesosSubmitRequestServlet( val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") val superviseDriver = sparkProperties.get("spark.driver.supervise") val driverMemory = sparkProperties.get("spark.driver.memory") + val driverCores = sparkProperties.get("spark.driver.cores") val appArgs = request.appArgs val environmentVariables = request.environmentVariables // Construct driver description val conf = new SparkConf(false) .setAll(sparkProperties) - .set("spark.master", masterUrl) + val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = new Command( - "org.apache.spark.deploy.SparkSubmit", - Seq("--master", masterUrl, "--class", mainClass, "{{USER_JAR}}") ++ appArgs, - environmentVariables, extraClassPath, extraLibraryPath, javaOpts) + mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) + val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES) - // Cores and memory are ignored in Mesos mode. We override the memory to be driver memory - // as the memory is used for JVM heap size of the driver later. new DriverDescription( - appResource, actualDriverMemory, 0, actualSuperviseDriver, command) + appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command) } protected override def handleSubmit( @@ -125,15 +119,13 @@ class MesosSubmitRequestServlet( responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { requestMessage match { case submitRequest: CreateSubmissionRequest => - val askTimeout = AkkaUtils.askTimeout(conf) val driverDescription = buildDriverDescription(submitRequest) - val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( - DeployMessages.RequestSubmitDriver(driverDescription), dispatcherActor, askTimeout) + val response = scheduler.submitDriver(driverDescription) val submitResponse = new CreateSubmissionResponse submitResponse.serverSparkVersion = sparkVersion - submitResponse.message = response.message + submitResponse.message = response.message.orNull submitResponse.success = response.success - submitResponse.submissionId = response.driverId.orNull + submitResponse.submissionId = response.id val unknownFields = findUnknownFields(requestMessageJson, requestMessage) if (unknownFields.nonEmpty) { // If there are fields that the server does not know about, warn the client @@ -147,36 +139,30 @@ class MesosSubmitRequestServlet( } } -class MesosKillRequestServlet( - dispatcherActor: ActorRef, - conf: SparkConf) extends KillRequestServlet { +class MesosKillRequestServlet(scheduler: ClusterScheduler, conf: SparkConf) + extends KillRequestServlet { protected override def handleKill(submissionId: String): KillSubmissionResponse = { - val askTimeout = AkkaUtils.askTimeout(conf) - val response = AkkaUtils.askWithReply[DeployMessages.KillDriverResponse]( - DeployMessages.RequestKillDriver(submissionId), dispatcherActor, askTimeout) + val response = scheduler.killDriver(submissionId) val k = new KillSubmissionResponse k.serverSparkVersion = sparkVersion - k.message = response.message + k.message = response.message.orNull k.submissionId = submissionId k.success = response.success k } } -class MesosStatusRequestServlet( - dispatcherActor: ActorRef, - conf: SparkConf) extends StatusRequestServlet { +class MesosStatusRequestServlet(scheduler: ClusterScheduler, conf: SparkConf) + extends StatusRequestServlet { protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { - val askTimeout = AkkaUtils.askTimeout(conf) - val response = AkkaUtils.askWithReply[DeployMessages.DriverStatusResponse]( - DeployMessages.RequestDriverStatus(submissionId), dispatcherActor, askTimeout) - val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } + val response = scheduler.getStatus(submissionId) + //val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } val d = new SubmissionStatusResponse d.serverSparkVersion = sparkVersion - d.submissionId = submissionId - d.success = response.found - d.driverState = response.state.map(_.toString).orNull - d.message = message.orNull + d.submissionId = response.id + d.success = response.success + //d.driverState = response.state.map(_.toString).orNull + d.message = response.message.orNull d } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index b037a4966ced0..d0be3df23f13b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -49,17 +49,10 @@ private[spark] class CoarseMesosSchedulerBackend( master: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with MScheduler - with Logging { + with MesosSchedulerHelper { val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures - // Lock used to wait for scheduler to be registered - var isRegistered = false - val registeredLock = new Object() - - // Driver for talking to Mesos - var driver: SchedulerDriver = null - // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt @@ -87,26 +80,8 @@ private[spark] class CoarseMesosSchedulerBackend( override def start() { super.start() - - synchronized { - new Thread("CoarseMesosSchedulerBackend driver") { - setDaemon(true) - override def run() { - val scheduler = CoarseMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() - driver = new MesosSchedulerDriver(scheduler, fwInfo, master) - try { { - val ret = driver.run() - logInfo("driver.run() returned with code " + ret) - } - } catch { - case e: Exception => logError("driver.run() failed", e) - } - } - }.start() - - waitForRegister() - } + val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() + startScheduler("CoarseMesosSchedulerBackend", master, CoarseMesosSchedulerBackend.this, fwInfo) } def createCommand(offer: Offer, numCores: Int): CommandInfo = { @@ -183,18 +158,7 @@ private[spark] class CoarseMesosSchedulerBackend( override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { appId = frameworkId.getValue logInfo("Registered as framework ID " + appId) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() - } - } - - def waitForRegister() { - registeredLock.synchronized { - while (!isRegistered) { - registeredLock.wait() - } - } + markRegistered() } override def disconnected(d: SchedulerDriver) {} @@ -245,14 +209,6 @@ private[spark] class CoarseMesosSchedulerBackend( } } - /** Helper function to pull out a resource from a Mesos Resources protobuf */ - private def getResource(res: JList[Resource], name: String): Double = { - for (r <- res if r.getName == name) { - return r.getScalar.getValue - } - 0 - } - /** Build a Mesos resource protobuf object */ private def createResource(resourceName: String, quantity: Double): Protos.Resource = { Resource.newBuilder() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala new file mode 100644 index 0000000000000..1b1628877b068 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -0,0 +1,343 @@ +/* + * 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.scheduler.cluster.mesos + +import java.text.SimpleDateFormat + +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import java.util.{List => JList} +import java.util.{Collections, Date} + +import org.apache.mesos.{SchedulerDriver, Scheduler} +import org.apache.mesos.Protos._ + +import org.apache.spark.deploy.DriverDescription +import org.apache.spark.deploy.master.DriverState +import org.apache.spark.deploy.master.DriverState.DriverState + +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils + +import scala.collection.mutable +import scala.collection.JavaConversions._ +import org.apache.mesos.Protos.Environment.Variable + + +private[spark] class DriverSubmission( + val submissionId: String, + val desc: DriverDescription, + val submitDate: Date) { + + def canEqual(other: Any): Boolean = other.isInstanceOf[DriverSubmission] + + override def equals(other: Any): Boolean = other match { + case that: DriverSubmission => + (that canEqual this) && + submissionId == that.submissionId + case _ => false + } +} + +private [spark] case class ClusterTaskState( + val submission: DriverSubmission, + val taskId: TaskID, + val slaveId: SlaveID, + var taskState: Option[TaskStatus], + var driverState: DriverState, + var startDate: Date) { + def copy(): ClusterTaskState = { + ClusterTaskState(submission, taskId, slaveId, taskState, driverState, startDate) + } +} + +private[spark] case class SubmitResponse(id: String, success: Boolean, message: Option[String]) +private[spark] case class StatusResponse(id: String, success: Boolean, message: Option[String]) +private[spark] case class KillResponse(id: String, success: Boolean, message: Option[String]) + +private[spark] case class ClusterSchedulerState( + queuedDrivers: Iterable[DriverSubmission], + launchedDrivers: Iterable[ClusterTaskState], + finishedDrivers: Iterable[ClusterTaskState]) + +private[spark] trait ClusterScheduler { + def submitDriver(desc: DriverDescription): SubmitResponse + def killDriver(submissionId: String): KillResponse + def getStatus(submissionId: String): StatusResponse + def getState(): ClusterSchedulerState +} + +private[spark] class MesosClusterScheduler(conf: SparkConf) + extends Scheduler with MesosSchedulerHelper with ClusterScheduler { + + val master = conf.get("spark.master") + val appName = conf.get("spark.app.name") + val capacity = conf.getInt("spark.mesos.driver.capacity", 200) + val executorUri = conf.getOption("spark.executor.uri") + val stateLock = new Object + val launchedDrivers = new mutable.HashMap[String, ClusterTaskState]() + + // TODO: Bound this finished drivers map or make it a array + val finishedDrivers = new mutable.HashMap[String, ClusterTaskState]() + val nextDriverNumber: AtomicLong = new AtomicLong(0) + var appId: String = _ + private val queue = new LinkedBlockingQueue[DriverSubmission](capacity) + + def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + + private def newDriverId(submitDate: Date): String = { + "driver-%s-%04d".format( + createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) + } + + def submitDriver(desc: DriverDescription): SubmitResponse = { + val submitDate: Date = new Date() + val submissionId: String = newDriverId(submitDate) + val submission = new DriverSubmission(submissionId, desc, submitDate) + if (queue.offer(submission)) { + SubmitResponse(submissionId, true, None) + } else { + SubmitResponse(submissionId, false, Option("Already reached maximum submission size")) + } + } + + def killDriver(submissionId: String): KillResponse = { + stateLock.synchronized { + if (launchedDrivers.contains(submissionId)) { + // Kill the JOB!!!! + return KillResponse(submissionId, true, Option("Killing running driver")) + } + } + + // Check if submission is queued + if (queue.remove(new DriverSubmission(submissionId, null, null))) { + return KillResponse(submissionId, true, Option("Removed driver while it's still pending")) + } + + KillResponse(submissionId, false, Option("Cannot find driver")) + } + + def start() { + val fwInfo = FrameworkInfo.newBuilder() + .setUser(Utils.getCurrentUserName()).setName(appName).build() + startScheduler("MesosClusterScheduler", master, MesosClusterScheduler.this, fwInfo) + } + + override def registered( + driver: SchedulerDriver, + frameworkId: FrameworkID, + masterInfo: MasterInfo): Unit = { + appId = frameworkId.getValue + logInfo("Registered as framework ID " + appId) + markRegistered() + } + + private def buildCommand(desc: DriverDescription): CommandInfo = { + val builder = CommandInfo.newBuilder() + .addUris(CommandInfo.URI.newBuilder().setValue(desc.jarUrl).build()) + + val entries = + (conf.getOption("spark.executor.extraLibraryPath").toList ++ desc.command.libraryPathEntries) + + val prefixEnv = if (!entries.isEmpty) { + Utils.libraryPathEnvPrefix(entries) + } else { + "" + } + + val stringBuilder = new StringBuilder + stringBuilder + .append(desc.command.mainClass) + .append(" ") + .append(desc.command.arguments.mkString("\"", "\"", "\"")) + + val envBuilder = Environment.newBuilder() + desc.command.environment.foreach { + case (k, v) => + envBuilder.addVariables( + Variable.newBuilder().setName(k).setValue(v).build()) + } + + builder.setEnvironment(envBuilder.build()) + + if (executorUri.isDefined) { + builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) + val basename = executorUri.get.split('/').last.split('.').head + builder.setValue( + s"cd $basename*; $prefixEnv ${stringBuilder.toString()}") + } else { + builder.setValue( + s"$prefixEnv ${stringBuilder.toString()}") + } + + builder.build + } + + override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { + // We should try to schedule all the drivers if the offers fit. + + // Non-blocking poll. + val submissionOption = Option(queue.poll(0, TimeUnit.SECONDS)) + + if (submissionOption.isEmpty) { + offers.foreach(o => driver.declineOffer(o.getId)) + return + } + + val submission = submissionOption.get + + var remainingOffers = offers + + val driverCpu = submission.desc.cores + val driverMem = submission.desc.mem + + // Should use the passed in driver cpu and memory. + val offerOption = offers.find { o => + getResource(o.getResourcesList, "cpus") >= driverCpu && + getResource(o.getResourcesList, "mem") >= driverMem + } + + offerOption.foreach { offer => + val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() + + val cpuResource = Resource.newBuilder() + .setName("cpus").setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() + + val memResource = Resource.newBuilder() + .setName("mem").setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() + + val commandInfo = buildCommand(submission.desc) + + val taskInfo = TaskInfo.newBuilder() + .setTaskId(taskId) + .setSlaveId(offer.getSlaveId) + .setCommand(commandInfo) + .addResources(cpuResource) + .addResources(memResource) + .build + + //TODO: logDebug("") + driver.launchTasks(Collections.singleton(offer.getId), Collections.singleton(taskInfo)) + + stateLock.synchronized { + launchedDrivers(submission.submissionId) = + ClusterTaskState(submission, taskId, offer.getSlaveId, + None, DriverState.SUBMITTED, new Date()) + } + + remainingOffers = offers.filter(o => o.getId.equals(offer.getId)) + } + + remainingOffers.foreach(o => driver.declineOffer(o.getId)) + } + + def getState(): ClusterSchedulerState = { + def copyDriverStates(states: Iterable[ClusterTaskState]): Iterable[ClusterTaskState] = { + states.collect { case s => s.copy() } + } + + stateLock.synchronized { + val queueCopy = new Array[DriverSubmission](queue.size()) + queue.copyToArray(queueCopy) + ClusterSchedulerState( + queueCopy, + copyDriverStates(launchedDrivers.values), + copyDriverStates(finishedDrivers.values)) + } + } + + def getStatus(submissionId: String): StatusResponse = { + stateLock.synchronized { + if (queue.contains(new DriverSubmission(submissionId, null, null))) { + return StatusResponse(submissionId, true, Option("Driver is queued for launch")) + } else if (launchedDrivers.contains(submissionId)) { + return StatusResponse(submissionId, true, Option("Driver is running")) + } else if (finishedDrivers.contains(submissionId)) { + return StatusResponse(submissionId, true, Option("Driver already finished")) + } else { + return StatusResponse(submissionId, false, None) + } + } + } + + override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {} + + override def disconnected(driver: SchedulerDriver): Unit = {} + + override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = {} + + override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {} + + override def error(driver: SchedulerDriver, error: String): Unit = {} + + def getDriverState(state: TaskState): DriverState = { + state match { + case TaskState.TASK_FAILED => DriverState.FAILED + case TaskState.TASK_ERROR => DriverState.ERROR + case TaskState.TASK_FINISHED => DriverState.FINISHED + case TaskState.TASK_KILLED => DriverState.KILLED + case TaskState.TASK_LOST => DriverState.ERROR + case TaskState.TASK_RUNNING => DriverState.RUNNING + case TaskState.TASK_STARTING | TaskState.TASK_STAGING => DriverState.SUBMITTED + case _ => DriverState.UNKNOWN + } + } + + def canRelaunch(state: TaskState): Boolean = { + state == TaskState.TASK_FAILED || + state == TaskState.TASK_KILLED || + state == TaskState.TASK_LOST + } + + override def statusUpdate(driver: SchedulerDriver, status: TaskStatus): Unit = { + val taskId = status.getTaskId.getValue + stateLock.synchronized { + if (launchedDrivers.contains(taskId)) { + if (canRelaunch(status.getState)) { + // TODO: We should try to relaunch if supervise is turned on. + // Also check how many times we've retried. + } + + val driverState = getDriverState(status.getState) + val state = if (isFinished(status.getState)) { + val launchedState = launchedDrivers.remove(taskId).get + finishedDrivers(taskId) = launchedState + launchedState + } else { + launchedDrivers(taskId) + } + state.taskState = Option(status) + state.driverState = driverState + } else { + logError("Unable to find driver " + taskId + " in status update") + } + } + } + + override def frameworkMessage( + driver: SchedulerDriver, + executorId: ExecutorID, + slaveId: SlaveID, + message: Array[Byte]): Unit = {} + + override def executorLost( + driver: SchedulerDriver, + executorId: ExecutorID, + slaveId: SlaveID, + status: Int): Unit = {} +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index d9d62b0e287ed..b145ecb4c40ef 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -47,14 +47,7 @@ private[spark] class MesosSchedulerBackend( master: String) extends SchedulerBackend with MScheduler - with Logging { - - // Lock used to wait for scheduler to be registered - var isRegistered = false - val registeredLock = new Object() - - // Driver for talking to Mesos - var driver: SchedulerDriver = null + with MesosSchedulerHelper { // Which slave IDs we have executors on val slaveIdsWithExecutors = new HashSet[String] @@ -73,26 +66,9 @@ private[spark] class MesosSchedulerBackend( @volatile var appId: String = _ override def start() { - synchronized { - classLoader = Thread.currentThread.getContextClassLoader - - new Thread("MesosSchedulerBackend driver") { - setDaemon(true) - override def run() { - val scheduler = MesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() - driver = new MesosSchedulerDriver(scheduler, fwInfo, master) - try { - val ret = driver.run() - logInfo("driver.run() returned with code " + ret) - } catch { - case e: Exception => logError("driver.run() failed", e) - } - } - }.start() - - waitForRegister() - } + val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() + classLoader = Thread.currentThread.getContextClassLoader + startScheduler("MesosSchedulerBackend", master, MesosSchedulerBackend.this, fwInfo) } def createExecutorInfo(execId: String): MesosExecutorInfo = { @@ -181,18 +157,7 @@ private[spark] class MesosSchedulerBackend( inClassLoader() { appId = frameworkId.getValue logInfo("Registered as framework ID " + appId) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() - } - } - } - - def waitForRegister() { - registeredLock.synchronized { - while (!isRegistered) { - registeredLock.wait() - } + markRegistered() } } @@ -287,14 +252,6 @@ private[spark] class MesosSchedulerBackend( } } - /** Helper function to pull out a resource from a Mesos Resources protobuf */ - def getResource(res: JList[Resource], name: String): Double = { - for (r <- res if r.getName == name) { - return r.getScalar.getValue - } - 0 - } - /** Turn a Spark TaskDescription into a Mesos task */ def createMesosTask(task: TaskDescription, slaveId: String): MesosTaskInfo = { val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerHelper.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerHelper.scala new file mode 100644 index 0000000000000..8e43b0ad96d9c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerHelper.scala @@ -0,0 +1,84 @@ +/* + * 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.scheduler.cluster.mesos + +import org.apache.mesos.{Scheduler, MesosSchedulerDriver, SchedulerDriver} +import org.apache.mesos.Protos.{TaskState, Resource, FrameworkInfo} +import org.apache.spark.Logging + +import java.util.concurrent.CountDownLatch +import java.util.List + +import scala.collection.JavaConversions._ + +private[spark] trait MesosSchedulerHelper extends Logging { + // Lock used to wait for scheduler to be registered + final val registerLatch = new CountDownLatch(1) + + // Driver for talking to Mesos + var driver: SchedulerDriver = null + + def startScheduler(name: String, masterUrl: String, scheduler: Scheduler, fwInfo: FrameworkInfo) { + synchronized { + if (driver != null) { + waitForRegister() + return + } + + new Thread(name + " driver") { + setDaemon(true) + + override def run() { + driver = new MesosSchedulerDriver(scheduler, fwInfo, masterUrl) + try { + val ret = driver.run() + logInfo("driver.run() returned with code " + ret) + } catch { + case e: Exception => logError("driver.run() failed", e) + } + } + }.start() + + waitForRegister() + } + } + + private def waitForRegister() { + registerLatch.await() + } + + def markRegistered() { + registerLatch.countDown() + } + + def getResource(res: List[Resource], name: String): Double = { + for (r <- res if r.getName == name) { + return r.getScalar.getValue + } + 0.0 + } + + /** Check whether a Mesos task state represents a finished task */ + def isFinished(state: TaskState) = { + state == TaskState.TASK_FINISHED || + state == TaskState.TASK_FAILED || + state == TaskState.TASK_KILLED || + state == TaskState.TASK_LOST || + state == TaskState.TASK_ERROR + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala new file mode 100644 index 0000000000000..da607a1d88f11 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -0,0 +1,65 @@ +/* + * 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.scheduler.mesos + +import org.scalatest.FunSuite +import org.apache.spark.{SparkConf, LocalSparkContext} +import org.scalatest.mock.MockitoSugar +import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler +import org.apache.spark.deploy.{Command, DriverDescription} + +class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar { + def createCommand: Command = { + new Command( + "mainClass", Seq("arg"), null, null, null, null) + } + + test("can queue drivers") { + val conf = new SparkConf() + conf.setMaster("mesos://localhost:5050") + conf.setAppName("spark mesos") + val scheduler = new MesosClusterScheduler(conf) + val response = + scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand)) + assert(response.success) + + val response2 = + scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand)) + assert(response2.success) + + val state = scheduler.getState() + assert(state.queuedDrivers.exists(d => d.submissionId == response.id)) + assert(state.queuedDrivers.exists(d => d.submissionId == response2.id)) + } + + test("can kill queued drivers") { + val conf = new SparkConf() + conf.setMaster("mesos://localhost:5050") + conf.setAppName("spark mesos") + val scheduler = new MesosClusterScheduler(conf) + val response = + scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand)) + assert(response.success) + + val killResponse = scheduler.killDriver(response.id) + assert(killResponse.success) + + val state = scheduler.getState() + assert(state.queuedDrivers.isEmpty) + } +} From 0fa77800539c46c70ef636d3b6cf60c89cd8bfa0 Mon Sep 17 00:00:00 2001 From: Luc Bourlier Date: Wed, 11 Mar 2015 00:49:37 +0100 Subject: [PATCH 09/36] Launch task through the mesos scheduler --- .../deploy/mesos/MesosClusterDispatcher.scala | 8 ++- .../deploy/mesos/ui/DriverOutputPage.scala | 4 +- .../spark/deploy/rest/MesosRestServer.scala | 11 +-- .../cluster/mesos/MesosClusterScheduler.scala | 72 ++++++++++++------- .../mesos/MesosClusterSchedulerSuite.scala | 7 +- 5 files changed, 67 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index c592c3f200d08..ce1caad8b10f4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -66,6 +66,8 @@ object MesosClusterDispatcher { def main(args: Array[String]) { val conf = new SparkConf val dispatcherArgs = new ClusterDispatcherArguments(args, conf) + conf.setMaster(dispatcherArgs.masterUrl) + conf.setAppName("Mesos Cluster Dispatcher") val scheduler = new MesosClusterScheduler(conf) scheduler.start() new MesosClusterDispatcher( @@ -74,6 +76,10 @@ object MesosClusterDispatcher { dispatcherArgs.webUiPort, conf, scheduler).start() + this.synchronized { + // block indefinitely + this.wait() // TODO: bad + } } class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { @@ -103,7 +109,7 @@ object MesosClusterDispatcher { System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") System.exit(1) } - masterUrl = value + masterUrl = value.stripPrefix("mesos://") parse(tail) case ("--help") :: tail => diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala index cbd7384aed769..c7a7f5026cc8d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala @@ -55,7 +55,7 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { {submission.submissionId} {submission.submitDate} - {submission.desc.command.mainClass} + {submission.req.desc.command.mainClass} } @@ -63,7 +63,7 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { {state.submission.submissionId} {state.submission.submitDate} - {state.submission.desc.command.mainClass} + {state.submission.req.desc.command.mainClass} {state.startDate} {state.slaveId} {state.taskId} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala index fcd86b3c73ee2..60a14c01af869 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala @@ -23,10 +23,10 @@ import javax.servlet.http.HttpServletResponse import org.apache.spark.deploy.DriverDescription import org.apache.spark.deploy.ClientArguments._ import org.apache.spark.deploy.Command - import org.apache.spark.{SparkConf, SPARK_VERSION => sparkVersion} import org.apache.spark.util.Utils import org.apache.spark.scheduler.cluster.mesos.ClusterScheduler +import org.apache.spark.scheduler.cluster.mesos.DriverRequest /** * A server that responds to requests submitted by the [[RestClient]]. @@ -74,7 +74,7 @@ class MesosSubmitRequestServlet( * This does not currently consider fields used by python applications since python * is not supported in mesos cluster mode yet. */ - private def buildDriverDescription(request: CreateSubmissionRequest): DriverDescription = { + private def buildDriverRequest(request: CreateSubmissionRequest): DriverRequest = { // Required fields, including the main class because python is not yet supported val appResource = Option(request.appResource).getOrElse { throw new SubmitRestMissingFieldException("Application jar is missing.") @@ -109,8 +109,9 @@ class MesosSubmitRequestServlet( val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES) - new DriverDescription( - appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command) + DriverRequest(new DriverDescription( + appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command), + conf) } protected override def handleSubmit( @@ -119,7 +120,7 @@ class MesosSubmitRequestServlet( responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { requestMessage match { case submitRequest: CreateSubmissionRequest => - val driverDescription = buildDriverDescription(submitRequest) + val driverDescription = buildDriverRequest(submitRequest) val response = scheduler.submitDriver(driverDescription) val submitResponse = new CreateSubmissionResponse submitResponse.serverSparkVersion = sparkVersion diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 1b1628877b068..d1dbbd1b795da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -18,30 +18,28 @@ package org.apache.spark.scheduler.cluster.mesos import java.text.SimpleDateFormat - import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} import java.util.{List => JList} import java.util.{Collections, Date} - import org.apache.mesos.{SchedulerDriver, Scheduler} import org.apache.mesos.Protos._ - import org.apache.spark.deploy.DriverDescription import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState - import org.apache.spark.SparkConf import org.apache.spark.util.Utils - import scala.collection.mutable import scala.collection.JavaConversions._ import org.apache.mesos.Protos.Environment.Variable +import org.apache.spark.SparkException +import java.io.File +case class DriverRequest(desc: DriverDescription, conf: SparkConf) private[spark] class DriverSubmission( val submissionId: String, - val desc: DriverDescription, + val req: DriverRequest, val submitDate: Date) { def canEqual(other: Any): Boolean = other.isInstanceOf[DriverSubmission] @@ -76,7 +74,7 @@ private[spark] case class ClusterSchedulerState( finishedDrivers: Iterable[ClusterTaskState]) private[spark] trait ClusterScheduler { - def submitDriver(desc: DriverDescription): SubmitResponse + def submitDriver(desc: DriverRequest): SubmitResponse def killDriver(submissionId: String): KillResponse def getStatus(submissionId: String): StatusResponse def getState(): ClusterSchedulerState @@ -88,7 +86,6 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) val master = conf.get("spark.master") val appName = conf.get("spark.app.name") val capacity = conf.getInt("spark.mesos.driver.capacity", 200) - val executorUri = conf.getOption("spark.executor.uri") val stateLock = new Object val launchedDrivers = new mutable.HashMap[String, ClusterTaskState]() @@ -105,10 +102,10 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) } - def submitDriver(desc: DriverDescription): SubmitResponse = { + def submitDriver(req: DriverRequest): SubmitResponse = { val submitDate: Date = new Date() val submissionId: String = newDriverId(submitDate) - val submission = new DriverSubmission(submissionId, desc, submitDate) + val submission = new DriverSubmission(submissionId, req, submitDate) if (queue.offer(submission)) { SubmitResponse(submissionId, true, None) } else { @@ -147,9 +144,16 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) markRegistered() } - private def buildCommand(desc: DriverDescription): CommandInfo = { + private def buildCommand(req: DriverRequest): CommandInfo = { + + val desc = req.desc + + val cleanedJarUrl = desc.jarUrl.stripPrefix("file:") + + logInfo(s"jarUrl: $cleanedJarUrl") + val builder = CommandInfo.newBuilder() - .addUris(CommandInfo.URI.newBuilder().setValue(desc.jarUrl).build()) + .addUris(CommandInfo.URI.newBuilder().setValue(cleanedJarUrl).build()) val entries = (conf.getOption("spark.executor.extraLibraryPath").toList ++ desc.command.libraryPathEntries) @@ -160,11 +164,7 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) "" } - val stringBuilder = new StringBuilder - stringBuilder - .append(desc.command.mainClass) - .append(" ") - .append(desc.command.arguments.mkString("\"", "\"", "\"")) + // TODO: add support for more spark-submit parameters val envBuilder = Environment.newBuilder() desc.command.environment.foreach { @@ -175,14 +175,35 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) builder.setEnvironment(envBuilder.build()) + val executorUri = req.conf.getOption("spark.executor.uri") if (executorUri.isDefined) { builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) + val basename = executorUri.get.split('/').last.split('.').head + val cmd = + Seq("bin/spark-submit", + "--class", desc.command.mainClass, + "--master", s"mesos://${conf.get("spark.master")}", + s"../${desc.jarUrl.split("/").last}") + .mkString(" ") + builder.setValue( - s"cd $basename*; $prefixEnv ${stringBuilder.toString()}") + s"cd $basename*; $prefixEnv $cmd") } else { + val executorSparkHome = req.conf.getOption("spark.mesos.executor.home") + .getOrElse { + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } + + val cmd = + Seq(new File(executorSparkHome, "./bin/spark-submit"), + "--class", desc.command.mainClass, + "--master", s"mesos://${conf.get("spark.master")}", + desc.jarUrl.split("/").last) + .mkString(" ") + builder.setValue( - s"$prefixEnv ${stringBuilder.toString()}") + s"$prefixEnv $cmd") } builder.build @@ -203,8 +224,8 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) var remainingOffers = offers - val driverCpu = submission.desc.cores - val driverMem = submission.desc.mem + val driverCpu = submission.req.desc.cores + val driverMem = submission.req.desc.mem // Should use the passed in driver cpu and memory. val offerOption = offers.find { o => @@ -216,15 +237,18 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() val cpuResource = Resource.newBuilder() - .setName("cpus").setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() + .setName("cpus").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() val memResource = Resource.newBuilder() - .setName("mem").setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() + .setName("mem").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() - val commandInfo = buildCommand(submission.desc) + val commandInfo = buildCommand(submission.req) val taskInfo = TaskInfo.newBuilder() .setTaskId(taskId) + .setName(s"driver for ${submission.req.desc.command.mainClass}") .setSlaveId(offer.getSlaveId) .setCommand(commandInfo) .addResources(cpuResource) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index da607a1d88f11..e149d61ee704b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.FunSuite import org.apache.spark.{SparkConf, LocalSparkContext} import org.scalatest.mock.MockitoSugar import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler +import org.apache.spark.scheduler.cluster.mesos.DriverRequest import org.apache.spark.deploy.{Command, DriverDescription} class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar { @@ -35,11 +36,11 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo conf.setAppName("spark mesos") val scheduler = new MesosClusterScheduler(conf) val response = - scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand)) + scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf)) assert(response.success) val response2 = - scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand)) + scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf)) assert(response2.success) val state = scheduler.getState() @@ -53,7 +54,7 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo conf.setAppName("spark mesos") val scheduler = new MesosClusterScheduler(conf) val response = - scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand)) + scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf)) assert(response.success) val killResponse = scheduler.killDriver(response.id) From b8e7181652becaebb68eb8732446240f582fd2c8 Mon Sep 17 00:00:00 2001 From: Luc Bourlier Date: Wed, 11 Mar 2015 01:41:11 +0100 Subject: [PATCH 10/36] Adds a shutdown latch to keep the deamon running --- .../spark/deploy/mesos/MesosClusterDispatcher.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index ce1caad8b10f4..586e7a1b4bf2a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -19,11 +19,11 @@ package org.apache.spark.deploy.mesos import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{IntParam, Utils} - import java.io.File import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.MesosRestServer import org.apache.spark.scheduler.cluster.mesos.{ClusterScheduler, MesosClusterScheduler} +import java.util.concurrent.CountDownLatch /* * A dispatcher actor that is responsible for managing drivers, that is intended to @@ -63,6 +63,9 @@ private [spark] class MesosClusterDispatcher( } object MesosClusterDispatcher { + + val shutdownLatch = new CountDownLatch(1) + def main(args: Array[String]) { val conf = new SparkConf val dispatcherArgs = new ClusterDispatcherArguments(args, conf) @@ -76,10 +79,8 @@ object MesosClusterDispatcher { dispatcherArgs.webUiPort, conf, scheduler).start() - this.synchronized { - // block indefinitely - this.wait() // TODO: bad - } + + shutdownLatch.await() } class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { From 825afa05907639e2b594e757aa2f54188b4e3662 Mon Sep 17 00:00:00 2001 From: Luc Bourlier Date: Wed, 11 Mar 2015 19:40:40 +0100 Subject: [PATCH 11/36] Supports more spark-submit parameters --- .../spark/deploy/DriverDescription.scala | 2 +- .../cluster/mesos/MesosClusterScheduler.scala | 47 +++++++++++-------- 2 files changed, 28 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala index 659fb434a80f5..458286f139b6f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -private[deploy] class DriverDescription( +private[spark] case class DriverDescription( val jarUrl: String, val mem: Int, val cores: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index d1dbbd1b795da..f03dccd57618d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -164,8 +164,6 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) "" } - // TODO: add support for more spark-submit parameters - val envBuilder = Environment.newBuilder() desc.command.environment.foreach { case (k, v) => @@ -175,40 +173,49 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) builder.setEnvironment(envBuilder.build()) + val cmdOptions = generateCmdOption(req) + val executorUri = req.conf.getOption("spark.executor.uri") - if (executorUri.isDefined) { + val cmd = if (executorUri.isDefined) { builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) - val basename = executorUri.get.split('/').last.split('.').head - val cmd = - Seq("bin/spark-submit", - "--class", desc.command.mainClass, - "--master", s"mesos://${conf.get("spark.master")}", - s"../${desc.jarUrl.split("/").last}") - .mkString(" ") + val folderBasename = executorUri.get.split('/').last.split('.').head + + val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit" + + val cmdJar = s"../${desc.jarUrl.split("/").last}" - builder.setValue( - s"cd $basename*; $prefixEnv $cmd") + s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar" } else { val executorSparkHome = req.conf.getOption("spark.mesos.executor.home") .getOrElse { throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") } - val cmd = - Seq(new File(executorSparkHome, "./bin/spark-submit"), - "--class", desc.command.mainClass, - "--master", s"mesos://${conf.get("spark.master")}", - desc.jarUrl.split("/").last) - .mkString(" ") + val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getCanonicalPath - builder.setValue( - s"$prefixEnv $cmd") + val cmdJar = desc.jarUrl.split("/").last + + s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar" } + builder.setValue(cmd) + builder.build } + private def generateCmdOption(req: DriverRequest): Seq[String] = { + Seq( + "--name", req.conf.get("spark.app.name"), + "--class", req.desc.command.mainClass, + "--master", s"mesos://${conf.get("spark.master")}", + "--driver-cores", req.desc.cores.toString, + "--driver-memory", s"${req.desc.mem}M", + "--executor-memory", req.conf.get("spark.executor.memory"), + "--total-executor-cores", req.conf.get("spark.cores.max") + ) + } + override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { // We should try to schedule all the drivers if the offers fit. From d57d77d942d55f94475d5676512f5c2487e96bac Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 11 Mar 2015 13:20:15 -0700 Subject: [PATCH 12/36] Add documentation --- docs/running-on-mesos.md | 23 ++++++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 594bf78b67713..5a34b0baf3ad5 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -78,6 +78,9 @@ To verify that the Mesos cluster is ready for Spark, navigate to the Mesos maste To use Mesos from Spark, you need a Spark binary package available in a place accessible by Mesos, and a Spark driver program configured to connect to Mesos. +Alternatively, you can also install Spark in the same location in all the Mesos slaves, and configure +`spark.mesos.executor.home` (defaults to SPARK_HOME) to point to that location. + ## Uploading Spark Package When Mesos runs a task on a Mesos slave for the first time, that slave must have a Spark binary @@ -107,7 +110,11 @@ the `make-distribution.sh` script included in a Spark source tarball/checkout. The Master URLs for Mesos are in the form `mesos://host:5050` for a single-master Mesos cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooKeeper. -The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: +## Client Mode + +In client mode, a Spark Mesos framework is launched directly on the client machine and waits for the driver output. + +The driver needs some configuration in `spark-env.sh` to interact properly with Mesos: 1. In `spark-env.sh` set some environment variables: * `export MESOS_NATIVE_JAVA_LIBRARY=`. This path is typically @@ -129,8 +136,7 @@ val sc = new SparkContext(conf) {% endhighlight %} (You can also use [`spark-submit`](submitting-applications.html) and configure `spark.executor.uri` -in the [conf/spark-defaults.conf](configuration.html#loading-default-configurations) file. Note -that `spark-submit` currently only supports deploying the Spark driver in `client` mode for Mesos.) +in the [conf/spark-defaults.conf](configuration.html#loading-default-configurations) file.) When running a shell, the `spark.executor.uri` parameter is inherited from `SPARK_EXECUTOR_URI`, so it does not need to be redundantly passed in as a system property. @@ -139,6 +145,17 @@ it does not need to be redundantly passed in as a system property. ./bin/spark-shell --master mesos://host:5050 {% endhighlight %} +## Cluster mode + +Spark on Mesos also supports cluster mode, where the driver is launched in the cluster and the client +can find the results of the driver from the Mesos Web UI. + +To use cluster mode, you must start the MesosClusterDispatcher in your cluster via the `sbin/start-mesos-dispatcher.sh` script, +passing in the Mesos master url (ie: mesos://host:5050). + +From the client, you can submit a job to Mesos cluster by running `spark-submit` and specifying the master url +to the url of the MesosClusterDispatcher (ie: mesos://dispatcher:7077). You can view driver statuses on the +Spark cluster Web UI. # Mesos Run Modes From 8ec76bcde0241e96f475f4514df2931310400060 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 11 Mar 2015 11:41:08 -0700 Subject: [PATCH 13/36] Fix Mesos dispatcher UI. --- .../deploy/mesos/MesosClusterDispatcher.scala | 38 +++++++++----- .../deploy/mesos/ui/DriverOutputPage.scala | 49 ++++++++++++++----- .../cluster/mesos/MesosClusterScheduler.scala | 9 +++- 3 files changed, 71 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 586e7a1b4bf2a..c34255f1cc5d9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -19,11 +19,14 @@ package org.apache.spark.deploy.mesos import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{IntParam, Utils} + import java.io.File +import java.util.concurrent.CountDownLatch + import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.MesosRestServer import org.apache.spark.scheduler.cluster.mesos.{ClusterScheduler, MesosClusterScheduler} -import java.util.concurrent.CountDownLatch + /* * A dispatcher actor that is responsible for managing drivers, that is intended to @@ -34,26 +37,18 @@ import java.util.concurrent.CountDownLatch private [spark] class MesosClusterDispatcher( host: String, serverPort: Int, - webUiPort: Int, conf: SparkConf, + webUi: MesosClusterUI, scheduler: ClusterScheduler) extends Logging { val server = new MesosRestServer(host, serverPort, conf, scheduler) - val dispatcherPublicAddress = { - val envVar = System.getenv("SPARK_PUBLIC_DNS") - if (envVar != null) envVar else host - } - - val webUi = new MesosClusterUI( - new SecurityManager(conf), webUiPort, conf, dispatcherPublicAddress, scheduler) - val sparkHome = new File(sys.env.get("SPARK_HOME").getOrElse(".")) def start() { server.start() - webUi.bind() + // We assume web ui is already started as the scheduler needs the bound port. } def stop() { @@ -63,6 +58,10 @@ private [spark] class MesosClusterDispatcher( } object MesosClusterDispatcher { + def dispatcherPublicAddress(conf: SparkConf, host: String): String = { + val envVar = conf.getenv("SPARK_PUBLIC_DNS") + if (envVar != null) envVar else host + } val shutdownLatch = new CountDownLatch(1) @@ -72,12 +71,27 @@ object MesosClusterDispatcher { conf.setMaster(dispatcherArgs.masterUrl) conf.setAppName("Mesos Cluster Dispatcher") val scheduler = new MesosClusterScheduler(conf) + + // We have to create the webui and bind it early as we need to + // pass the framework web ui url to Mesos which is before the + // scheduler starts. + val webUi = + new MesosClusterUI( + new SecurityManager(conf), + dispatcherArgs.webUiPort, + conf, + dispatcherPublicAddress(conf, dispatcherArgs.host), + scheduler) + + webUi.bind() + + scheduler.frameworkUrl = webUi.activeWebUiUrl scheduler.start() new MesosClusterDispatcher( dispatcherArgs.host, dispatcherArgs.port, - dispatcherArgs.webUiPort, conf, + webUi, scheduler).start() shutdownLatch.await() diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala index c7a7f5026cc8d..9bb872c44a5f9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala @@ -20,8 +20,8 @@ package org.apache.spark.deploy.mesos.ui import org.apache.spark.ui.{UIUtils, WebUIPage} import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.deploy.master.DriverInfo -import org.apache.spark.scheduler.cluster.mesos.{ClusterTaskState, DriverSubmission, ClusterScheduler} +import org.apache.spark.scheduler.cluster.mesos.{ClusterTaskState, DriverSubmission} +import org.apache.mesos.Protos.TaskStatus class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { @@ -30,7 +30,7 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { val queuedHeaders = Seq("DriverID", "Submit Date", "Description") val driverHeaders = queuedHeaders ++ - Seq("Start Date", "Mesos Slave ID", "Mesos Task ID", "State", "Logs") + Seq("Start Date", "Mesos Slave ID", "Mesos Task ID", "State") val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) @@ -47,8 +47,6 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { ; UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster") - - null } def queuedRow(submission: DriverSubmission): Seq[Node] = { @@ -65,13 +63,40 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { {state.submission.submitDate} {state.submission.req.desc.command.mainClass} {state.startDate} - {state.slaveId} - {state.taskId} - {state.taskState} - - stdout, - stderr - + {state.slaveId.getValue} + {state.taskId.getValue} + {stateString(state.taskState)} } + + def stateString(status: Option[TaskStatus]): String = { + if (status.isEmpty) { + return "" + } + + val sb = new StringBuilder + sb.append(s"State: ${status.get.getState}") + + if (status.get.hasMessage) { + sb.append(s", Message: ${status.get.getMessage}") + } + + if (status.get.hasHealthy) { + sb.append(s", Healthy: ${status.get.getHealthy}") + } + + if (status.get.hasSource) { + sb.append(s", Source: ${status.get.getSource}") + } + + if (status.get.hasReason) { + sb.append(s", Reason: ${status.get.getReason}") + } + + if (status.get.hasTimestamp) { + sb.append(s", Time: ${status.get.getTimestamp}") + } + + sb.toString() + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index f03dccd57618d..7d4001994c0af 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -83,6 +83,7 @@ private[spark] trait ClusterScheduler { private[spark] class MesosClusterScheduler(conf: SparkConf) extends Scheduler with MesosSchedulerHelper with ClusterScheduler { + var frameworkUrl: String = _ val master = conf.get("spark.master") val appName = conf.get("spark.app.name") val capacity = conf.getInt("spark.mesos.driver.capacity", 200) @@ -131,7 +132,11 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) def start() { val fwInfo = FrameworkInfo.newBuilder() - .setUser(Utils.getCurrentUserName()).setName(appName).build() + .setUser(Utils.getCurrentUserName()) + .setName(appName) + .setWebuiUrl(frameworkUrl) + .setCheckpoint(true) + .build() startScheduler("MesosClusterScheduler", master, MesosClusterScheduler.this, fwInfo) } @@ -188,6 +193,8 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar" } else { val executorSparkHome = req.conf.getOption("spark.mesos.executor.home") + .orElse(conf.getOption("spark.home")) + .orElse(Option(System.getenv("SPARK_HOME"))) .getOrElse { throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") } From 6887e5ec37fd136f77f73130cdd36bf8e08c53b6 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 11 Mar 2015 17:48:19 -0700 Subject: [PATCH 14/36] Support looking at SPARK_EXECUTOR_URI env variable in schedulers --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 10 ++++++---- .../cluster/mesos/MesosClusterScheduler.scala | 2 ++ .../cluster/mesos/MesosSchedulerBackend.scala | 10 ++++++---- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d0be3df23f13b..921140e367d1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -125,8 +125,10 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ENDPOINT_NAME) - val uri = conf.get("spark.executor.uri", null) - if (uri == null) { + val uri = conf.getOption("spark.executor.uri") + .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + + if (uri.isEmpty) { val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath command.setValue( "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" @@ -139,7 +141,7 @@ private[spark] class CoarseMesosSchedulerBackend( } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". - val basename = uri.split('/').last.split('.').head + val basename = uri.get.split('/').last.split('.').head command.setValue( s"cd $basename*; $prefixEnv " + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + @@ -148,7 +150,7 @@ private[spark] class CoarseMesosSchedulerBackend( s" --hostname ${offer.getHostname}" + s" --cores $numCores" + s" --app-id $appId") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) } command.build() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 7d4001994c0af..be5ea6f700bca 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -181,6 +181,8 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) val cmdOptions = generateCmdOption(req) val executorUri = req.conf.getOption("spark.executor.uri") + .orElse(req.desc.command.environment.get("SPARK_EXECUTOR_URI")) + val cmd = if (executorUri.isDefined) { builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index b145ecb4c40ef..833c2cd291a6d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -101,17 +101,19 @@ private[spark] class MesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = sc.conf.get("spark.executor.uri", null) + val uri = sc.conf.getOption("spark.executor.uri") + .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + val executorBackendName = classOf[MesosExecutorBackend].getName - if (uri == null) { + if (uri.isEmpty) { val executorPath = new File(executorSparkHome, "/bin/spark-class").getCanonicalPath command.setValue(s"$prefixEnv $executorPath $executorBackendName") } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". - val basename = uri.split('/').last.split('.').head + val basename = uri.get.split('/').last.split('.').head command.setValue(s"cd ${basename}*; $prefixEnv ./bin/spark-class $executorBackendName") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) } val cpus = Resource.newBuilder() .setName("cpus") From 543a98d8a51085f434bd49aad68363c5e1f4e677 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Thu, 12 Mar 2015 01:01:05 -0700 Subject: [PATCH 15/36] Schedule multiple jobs --- .../cluster/mesos/MesosClusterScheduler.scala | 104 +++++++++++------- 1 file changed, 63 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index be5ea6f700bca..0e1bf5eb5bd64 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -75,8 +75,11 @@ private[spark] case class ClusterSchedulerState( private[spark] trait ClusterScheduler { def submitDriver(desc: DriverRequest): SubmitResponse + def killDriver(submissionId: String): KillResponse + def getStatus(submissionId: String): StatusResponse + def getState(): ClusterSchedulerState } @@ -225,65 +228,84 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) ) } + private [spark] case class ResourceOffer(val offer: Offer, var cpu: Double, var mem: Double) + override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { - // We should try to schedule all the drivers if the offers fit. + var submission = queue.peek - // Non-blocking poll. - val submissionOption = Option(queue.poll(0, TimeUnit.SECONDS)) + val usedOffers = new mutable.HashSet[OfferID] - if (submissionOption.isEmpty) { - offers.foreach(o => driver.declineOffer(o.getId)) - return + val currentOffers = offers.map { + o => + ResourceOffer( + o, + getResource(o.getResourcesList, "cpus"), + getResource(o.getResourcesList, "mem")) } - val submission = submissionOption.get + var canSchedule = true - var remainingOffers = offers + while (canSchedule && submission != null) { + val driverCpu = submission.req.desc.cores + val driverMem = submission.req.desc.mem - val driverCpu = submission.req.desc.cores - val driverMem = submission.req.desc.mem + val offerOption = currentOffers.find { o => + o.cpu >= driverCpu && o.mem >= driverMem + } - // Should use the passed in driver cpu and memory. - val offerOption = offers.find { o => - getResource(o.getResourcesList, "cpus") >= driverCpu && - getResource(o.getResourcesList, "mem") >= driverMem - } + if (offerOption.isDefined) { + val offer = offerOption.get - offerOption.foreach { offer => - val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() + offer.cpu -= driverCpu + offer.mem -= driverMem - val cpuResource = Resource.newBuilder() - .setName("cpus").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() + val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() - val memResource = Resource.newBuilder() - .setName("mem").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() + val cpuResource = Resource.newBuilder() + .setName("cpus").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() - val commandInfo = buildCommand(submission.req) + val memResource = Resource.newBuilder() + .setName("mem").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() - val taskInfo = TaskInfo.newBuilder() - .setTaskId(taskId) - .setName(s"driver for ${submission.req.desc.command.mainClass}") - .setSlaveId(offer.getSlaveId) - .setCommand(commandInfo) - .addResources(cpuResource) - .addResources(memResource) - .build + val commandInfo = buildCommand(submission.req) - //TODO: logDebug("") - driver.launchTasks(Collections.singleton(offer.getId), Collections.singleton(taskInfo)) + val taskInfo = TaskInfo.newBuilder() + .setTaskId(taskId) + .setName(s"driver for ${submission.req.desc.command.mainClass}") + .setSlaveId(offer.offer.getSlaveId) + .setCommand(commandInfo) + .addResources(cpuResource) + .addResources(memResource) + .build - stateLock.synchronized { - launchedDrivers(submission.submissionId) = - ClusterTaskState(submission, taskId, offer.getSlaveId, - None, DriverState.SUBMITTED, new Date()) - } + logDebug(s"Launching task ${taskInfo}, with offer: ${offer.offer}") + driver.launchTasks(Collections.singleton(offer.offer.getId), Collections.singleton(taskInfo)) - remainingOffers = offers.filter(o => o.getId.equals(offer.getId)) + stateLock.synchronized { + launchedDrivers(submission.submissionId) = + ClusterTaskState(submission, taskId, offer.offer.getSlaveId, + None, DriverState.SUBMITTED, new Date()) + } + + usedOffers += offer.offer.getId + + // remove driver from queue. + queue.poll(0, TimeUnit.SECONDS) + + submission = queue.peek + } else { + // We can stop at very first driver that we cannot schedule on. + // TODO: We should remove the top driver that cannot be scheduled + // over a configurable time period. + canSchedule = false + } } - remainingOffers.foreach(o => driver.declineOffer(o.getId)) + offers + .filter(o => !usedOffers.contains(o.getId)) + .foreach(o => driver.declineOffer(o.getId)) } def getState(): ClusterSchedulerState = { From febfabab3b141c712f640759113d15cd0b497e15 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Thu, 12 Mar 2015 13:59:52 -0700 Subject: [PATCH 16/36] Bound the finished drivers in memory --- .../cluster/mesos/MesosClusterScheduler.scala | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 0e1bf5eb5bd64..914990bbc264f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -89,15 +89,17 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) var frameworkUrl: String = _ val master = conf.get("spark.master") val appName = conf.get("spark.app.name") - val capacity = conf.getInt("spark.mesos.driver.capacity", 200) + val queuedCapacity = conf.getInt("spark.deploy.mesos.queuedDrivers", 200) + val retainedDrivers = conf.getInt("spark.deploy.retainedDrivers", 200) + val stateLock = new Object val launchedDrivers = new mutable.HashMap[String, ClusterTaskState]() - // TODO: Bound this finished drivers map or make it a array - val finishedDrivers = new mutable.HashMap[String, ClusterTaskState]() + val finishedDrivers = new mutable.ArrayBuffer[ClusterTaskState](retainedDrivers) val nextDriverNumber: AtomicLong = new AtomicLong(0) var appId: String = _ - private val queue = new LinkedBlockingQueue[DriverSubmission](capacity) + + private val queue = new LinkedBlockingQueue[DriverSubmission](queuedCapacity) def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs @@ -319,7 +321,7 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) ClusterSchedulerState( queueCopy, copyDriverStates(launchedDrivers.values), - copyDriverStates(finishedDrivers.values)) + copyDriverStates(finishedDrivers)) } } @@ -378,11 +380,18 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) val driverState = getDriverState(status.getState) val state = if (isFinished(status.getState)) { val launchedState = launchedDrivers.remove(taskId).get - finishedDrivers(taskId) = launchedState + if (finishedDrivers.size >= retainedDrivers) { + val toRemove = math.max(retainedDrivers / 10, 1) + finishedDrivers.trimStart(toRemove) + } + + finishedDrivers += launchedState + launchedState } else { launchedDrivers(taskId) } + state.taskState = Option(status) state.driverState = driverState } else { From 3d4dfa1308c18619112b3735ffa306cd21b71ee1 Mon Sep 17 00:00:00 2001 From: Luc Bourlier Date: Thu, 12 Mar 2015 00:05:04 +0100 Subject: [PATCH 17/36] Adds support to kill submissions --- .../cluster/mesos/MesosClusterScheduler.scala | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 914990bbc264f..fa7a91b590a58 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -122,17 +122,23 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) def killDriver(submissionId: String): KillResponse = { stateLock.synchronized { if (launchedDrivers.contains(submissionId)) { - // Kill the JOB!!!! - return KillResponse(submissionId, true, Option("Killing running driver")) + // Check if submission is running + val task = launchedDrivers(submissionId) + driver.killTask(task.taskId) + Some(KillResponse(submissionId, true, Option("Killing running driver"))) + } else { + None } + }.orElse { + // Check if submission is queued + if (queue.remove(new DriverSubmission(submissionId, null, null))) { + Some(KillResponse(submissionId, true, Option("Removed driver while it's still pending"))) + } else { + None + } + }.getOrElse{ + KillResponse(submissionId, false, Option("Cannot find driver")) } - - // Check if submission is queued - if (queue.remove(new DriverSubmission(submissionId, null, null))) { - return KillResponse(submissionId, true, Option("Removed driver while it's still pending")) - } - - KillResponse(submissionId, false, Option("Cannot find driver")) } def start() { From 371ce65e1a1b5366bee2787235a29654ad5e6f1b Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 18 Mar 2015 15:01:38 -0700 Subject: [PATCH 18/36] Handle cluster mode recovery and state persistence. --- .../{master => }/SparkCuratorUtil.scala | 4 +- .../spark/deploy/master/DriverState.scala | 2 +- .../master/ZooKeeperLeaderElectionAgent.scala | 1 + .../master/ZooKeeperPersistenceEngine.scala | 1 + .../deploy/mesos/MesosClusterDispatcher.scala | 118 ++++--- .../deploy/mesos/MesosDriverDescription.scala | 26 ++ .../deploy/mesos/ui/DriverOutputPage.scala | 8 +- .../deploy/mesos/ui/MesosClusterUI.scala | 3 +- .../spark/deploy/rest/MesosRestServer.scala | 33 +- .../spark/deploy/worker/DriverRunner.scala | 6 +- .../apache/spark/deploy/worker/Worker.scala | 13 +- .../spark/deploy/worker/ui/LogPage.scala | 6 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- .../mesos/ClusterPersistenceEngine.scala | 112 +++++++ .../scheduler/cluster/mesos/DriverQueue.scala | 92 +++++ .../cluster/mesos/LaunchedDrivers.scala | 68 ++++ .../cluster/mesos/MesosClusterScheduler.scala | 315 +++++++++++------- .../scheduler/cluster/mesos/MesosState.scala | 108 ++++++ .../spark/deploy/JsonProtocolSuite.scala | 7 +- .../deploy/worker/DriverRunnerTest.scala | 7 +- .../mesos/MesosClusterSchedulerSuite.scala | 24 +- 21 files changed, 755 insertions(+), 201 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/{master => }/SparkCuratorUtil.scala (95%) create mode 100644 core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/ClusterPersistenceEngine.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosState.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala rename to core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala index 5b22481ea8c5f..52f4582107e85 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.master +package org.apache.spark.deploy import scala.collection.JavaConversions._ @@ -25,7 +25,7 @@ import org.apache.zookeeper.KeeperException import org.apache.spark.{Logging, SparkConf} -private[deploy] object SparkCuratorUtil extends Logging { +private[spark] object SparkCuratorUtil extends Logging { private val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 private val ZK_SESSION_TIMEOUT_MILLIS = 60000 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala index 35ff33a61653c..26a68bade3c60 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[deploy] object DriverState extends Enumeration { +private[spark] object DriverState extends Enumeration { type DriverState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 4823fd7cac0cb..52758d6a7c4be 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 @@ -23,6 +23,7 @@ import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.master.MasterMessages._ import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} +import org.apache.spark.deploy.SparkCuratorUtil private[master] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { 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 a285783f72000..80db6d474b5c1 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 @@ -26,6 +26,7 @@ import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkCuratorUtil private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index c34255f1cc5d9..7aada65675fa8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -18,15 +18,16 @@ package org.apache.spark.deploy.mesos import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.util.{IntParam, Utils} +import org.apache.spark.util.{SignalLogger, IntParam, Utils} import java.io.File -import java.util.concurrent.CountDownLatch +import java.util.concurrent.{TimeUnit, CountDownLatch} import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.MesosRestServer -import org.apache.spark.scheduler.cluster.mesos.{ClusterScheduler, MesosClusterScheduler} - +import org.apache.spark.scheduler.cluster.mesos._ +import org.apache.spark.deploy.mesos.MesosClusterDispatcher.ClusterDispatcherArguments +import org.apache.mesos.state.{ZooKeeperState, InMemoryState} /* * A dispatcher actor that is responsible for managing drivers, that is intended to @@ -35,76 +36,101 @@ import org.apache.spark.scheduler.cluster.mesos.{ClusterScheduler, MesosClusterS * a daemon to launch drivers as Mesos frameworks upon request. */ private [spark] class MesosClusterDispatcher( - host: String, - serverPort: Int, - conf: SparkConf, - webUi: MesosClusterUI, - scheduler: ClusterScheduler) extends Logging { + args: ClusterDispatcherArguments, + conf: SparkConf) extends Logging { + + def dispatcherPublicAddress(conf: SparkConf, host: String): String = { + val envVar = conf.getenv("SPARK_PUBLIC_DNS") + if (envVar != null) envVar else host + } + + val recoveryMode = conf.get("spark.deploy.recoveryMode", "NONE").toUpperCase() + logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode) + + val engineFactory = recoveryMode match { + case "NONE" => new BlackHolePersistenceEngineFactory + case "ZOOKEEPER" => { + new ZookeeperClusterPersistenceEngineFactory(conf) + } + } - val server = new MesosRestServer(host, serverPort, conf, scheduler) + val scheduler = new MesosClusterScheduler( + engineFactory, + conf) + + val server = new MesosRestServer(args.host, args.port, conf, scheduler) + + val webUi = new MesosClusterUI( + new SecurityManager(conf), + args.webUiPort, + conf, + dispatcherPublicAddress(conf, args.host), + scheduler) + + val shutdownLatch = new CountDownLatch(1) val sparkHome = new File(sys.env.get("SPARK_HOME").getOrElse(".")) def start() { + webUi.bind() + scheduler.frameworkUrl = webUi.activeWebUiUrl + scheduler.start() server.start() - // We assume web ui is already started as the scheduler needs the bound port. + } + + def awaitShutdown() { + shutdownLatch.await() } def stop() { webUi.stop() server.stop() + scheduler.stop() + shutdownLatch.countDown() } } -object MesosClusterDispatcher { - def dispatcherPublicAddress(conf: SparkConf, host: String): String = { - val envVar = conf.getenv("SPARK_PUBLIC_DNS") - if (envVar != null) envVar else host - } - - val shutdownLatch = new CountDownLatch(1) - +private[mesos] object MesosClusterDispatcher extends Logging { def main(args: Array[String]) { + SignalLogger.register(log) + val conf = new SparkConf val dispatcherArgs = new ClusterDispatcherArguments(args, conf) + conf.setMaster(dispatcherArgs.masterUrl) conf.setAppName("Mesos Cluster Dispatcher") - val scheduler = new MesosClusterScheduler(conf) - - // We have to create the webui and bind it early as we need to - // pass the framework web ui url to Mesos which is before the - // scheduler starts. - val webUi = - new MesosClusterUI( - new SecurityManager(conf), - dispatcherArgs.webUiPort, - conf, - dispatcherPublicAddress(conf, dispatcherArgs.host), - scheduler) - webUi.bind() + val dispatcher = new MesosClusterDispatcher( + dispatcherArgs, + conf) - scheduler.frameworkUrl = webUi.activeWebUiUrl - scheduler.start() - new MesosClusterDispatcher( - dispatcherArgs.host, - dispatcherArgs.port, - conf, - webUi, - scheduler).start() + dispatcher.start() - shutdownLatch.await() + val shutdownHook = new Thread() { + override def run() { + logInfo("Shutdown hook is shutting down dispatcher") + dispatcher.stop() + dispatcher.awaitShutdown() + } + } + + Runtime.getRuntime.addShutdownHook(shutdownHook) + + dispatcher.awaitShutdown() } class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8081 - var masterUrl: String = null + var masterUrl: String = _ + var propertiesFile: String = _ parse(args.toList) + propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) + def parse(args: List[String]): Unit = args match { case ("--host" | "-h") :: value :: tail => Utils.checkHost(value, "Please use hostname " + value) @@ -127,6 +153,10 @@ object MesosClusterDispatcher { masterUrl = value.stripPrefix("mesos://") parse(tail) + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + case ("--help") :: tail => printUsageAndExit(0) @@ -152,7 +182,9 @@ object MesosClusterDispatcher { " -h HOST, --host HOST Hostname to listen on\n" + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + - " -m --master MASTER URI for connecting to Mesos master\n") + " -m --master MASTER URI for connecting to Mesos master\n" + + " --properties-file FILE Path to a custom Spark properties file.\n" + + " Default is conf/spark-defaults.conf.") System.exit(exitCode) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala new file mode 100644 index 0000000000000..ab764296e7a25 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.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.mesos + +import org.apache.spark.deploy.DriverDescription +import scala.collection.mutable + +private[spark] class MesosDriverDescription( + val desc: DriverDescription, + val schedulerProperties: mutable.HashMap[String, String]) extends Serializable { +} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala index 9bb872c44a5f9..b0e520415a42a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala @@ -30,12 +30,13 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { val queuedHeaders = Seq("DriverID", "Submit Date", "Description") val driverHeaders = queuedHeaders ++ - Seq("Start Date", "Mesos Slave ID", "Mesos Task ID", "State") + Seq("Start Date", "Mesos Slave ID", "State") val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers) val content = +

Mesos Framework ID: {state.appId}

Queued Drivers:

@@ -53,7 +54,7 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { {submission.submissionId} {submission.submitDate} - {submission.req.desc.command.mainClass} + {submission.desc.desc.command.mainClass} } @@ -61,10 +62,9 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { {state.submission.submissionId} {state.submission.submitDate} - {state.submission.req.desc.command.mainClass} + {state.submission.desc.desc.command.mainClass} {state.startDate} {state.slaveId.getValue} - {state.taskId.getValue} {stateString(state.taskState)} } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala index ff1427fe9aad7..a7382a401a63f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala @@ -21,7 +21,6 @@ import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.SparkConf import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.deploy.worker.ui.ActiveWebUiUrlAccessor import org.apache.spark.scheduler.cluster.mesos.ClusterScheduler /** @@ -33,7 +32,7 @@ private [spark] class MesosClusterUI( conf: SparkConf, dispatcherPublicAddress: String, val scheduler: ClusterScheduler) - extends WebUI(securityManager, port, conf) with ActiveWebUiUrlAccessor { + extends WebUI(securityManager, port, conf) { initialize() diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala index 60a14c01af869..0184965f47e26 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala @@ -26,7 +26,8 @@ import org.apache.spark.deploy.Command import org.apache.spark.{SparkConf, SPARK_VERSION => sparkVersion} import org.apache.spark.util.Utils import org.apache.spark.scheduler.cluster.mesos.ClusterScheduler -import org.apache.spark.scheduler.cluster.mesos.DriverRequest +import scala.collection.mutable +import org.apache.spark.deploy.mesos.MesosDriverDescription /** * A server that responds to requests submitted by the [[RestClient]]. @@ -74,7 +75,7 @@ class MesosSubmitRequestServlet( * This does not currently consider fields used by python applications since python * is not supported in mesos cluster mode yet. */ - private def buildDriverRequest(request: CreateSubmissionRequest): DriverRequest = { + private def buildDriverDescription(request: CreateSubmissionRequest): MesosDriverDescription = { // Required fields, including the main class because python is not yet supported val appResource = Option(request.appResource).getOrElse { throw new SubmitRestMissingFieldException("Application jar is missing.") @@ -93,6 +94,25 @@ class MesosSubmitRequestServlet( val driverCores = sparkProperties.get("spark.driver.cores") val appArgs = request.appArgs val environmentVariables = request.environmentVariables + val schedulerProperties = new mutable.HashMap[String, String] + // Store Spark submit specific arguments here to pass to the scheduler. + schedulerProperties("spark.app.name") = sparkProperties.getOrElse("spark.app.name", mainClass) + + sparkProperties.get("spark.executor.memory").foreach { + v => schedulerProperties("spark.executor.memory") = v + } + + sparkProperties.get("spark.cores.max").foreach { + v => schedulerProperties("spark.cores.max") = v + } + + sparkProperties.get("spark.executor.uri").foreach { + v => schedulerProperties("spark.executor.uri") = v + } + + sparkProperties.get("spark.mesos.executor.home").foreach { + v => schedulerProperties("spark.mesos.executor.home") = v + } // Construct driver description val conf = new SparkConf(false) @@ -109,9 +129,10 @@ class MesosSubmitRequestServlet( val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES) - DriverRequest(new DriverDescription( - appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command), - conf) + val desc = new DriverDescription( + appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command) + + new MesosDriverDescription(desc, schedulerProperties) } protected override def handleSubmit( @@ -120,7 +141,7 @@ class MesosSubmitRequestServlet( responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { requestMessage match { case submitRequest: CreateSubmissionRequest => - val driverDescription = buildDriverRequest(submitRequest) + val driverDescription = buildDriverDescription(submitRequest) val response = scheduler.submitDriver(driverDescription) val submitResponse = new CreateSubmissionResponse submitResponse.serverSparkVersion = sparkVersion diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index ef7a703bffe67..e2bb42787a016 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -44,8 +44,7 @@ private[deploy] class DriverRunner( val sparkHome: File, val driverDesc: DriverDescription, val worker: ActorRef, - val workerUrl: String, - val securityManager: SecurityManager) + val workerUrl: String) extends Logging { @volatile private var process: Option[Process] = None @@ -137,9 +136,12 @@ private[deploy] class DriverRunner( * Will throw an exception if there are errors downloading the jar. */ private def downloadUserJar(driverDir: File): String = { + val jarPath = new Path(driverDesc.jarUrl) val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val jarFileSystem = jarPath.getFileSystem(hadoopConf) + val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) val jarFileName = jarPath.getName val localJarFile = new File(driverDir, jarFileName) 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 1c9802ca81678..c4c24a7866aa3 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -61,7 +61,7 @@ private[worker] class Worker( assert (port > 0) // For worker and executor IDs - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // Send a heartbeat every (heartbeat timeout) / 4 milliseconds private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 @@ -85,10 +85,10 @@ private[worker] class Worker( private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders - private val CLEANUP_INTERVAL_MILLIS = + private val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - private val APP_DATA_RETENTION_SECS = + private val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) private val testing: Boolean = sys.props.contains("spark.testing") @@ -112,7 +112,7 @@ private[worker] class Worker( } else { new File(sys.env.get("SPARK_HOME").getOrElse(".")) } - + var workDir: File = null val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] @@ -134,7 +134,7 @@ private[worker] class Worker( private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) private val workerSource = new WorkerSource(this) - + private var registrationRetryTimer: Option[Cancellable] = None var coresUsed = 0 @@ -436,8 +436,7 @@ private[worker] class Worker( sparkHome, driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), self, - akkaUrl, - securityMgr) + akkaUrl) drivers(driverId) = driver driver.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 839a38a33194e..88170d4df3053 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.worker.ui -import java.io.File import javax.servlet.http.HttpServletRequest import scala.xml.Node @@ -31,9 +30,6 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with private val worker = parent.worker private val workDir = parent.workDir -private[spark] class LogPage( - urlAccessor: ActiveWebUiUrlAccessor, - workDir: File) extends WebUIPage("logPage") with Logging { def renderLog(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 @@ -77,7 +73,7 @@ private[spark] class LogPage( } val (logText, startByte, endByte, logLength) = getLog(logDir, logType, offset, byteLength) - val linkToMaster =

Back to Master

+ val linkToMaster =

Back to Master

val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} val backButton = 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 8eb5fe0376598..b3bb5f911dbd7 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 @@ -44,7 +44,7 @@ class WorkerWebUI( /** Initialize all components of the server. */ def initialize() { - val logPage = new LogPage(worker, workDir) + val logPage = new LogPage(this) attachPage(logPage) attachPage(new WorkerPage(this)) attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static")) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/ClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/ClusterPersistenceEngine.scala new file mode 100644 index 0000000000000..ecca4a5817495 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/ClusterPersistenceEngine.scala @@ -0,0 +1,112 @@ +/* + * 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.scheduler.cluster.mesos + +import scala.collection.JavaConversions._ + +import org.apache.curator.framework.CuratorFramework +import org.apache.spark.deploy.SparkCuratorUtil + + +import org.apache.zookeeper.CreateMode +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.Utils +import org.apache.zookeeper.KeeperException.NoNodeException + +abstract class ClusterPersistenceEngineFactory(conf: SparkConf) { + def createEngine(path: String): ClusterPersistenceEngine +} + +private[spark] class ZookeeperClusterPersistenceEngineFactory(conf: SparkConf) + extends ClusterPersistenceEngineFactory(conf) { + + lazy val zk = SparkCuratorUtil.newClient(conf) + + def createEngine(path: String): ClusterPersistenceEngine = { + new ZookeeperClusterPersistenceEngine(path, zk, conf) + } +} + +private[spark] class BlackHolePersistenceEngineFactory + extends ClusterPersistenceEngineFactory(null) { + def createEngine(path: String): ClusterPersistenceEngine = { + new BlackHoleClusterPersistenceEngine + } +} + +trait ClusterPersistenceEngine { + def persist(name: String, obj: Object) + def expunge(name: String) + def fetch[T](name: String): Option[T] + def fetchAll[T](): Iterable[T] +} + +private[spark] class BlackHoleClusterPersistenceEngine extends ClusterPersistenceEngine { + override def persist(name: String, obj: Object): Unit = {} + + override def fetch[T](name: String): Option[T] = None + + override def expunge(name: String): Unit = {} + + override def fetchAll[T](): Iterable[T] = Iterable.empty[T] +} + +private[spark] class ZookeeperClusterPersistenceEngine( + baseDir: String, + zk: CuratorFramework, + conf: SparkConf) + extends ClusterPersistenceEngine with Logging { + private val WORKING_DIR = + conf.get("spark.deploy.zookeeper.dir", "/spark_mesos_dispatcher") + "/" + baseDir + + SparkCuratorUtil.mkdir(zk, WORKING_DIR) + + def path(name: String): String = { + WORKING_DIR + "/" + name + } + + override def expunge(name: String) { + zk.delete().forPath(path(name)) + } + + override def persist(name: String, obj: Object) { + val serialized = Utils.serialize(obj) + val zkPath = path(name) + zk.create().withMode(CreateMode.PERSISTENT).forPath(zkPath, serialized) + } + + override def fetch[T](name: String): Option[T] = { + val zkPath = path(name) + + try { + val fileData = zk.getData().forPath(zkPath) + Some(Utils.deserialize[T](fileData)) + } catch { + case e: NoNodeException => None + case e: Exception => { + logWarning("Exception while reading persisted file, deleting", e) + zk.delete().forPath(zkPath) + None + } + } + } + + override def fetchAll[T](): Iterable[T] = { + zk.getChildren.forPath(WORKING_DIR).map(fetch[T]).flatten + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala new file mode 100644 index 0000000000000..55ab372d518a3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala @@ -0,0 +1,92 @@ +/* + * 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.scheduler.cluster.mesos + +import scala.collection.mutable +import javax.annotation.concurrent.NotThreadSafe + +/** + * A request queue for launching drivers in Mesos cluster mode. + * This queue automatically stores the state after each pop/push + * so it can be recovered later. + * This queue is also bounded and rejects offers when it's full. + * @param state Mesos state abstraction to fetch persistent state. + */ +@NotThreadSafe +class DriverQueue(state: ClusterPersistenceEngine, capacity: Int) { + var queue: mutable.Queue[DriverSubmission] = new mutable.Queue[DriverSubmission]() + private var count = 0 + + initialize() + + def initialize() { + state.fetchAll[DriverSubmission]().foreach(d => queue.enqueue(d)) + + // This size might be larger than the passed in capacity, but we allow + // this so we don't lose queued drivers. + count = queue.size + } + + def isFull = count >= capacity + + def contains(submissionId: String): Boolean = { + queue.exists(s => s.submissionId.equals(submissionId)) + } + + def offer(submission: DriverSubmission): Boolean = { + if (isFull) { + return false + } + + queue.enqueue(submission) + state.persist(submission.submissionId, submission) + true + } + + def remove(submissionId: String): Boolean = { + val removed = queue.dequeueFirst(d => d.submissionId.equals(submissionId)) + if (removed.isDefined) { + state.expunge(removed.get.submissionId) + true + } + + false + } + + def peek(): Option[DriverSubmission] = { + queue.headOption + } + + def poll(): Option[DriverSubmission] = { + if (queue.isEmpty) { + None + } else { + val item = queue.dequeue() + state.expunge(item.submissionId) + Some(item) + } + } + + // Returns a copy of the queued drivers. + def drivers: Iterable[DriverSubmission] = { + val buffer = new Array[DriverSubmission](queue.size) + queue.copyToArray(buffer) + buffer + } + +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala new file mode 100644 index 0000000000000..9659ca6c6e426 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala @@ -0,0 +1,68 @@ +/* + * 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.scheduler.cluster.mesos + +import scala.collection.mutable +import org.apache.mesos.Protos.{TaskID, SlaveID} + +class LaunchedDrivers(state: ClusterPersistenceEngine) { + private val drivers = new mutable.HashMap[String, ClusterTaskState] + + // Holds the list of tasks that needs to reconciliation from the master. + // All states that are loaded after failover are added here. + val pendingRecover = new mutable.HashMap[String, SlaveID] + + initialize() + + def initialize() { + state.fetchAll[ClusterTaskState]().foreach { + case state => + drivers(state.taskId.getValue) = state + pendingRecover(state.taskId.getValue) = state.slaveId + } + } + + def get(submissionId: String) = drivers(submissionId) + + def states: Iterable[ClusterTaskState] = { + drivers.values.map(_.copy()).toList + } + + def contains(submissionId: String): Boolean = drivers.contains(submissionId) + + def remove(submissionId: String): Option[ClusterTaskState] = { + if (pendingRecover.contains(submissionId)) { + pendingRecover.remove(submissionId) + } + + val removedState = drivers.remove(submissionId) + state.expunge(submissionId) + removedState + } + + def set(submissionId: String, newState: ClusterTaskState) { + if (pendingRecover.contains(newState.taskId.getValue)) { + pendingRecover.remove(newState.taskId.getValue) + } + drivers(submissionId) = newState + state.persist(submissionId, newState) + } + + def isEmpty = drivers.isEmpty + +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index fa7a91b590a58..bed36b5846a2d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -17,30 +17,33 @@ package org.apache.spark.scheduler.cluster.mesos +import java.io.File import java.text.SimpleDateFormat import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} -import java.util.{List => JList} -import java.util.{Collections, Date} +import java.util.concurrent.locks.ReentrantLock +import java.util.{Date, List => JList} + import org.apache.mesos.{SchedulerDriver, Scheduler} import org.apache.mesos.Protos._ -import org.apache.spark.deploy.DriverDescription import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.SparkConf +import org.apache.spark.SparkException import org.apache.spark.util.Utils + import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ -import org.apache.mesos.Protos.Environment.Variable -import org.apache.spark.SparkException -import java.io.File -case class DriverRequest(desc: DriverDescription, conf: SparkConf) +import scala.concurrent.duration.Duration +import org.apache.mesos.Protos.Environment.Variable +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.mesos.Protos.TaskStatus.Reason private[spark] class DriverSubmission( val submissionId: String, - val req: DriverRequest, - val submitDate: Date) { + val desc: MesosDriverDescription, + val submitDate: Date) extends Serializable { def canEqual(other: Any): Boolean = other.isInstanceOf[DriverSubmission] @@ -58,7 +61,7 @@ private [spark] case class ClusterTaskState( val slaveId: SlaveID, var taskState: Option[TaskStatus], var driverState: DriverState, - var startDate: Date) { + var startDate: Date) extends Serializable { def copy(): ClusterTaskState = { ClusterTaskState(submission, taskId, slaveId, taskState, driverState, startDate) } @@ -69,12 +72,13 @@ private[spark] case class StatusResponse(id: String, success: Boolean, message: private[spark] case class KillResponse(id: String, success: Boolean, message: Option[String]) private[spark] case class ClusterSchedulerState( + appId: String, queuedDrivers: Iterable[DriverSubmission], launchedDrivers: Iterable[ClusterTaskState], finishedDrivers: Iterable[ClusterTaskState]) private[spark] trait ClusterScheduler { - def submitDriver(desc: DriverRequest): SubmitResponse + def submitDriver(desc: MesosDriverDescription): SubmitResponse def killDriver(submissionId: String): KillResponse @@ -83,23 +87,30 @@ private[spark] trait ClusterScheduler { def getState(): ClusterSchedulerState } -private[spark] class MesosClusterScheduler(conf: SparkConf) - extends Scheduler with MesosSchedulerHelper with ClusterScheduler { +private[spark] class MesosClusterScheduler( + engineFactory: ClusterPersistenceEngineFactory, + conf: SparkConf) extends Scheduler with MesosSchedulerHelper with ClusterScheduler { var frameworkUrl: String = _ + val master = conf.get("spark.master") val appName = conf.get("spark.app.name") val queuedCapacity = conf.getInt("spark.deploy.mesos.queuedDrivers", 200) val retainedDrivers = conf.getInt("spark.deploy.retainedDrivers", 200) + val state = engineFactory.createEngine("scheduler") + val stateTimeout = + Duration.create(conf.getLong("spark.mesos.cluster.recover.timeout", 30), "seconds") - val stateLock = new Object - val launchedDrivers = new mutable.HashMap[String, ClusterTaskState]() + val stateLock = new ReentrantLock() val finishedDrivers = new mutable.ArrayBuffer[ClusterTaskState](retainedDrivers) + val nextDriverNumber: AtomicLong = new AtomicLong(0) - var appId: String = _ + var appId: String = null - private val queue = new LinkedBlockingQueue[DriverSubmission](queuedCapacity) + private var launchedDrivers: LaunchedDrivers = _ + + private var queue: DriverQueue = _ def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs @@ -108,30 +119,33 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) } - def submitDriver(req: DriverRequest): SubmitResponse = { - val submitDate: Date = new Date() - val submissionId: String = newDriverId(submitDate) - val submission = new DriverSubmission(submissionId, req, submitDate) - if (queue.offer(submission)) { + def submitDriver(desc: MesosDriverDescription): SubmitResponse = { + stateLock.synchronized { + if (queue.isFull) { + return SubmitResponse("", false, Option("Already reached maximum submission size")) + } + + val submitDate: Date = new Date() + val submissionId: String = newDriverId(submitDate) + val submission = new DriverSubmission(submissionId, desc, submitDate) + queue.offer(submission) SubmitResponse(submissionId, true, None) - } else { - SubmitResponse(submissionId, false, Option("Already reached maximum submission size")) } } def killDriver(submissionId: String): KillResponse = { stateLock.synchronized { + // First check if submission is running or launched. if (launchedDrivers.contains(submissionId)) { - // Check if submission is running - val task = launchedDrivers(submissionId) + val task = launchedDrivers.get(submissionId) driver.killTask(task.taskId) Some(KillResponse(submissionId, true, Option("Killing running driver"))) } else { None } }.orElse { - // Check if submission is queued - if (queue.remove(new DriverSubmission(submissionId, null, null))) { + // Second we check if it's still queued. + if (queue.remove(submissionId)) { Some(KillResponse(submissionId, true, Option("Removed driver while it's still pending"))) } else { None @@ -141,38 +155,88 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) } } + private def recoverState { + stateLock.synchronized { + queue = new DriverQueue(engineFactory.createEngine("driverQueue"), queuedCapacity) + + launchedDrivers = new LaunchedDrivers(engineFactory.createEngine("launchedDrivers")) + + // TODO: Consider storing finished drivers so we can show them on the UI after + // failover. For now we clear the history on each recovery. + finishedDrivers.clear() + } + } + def start() { - val fwInfo = FrameworkInfo.newBuilder() + //TODO: Implement leader election to make sure only one framework running in the cluster. + val fwId = state.fetch[String]("frameworkId") + + val builder = FrameworkInfo.newBuilder() .setUser(Utils.getCurrentUserName()) .setName(appName) .setWebuiUrl(frameworkUrl) .setCheckpoint(true) - .build() - startScheduler("MesosClusterScheduler", master, MesosClusterScheduler.this, fwInfo) + .setFailoverTimeout(Integer.MAX_VALUE) // Setting to max for tasks keep running until recovery + + fwId.foreach { id => + builder.setId(FrameworkID.newBuilder().setValue(id).build()) + appId = id + } + + // Recover scheduler state that is persisted. + // We still need to do task reconciliation to be up to date of the latest task states + // as it might have changed while the scheduler is failing over. + recoverState + startScheduler("MesosClusterScheduler", master, MesosClusterScheduler.this, builder.build()) + } + + def stop() { + driver.stop(true) } override def registered( driver: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo): Unit = { - appId = frameworkId.getValue - logInfo("Registered as framework ID " + appId) + logInfo("Registered as framework ID " + frameworkId.getValue) + if (frameworkId.getValue != appId) { + appId = frameworkId.getValue + state.persist("frameworkId", appId) + } markRegistered() - } - private def buildCommand(req: DriverRequest): CommandInfo = { + stateLock.synchronized { + if (!launchedDrivers.pendingRecover.isEmpty) { + // Start task reconciliation if we need to recover. + val statuses = launchedDrivers.pendingRecover.collect { + case (taskId, slaveId) => + launchedDrivers.get(taskId).taskState.getOrElse( + TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(taskId).build()) + .setSlaveId(slaveId) + .setState(TaskState.TASK_STAGING) + .build) + } + + // TODO: Page the status updates to avoid trying to reconcile + // a large amount of tasks at once. + driver.reconcileTasks(statuses) + } + } + } + private def buildCommand(req: DriverSubmission): CommandInfo = { val desc = req.desc - val cleanedJarUrl = desc.jarUrl.stripPrefix("file:") - - logInfo(s"jarUrl: $cleanedJarUrl") + val appJar = CommandInfo.URI.newBuilder() + .setValue(desc.desc.jarUrl.stripPrefix("file:").stripPrefix("local:")).build() val builder = CommandInfo.newBuilder() - .addUris(CommandInfo.URI.newBuilder().setValue(cleanedJarUrl).build()) + .addUris(appJar) val entries = - (conf.getOption("spark.executor.extraLibraryPath").toList ++ desc.command.libraryPathEntries) + (conf.getOption("spark.executor.extraLibraryPath").toList ++ + desc.desc.command.libraryPathEntries) val prefixEnv = if (!entries.isEmpty) { Utils.libraryPathEnvPrefix(entries) @@ -181,7 +245,7 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) } val envBuilder = Environment.newBuilder() - desc.command.environment.foreach { + desc.desc.command.environment.foreach { case (k, v) => envBuilder.addVariables( Variable.newBuilder().setName(k).setValue(v).build()) @@ -191,8 +255,8 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) val cmdOptions = generateCmdOption(req) - val executorUri = req.conf.getOption("spark.executor.uri") - .orElse(req.desc.command.environment.get("SPARK_EXECUTOR_URI")) + val executorUri = req.desc.schedulerProperties.get("spark.executor.uri") + .orElse(req.desc.desc.command.environment.get("SPARK_EXECUTOR_URI")) val cmd = if (executorUri.isDefined) { builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) @@ -201,11 +265,13 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit" - val cmdJar = s"../${desc.jarUrl.split("/").last}" + val cmdJar = s"../${desc.desc.jarUrl.split("/").last}" - s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar" + val appArguments = desc.desc.command.arguments.mkString(" ") + + s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar $appArguments" } else { - val executorSparkHome = req.conf.getOption("spark.mesos.executor.home") + val executorSparkHome = req.desc.schedulerProperties.get("spark.mesos.executor.home") .orElse(conf.getOption("spark.home")) .orElse(Option(System.getenv("SPARK_HOME"))) .getOrElse { @@ -214,7 +280,7 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getCanonicalPath - val cmdJar = desc.jarUrl.split("/").last + val cmdJar = desc.desc.jarUrl.split("/").last s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar" } @@ -224,25 +290,28 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) builder.build } - private def generateCmdOption(req: DriverRequest): Seq[String] = { - Seq( - "--name", req.conf.get("spark.app.name"), - "--class", req.desc.command.mainClass, + private def generateCmdOption(req: DriverSubmission): Seq[String] = { + var options = Seq( + "--name", req.desc.schedulerProperties("spark.app.name"), + "--class", req.desc.desc.command.mainClass, "--master", s"mesos://${conf.get("spark.master")}", - "--driver-cores", req.desc.cores.toString, - "--driver-memory", s"${req.desc.mem}M", - "--executor-memory", req.conf.get("spark.executor.memory"), - "--total-executor-cores", req.conf.get("spark.cores.max") - ) + "--driver-cores", req.desc.desc.cores.toString, + "--driver-memory", s"${req.desc.desc.mem}M") + + req.desc.schedulerProperties.get("spark.executor.memory").map { v => + options ++= Seq("--executor-memory", v) + } + + req.desc.schedulerProperties.get("spark.cores.max").map { v => + options ++= Seq("--total-executor-cores", v) + } + + options } private [spark] case class ResourceOffer(val offer: Offer, var cpu: Double, var mem: Double) override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { - var submission = queue.peek - - val usedOffers = new mutable.HashSet[OfferID] - val currentOffers = offers.map { o => ResourceOffer( @@ -253,66 +322,81 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) var canSchedule = true - while (canSchedule && submission != null) { - val driverCpu = submission.req.desc.cores - val driverMem = submission.req.desc.mem + val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() - val offerOption = currentOffers.find { o => - o.cpu >= driverCpu && o.mem >= driverMem - } + stateLock.synchronized { + var submissionOption = queue.peek - if (offerOption.isDefined) { - val offer = offerOption.get + while (canSchedule && submissionOption.isDefined) { + val submission = submissionOption.get + val driverCpu = submission.desc.desc.cores + val driverMem = submission.desc.desc.mem - offer.cpu -= driverCpu - offer.mem -= driverMem + val offerOption = currentOffers.find { o => + o.cpu >= driverCpu && o.mem >= driverMem + } - val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() + if (offerOption.isDefined) { + val offer = offerOption.get - val cpuResource = Resource.newBuilder() - .setName("cpus").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() + offer.cpu -= driverCpu + offer.mem -= driverMem - val memResource = Resource.newBuilder() - .setName("mem").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() + val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() - val commandInfo = buildCommand(submission.req) + val cpuResource = Resource.newBuilder() + .setName("cpus").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() - val taskInfo = TaskInfo.newBuilder() - .setTaskId(taskId) - .setName(s"driver for ${submission.req.desc.command.mainClass}") - .setSlaveId(offer.offer.getSlaveId) - .setCommand(commandInfo) - .addResources(cpuResource) - .addResources(memResource) - .build + val memResource = Resource.newBuilder() + .setName("mem").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() - logDebug(s"Launching task ${taskInfo}, with offer: ${offer.offer}") - driver.launchTasks(Collections.singleton(offer.offer.getId), Collections.singleton(taskInfo)) + val commandInfo = buildCommand(submission) - stateLock.synchronized { - launchedDrivers(submission.submissionId) = - ClusterTaskState(submission, taskId, offer.offer.getSlaveId, - None, DriverState.SUBMITTED, new Date()) - } + val taskInfo = TaskInfo.newBuilder() + .setTaskId(taskId) + .setName(s"driver for ${submission.desc.desc.command.mainClass}") + .setSlaveId(offer.offer.getSlaveId) + .setCommand(commandInfo) + .addResources(cpuResource) + .addResources(memResource) + .build + + val queuedTasks = if (!tasks.contains(offer.offer.getId)) { + val buffer = new ArrayBuffer[TaskInfo] + tasks(offer.offer.getId) = buffer + buffer + } else { + tasks(offer.offer.getId) + } + + queuedTasks += taskInfo - usedOffers += offer.offer.getId + launchedDrivers.set( + submission.submissionId, + ClusterTaskState(submission, taskId, offer.offer.getSlaveId, + None, DriverState.SUBMITTED, new Date())) - // remove driver from queue. - queue.poll(0, TimeUnit.SECONDS) + // remove driver from queue. + queue.poll - submission = queue.peek - } else { - // We can stop at very first driver that we cannot schedule on. - // TODO: We should remove the top driver that cannot be scheduled - // over a configurable time period. - canSchedule = false + submissionOption = queue.peek + } else { + // We can stop at very first driver that we cannot schedule on. + // TODO: We should remove a driver which cannot be scheduled + // over a configurable time period. + canSchedule = false + } } } + if (!tasks.isEmpty) { + driver.launchTasks(tasks.keySet, tasks.values.flatten) + } + offers - .filter(o => !usedOffers.contains(o.getId)) + .filter(o => !tasks.keySet.contains(o.getId)) .foreach(o => driver.declineOffer(o.getId)) } @@ -322,18 +406,17 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) } stateLock.synchronized { - val queueCopy = new Array[DriverSubmission](queue.size()) - queue.copyToArray(queueCopy) ClusterSchedulerState( - queueCopy, - copyDriverStates(launchedDrivers.values), + appId, + queue.drivers, + launchedDrivers.states, copyDriverStates(finishedDrivers)) } } def getStatus(submissionId: String): StatusResponse = { stateLock.synchronized { - if (queue.contains(new DriverSubmission(submissionId, null, null))) { + if (queue.contains(submissionId)) { return StatusResponse(submissionId, true, Option("Driver is queued for launch")) } else if (launchedDrivers.contains(submissionId)) { return StatusResponse(submissionId, true, Option("Driver is running")) @@ -349,7 +432,9 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) override def disconnected(driver: SchedulerDriver): Unit = {} - override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = {} + override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = { + logInfo("Framework re-registered.") + } override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {} @@ -378,6 +463,12 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) val taskId = status.getTaskId.getValue stateLock.synchronized { if (launchedDrivers.contains(taskId)) { + if (status.getReason == Reason.REASON_RECONCILIATION && + !launchedDrivers.pendingRecover.contains(taskId)) { + // Task has already received update and no longer requires reconciliation. + return + } + if (canRelaunch(status.getState)) { // TODO: We should try to relaunch if supervise is turned on. // Also check how many times we've retried. @@ -395,13 +486,13 @@ private[spark] class MesosClusterScheduler(conf: SparkConf) launchedState } else { - launchedDrivers(taskId) + launchedDrivers.get(taskId) } state.taskState = Option(status) state.driverState = driverState } else { - logError("Unable to find driver " + taskId + " in status update") + logError(s"Unable to find driver $taskId in status update") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosState.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosState.scala new file mode 100644 index 0000000000000..0b6b1e1cd9c7e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosState.scala @@ -0,0 +1,108 @@ +/* + * 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.scheduler.cluster.mesos + +import org.apache.mesos.state._ +import scala.concurrent.duration.Duration +import java.util.concurrent._ +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.util.Utils +import java.lang.Boolean +import java.util +import java.util.Collections +import org.apache.mesos.MesosNativeLibrary +import org.apache.spark.deploy.SparkCuratorUtil + +/** + * An abstraction over Mesos state abstraction. This to provide automatic retries and + * serialization of objects, and translating state exceptions into Spark exceptions. + */ +private[spark] class MesosState(conf: SparkConf) { + val defaultFetchTimeoutMs = conf.getLong("spark.mesos.state.fetch.timeout.ms", 2000) + val defaultStoreTimeoutMs = conf.getLong("spark.mesos.state.store.timeout.ms", 5000) + val retries = conf.getInt("spark.mesos.state.store.retries", 3) + val quorum = conf.getInt("spark.mesos.state.quorum", 1) + val path = conf.get("spark.mesos.state.path", "/.spark_mesos_dispatcher") + private val ZK_SESSION_TIMEOUT_MILLIS = 60000 + + private val state = conf.get("spark.deploy.recoveryMode", "NONE").toUpperCase() match { + case "NONE" => new InMemoryState() + case "ZOOKEEPER" => { + val servers = conf.get("spark.deploy.zookeeper.url") + new ZooKeeperState( + servers, + ZK_SESSION_TIMEOUT_MILLIS, + TimeUnit.MILLISECONDS, + "/spark_mesos_dispatcher") + } + } + + assert(retries >= 0, s"Retries must be larger or equal than zero, retries: $retries") + + def fetch[T](name: String, timeout: Option[Duration] = None): Option[(Variable, T)] = { + val finalTimeout = + timeout.getOrElse(Duration.create(defaultFetchTimeoutMs, TimeUnit.MILLISECONDS)) + try { + val variable = state.fetch(name).get(finalTimeout.toMillis, TimeUnit.MILLISECONDS) + if (variable == null || variable.value().size == 0) { + None + } else { + Option((variable, Utils.deserialize(variable.value()).asInstanceOf[T])) + } + } catch { + case e: TimeoutException => + throw new SparkException(s"Timed out fetching $name, timeout: $finalTimeout") + case e: ExecutionException => + throw new SparkException(s"Failed to fetch $name, error: $e") + case e: CancellationException => + throw new SparkException("Fetch operation is discarded") + } + } + + def store[T]( + name: String, + variable: Variable, + value: T, + timeout: Option[Duration] = None): Variable = { + val finalTimeout = + timeout.getOrElse(Duration.create(defaultStoreTimeoutMs, TimeUnit.MILLISECONDS)) + val newVariable = variable.mutate(Utils.serialize(value)) + val future = state.store(newVariable) + var remainingRuns = retries + 1 + while (remainingRuns > 0) { + try { + future.get(finalTimeout.toMillis, TimeUnit.MILLISECONDS) + } catch { + case e: TimeoutException => + throw new SparkException(s"Timed out storing $name, timeout: $finalTimeout") + case e: ExecutionException => + throw new SparkException(s"Failed to storing $name, error: $e") + case e: CancellationException => + throw new SparkException("Store operation is discarded") + } + + val status = future.get() + if (status != null) { + status + } + remainingRuns -= 1 + } + + throw new SparkException(s"Unable to store variable $name after $retries retries") + } +} 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 b58d62567afe1..2071701b313db 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.SparkConf class JsonProtocolSuite extends FunSuite { @@ -124,9 +124,8 @@ class JsonProtocolSuite extends FunSuite { } def createDriverRunner(): DriverRunner = { - val conf = new SparkConf() - new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), - createDriverDesc(), null, "akka://worker", new SecurityManager(conf)) + new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), + createDriverDesc(), null, "akka://worker") } def assertValidJson(json: JValue) { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index 2159fd8c16c6f..aa6e4874cecde 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -25,7 +25,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.FunSuite -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.util.Clock @@ -33,9 +33,8 @@ class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) - val conf = new SparkConf() - new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), - driverDescription, null, "akka://1.2.3.4/worker/", new SecurityManager(conf)) + new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), + driverDescription, null, "akka://1.2.3.4/worker/") } private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index e149d61ee704b..6fe8a74a29333 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -20,9 +20,11 @@ package org.apache.spark.scheduler.mesos import org.scalatest.FunSuite import org.apache.spark.{SparkConf, LocalSparkContext} import org.scalatest.mock.MockitoSugar -import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler -import org.apache.spark.scheduler.cluster.mesos.DriverRequest -import org.apache.spark.deploy.{Command, DriverDescription} +import org.apache.spark.scheduler.cluster.mesos._ +import org.apache.spark.deploy.Command +import org.apache.spark.deploy.DriverDescription +import org.apache.spark.deploy.mesos.MesosDriverDescription +import scala.collection.mutable class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar { def createCommand: Command = { @@ -34,13 +36,17 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo val conf = new SparkConf() conf.setMaster("mesos://localhost:5050") conf.setAppName("spark mesos") - val scheduler = new MesosClusterScheduler(conf) + val scheduler = new MesosClusterScheduler(new BlackHolePersistenceEngineFactory, conf) val response = - scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf)) + scheduler.submitDriver( + new MesosDriverDescription(new DriverDescription("jar", 1000, 1, true, createCommand), + new mutable.HashMap[String, String]())) assert(response.success) val response2 = - scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf)) + scheduler.submitDriver(new MesosDriverDescription( + new DriverDescription("jar", 1000, 1, true, createCommand), + new mutable.HashMap[String, String]())) assert(response2.success) val state = scheduler.getState() @@ -52,9 +58,11 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo val conf = new SparkConf() conf.setMaster("mesos://localhost:5050") conf.setAppName("spark mesos") - val scheduler = new MesosClusterScheduler(conf) + val scheduler = new MesosClusterScheduler(new BlackHolePersistenceEngineFactory, conf) val response = - scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf)) + scheduler.submitDriver(new MesosDriverDescription( + new DriverDescription("jar", 1000, 1, true, createCommand), + new mutable.HashMap[String, String]())) assert(response.success) val killResponse = scheduler.killDriver(response.id) From e0f33f71dbe3ebf487d59432ee2d0f88d7428376 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Mon, 23 Mar 2015 14:49:21 -0700 Subject: [PATCH 19/36] Add supervise support and persist retries. --- .../spark/deploy/FaultToleranceTest.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +- .../deploy/mesos/ui/DriverOutputPage.scala | 18 +- .../spark/deploy/rest/MesosRestServer.scala | 2 +- .../scheduler/cluster/mesos/DriverQueue.scala | 4 +- .../cluster/mesos/LaunchedDrivers.scala | 8 +- .../cluster/mesos/MesosClusterScheduler.scala | 193 ++++++++++-------- .../cluster/mesos/SuperviseRetryList.scala | 80 ++++++++ 8 files changed, 223 insertions(+), 94 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala 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 a7c89276a045e..82df15bedd44d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -32,7 +32,7 @@ import org.json4s._ import org.json4s.jackson.JsonMethods import org.apache.spark.{Logging, SparkConf, SparkContext} -import org.apache.spark.deploy.master.{RecoveryState, SparkCuratorUtil} +import org.apache.spark.deploy.master.{RecoveryState} import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index afce8ccf4a610..3e4edec7f7693 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -381,11 +381,13 @@ object SparkSubmit { // Standalone cluster only // Do not set CL arguments here because there are multiple possibilities for the main class - OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), + OptionAssigner(args.jars, STANDALONE | MESOS, CLUSTER, sysProp = "spark.jars"), OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy"), - OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, sysProp = "spark.driver.memory"), - OptionAssigner(args.driverCores, STANDALONE, CLUSTER, sysProp = "spark.driver.cores"), - OptionAssigner(args.supervise.toString, STANDALONE, CLUSTER, + OptionAssigner(args.driverMemory, STANDALONE | MESOS, CLUSTER, + sysProp = "spark.driver.memory"), + OptionAssigner(args.driverCores, STANDALONE | MESOS, CLUSTER, + sysProp = "spark.driver.cores"), + OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, sysProp = "spark.driver.supervise"), // Yarn client only diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala index b0e520415a42a..fa974e0c68f8b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.mesos.ui import org.apache.spark.ui.{UIUtils, WebUIPage} import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.scheduler.cluster.mesos.{ClusterTaskState, DriverSubmission} +import org.apache.spark.scheduler.cluster.mesos.{RetryState, ClusterTaskState, DriverSubmission} import org.apache.mesos.Protos.TaskStatus class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { @@ -31,10 +31,13 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { val queuedHeaders = Seq("DriverID", "Submit Date", "Description") val driverHeaders = queuedHeaders ++ Seq("Start Date", "Mesos Slave ID", "State") + val retryHeaders = Seq("DriverID", "Submit Date", "Description") ++ + Seq("Last Failed Status", "Next Retry Time", "Attempt Count") val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers) + val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.retryList) val content =

Mesos Framework ID: {state.appId}

@@ -45,6 +48,8 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { {launchedTable}

Finished Drivers:

{finishedTable} +

Supervise drivers waiting for retry:

+ {retryTable}
; UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster") @@ -69,6 +74,17 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { } + def retryRow(state: RetryState): Seq[Node] = { + + {state.submission.submissionId} + {state.submission.submitDate} + {state.submission.desc.desc.command.mainClass} + {state.lastFailureStatus} + {state.nextRetry} + {state.retries} + + } + def stateString(status: Option[TaskStatus]): String = { if (status.isEmpty) { return "" diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala index 0184965f47e26..e2a43e4b8666a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala @@ -167,7 +167,7 @@ class MesosKillRequestServlet(scheduler: ClusterScheduler, conf: SparkConf) val response = scheduler.killDriver(submissionId) val k = new KillSubmissionResponse k.serverSparkVersion = sparkVersion - k.message = response.message.orNull + k.message = response.message k.submissionId = submissionId k.success = response.success k diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala index 55ab372d518a3..fdec71af18fa5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler.cluster.mesos import scala.collection.mutable -import javax.annotation.concurrent.NotThreadSafe /** * A request queue for launching drivers in Mesos cluster mode. @@ -27,8 +26,7 @@ import javax.annotation.concurrent.NotThreadSafe * This queue is also bounded and rejects offers when it's full. * @param state Mesos state abstraction to fetch persistent state. */ -@NotThreadSafe -class DriverQueue(state: ClusterPersistenceEngine, capacity: Int) { +private[mesos] class DriverQueue(state: ClusterPersistenceEngine, capacity: Int) { var queue: mutable.Queue[DriverSubmission] = new mutable.Queue[DriverSubmission]() private var count = 0 diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala index 9659ca6c6e426..27570409e5f50 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala @@ -18,9 +18,13 @@ package org.apache.spark.scheduler.cluster.mesos import scala.collection.mutable -import org.apache.mesos.Protos.{TaskID, SlaveID} +import org.apache.mesos.Protos.SlaveID -class LaunchedDrivers(state: ClusterPersistenceEngine) { +/** + * Tracks all the launched or running drivers in the Mesos cluster scheduler. + * @param state Persistence engine to store state. + */ +private[mesos] class LaunchedDrivers(state: ClusterPersistenceEngine) { private val drivers = new mutable.HashMap[String, ClusterTaskState] // Holds the list of tasks that needs to reconciliation from the master. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index bed36b5846a2d..d05bffc5a667b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -61,21 +61,24 @@ private [spark] case class ClusterTaskState( val slaveId: SlaveID, var taskState: Option[TaskStatus], var driverState: DriverState, - var startDate: Date) extends Serializable { + var startDate: Date, + val lastRetry: Option[RetryState] = None) extends Serializable { + def copy(): ClusterTaskState = { - ClusterTaskState(submission, taskId, slaveId, taskState, driverState, startDate) + ClusterTaskState(submission, taskId, slaveId, taskState, driverState, startDate, lastRetry) } } private[spark] case class SubmitResponse(id: String, success: Boolean, message: Option[String]) private[spark] case class StatusResponse(id: String, success: Boolean, message: Option[String]) -private[spark] case class KillResponse(id: String, success: Boolean, message: Option[String]) +private[spark] case class KillResponse(id: String, success: Boolean, message: String) private[spark] case class ClusterSchedulerState( appId: String, queuedDrivers: Iterable[DriverSubmission], launchedDrivers: Iterable[ClusterTaskState], - finishedDrivers: Iterable[ClusterTaskState]) + finishedDrivers: Iterable[ClusterTaskState], + retryList: Iterable[RetryState]) private[spark] trait ClusterScheduler { def submitDriver(desc: MesosDriverDescription): SubmitResponse @@ -97,6 +100,7 @@ private[spark] class MesosClusterScheduler( val appName = conf.get("spark.app.name") val queuedCapacity = conf.getInt("spark.deploy.mesos.queuedDrivers", 200) val retainedDrivers = conf.getInt("spark.deploy.retainedDrivers", 200) + val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute val state = engineFactory.createEngine("scheduler") val stateTimeout = Duration.create(conf.getLong("spark.mesos.cluster.recover.timeout", 30), "seconds") @@ -114,6 +118,8 @@ private[spark] class MesosClusterScheduler( def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + private var superviseRetryList: SuperviseRetryList = _ + private def newDriverId(submitDate: Date): String = { "driver-%s-%04d".format( createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) @@ -135,23 +141,21 @@ private[spark] class MesosClusterScheduler( def killDriver(submissionId: String): KillResponse = { stateLock.synchronized { - // First check if submission is running or launched. + // We look for the requested driver in the following places: + // 1. Check if submission is running or launched. + // 2. Check if it's still queued. + // 3. Check if it's in the retry list. if (launchedDrivers.contains(submissionId)) { val task = launchedDrivers.get(submissionId) driver.killTask(task.taskId) - Some(KillResponse(submissionId, true, Option("Killing running driver"))) - } else { - None - } - }.orElse { - // Second we check if it's still queued. - if (queue.remove(submissionId)) { - Some(KillResponse(submissionId, true, Option("Removed driver while it's still pending"))) + return KillResponse(submissionId, true, "Killing running driver") + } else if (queue.remove(submissionId)) { + return KillResponse(submissionId, true, "Removed driver while it's still pending") + } else if (superviseRetryList.remove(submissionId)) { + return KillResponse(submissionId, true, "Removed driver while it's retrying") } else { - None + return KillResponse(submissionId, false, "Cannot find driver") } - }.getOrElse{ - KillResponse(submissionId, false, Option("Cannot find driver")) } } @@ -161,6 +165,18 @@ private[spark] class MesosClusterScheduler( launchedDrivers = new LaunchedDrivers(engineFactory.createEngine("launchedDrivers")) + // There is potential timing issue where a queued driver might have been launched + // but the scheduler shuts down before the queued driver was able to be removed + // from the queue. We try to mitigate this issue by walking through all queued drivers + // and remove if they're already launched. + queue.drivers.foreach { + d => if (launchedDrivers.contains(d.submissionId)) { + queue.remove(d.submissionId) + } + } + + superviseRetryList = new SuperviseRetryList(engineFactory.createEngine("retryList")) + // TODO: Consider storing finished drivers so we can show them on the UI after // failover. For now we clear the history on each recovery. finishedDrivers.clear() @@ -320,15 +336,21 @@ private[spark] class MesosClusterScheduler( getResource(o.getResourcesList, "mem")) } - var canSchedule = true - val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() - stateLock.synchronized { - var submissionOption = queue.peek + val currentTime = new Date() + + def scheduleTasks( + taskFunc: () => (Option[DriverSubmission], Option[RetryState]), + scheduledCallback: (String) => Unit) { + var nextItem = taskFunc() + // TODO: We should not stop scheduling at the very first task + // that cannot be scheduled. Instead we should exhaust the + // candidate list and remove drivers that cannot scheduled + // over a configurable period of time. + while (nextItem._1.isDefined) { + val (submission, retryState) = (nextItem._1.get, nextItem._2) - while (canSchedule && submissionOption.isDefined) { - val submission = submissionOption.get val driverCpu = submission.desc.desc.cores val driverMem = submission.desc.desc.mem @@ -336,63 +358,68 @@ private[spark] class MesosClusterScheduler( o.cpu >= driverCpu && o.mem >= driverMem } - if (offerOption.isDefined) { - val offer = offerOption.get + if (offerOption.isEmpty) { + return + } - offer.cpu -= driverCpu - offer.mem -= driverMem + val offer = offerOption.get - val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() + offer.cpu -= driverCpu + offer.mem -= driverMem - val cpuResource = Resource.newBuilder() - .setName("cpus").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() + val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() - val memResource = Resource.newBuilder() - .setName("mem").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() + val cpuResource = Resource.newBuilder() + .setName("cpus").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() - val commandInfo = buildCommand(submission) + val memResource = Resource.newBuilder() + .setName("mem").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() - val taskInfo = TaskInfo.newBuilder() - .setTaskId(taskId) - .setName(s"driver for ${submission.desc.desc.command.mainClass}") - .setSlaveId(offer.offer.getSlaveId) - .setCommand(commandInfo) - .addResources(cpuResource) - .addResources(memResource) - .build + val commandInfo = buildCommand(submission) - val queuedTasks = if (!tasks.contains(offer.offer.getId)) { - val buffer = new ArrayBuffer[TaskInfo] - tasks(offer.offer.getId) = buffer - buffer - } else { - tasks(offer.offer.getId) - } + val taskInfo = TaskInfo.newBuilder() + .setTaskId(taskId) + .setName(s"driver for ${submission.desc.desc.command.mainClass}") + .setSlaveId(offer.offer.getSlaveId) + .setCommand(commandInfo) + .addResources(cpuResource) + .addResources(memResource) + .build - queuedTasks += taskInfo + val queuedTasks = if (!tasks.contains(offer.offer.getId)) { + val buffer = new ArrayBuffer[TaskInfo] + tasks(offer.offer.getId) = buffer + buffer + } else { + tasks(offer.offer.getId) + } - launchedDrivers.set( - submission.submissionId, - ClusterTaskState(submission, taskId, offer.offer.getSlaveId, - None, DriverState.SUBMITTED, new Date())) + queuedTasks += taskInfo - // remove driver from queue. - queue.poll + launchedDrivers.set( + submission.submissionId, + ClusterTaskState(submission, taskId, offer.offer.getSlaveId, + None, DriverState.SUBMITTED, new Date(), retryState)) - submissionOption = queue.peek - } else { - // We can stop at very first driver that we cannot schedule on. - // TODO: We should remove a driver which cannot be scheduled - // over a configurable time period. - canSchedule = false - } + scheduledCallback(submission.submissionId) + + nextItem = taskFunc() } } - if (!tasks.isEmpty) { - driver.launchTasks(tasks.keySet, tasks.values.flatten) + stateLock.synchronized { + scheduleTasks(() => { + superviseRetryList.getNextRetry(currentTime) + }, (id: String) => { + superviseRetryList.remove(id) + }) + scheduleTasks(() => (queue.peek, None), (_) => queue.poll) + } + + tasks.foreach { + case (offerId, tasks) => driver.launchTasks(offerId, tasks) } offers @@ -401,16 +428,13 @@ private[spark] class MesosClusterScheduler( } def getState(): ClusterSchedulerState = { - def copyDriverStates(states: Iterable[ClusterTaskState]): Iterable[ClusterTaskState] = { - states.collect { case s => s.copy() } - } - stateLock.synchronized { ClusterSchedulerState( appId, queue.drivers, launchedDrivers.states, - copyDriverStates(finishedDrivers)) + finishedDrivers.collect { case s => s.copy() }, + superviseRetryList.retries) } } @@ -433,7 +457,7 @@ private[spark] class MesosClusterScheduler( override def disconnected(driver: SchedulerDriver): Unit = {} override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = { - logInfo("Framework re-registered.") + logInfo(s"Framework re-registered with master ${masterInfo.getId}") } override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {} @@ -453,7 +477,7 @@ private[spark] class MesosClusterScheduler( } } - def canRelaunch(state: TaskState): Boolean = { + def shouldRelaunch(state: TaskState): Boolean = { state == TaskState.TASK_FAILED || state == TaskState.TASK_KILLED || state == TaskState.TASK_LOST @@ -469,24 +493,29 @@ private[spark] class MesosClusterScheduler( return } - if (canRelaunch(status.getState)) { - // TODO: We should try to relaunch if supervise is turned on. - // Also check how many times we've retried. + val state = launchedDrivers.get(taskId) + + if (state.submission.desc.desc.supervise && shouldRelaunch(status.getState)) { + val (retries, waitTimeSec) = if (state.lastRetry.isDefined) { + (state.lastRetry.get.retries + 1, + Math.min(maxRetryWaitTime, state.lastRetry.get.waitTime * 2)) + } else { + (1, 1) + } + val nextRetry = new Date(new Date().getTime + waitTimeSec * 1000L) + superviseRetryList.add( + RetryState(state.submission, status, retries, nextRetry, waitTimeSec)) } val driverState = getDriverState(status.getState) - val state = if (isFinished(status.getState)) { - val launchedState = launchedDrivers.remove(taskId).get + if (isFinished(status.getState)) { + launchedDrivers.remove(taskId) if (finishedDrivers.size >= retainedDrivers) { val toRemove = math.max(retainedDrivers / 10, 1) finishedDrivers.trimStart(toRemove) } - finishedDrivers += launchedState - - launchedState - } else { - launchedDrivers.get(taskId) + finishedDrivers += state } state.taskState = Option(status) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala new file mode 100644 index 0000000000000..2462477b1a271 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala @@ -0,0 +1,80 @@ +/* + * 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.scheduler.cluster.mesos + +import org.apache.mesos.Protos.TaskStatus +import java.util.Date +import scala.collection.mutable.ArrayBuffer + +private[mesos] case class RetryState( + submission: DriverSubmission, + lastFailureStatus: TaskStatus, + retries: Int, + nextRetry: Date, + waitTime: Int) extends Serializable { + def copy() = new RetryState(submission, lastFailureStatus, retries, nextRetry, waitTime) +} + +/** + * Tracks all the drivers that were submitted under supervise which failed to run + * and waiting to be scheduled again. + * @param state Persistence engine to store state. + */ +private[mesos] class SuperviseRetryList(state: ClusterPersistenceEngine) { + val drivers = new ArrayBuffer[RetryState] + + initialize() + + def initialize() { + state.fetchAll[RetryState]().foreach { + s => drivers += s + } + } + + def getNextRetry(currentTime: Date): (Option[DriverSubmission], Option[RetryState]) = { + val retry = drivers.find(d => d.nextRetry.before(currentTime)) + if (retry.isDefined) { + (Some(retry.get.submission), retry) + } else { + (None, None) + } + } + + def retries: Iterable[RetryState] = { + drivers.collect { case d => d.copy}.toList + } + + def remove(submissionId: String): Boolean = { + val index = + drivers.indexWhere( + s => s.submission.submissionId.equals(submissionId)) + + if (index != -1) { + drivers.remove(index) + state.expunge(submissionId) + true + } + + index != -1 + } + + def add(retryState: RetryState) { + drivers += retryState + state.persist(retryState.submission.submissionId, retryState) + } +} From 7f214c2a8bb8c4b38115de6d7bfd670bc15c9f04 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Mon, 23 Mar 2015 16:07:57 -0700 Subject: [PATCH 20/36] Fix RetryState visibility --- .../spark/scheduler/cluster/mesos/SuperviseRetryList.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala index 2462477b1a271..61729127a05d9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala @@ -21,7 +21,7 @@ import org.apache.mesos.Protos.TaskStatus import java.util.Date import scala.collection.mutable.ArrayBuffer -private[mesos] case class RetryState( +private[spark] case class RetryState( submission: DriverSubmission, lastFailureStatus: TaskStatus, retries: Int, From 862b5b5d7002105f0d0fb736d4e01963b78eb3c2 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Mon, 23 Mar 2015 17:24:23 -0700 Subject: [PATCH 21/36] Support asking driver status when it's retrying. --- .../spark/scheduler/cluster/mesos/MesosClusterScheduler.scala | 2 ++ .../spark/scheduler/cluster/mesos/SuperviseRetryList.scala | 3 +++ 2 files changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index d05bffc5a667b..4986577a152d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -446,6 +446,8 @@ private[spark] class MesosClusterScheduler( return StatusResponse(submissionId, true, Option("Driver is running")) } else if (finishedDrivers.contains(submissionId)) { return StatusResponse(submissionId, true, Option("Driver already finished")) + } else if (superviseRetryList.contains(submissionId)) { + return StatusResponse(submissionId, true, Option("Driver is retrying")) } else { return StatusResponse(submissionId, false, None) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala index 61729127a05d9..75ab44499928c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala @@ -46,6 +46,9 @@ private[mesos] class SuperviseRetryList(state: ClusterPersistenceEngine) { } } + def contains(submissionId: String) = + drivers.exists(d => d.submission.submissionId.equals(submissionId)) + def getNextRetry(currentTime: Date): (Option[DriverSubmission], Option[RetryState]) = { val retry = drivers.find(d => d.nextRetry.before(currentTime)) if (retry.isDefined) { From 920fc4b7c0258332e9bac58412d77bf8badb9998 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Mon, 23 Mar 2015 18:05:34 -0700 Subject: [PATCH 22/36] Fix scala style issues. --- .../spark/deploy/rest/MesosRestServer.scala | 7 ++--- .../cluster/mesos/MesosClusterScheduler.scala | 30 ++++++++++++------- .../cluster/mesos/SuperviseRetryList.scala | 4 +++ 3 files changed, 27 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala index e2a43e4b8666a..dce3100d9f7c3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala @@ -145,7 +145,7 @@ class MesosSubmitRequestServlet( val response = scheduler.submitDriver(driverDescription) val submitResponse = new CreateSubmissionResponse submitResponse.serverSparkVersion = sparkVersion - submitResponse.message = response.message.orNull + submitResponse.message = response.message submitResponse.success = response.success submitResponse.submissionId = response.id val unknownFields = findUnknownFields(requestMessageJson, requestMessage) @@ -178,13 +178,12 @@ class MesosStatusRequestServlet(scheduler: ClusterScheduler, conf: SparkConf) extends StatusRequestServlet { protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { val response = scheduler.getStatus(submissionId) - //val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } val d = new SubmissionStatusResponse + d.driverState = response.state d.serverSparkVersion = sparkVersion d.submissionId = response.id d.success = response.success - //d.driverState = response.state.map(_.toString).orNull - d.message = response.message.orNull + d.message = response.status.map { s => s.toString }.getOrElse("") d } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 4986577a152d8..5a489b084d587 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -69,8 +69,14 @@ private [spark] case class ClusterTaskState( } } -private[spark] case class SubmitResponse(id: String, success: Boolean, message: Option[String]) -private[spark] case class StatusResponse(id: String, success: Boolean, message: Option[String]) +private[spark] case class SubmitResponse(id: String, success: Boolean, message: String) + +private[spark] case class StatusResponse( + id: String, + success: Boolean, + state: String, + status: Option[TaskStatus] = None) + private[spark] case class KillResponse(id: String, success: Boolean, message: String) private[spark] case class ClusterSchedulerState( @@ -128,14 +134,14 @@ private[spark] class MesosClusterScheduler( def submitDriver(desc: MesosDriverDescription): SubmitResponse = { stateLock.synchronized { if (queue.isFull) { - return SubmitResponse("", false, Option("Already reached maximum submission size")) + return SubmitResponse("", false, "Already reached maximum submission size") } val submitDate: Date = new Date() val submissionId: String = newDriverId(submitDate) val submission = new DriverSubmission(submissionId, desc, submitDate) queue.offer(submission) - SubmitResponse(submissionId, true, None) + SubmitResponse(submissionId, true, "") } } @@ -184,7 +190,7 @@ private[spark] class MesosClusterScheduler( } def start() { - //TODO: Implement leader election to make sure only one framework running in the cluster. + // TODO: Implement leader election to make sure only one framework running in the cluster. val fwId = state.fetch[String]("frameworkId") val builder = FrameworkInfo.newBuilder() @@ -441,15 +447,19 @@ private[spark] class MesosClusterScheduler( def getStatus(submissionId: String): StatusResponse = { stateLock.synchronized { if (queue.contains(submissionId)) { - return StatusResponse(submissionId, true, Option("Driver is queued for launch")) + return StatusResponse(submissionId, true, "Driver is queued for launch") } else if (launchedDrivers.contains(submissionId)) { - return StatusResponse(submissionId, true, Option("Driver is running")) + val status = launchedDrivers.get(submissionId).taskState + return StatusResponse(submissionId, true, "Driver is running", status) } else if (finishedDrivers.contains(submissionId)) { - return StatusResponse(submissionId, true, Option("Driver already finished")) + val status = + finishedDrivers.find(d => d.submission.submissionId.equals(submissionId)).get.taskState + return StatusResponse(submissionId, true, "Driver already finished", status) } else if (superviseRetryList.contains(submissionId)) { - return StatusResponse(submissionId, true, Option("Driver is retrying")) + val status = superviseRetryList.get(submissionId).get.lastFailureStatus + return StatusResponse(submissionId, true, "Driver failed and retrying", Some(status)) } else { - return StatusResponse(submissionId, false, None) + return StatusResponse(submissionId, false, "Driver not found") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala index 75ab44499928c..0c82436253e15 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala @@ -58,6 +58,10 @@ private[mesos] class SuperviseRetryList(state: ClusterPersistenceEngine) { } } + def get(submissionId: String): Option[RetryState] = { + drivers.find(d => d.submission.submissionId.equals(submissionId)) + } + def retries: Iterable[RetryState] = { drivers.collect { case d => d.copy}.toList } From a46ad66eca728121052be7ec0329b722c1b6105e Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 24 Mar 2015 16:31:39 -0700 Subject: [PATCH 23/36] Allow zk cli param override. --- .../spark/deploy/mesos/MesosClusterDispatcher.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 7aada65675fa8..936c6de29e1b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -101,6 +101,13 @@ private[mesos] object MesosClusterDispatcher extends Logging { conf.setMaster(dispatcherArgs.masterUrl) conf.setAppName("Mesos Cluster Dispatcher") + dispatcherArgs.zookeeperUrl.foreach { + z => { + conf.set("spark.deploy.recoveryMode", "ZOOKEEPER") + conf.set("spark.deploy.zookeeper.url", z) + } + } + val dispatcher = new MesosClusterDispatcher( dispatcherArgs, conf) @@ -125,6 +132,7 @@ private[mesos] object MesosClusterDispatcher extends Logging { var port = 7077 var webUiPort = 8081 var masterUrl: String = _ + var zookeeperUrl: Option[String] = None var propertiesFile: String = _ parse(args.toList) @@ -145,6 +153,10 @@ private[mesos] object MesosClusterDispatcher extends Logging { webUiPort = value parse(tail) + case ("--zk" | "-z") :: value :: tail => + zookeeperUrl = Some(value) + parse(tail) + case ("--master" | "-m") :: value :: tail => if (!value.startsWith("mesos://")) { System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") @@ -183,6 +195,7 @@ private[mesos] object MesosClusterDispatcher extends Logging { " -p PORT, --port PORT Port to listen on (default: 7077)\n" + " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + " -m --master MASTER URI for connecting to Mesos master\n" + + " -z --zk ZOOKEEPER URLs for connecting to Zookeeper for persistence\n" + " --properties-file FILE Path to a custom Spark properties file.\n" + " Default is conf/spark-defaults.conf.") System.exit(exitCode) From 72526123c599db2c661e3d6b7b76174dd3a6d26c Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 25 Mar 2015 00:54:50 -0700 Subject: [PATCH 24/36] Fix tests. --- .../org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala | 3 +-- .../spark/scheduler/cluster/mesos/MesosClusterScheduler.scala | 2 +- .../spark/scheduler/mesos/MesosClusterSchedulerSuite.scala | 2 ++ 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala index fdec71af18fa5..a03ae06bac602 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala @@ -60,10 +60,9 @@ private[mesos] class DriverQueue(state: ClusterPersistenceEngine, capacity: Int) val removed = queue.dequeueFirst(d => d.submissionId.equals(submissionId)) if (removed.isDefined) { state.expunge(removed.get.submissionId) - true } - false + removed.isDefined } def peek(): Option[DriverSubmission] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 5a489b084d587..1acac7f7e067c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -165,7 +165,7 @@ private[spark] class MesosClusterScheduler( } } - private def recoverState { + def recoverState { stateLock.synchronized { queue = new DriverQueue(engineFactory.createEngine("driverQueue"), queuedCapacity) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index 6fe8a74a29333..4db22bb74d0a7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -37,6 +37,7 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo conf.setMaster("mesos://localhost:5050") conf.setAppName("spark mesos") val scheduler = new MesosClusterScheduler(new BlackHolePersistenceEngineFactory, conf) + scheduler.recoverState val response = scheduler.submitDriver( new MesosDriverDescription(new DriverDescription("jar", 1000, 1, true, createCommand), @@ -59,6 +60,7 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo conf.setMaster("mesos://localhost:5050") conf.setAppName("spark mesos") val scheduler = new MesosClusterScheduler(new BlackHolePersistenceEngineFactory, conf) + scheduler.recoverState val response = scheduler.submitDriver(new MesosDriverDescription( new DriverDescription("jar", 1000, 1, true, createCommand), From 20f72843c609e4a9bed533ccea4d3e253579c5a9 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Sat, 28 Mar 2015 01:20:52 -0700 Subject: [PATCH 25/36] Address review comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 31 +- .../deploy/mesos/MesosClusterDispatcher.scala | 95 ++-- .../deploy/mesos/MesosDriverDescription.scala | 30 +- ...utputPage.scala => MesosClusterPage.scala} | 40 +- .../deploy/mesos/ui/MesosClusterUI.scala | 11 +- .../apache/spark/deploy/rest/RestServer.scala | 23 +- .../deploy/rest/StandaloneRestServer.scala | 19 +- .../rest/SubmitRestProtocolResponse.scala | 6 +- .../rest/{ => mesos}/MesosRestServer.scala | 124 ++--- .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../scheduler/cluster/mesos/DriverQueue.scala | 26 +- .../cluster/mesos/LaunchedDrivers.scala | 25 +- ...la => MesosClusterPersistenceEngine.scala} | 80 +-- .../cluster/mesos/MesosClusterScheduler.scala | 456 ++++++++---------- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- ...Helper.scala => MesosSchedulerUtils.scala} | 50 +- .../cluster/mesos/SuperviseRetryList.scala | 36 +- .../mesos/MesosClusterSchedulerSuite.scala | 35 +- sbin/start-mesos-dispatcher.sh | 6 +- sbin/stop-mesos-dispatcher.sh | 3 +- 20 files changed, 551 insertions(+), 549 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/mesos/ui/{DriverOutputPage.scala => MesosClusterPage.scala} (79%) rename core/src/main/scala/org/apache/spark/deploy/rest/{ => mesos}/MesosRestServer.scala (57%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/{ClusterPersistenceEngine.scala => MesosClusterPersistenceEngine.scala} (55%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/{MesosSchedulerHelper.scala => MesosSchedulerUtils.scala} (64%) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 3e4edec7f7693..bf04e88faf7e3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -22,8 +22,6 @@ import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowab import java.net.URL import java.security.PrivilegedExceptionAction -import scala.collection.mutable.{ArrayBuffer, HashMap, Map} - import org.apache.hadoop.fs.Path import org.apache.hadoop.security.UserGroupInformation import org.apache.ivy.Ivy @@ -36,11 +34,12 @@ import org.apache.ivy.core.retrieve.RetrieveOptions import org.apache.ivy.core.settings.IvySettings import org.apache.ivy.plugins.matcher.GlobPatternMatcher import org.apache.ivy.plugins.resolver.{ChainResolver, IBiblioResolver} - import org.apache.spark.SPARK_VERSION import org.apache.spark.deploy.rest._ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} + /** * Whether to submit, kill, or request the status of an application. * The latter two operations are currently supported only for standalone cluster mode. @@ -114,7 +113,9 @@ object SparkSubmit { } } - /** Kill an existing submission using the REST protocol. Standalone cluster mode only. */ + /** + * Kill an existing submission using the REST protocol. Standalone and Mesos cluster mode only. + */ private def kill(args: SparkSubmitArguments): Unit = { new RestClient() .killSubmission(args.master, args.submissionToKill) @@ -122,7 +123,7 @@ object SparkSubmit { /** * Request the status of an existing submission using the REST protocol. - * Standalone cluster mode only. + * Standalone and Mesos cluster mode only. */ private def requestStatus(args: SparkSubmitArguments): Unit = { new RestClient() @@ -297,7 +298,7 @@ object SparkSubmit { (clusterManager, deployMode) match { case (MESOS, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + - "on Mesos clusters.") + "applications on Mesos clusters.") case (STANDALONE, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + "applications on standalone clusters.") @@ -381,14 +382,7 @@ object SparkSubmit { // Standalone cluster only // Do not set CL arguments here because there are multiple possibilities for the main class - OptionAssigner(args.jars, STANDALONE | MESOS, CLUSTER, sysProp = "spark.jars"), OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy"), - OptionAssigner(args.driverMemory, STANDALONE | MESOS, CLUSTER, - sysProp = "spark.driver.memory"), - OptionAssigner(args.driverCores, STANDALONE | MESOS, CLUSTER, - sysProp = "spark.driver.cores"), - OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, - sysProp = "spark.driver.supervise"), // Yarn client only OptionAssigner(args.queue, YARN, CLIENT, sysProp = "spark.yarn.queue"), @@ -417,7 +411,14 @@ object SparkSubmit { OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, - sysProp = "spark.files") + sysProp = "spark.files"), + OptionAssigner(args.jars, STANDALONE | MESOS, CLUSTER, sysProp = "spark.jars"), + OptionAssigner(args.driverMemory, STANDALONE | MESOS, CLUSTER, + sysProp = "spark.driver.memory"), + OptionAssigner(args.driverCores, STANDALONE | MESOS, CLUSTER, + sysProp = "spark.driver.cores"), + OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, + sysProp = "spark.driver.supervise") ) // In client mode, launch the application main class directly @@ -501,7 +502,7 @@ object SparkSubmit { } if (isMesosCluster) { - // Mesos cluster dispatcher only supports the REST interface + assert(args.useRest, "Mesos cluster mode is only supported through the REST submission API") childMainClass = "org.apache.spark.deploy.rest.RestClient" childArgs += (args.primaryResource, args.mainClass) if (args.childArgs != null) { diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 936c6de29e1b4..204466eee9ad3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -17,73 +17,77 @@ package org.apache.spark.deploy.mesos -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.util.{SignalLogger, IntParam, Utils} - import java.io.File -import java.util.concurrent.{TimeUnit, CountDownLatch} +import java.util.concurrent.CountDownLatch +import org.apache.spark +import org.apache.spark.deploy.mesos.MesosClusterDispatcher.ClusterDispatcherArguments import org.apache.spark.deploy.mesos.ui.MesosClusterUI -import org.apache.spark.deploy.rest.MesosRestServer +import org.apache.spark.deploy.rest.mesos.MesosRestServer import org.apache.spark.scheduler.cluster.mesos._ -import org.apache.spark.deploy.mesos.MesosClusterDispatcher.ClusterDispatcherArguments -import org.apache.mesos.state.{ZooKeeperState, InMemoryState} +import org.apache.spark.util.{IntParam, SignalLogger, Utils} +import org.apache.spark.{Logging, SecurityManager, SparkConf} /* - * A dispatcher actor that is responsible for managing drivers, that is intended to - * used for Mesos cluster mode. + * A dispatcher that is responsible for managing and launching drivers, and is intended to + * be used for Mesos cluster mode. The dispatcher ls launched by the user in the cluster, + * which it launches a [[MesosRestServer]] for listening for driver requests, and launches a + * [[MesoClusterScheduler]] to launch these drivers in the Mesos cluster. + * + * A typical new driver lifecycle is the following: + * + * - Driver submitted via spark-submit talking to the [[MesosRestServer]] + * - [[MesosRestServer]] queues the driver request to [[MesosClusterScheduler]] + * - [[MesosClusterScheduler]] gets resource offers and launches the drivers that are in queue + * + * This dispatcher supports both Mesos fine-grain or coarse-grain mode as the mode is configurable + * per driver launched. * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as * a daemon to launch drivers as Mesos frameworks upon request. */ -private [spark] class MesosClusterDispatcher( +private[mesos] class MesosClusterDispatcher( args: ClusterDispatcherArguments, - conf: SparkConf) extends Logging { + conf: SparkConf) + extends Logging { - def dispatcherPublicAddress(conf: SparkConf, host: String): String = { + private def publicAddress(conf: SparkConf, host: String): String = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - val recoveryMode = conf.get("spark.deploy.recoveryMode", "NONE").toUpperCase() + private val recoveryMode = conf.get("spark.deploy.recoveryMode", "NONE").toUpperCase() logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode) - val engineFactory = recoveryMode match { - case "NONE" => new BlackHolePersistenceEngineFactory - case "ZOOKEEPER" => { - new ZookeeperClusterPersistenceEngineFactory(conf) - } + private val engineFactory = recoveryMode match { + case "NONE" => new BlackHoleMesosClusterPersistenceEngineFactory + case "ZOOKEEPER" => new ZookeeperMesosClusterPersistenceEngineFactory(conf) } - val scheduler = new MesosClusterScheduler( - engineFactory, - conf) + private val scheduler = new MesosClusterSchedulerDriver(engineFactory, conf) - val server = new MesosRestServer(args.host, args.port, conf, scheduler) - - val webUi = new MesosClusterUI( + private val server = new MesosRestServer(args.host, args.port, conf, scheduler) + private val webUi = new MesosClusterUI( new SecurityManager(conf), args.webUiPort, conf, - dispatcherPublicAddress(conf, args.host), + publicAddress(conf, args.host), scheduler) - val shutdownLatch = new CountDownLatch(1) - - val sparkHome = - new File(sys.env.get("SPARK_HOME").getOrElse(".")) + private val shutdownLatch = new CountDownLatch(1) + private val sparkHome = new File(Option(conf.getenv("SPARK_HOME")).getOrElse(".")) - def start() { + def start(): Unit = { webUi.bind() scheduler.frameworkUrl = webUi.activeWebUiUrl scheduler.start() server.start() } - def awaitShutdown() { + def awaitShutdown(): Unit = { shutdownLatch.await() } - def stop() { + def stop(): Unit = { webUi.stop() server.stop() scheduler.stop() @@ -91,7 +95,7 @@ private [spark] class MesosClusterDispatcher( } } -private[mesos] object MesosClusterDispatcher extends Logging { +private[mesos] object MesosClusterDispatcher extends spark.Logging { def main(args: Array[String]) { SignalLogger.register(log) @@ -101,11 +105,9 @@ private[mesos] object MesosClusterDispatcher extends Logging { conf.setMaster(dispatcherArgs.masterUrl) conf.setAppName("Mesos Cluster Dispatcher") - dispatcherArgs.zookeeperUrl.foreach { - z => { - conf.set("spark.deploy.recoveryMode", "ZOOKEEPER") - conf.set("spark.deploy.zookeeper.url", z) - } + dispatcherArgs.zookeeperUrl.foreach { z => + conf.set("spark.deploy.recoveryMode", "ZOOKEEPER") + conf.set("spark.deploy.zookeeper.url", z) } val dispatcher = new MesosClusterDispatcher( @@ -127,7 +129,7 @@ private[mesos] object MesosClusterDispatcher extends Logging { dispatcher.awaitShutdown() } - class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { + private class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8081 @@ -186,18 +188,19 @@ private[mesos] object MesosClusterDispatcher extends Logging { /** * Print usage and exit JVM with the given exit code. */ - def printUsageAndExit(exitCode: Int) { + def printUsageAndExit(exitCode: Int): Unit = { System.err.println( "Usage: MesosClusterDispatcher [options]\n" + "\n" + "Options:\n" + - " -h HOST, --host HOST Hostname to listen on\n" + - " -p PORT, --port PORT Port to listen on (default: 7077)\n" + - " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + + " -h HOST, --host HOST Hostname to listen on\n" + + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + + " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + " -m --master MASTER URI for connecting to Mesos master\n" + - " -z --zk ZOOKEEPER URLs for connecting to Zookeeper for persistence\n" + - " --properties-file FILE Path to a custom Spark properties file.\n" + - " Default is conf/spark-defaults.conf.") + " -z --zk ZOOKEEPER Comma delimited URLs for connecting to \n" + + " Zookeeper for persistence\n" + + " --properties-file FILE Path to a custom Spark properties file.\n" + + " Default is conf/spark-defaults.conf.") System.exit(exitCode) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala index ab764296e7a25..05442bb82ac5e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala @@ -17,10 +17,34 @@ package org.apache.spark.deploy.mesos -import org.apache.spark.deploy.DriverDescription +import java.util.Date + import scala.collection.mutable +import org.apache.spark.deploy.Command + +/** + * Describes a Spark driver that is submitted from the + * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]], to be launched by + * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]]. + * @param jarUrl URL to the application jar + * @param mem Amount of memory for the driver + * @param cores Amount of cores for the driver + * @param supervise Supervise the driver for long running app + * @param command The command to launch the driver. + * @param schedulerProperties Extra properties to pass the Mesos scheduler + */ private[spark] class MesosDriverDescription( - val desc: DriverDescription, - val schedulerProperties: mutable.HashMap[String, String]) extends Serializable { + val jarUrl: String, + val mem: Int, + val cores: Int, + val supervise: Boolean, + val command: Command, + val schedulerProperties: mutable.HashMap[String, String]) + extends Serializable { + + var submissionId: Option[String] = None + var submissionDate: Option[Date] = None + + override def toString: String = s"MesosDriverDescription (${command.mainClass})" } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala similarity index 79% rename from core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala rename to core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala index fa974e0c68f8b..0189267ddc2ba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -17,29 +17,30 @@ package org.apache.spark.deploy.mesos.ui -import org.apache.spark.ui.{UIUtils, WebUIPage} import javax.servlet.http.HttpServletRequest + import scala.xml.Node -import org.apache.spark.scheduler.cluster.mesos.{RetryState, ClusterTaskState, DriverSubmission} + import org.apache.mesos.Protos.TaskStatus -class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.scheduler.cluster.mesos.{RetryState, MesosClusterTaskState} +import org.apache.spark.ui.{UIUtils, WebUIPage} +private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val state = parent.scheduler.getState() - val queuedHeaders = Seq("DriverID", "Submit Date", "Description") val driverHeaders = queuedHeaders ++ Seq("Start Date", "Mesos Slave ID", "State") val retryHeaders = Seq("DriverID", "Submit Date", "Description") ++ Seq("Last Failed Status", "Next Retry Time", "Attempt Count") - val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers) val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.retryList) val content = -

Mesos Framework ID: {state.appId}

+

Mesos Framework ID: {state.frameworkId}

Queued Drivers:

@@ -55,64 +56,57 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") { UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster") } - def queuedRow(submission: DriverSubmission): Seq[Node] = { + private def queuedRow(submission: MesosDriverDescription): Seq[Node] = { {submission.submissionId} - {submission.submitDate} - {submission.desc.desc.command.mainClass} + {submission.submissionDate} + {submission.command.mainClass} } - def driverRow(state: ClusterTaskState): Seq[Node] = { + private def driverRow(state: MesosClusterTaskState): Seq[Node] = { {state.submission.submissionId} - {state.submission.submitDate} - {state.submission.desc.desc.command.mainClass} + {state.submission.submissionDate} + {state.submission.command.mainClass} {state.startDate} {state.slaveId.getValue} {stateString(state.taskState)} } - def retryRow(state: RetryState): Seq[Node] = { + private def retryRow(state: RetryState): Seq[Node] = { {state.submission.submissionId} - {state.submission.submitDate} - {state.submission.desc.desc.command.mainClass} + {state.submission.submissionDate} + {state.submission.command.mainClass} {state.lastFailureStatus} {state.nextRetry} {state.retries} } - def stateString(status: Option[TaskStatus]): String = { + private def stateString(status: Option[TaskStatus]): String = { if (status.isEmpty) { return "" } - val sb = new StringBuilder sb.append(s"State: ${status.get.getState}") - if (status.get.hasMessage) { sb.append(s", Message: ${status.get.getMessage}") } - if (status.get.hasHealthy) { sb.append(s", Healthy: ${status.get.getHealthy}") } - if (status.get.hasSource) { sb.append(s", Source: ${status.get.getSource}") } - if (status.get.hasReason) { sb.append(s", Reason: ${status.get.getReason}") } - if (status.get.hasTimestamp) { sb.append(s", Time: ${status.get.getTimestamp}") } - sb.toString() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala index a7382a401a63f..24c64e4be1f00 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala @@ -17,11 +17,10 @@ package org.apache.spark.deploy.mesos.ui -import org.apache.spark.ui.{SparkUI, WebUI} -import org.apache.spark.SparkConf -import org.apache.spark.SecurityManager +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.scheduler.cluster.mesos.ClusterScheduler +import org.apache.spark.ui.{SparkUI, WebUI} /** * UI that displays driver results from the [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]] @@ -31,7 +30,7 @@ private [spark] class MesosClusterUI( port: Int, conf: SparkConf, dispatcherPublicAddress: String, - val scheduler: ClusterScheduler) + val scheduler: MesosClusterScheduler) extends WebUI(securityManager, port, conf) { initialize() @@ -39,7 +38,7 @@ private [spark] class MesosClusterUI( def activeWebUiUrl: String = "http://" + dispatcherPublicAddress + ":" + boundPort override def initialize() { - attachPage(new DriverOutputPage(this)) + attachPage(new MesosClusterPage(this)) attachHandler(createStaticHandler(MesosClusterUI.STATIC_RESOURCE_DIR, "/static")) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala index 513a0cfa1b993..7ee730a1f3392 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala @@ -46,24 +46,19 @@ import org.apache.spark.util.Utils * fails, the response will consist of an empty body with a response code that indicates internal * server error. * - * @param host the address this server should bind to - * @param requestedPort the port this server will attempt to bind to */ -private[spark] abstract class RestServer( - host: String, - requestedPort: Int, - masterConf: SparkConf, - submitRequestServlet: SubmitRequestServlet, - killRequestServlet: KillRequestServlet, - statusRequestServlet: StatusRequestServlet) - extends Logging { - - import RestServer._ +private[spark] abstract class RestServer extends Logging { + val host: String + val requestedPort: Int + val masterConf: SparkConf + val submitRequestServlet: SubmitRequestServlet + val killRequestServlet: KillRequestServlet + val statusRequestServlet: StatusRequestServlet private var _server: Option[Server] = None // A mapping from URL prefixes to servlets that serve them. Exposed for testing. - protected val baseContext = s"/$PROTOCOL_VERSION/submissions" + protected val baseContext = s"/$RestServer.PROTOCOL_VERSION/submissions" protected val contextToServlet = Map[String, RestServlet]( s"$baseContext/create/*" -> submitRequestServlet, s"$baseContext/kill/*" -> killRequestServlet, @@ -110,7 +105,7 @@ private[rest] object RestServer { } /** - * An abstract servlet for handling requests passed to the [[StandaloneRestServer]]. + * An abstract servlet for handling requests passed to the [[RestServer]]. */ private[rest] abstract class RestServlet extends HttpServlet with Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 1f3c3e97e9818..5a2c1afe279b8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -50,18 +50,17 @@ import org.apache.spark.deploy.ClientArguments._ * @param masterConf the conf used by the Master */ private[deploy] class StandaloneRestServer( - host: String, - requestedPort: Int, + val host: String, + val requestedPort: Int, masterActor: ActorRef, masterUrl: String, - masterConf: SparkConf) - extends RestServer( - host, - requestedPort, - masterConf, - new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf), - new StandaloneKillRequestServlet(masterActor, masterConf), - new StandaloneStatusRequestServlet(masterActor, masterConf)) {} + val masterConf: SparkConf) + extends RestServer { + + val submitRequestServlet = new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) + val killRequestServlet = new StandaloneKillRequestServlet(masterActor, masterConf) + val statusRequestServlet = new StandaloneStatusRequestServlet(masterActor, masterConf) +} /** * A servlet for handling kill requests passed to the [[StandaloneRestServer]]. diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala index 8fde8c142a4c1..0e226ee294cab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala @@ -35,7 +35,7 @@ private[rest] abstract class SubmitRestProtocolResponse extends SubmitRestProtoc /** * A response to a [[CreateSubmissionRequest]] in the REST application submission protocol. */ -private[rest] class CreateSubmissionResponse extends SubmitRestProtocolResponse { +private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -46,7 +46,7 @@ private[rest] class CreateSubmissionResponse extends SubmitRestProtocolResponse /** * A response to a kill request in the REST application submission protocol. */ -private[rest] class KillSubmissionResponse extends SubmitRestProtocolResponse { +private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -58,7 +58,7 @@ private[rest] class KillSubmissionResponse extends SubmitRestProtocolResponse { /** * A response to a status request in the REST application submission protocol. */ -private[rest] class SubmissionStatusResponse extends SubmitRestProtocolResponse { +private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse { var submissionId: String = null var driverState: String = null var workerId: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala similarity index 57% rename from core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala rename to core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index dce3100d9f7c3..c4e964db667ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -15,59 +15,49 @@ * limitations under the License. */ -package org.apache.spark.deploy.rest +package org.apache.spark.deploy.rest.mesos import java.io.File import javax.servlet.http.HttpServletResponse -import org.apache.spark.deploy.DriverDescription -import org.apache.spark.deploy.ClientArguments._ -import org.apache.spark.deploy.Command -import org.apache.spark.{SparkConf, SPARK_VERSION => sparkVersion} -import org.apache.spark.util.Utils -import org.apache.spark.scheduler.cluster.mesos.ClusterScheduler import scala.collection.mutable + +import org.apache.spark.deploy.Command import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.deploy.rest._ +import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler +import org.apache.spark.util.Utils +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} + /** * A server that responds to requests submitted by the [[RestClient]]. - * This is intended to be used in Mesos cluster mode only. - * - * This server responds with different HTTP codes depending on the situation: - * 200 OK - Request was processed successfully - * 400 BAD REQUEST - Request was malformed, not successfully validated, or of unexpected type - * 468 UNKNOWN PROTOCOL VERSION - Request specified a protocol this server does not understand - * 500 INTERNAL SERVER ERROR - Server throws an exception internally while processing the request - * - * The server always includes a JSON representation of the relevant [[SubmitRestProtocolResponse]] - * in the HTTP body. If an error occurs, however, the server will include an [[ErrorResponse]] - * instead of the one expected by the client. If the construction of this error response itself - * fails, the response will consist of an empty body with a response code that indicates internal - * server error. + * This is intended to be used in Mesos cluster mode only, which forwards all requests to + * [[MesosClusterScheduler]]. * - * @param host the address this server should bind to - * @param requestedPort the port this server will attempt to bind to - * @param masterConf the conf used by the Master - * @param scheduler the scheduler that handles driver requests + * For more details about the RestServer Spark protocol and status codes please refer to + * [[RestServer]] javadocs. */ -private [spark] class MesosRestServer( - host: String, - requestedPort: Int, - masterConf: SparkConf, - scheduler: ClusterScheduler) - extends RestServer( - host, - requestedPort, - masterConf, - new MesosSubmitRequestServlet(scheduler, masterConf), - new MesosKillRequestServlet(scheduler, masterConf), - new MesosStatusRequestServlet(scheduler, masterConf)) {} - -class MesosSubmitRequestServlet( - scheduler: ClusterScheduler, +private[spark] class MesosRestServer( + val host: String, + val requestedPort: Int, + val masterConf: SparkConf, + scheduler: MesosClusterScheduler) + extends RestServer { + val submitRequestServlet = new MesosSubmitRequestServlet(scheduler, masterConf) + val killRequestServlet = new MesosKillRequestServlet(scheduler, masterConf) + val statusRequestServlet = new MesosStatusRequestServlet(scheduler, masterConf) +} + +private[mesos] class MesosSubmitRequestServlet( + scheduler: MesosClusterScheduler, conf: SparkConf) extends SubmitRequestServlet { + private val DEFAULT_SUPERVISE = false + private val DEFAULT_MEMORY = 512 // mb + private val DEFAULT_CORES = 1 + /** * Build a driver description from the fields specified in the submit request. * @@ -98,26 +88,22 @@ class MesosSubmitRequestServlet( // Store Spark submit specific arguments here to pass to the scheduler. schedulerProperties("spark.app.name") = sparkProperties.getOrElse("spark.app.name", mainClass) - sparkProperties.get("spark.executor.memory").foreach { - v => schedulerProperties("spark.executor.memory") = v + sparkProperties.get("spark.executor.memory").foreach { v => + schedulerProperties("spark.executor.memory") = v } - - sparkProperties.get("spark.cores.max").foreach { - v => schedulerProperties("spark.cores.max") = v + sparkProperties.get("spark.cores.max").foreach { v => + schedulerProperties("spark.cores.max") = v } - - sparkProperties.get("spark.executor.uri").foreach { - v => schedulerProperties("spark.executor.uri") = v + sparkProperties.get("spark.executor.uri").foreach { v => + schedulerProperties("spark.executor.uri") = v } - - sparkProperties.get("spark.mesos.executor.home").foreach { - v => schedulerProperties("spark.mesos.executor.home") = v + sparkProperties.get("spark.mesos.executor.home").foreach { v => + schedulerProperties("spark.mesos.executor.home") = v } // Construct driver description val conf = new SparkConf(false) .setAll(sparkProperties) - val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) @@ -129,10 +115,9 @@ class MesosSubmitRequestServlet( val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES) - val desc = new DriverDescription( - appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command) - - new MesosDriverDescription(desc, schedulerProperties) + new MesosDriverDescription( + appResource, actualDriverMemory, actualDriverCores, + actualSuperviseDriver, command, schedulerProperties) } protected override def handleSubmit( @@ -142,18 +127,14 @@ class MesosSubmitRequestServlet( requestMessage match { case submitRequest: CreateSubmissionRequest => val driverDescription = buildDriverDescription(submitRequest) - val response = scheduler.submitDriver(driverDescription) - val submitResponse = new CreateSubmissionResponse - submitResponse.serverSparkVersion = sparkVersion - submitResponse.message = response.message - submitResponse.success = response.success - submitResponse.submissionId = response.id + val s = scheduler.submitDriver(driverDescription) + s.serverSparkVersion = sparkVersion val unknownFields = findUnknownFields(requestMessageJson, requestMessage) if (unknownFields.nonEmpty) { // If there are fields that the server does not know about, warn the client - submitResponse.unknownFields = unknownFields + s.unknownFields = unknownFields } - submitResponse + s case unexpected => responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) handleError(s"Received message of unexpected type ${unexpected.messageType}.") @@ -161,29 +142,20 @@ class MesosSubmitRequestServlet( } } -class MesosKillRequestServlet(scheduler: ClusterScheduler, conf: SparkConf) +private[mesos] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) extends KillRequestServlet { protected override def handleKill(submissionId: String): KillSubmissionResponse = { - val response = scheduler.killDriver(submissionId) - val k = new KillSubmissionResponse + val k = scheduler.killDriver(submissionId) k.serverSparkVersion = sparkVersion - k.message = response.message - k.submissionId = submissionId - k.success = response.success k } } -class MesosStatusRequestServlet(scheduler: ClusterScheduler, conf: SparkConf) +private[mesos] class MesosStatusRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) extends StatusRequestServlet { protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { - val response = scheduler.getStatus(submissionId) - val d = new SubmissionStatusResponse - d.driverState = response.state + val d = scheduler.getStatus(submissionId) d.serverSparkVersion = sparkVersion - d.submissionId = response.id - d.success = response.success - d.message = response.status.map { s => s.toString }.getOrElse("") d } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 921140e367d1b..055d2e86063f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -49,7 +49,7 @@ private[spark] class CoarseMesosSchedulerBackend( master: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with MScheduler - with MesosSchedulerHelper { + with MesosSchedulerUtils { val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala index a03ae06bac602..78d6c545c41a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala @@ -19,6 +19,8 @@ package org.apache.spark.scheduler.cluster.mesos import scala.collection.mutable +import org.apache.spark.deploy.mesos.MesosDriverDescription + /** * A request queue for launching drivers in Mesos cluster mode. * This queue automatically stores the state after each pop/push @@ -26,14 +28,14 @@ import scala.collection.mutable * This queue is also bounded and rejects offers when it's full. * @param state Mesos state abstraction to fetch persistent state. */ -private[mesos] class DriverQueue(state: ClusterPersistenceEngine, capacity: Int) { - var queue: mutable.Queue[DriverSubmission] = new mutable.Queue[DriverSubmission]() +private[mesos] class DriverQueue(state: MesosClusterPersistenceEngine, capacity: Int) { + var queue: mutable.Queue[MesosDriverDescription] = new mutable.Queue[MesosDriverDescription]() private var count = 0 initialize() - def initialize() { - state.fetchAll[DriverSubmission]().foreach(d => queue.enqueue(d)) + def initialize(): Unit = { + state.fetchAll[MesosDriverDescription]().foreach(d => queue.enqueue(d)) // This size might be larger than the passed in capacity, but we allow // this so we don't lose queued drivers. @@ -46,42 +48,42 @@ private[mesos] class DriverQueue(state: ClusterPersistenceEngine, capacity: Int) queue.exists(s => s.submissionId.equals(submissionId)) } - def offer(submission: DriverSubmission): Boolean = { + def offer(submission: MesosDriverDescription): Boolean = { if (isFull) { return false } queue.enqueue(submission) - state.persist(submission.submissionId, submission) + state.persist(submission.submissionId.get, submission) true } def remove(submissionId: String): Boolean = { val removed = queue.dequeueFirst(d => d.submissionId.equals(submissionId)) if (removed.isDefined) { - state.expunge(removed.get.submissionId) + state.expunge(removed.get.submissionId.get) } removed.isDefined } - def peek(): Option[DriverSubmission] = { + def peek(): Option[MesosDriverDescription] = { queue.headOption } - def poll(): Option[DriverSubmission] = { + def poll(): Option[MesosDriverDescription] = { if (queue.isEmpty) { None } else { val item = queue.dequeue() - state.expunge(item.submissionId) + state.expunge(item.submissionId.get) Some(item) } } // Returns a copy of the queued drivers. - def drivers: Iterable[DriverSubmission] = { - val buffer = new Array[DriverSubmission](queue.size) + def drivers: Iterable[MesosDriverDescription] = { + val buffer = new Array[MesosDriverDescription](queue.size) queue.copyToArray(buffer) buffer } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala index 27570409e5f50..66cca24ac61b0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala @@ -22,10 +22,10 @@ import org.apache.mesos.Protos.SlaveID /** * Tracks all the launched or running drivers in the Mesos cluster scheduler. - * @param state Persistence engine to store state. + * @param state Persistence engine to store tasks state. */ -private[mesos] class LaunchedDrivers(state: ClusterPersistenceEngine) { - private val drivers = new mutable.HashMap[String, ClusterTaskState] +private[mesos] class LaunchedDrivers(state: MesosClusterPersistenceEngine) { + private val drivers = new mutable.HashMap[String, MesosClusterTaskState] // Holds the list of tasks that needs to reconciliation from the master. // All states that are loaded after failover are added here. @@ -33,23 +33,22 @@ private[mesos] class LaunchedDrivers(state: ClusterPersistenceEngine) { initialize() - def initialize() { - state.fetchAll[ClusterTaskState]().foreach { - case state => - drivers(state.taskId.getValue) = state - pendingRecover(state.taskId.getValue) = state.slaveId + def initialize(): Unit = { + state.fetchAll[MesosClusterTaskState]().foreach { case state => + drivers(state.taskId.getValue) = state + pendingRecover(state.taskId.getValue) = state.slaveId } } - def get(submissionId: String) = drivers(submissionId) + def get(submissionId: String): MesosClusterTaskState = drivers(submissionId) - def states: Iterable[ClusterTaskState] = { + def states: Iterable[MesosClusterTaskState] = { drivers.values.map(_.copy()).toList } def contains(submissionId: String): Boolean = drivers.contains(submissionId) - def remove(submissionId: String): Option[ClusterTaskState] = { + def remove(submissionId: String): Option[MesosClusterTaskState] = { if (pendingRecover.contains(submissionId)) { pendingRecover.remove(submissionId) } @@ -59,7 +58,7 @@ private[mesos] class LaunchedDrivers(state: ClusterPersistenceEngine) { removedState } - def set(submissionId: String, newState: ClusterTaskState) { + def set(submissionId: String, newState: MesosClusterTaskState): Unit = { if (pendingRecover.contains(newState.taskId.getValue)) { pendingRecover.remove(newState.taskId.getValue) } @@ -67,6 +66,6 @@ private[mesos] class LaunchedDrivers(state: ClusterPersistenceEngine) { state.persist(submissionId, newState) } - def isEmpty = drivers.isEmpty + def isEmpty: Boolean = drivers.isEmpty } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/ClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala similarity index 55% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/ClusterPersistenceEngine.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala index ecca4a5817495..33ed264196b49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/ClusterPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -20,57 +20,79 @@ package org.apache.spark.scheduler.cluster.mesos import scala.collection.JavaConversions._ import org.apache.curator.framework.CuratorFramework -import org.apache.spark.deploy.SparkCuratorUtil - - import org.apache.zookeeper.CreateMode +import org.apache.zookeeper.KeeperException.NoNodeException + import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.util.Utils -import org.apache.zookeeper.KeeperException.NoNodeException -abstract class ClusterPersistenceEngineFactory(conf: SparkConf) { - def createEngine(path: String): ClusterPersistenceEngine +/** + * Persistence engine factory that is responsible for creating new persistence engines + * to store Mesos cluster mode state. + */ +private[spark] abstract class MesosClusterPersistenceEngineFactory(conf: SparkConf) { + def createEngine(path: String): MesosClusterPersistenceEngine +} + +/** + * Mesos cluster persistence engine is responsible for persisting Mesos cluster mode + * specific state, so that on failover all the state can be recovered and the scheduler + * can resume managing the drivers. + */ +private[spark] trait MesosClusterPersistenceEngine { + def persist(name: String, obj: Object): Unit + def expunge(name: String): Unit + def fetch[T](name: String): Option[T] + def fetchAll[T](): Iterable[T] } -private[spark] class ZookeeperClusterPersistenceEngineFactory(conf: SparkConf) - extends ClusterPersistenceEngineFactory(conf) { +/** + * Zookeeper backed persistence engine factory. + * All Zk engines created from this factory shares the same Zookeeper client, so + * all of them reuses the same connection pool. + */ +private[spark] class ZookeeperMesosClusterPersistenceEngineFactory(conf: SparkConf) + extends MesosClusterPersistenceEngineFactory(conf) { lazy val zk = SparkCuratorUtil.newClient(conf) - def createEngine(path: String): ClusterPersistenceEngine = { - new ZookeeperClusterPersistenceEngine(path, zk, conf) + def createEngine(path: String): MesosClusterPersistenceEngine = { + new ZookeeperMesosClusterPersistenceEngine(path, zk, conf) } } -private[spark] class BlackHolePersistenceEngineFactory - extends ClusterPersistenceEngineFactory(null) { - def createEngine(path: String): ClusterPersistenceEngine = { - new BlackHoleClusterPersistenceEngine +/** + * Black hole persistence engine factory that creates black hole + * persistence engines, which stores nothing. + */ +private[spark] class BlackHoleMesosClusterPersistenceEngineFactory + extends MesosClusterPersistenceEngineFactory(null) { + def createEngine(path: String): MesosClusterPersistenceEngine = { + new BlackHoleMesosClusterPersistenceEngine } } -trait ClusterPersistenceEngine { - def persist(name: String, obj: Object) - def expunge(name: String) - def fetch[T](name: String): Option[T] - def fetchAll[T](): Iterable[T] -} - -private[spark] class BlackHoleClusterPersistenceEngine extends ClusterPersistenceEngine { +/** + * Black hole persistence engine that stores nothing. + */ +private[spark] class BlackHoleMesosClusterPersistenceEngine extends MesosClusterPersistenceEngine { override def persist(name: String, obj: Object): Unit = {} - override def fetch[T](name: String): Option[T] = None - override def expunge(name: String): Unit = {} - override def fetchAll[T](): Iterable[T] = Iterable.empty[T] } -private[spark] class ZookeeperClusterPersistenceEngine( +/** + * Zookeeper based Mesos cluster persistence engine, that stores cluster mode state + * into Zookeeper. Each engine object is operating under one folder in Zookeeper, but + * reuses a shared Zookeeper client. + */ +private[spark] class ZookeeperMesosClusterPersistenceEngine( baseDir: String, zk: CuratorFramework, conf: SparkConf) - extends ClusterPersistenceEngine with Logging { + extends MesosClusterPersistenceEngine with Logging { private val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark_mesos_dispatcher") + "/" + baseDir @@ -80,11 +102,11 @@ private[spark] class ZookeeperClusterPersistenceEngine( WORKING_DIR + "/" + name } - override def expunge(name: String) { + override def expunge(name: String): Unit = { zk.delete().forPath(path(name)) } - override def persist(name: String, obj: Object) { + override def persist(name: String, obj: Object): Unit = { val serialized = Utils.serialize(obj) val zkPath = path(name) zk.create().withMode(CreateMode.PERSISTENT).forPath(zkPath, serialized) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 1acac7f7e067c..3dd5dce309bec 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -21,213 +21,229 @@ import java.io.File import java.text.SimpleDateFormat import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.locks.ReentrantLock -import java.util.{Date, List => JList} - -import org.apache.mesos.{SchedulerDriver, Scheduler} -import org.apache.mesos.Protos._ -import org.apache.spark.deploy.master.DriverState -import org.apache.spark.deploy.master.DriverState.DriverState -import org.apache.spark.SparkConf -import org.apache.spark.SparkException -import org.apache.spark.util.Utils - -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ +import java.util.{Collections, Date, List => JList} -import scala.concurrent.duration.Duration import org.apache.mesos.Protos.Environment.Variable -import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.mesos.Protos.TaskStatus.Reason +import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} +import org.apache.mesos.{Scheduler, SchedulerDriver} +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} +import org.apache.spark.util.Utils +import org.apache.spark.{SparkConf, SparkException, TaskState} -private[spark] class DriverSubmission( - val submissionId: String, - val desc: MesosDriverDescription, - val submitDate: Date) extends Serializable { - - def canEqual(other: Any): Boolean = other.isInstanceOf[DriverSubmission] +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer - override def equals(other: Any): Boolean = other match { - case that: DriverSubmission => - (that canEqual this) && - submissionId == that.submissionId - case _ => false - } -} -private [spark] case class ClusterTaskState( - val submission: DriverSubmission, +/** + * Tracks the current state of a Mesos Task that runs a Spark driver. + * @param submission Submitted driver description from + * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] + * @param taskId Mesos TaskID generated for the task + * @param slaveId Slave ID that the task is assigned to + * @param taskState The last known task status update. + * @param startDate The date the task was launched + * @param retryState Retry state for this task (only applicable to supervised drivers) + */ +private[spark] class MesosClusterTaskState( + val submission: MesosDriverDescription, val taskId: TaskID, val slaveId: SlaveID, var taskState: Option[TaskStatus], - var driverState: DriverState, var startDate: Date, - val lastRetry: Option[RetryState] = None) extends Serializable { + val retryState: Option[RetryState] = None) + extends Serializable { - def copy(): ClusterTaskState = { - ClusterTaskState(submission, taskId, slaveId, taskState, driverState, startDate, lastRetry) + def copy(): MesosClusterTaskState = { + new MesosClusterTaskState( + submission, taskId, slaveId, taskState, startDate, retryState) } } -private[spark] case class SubmitResponse(id: String, success: Boolean, message: String) - -private[spark] case class StatusResponse( - id: String, - success: Boolean, - state: String, - status: Option[TaskStatus] = None) - -private[spark] case class KillResponse(id: String, success: Boolean, message: String) - -private[spark] case class ClusterSchedulerState( - appId: String, - queuedDrivers: Iterable[DriverSubmission], - launchedDrivers: Iterable[ClusterTaskState], - finishedDrivers: Iterable[ClusterTaskState], - retryList: Iterable[RetryState]) - -private[spark] trait ClusterScheduler { - def submitDriver(desc: MesosDriverDescription): SubmitResponse - - def killDriver(submissionId: String): KillResponse - - def getStatus(submissionId: String): StatusResponse - - def getState(): ClusterSchedulerState +/** + * The full state of the cluster scheduler, currently being used for displaying + * information on the UI. + * @param frameworkId Mesos Framework id for the cluster scheduler. + * @param masterUrl The Mesos master url + * @param queuedDrivers All drivers queued to be launched + * @param launchedDrivers All launched or running drivers + * @param finishedDrivers All terminated drivers + * @param retryList All drivers pending to be retried + */ +private[spark] class MesosClusterSchedulerState( + val frameworkId: String, + val masterUrl: String, + val queuedDrivers: Iterable[MesosDriverDescription], + val launchedDrivers: Iterable[MesosClusterTaskState], + val finishedDrivers: Iterable[MesosClusterTaskState], + val retryList: Iterable[RetryState]) + +/** + * Mesos cluster scheduler for running, killing and requesting + * status of Spark drivers that will be launched in a Mesos cluster. + * This interface is mainly for [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] and + * [[org.apache.spark.deploy.mesos.ui.MesosClusterPage]] to interact with + * [[MesosClusterSchedulerDriver]] and hide all the Mesos specific methods that it doesn't + * need to care about. + */ +private[spark] trait MesosClusterScheduler { + def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse + def killDriver(submissionId: String): KillSubmissionResponse + def getStatus(submissionId: String): SubmissionStatusResponse + def getState(): MesosClusterSchedulerState } -private[spark] class MesosClusterScheduler( - engineFactory: ClusterPersistenceEngineFactory, - conf: SparkConf) extends Scheduler with MesosSchedulerHelper with ClusterScheduler { +/** + * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode + * as Mesos tasks in a Mesos cluster. + * All drivers are launched asynchronously by the framework, which will eventually be launched + * by one of the slaves in the cluster. The results of the driver will be stored in slave's task + * sandbox which is accessible by visiting the Mesos UI. + * This scheduler supports recovery by persisting all its state and performs task reconciliation + * on recover, which gets all the latest state for all the drivers from Mesos master. + */ +private[spark] class MesosClusterSchedulerDriver( + engineFactory: MesosClusterPersistenceEngineFactory, + conf: SparkConf) + extends Scheduler with MesosSchedulerUtils with MesosClusterScheduler { var frameworkUrl: String = _ - val master = conf.get("spark.master") - val appName = conf.get("spark.app.name") - val queuedCapacity = conf.getInt("spark.deploy.mesos.queuedDrivers", 200) - val retainedDrivers = conf.getInt("spark.deploy.retainedDrivers", 200) - val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute - val state = engineFactory.createEngine("scheduler") - val stateTimeout = - Duration.create(conf.getLong("spark.mesos.cluster.recover.timeout", 30), "seconds") - - val stateLock = new ReentrantLock() - - val finishedDrivers = new mutable.ArrayBuffer[ClusterTaskState](retainedDrivers) - - val nextDriverNumber: AtomicLong = new AtomicLong(0) - var appId: String = null - + private val master = conf.get("spark.master") + private val appName = conf.get("spark.app.name") + private val queuedCapacity = conf.getInt("spark.deploy.mesos.queuedDrivers", 200) + private val retainedDrivers = conf.getInt("spark.deploy.retainedDrivers", 200) + private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute + private val state = engineFactory.createEngine("scheduler") + private val stateLock = new ReentrantLock() + private val finishedDrivers = new mutable.ArrayBuffer[MesosClusterTaskState](retainedDrivers) + private val nextDriverNumber: AtomicLong = new AtomicLong(0) + private var frameworkId: String = null + + // Stores all the launched and running drivers' states. private var launchedDrivers: LaunchedDrivers = _ + // A queue that stores all the submitted drivers that hasn't been launched. private var queue: DriverQueue = _ - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - + // All supervised drivers that are waiting to retry after termination. private var superviseRetryList: SuperviseRetryList = _ + private var masterInfo: MasterInfo = _ + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs private def newDriverId(submitDate: Date): String = { "driver-%s-%04d".format( createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) } - def submitDriver(desc: MesosDriverDescription): SubmitResponse = { + def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { + val c = new CreateSubmissionResponse + val submitDate = new Date() + desc.submissionId = Some(newDriverId(submitDate)) + desc.submissionDate = Some(submitDate) + stateLock.synchronized { if (queue.isFull) { - return SubmitResponse("", false, "Already reached maximum submission size") + c.success = false + c.message = "Already reached maximum submission size" } - - val submitDate: Date = new Date() - val submissionId: String = newDriverId(submitDate) - val submission = new DriverSubmission(submissionId, desc, submitDate) - queue.offer(submission) - SubmitResponse(submissionId, true, "") + c.submissionId = desc.submissionId.get + queue.offer(desc) + c.success = true } + c } - def killDriver(submissionId: String): KillResponse = { + def killDriver(submissionId: String): KillSubmissionResponse = { + val k = new KillSubmissionResponse + k.submissionId = submissionId stateLock.synchronized { // We look for the requested driver in the following places: // 1. Check if submission is running or launched. // 2. Check if it's still queued. // 3. Check if it's in the retry list. + // 4. Check if it has already completed. if (launchedDrivers.contains(submissionId)) { val task = launchedDrivers.get(submissionId) driver.killTask(task.taskId) - return KillResponse(submissionId, true, "Killing running driver") + k.success = true + k.message = "Killing running driver" } else if (queue.remove(submissionId)) { - return KillResponse(submissionId, true, "Removed driver while it's still pending") + k.success = true + k.message = "Removed driver while it's still pending" } else if (superviseRetryList.remove(submissionId)) { - return KillResponse(submissionId, true, "Removed driver while it's retrying") + k.success = true + k.message = "Removed driver while it's retrying" + } else if (finishedDrivers.exists(s => s.submission.submissionId.equals(submissionId))) { + k.success = false + k.message = "Driver already terminated" } else { - return KillResponse(submissionId, false, "Cannot find driver") + k.success = false + k.message = "Cannot find driver" } } + k } - def recoverState { + /** + * Recover scheduler state that is persisted. + * We still need to do task reconciliation to be up to date of the latest task states + * as it might have changed while the scheduler is failing over. + */ + def recoverState: Unit = { stateLock.synchronized { queue = new DriverQueue(engineFactory.createEngine("driverQueue"), queuedCapacity) - launchedDrivers = new LaunchedDrivers(engineFactory.createEngine("launchedDrivers")) - // There is potential timing issue where a queued driver might have been launched // but the scheduler shuts down before the queued driver was able to be removed // from the queue. We try to mitigate this issue by walking through all queued drivers // and remove if they're already launched. - queue.drivers.foreach { - d => if (launchedDrivers.contains(d.submissionId)) { - queue.remove(d.submissionId) - } - } + queue.drivers.map(_.submissionId.get).filter(launchedDrivers.contains).foreach(queue.remove) superviseRetryList = new SuperviseRetryList(engineFactory.createEngine("retryList")) - // TODO: Consider storing finished drivers so we can show them on the UI after // failover. For now we clear the history on each recovery. finishedDrivers.clear() } } - def start() { + def start(): Unit = { // TODO: Implement leader election to make sure only one framework running in the cluster. val fwId = state.fetch[String]("frameworkId") - val builder = FrameworkInfo.newBuilder() .setUser(Utils.getCurrentUserName()) .setName(appName) .setWebuiUrl(frameworkUrl) .setCheckpoint(true) - .setFailoverTimeout(Integer.MAX_VALUE) // Setting to max for tasks keep running until recovery - + .setFailoverTimeout(Integer.MAX_VALUE) // Setting to max so tasks keep running on crash fwId.foreach { id => builder.setId(FrameworkID.newBuilder().setValue(id).build()) - appId = id + frameworkId = id } - - // Recover scheduler state that is persisted. - // We still need to do task reconciliation to be up to date of the latest task states - // as it might have changed while the scheduler is failing over. recoverState - startScheduler("MesosClusterScheduler", master, MesosClusterScheduler.this, builder.build()) + startScheduler( + "MesosClusterScheduler", master, MesosClusterSchedulerDriver.this, builder.build()) } - def stop() { + def stop(): Unit = { driver.stop(true) } override def registered( driver: SchedulerDriver, - frameworkId: FrameworkID, + newFrameworkId: FrameworkID, masterInfo: MasterInfo): Unit = { - logInfo("Registered as framework ID " + frameworkId.getValue) - if (frameworkId.getValue != appId) { - appId = frameworkId.getValue - state.persist("frameworkId", appId) + logInfo("Registered as framework ID " + newFrameworkId.getValue) + if (newFrameworkId.getValue != frameworkId) { + frameworkId = newFrameworkId.getValue + state.persist("frameworkId", frameworkId) } markRegistered() stateLock.synchronized { + this.masterInfo = masterInfo if (!launchedDrivers.pendingRecover.isEmpty) { // Start task reconciliation if we need to recover. val statuses = launchedDrivers.pendingRecover.collect { @@ -236,10 +252,9 @@ private[spark] class MesosClusterScheduler( TaskStatus.newBuilder() .setTaskId(TaskID.newBuilder().setValue(taskId).build()) .setSlaveId(slaveId) - .setState(TaskState.TASK_STAGING) - .build) + .setState(MesosTaskState.TASK_STAGING) + .build()) } - // TODO: Page the status updates to avoid trying to reconcile // a large amount of tasks at once. driver.reconcileTasks(statuses) @@ -247,107 +262,77 @@ private[spark] class MesosClusterScheduler( } } - private def buildCommand(req: DriverSubmission): CommandInfo = { - val desc = req.desc - + private def buildCommand(desc: MesosDriverDescription): CommandInfo = { val appJar = CommandInfo.URI.newBuilder() - .setValue(desc.desc.jarUrl.stripPrefix("file:").stripPrefix("local:")).build() - - val builder = CommandInfo.newBuilder() - .addUris(appJar) - + .setValue(desc.jarUrl.stripPrefix("file:").stripPrefix("local:")).build() + val builder = CommandInfo.newBuilder().addUris(appJar) val entries = (conf.getOption("spark.executor.extraLibraryPath").toList ++ - desc.desc.command.libraryPathEntries) - + desc.command.libraryPathEntries) val prefixEnv = if (!entries.isEmpty) { Utils.libraryPathEnvPrefix(entries) } else { "" } - val envBuilder = Environment.newBuilder() - desc.desc.command.environment.foreach { - case (k, v) => - envBuilder.addVariables( - Variable.newBuilder().setName(k).setValue(v).build()) + desc.command.environment.foreach { case (k, v) => + envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v).build()) } - builder.setEnvironment(envBuilder.build()) - - val cmdOptions = generateCmdOption(req) - - val executorUri = req.desc.schedulerProperties.get("spark.executor.uri") - .orElse(req.desc.desc.command.environment.get("SPARK_EXECUTOR_URI")) - + val cmdOptions = generateCmdOption(desc) + val executorUri = desc.schedulerProperties.get("spark.executor.uri") + .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) val cmd = if (executorUri.isDefined) { builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) - val folderBasename = executorUri.get.split('/').last.split('.').head - val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit" - - val cmdJar = s"../${desc.desc.jarUrl.split("/").last}" - - val appArguments = desc.desc.command.arguments.mkString(" ") - + val cmdJar = s"../${desc.jarUrl.split("/").last}" + val appArguments = desc.command.arguments.mkString(" ") s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar $appArguments" } else { - val executorSparkHome = req.desc.schedulerProperties.get("spark.mesos.executor.home") + val executorSparkHome = desc.schedulerProperties.get("spark.mesos.executor.home") .orElse(conf.getOption("spark.home")) .orElse(Option(System.getenv("SPARK_HOME"))) .getOrElse { throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") } - val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getCanonicalPath - - val cmdJar = desc.desc.jarUrl.split("/").last - + val cmdJar = desc.jarUrl.split("/").last s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar" } - builder.setValue(cmd) - builder.build } - private def generateCmdOption(req: DriverSubmission): Seq[String] = { + private def generateCmdOption(desc: MesosDriverDescription): Seq[String] = { var options = Seq( - "--name", req.desc.schedulerProperties("spark.app.name"), - "--class", req.desc.desc.command.mainClass, + "--name", desc.schedulerProperties("spark.app.name"), + "--class", desc.command.mainClass, "--master", s"mesos://${conf.get("spark.master")}", - "--driver-cores", req.desc.desc.cores.toString, - "--driver-memory", s"${req.desc.desc.mem}M") - - req.desc.schedulerProperties.get("spark.executor.memory").map { v => + "--driver-cores", desc.cores.toString, + "--driver-memory", s"${desc.mem}M") + desc.schedulerProperties.get("spark.executor.memory").map { v => options ++= Seq("--executor-memory", v) } - - req.desc.schedulerProperties.get("spark.cores.max").map { v => + desc.schedulerProperties.get("spark.cores.max").map { v => options ++= Seq("--total-executor-cores", v) } - options } - private [spark] case class ResourceOffer(val offer: Offer, var cpu: Double, var mem: Double) + private class ResourceOffer(val offer: Offer, var cpu: Double, var mem: Double) override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { - val currentOffers = offers.map { - o => - ResourceOffer( - o, - getResource(o.getResourcesList, "cpus"), - getResource(o.getResourcesList, "mem")) + val currentOffers = offers.map { o => + new ResourceOffer( + o, + getResource(o.getResourcesList, "cpus"), + getResource(o.getResourcesList, "mem")) } - val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() - val currentTime = new Date() - def scheduleTasks( - taskFunc: () => (Option[DriverSubmission], Option[RetryState]), + taskFunc: () => (Option[MesosDriverDescription], Option[RetryState]), scheduledCallback: (String) => Unit) { var nextItem = taskFunc() // TODO: We should not stop scheduling at the very first task @@ -356,10 +341,8 @@ private[spark] class MesosClusterScheduler( // over a configurable period of time. while (nextItem._1.isDefined) { val (submission, retryState) = (nextItem._1.get, nextItem._2) - - val driverCpu = submission.desc.desc.cores - val driverMem = submission.desc.desc.mem - + val driverCpu = submission.cores + val driverMem = submission.mem val offerOption = currentOffers.find { o => o.cpu >= driverCpu && o.mem >= driverMem } @@ -369,31 +352,24 @@ private[spark] class MesosClusterScheduler( } val offer = offerOption.get - offer.cpu -= driverCpu offer.mem -= driverMem - - val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() - + val taskId = TaskID.newBuilder().setValue(submission.submissionId.get).build() val cpuResource = Resource.newBuilder() .setName("cpus").setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() - val memResource = Resource.newBuilder() .setName("mem").setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() - val commandInfo = buildCommand(submission) - val taskInfo = TaskInfo.newBuilder() .setTaskId(taskId) - .setName(s"driver for ${submission.desc.desc.command.mainClass}") + .setName(s"driver for ${submission.command.mainClass}") .setSlaveId(offer.offer.getSlaveId) .setCommand(commandInfo) .addResources(cpuResource) .addResources(memResource) .build - val queuedTasks = if (!tasks.contains(offer.offer.getId)) { val buffer = new ArrayBuffer[TaskInfo] tasks(offer.offer.getId) = buffer @@ -401,16 +377,12 @@ private[spark] class MesosClusterScheduler( } else { tasks(offer.offer.getId) } - queuedTasks += taskInfo - launchedDrivers.set( - submission.submissionId, - ClusterTaskState(submission, taskId, offer.offer.getSlaveId, - None, DriverState.SUBMITTED, new Date(), retryState)) - - scheduledCallback(submission.submissionId) - + submission.submissionId.get, + new MesosClusterTaskState(submission, taskId, offer.offer.getSlaveId, + None, new Date(), retryState)) + scheduledCallback(submission.submissionId.get) nextItem = taskFunc() } } @@ -421,11 +393,11 @@ private[spark] class MesosClusterScheduler( }, (id: String) => { superviseRetryList.remove(id) }) - scheduleTasks(() => (queue.peek, None), (_) => queue.poll) + scheduleTasks(() => (queue.peek(), None), (_) => queue.poll()) } - tasks.foreach { - case (offerId, tasks) => driver.launchTasks(offerId, tasks) + tasks.foreach { case (offerId, tasks) => + driver.launchTasks(Collections.singleton(offerId), tasks) } offers @@ -433,10 +405,11 @@ private[spark] class MesosClusterScheduler( .foreach(o => driver.declineOffer(o.getId)) } - def getState(): ClusterSchedulerState = { + def getState(): MesosClusterSchedulerState = { stateLock.synchronized { - ClusterSchedulerState( - appId, + new MesosClusterSchedulerState( + frameworkId, + s"http://${masterInfo.getIp}:${masterInfo.getPort}", queue.drivers, launchedDrivers.states, finishedDrivers.collect { case s => s.copy() }, @@ -444,24 +417,33 @@ private[spark] class MesosClusterScheduler( } } - def getStatus(submissionId: String): StatusResponse = { + def getStatus(submissionId: String): SubmissionStatusResponse = { + val s = new SubmissionStatusResponse + s.submissionId = submissionId stateLock.synchronized { if (queue.contains(submissionId)) { - return StatusResponse(submissionId, true, "Driver is queued for launch") + s.success = true + s.driverState = "Driver is queued for launch" } else if (launchedDrivers.contains(submissionId)) { - val status = launchedDrivers.get(submissionId).taskState - return StatusResponse(submissionId, true, "Driver is running", status) - } else if (finishedDrivers.contains(submissionId)) { - val status = - finishedDrivers.find(d => d.submission.submissionId.equals(submissionId)).get.taskState - return StatusResponse(submissionId, true, "Driver already finished", status) + s.success = true + s.driverState = "Driver is running" + launchedDrivers.get(submissionId).taskState.foreach(state => s.message = state.toString) + } else if (finishedDrivers.exists(s => s.submission.submissionId.equals(submissionId))) { + s.success = true + s.driverState = "Driver already finished" + finishedDrivers.find(d => d.submission.submissionId.equals(submissionId)).get.taskState + .foreach(state => s.message = state.toString) } else if (superviseRetryList.contains(submissionId)) { val status = superviseRetryList.get(submissionId).get.lastFailureStatus - return StatusResponse(submissionId, true, "Driver failed and retrying", Some(status)) + s.success = true + s.driverState = "Driver failed and retrying" + s.message = status.toString } else { - return StatusResponse(submissionId, false, "Driver not found") + s.success = false + s.driverState = "Driver not found" } } + s } override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {} @@ -476,23 +458,15 @@ private[spark] class MesosClusterScheduler( override def error(driver: SchedulerDriver, error: String): Unit = {} - def getDriverState(state: TaskState): DriverState = { - state match { - case TaskState.TASK_FAILED => DriverState.FAILED - case TaskState.TASK_ERROR => DriverState.ERROR - case TaskState.TASK_FINISHED => DriverState.FINISHED - case TaskState.TASK_KILLED => DriverState.KILLED - case TaskState.TASK_LOST => DriverState.ERROR - case TaskState.TASK_RUNNING => DriverState.RUNNING - case TaskState.TASK_STARTING | TaskState.TASK_STAGING => DriverState.SUBMITTED - case _ => DriverState.UNKNOWN - } - } - - def shouldRelaunch(state: TaskState): Boolean = { - state == TaskState.TASK_FAILED || - state == TaskState.TASK_KILLED || - state == TaskState.TASK_LOST + /** + * Check if the task state is a recoverable state that we can relaunch the task. + * Task state like TASK_ERROR are not relaunchable state since it wasn't able + * to be validated by Mesos. + */ + def shouldRelaunch(state: MesosTaskState): Boolean = { + state == MesosTaskState.TASK_FAILED || + state == MesosTaskState.TASK_KILLED || + state == MesosTaskState.TASK_LOST } override def statusUpdate(driver: SchedulerDriver, status: TaskStatus): Unit = { @@ -504,13 +478,12 @@ private[spark] class MesosClusterScheduler( // Task has already received update and no longer requires reconciliation. return } - val state = launchedDrivers.get(taskId) - - if (state.submission.desc.desc.supervise && shouldRelaunch(status.getState)) { - val (retries, waitTimeSec) = if (state.lastRetry.isDefined) { - (state.lastRetry.get.retries + 1, - Math.min(maxRetryWaitTime, state.lastRetry.get.waitTime * 2)) + // Check if the driver is supervise enabled and can be relaunched. + if (state.submission.supervise && shouldRelaunch(status.getState)) { + val (retries, waitTimeSec) = if (state.retryState.isDefined) { + (state.retryState.get.retries + 1, + Math.min(maxRetryWaitTime, state.retryState.get.waitTime * 2)) } else { (1, 1) } @@ -518,20 +491,15 @@ private[spark] class MesosClusterScheduler( superviseRetryList.add( RetryState(state.submission, status, retries, nextRetry, waitTimeSec)) } - - val driverState = getDriverState(status.getState) - if (isFinished(status.getState)) { + if (TaskState.isFinished(TaskState.fromMesos(status.getState))) { launchedDrivers.remove(taskId) if (finishedDrivers.size >= retainedDrivers) { val toRemove = math.max(retainedDrivers / 10, 1) finishedDrivers.trimStart(toRemove) } - finishedDrivers += state } - state.taskState = Option(status) - state.driverState = driverState } else { logError(s"Unable to find driver $taskId in status update") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 833c2cd291a6d..41d77a329ce4b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -47,7 +47,7 @@ private[spark] class MesosSchedulerBackend( master: String) extends SchedulerBackend with MScheduler - with MesosSchedulerHelper { + with MesosSchedulerUtils { // Which slave IDs we have executors on val slaveIdsWithExecutors = new HashSet[String] diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerHelper.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala similarity index 64% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerHelper.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 8e43b0ad96d9c..705fb40646ded 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerHelper.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -17,23 +17,39 @@ package org.apache.spark.scheduler.cluster.mesos -import org.apache.mesos.{Scheduler, MesosSchedulerDriver, SchedulerDriver} -import org.apache.mesos.Protos.{TaskState, Resource, FrameworkInfo} -import org.apache.spark.Logging - -import java.util.concurrent.CountDownLatch import java.util.List +import java.util.concurrent.CountDownLatch + +import org.apache.mesos.Protos.{FrameworkInfo, Resource} +import org.apache.mesos.{MesosSchedulerDriver, Scheduler, SchedulerDriver} +import org.apache.spark.Logging import scala.collection.JavaConversions._ -private[spark] trait MesosSchedulerHelper extends Logging { +/** + * Shared trait for implementing a Mesos Scheduler. This holds common state and helper + * methods and Mesos scheduler will use. + */ +private[mesos] trait MesosSchedulerUtils extends Logging { // Lock used to wait for scheduler to be registered final val registerLatch = new CountDownLatch(1) // Driver for talking to Mesos var driver: SchedulerDriver = null - def startScheduler(name: String, masterUrl: String, scheduler: Scheduler, fwInfo: FrameworkInfo) { + /** + * Starts the MesosSchedulerDriver with the provided information. This method returns + * until the scheduler has registered with Mesos. + * @param name Name of the scheduler + * @param masterUrl Mesos master connection URL + * @param scheduler Scheduler object + * @param fwInfo FrameworkInfo to pass to the Mesos master + */ + def startScheduler( + name: String, + masterUrl: String, + scheduler: Scheduler, + fwInfo: FrameworkInfo): Unit = { synchronized { if (driver != null) { waitForRegister() @@ -58,11 +74,18 @@ private[spark] trait MesosSchedulerHelper extends Logging { } } - private def waitForRegister() { + /** + * Waits for the scheduler to be registered, which the scheduler will signal by calling + * markRegistered(). + */ + def waitForRegister(): Unit = { registerLatch.await() } - def markRegistered() { + /** + * Signal that the scheduler has registered with Mesos. + */ + def markRegistered(): Unit = { registerLatch.countDown() } @@ -72,13 +95,4 @@ private[spark] trait MesosSchedulerHelper extends Logging { } 0.0 } - - /** Check whether a Mesos task state represents a finished task */ - def isFinished(state: TaskState) = { - state == TaskState.TASK_FINISHED || - state == TaskState.TASK_FAILED || - state == TaskState.TASK_KILLED || - state == TaskState.TASK_LOST || - state == TaskState.TASK_ERROR - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala index 0c82436253e15..0925b6969ef72 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala @@ -19,10 +19,21 @@ package org.apache.spark.scheduler.cluster.mesos import org.apache.mesos.Protos.TaskStatus import java.util.Date +import org.apache.spark.deploy.mesos.MesosDriverDescription + import scala.collection.mutable.ArrayBuffer +/** + * Tracks the retry state of a driver, which includes the next time it should be scheduled + * and necessary information to do exponential backoff. + * @param submission Driver submission. + * @param lastFailureStatus Last Task status when it failed. + * @param retries Number of times it has retried. + * @param nextRetry Next retry time to be scheduled. + * @param waitTime The amount of time driver is scheduled to wait until next retry. + */ private[spark] case class RetryState( - submission: DriverSubmission, + submission: MesosDriverDescription, lastFailureStatus: TaskStatus, retries: Int, nextRetry: Date, @@ -35,21 +46,19 @@ private[spark] case class RetryState( * and waiting to be scheduled again. * @param state Persistence engine to store state. */ -private[mesos] class SuperviseRetryList(state: ClusterPersistenceEngine) { +private[mesos] class SuperviseRetryList(state: MesosClusterPersistenceEngine) { val drivers = new ArrayBuffer[RetryState] initialize() - def initialize() { - state.fetchAll[RetryState]().foreach { - s => drivers += s - } + def initialize(): Unit = { + state.fetchAll[RetryState]().foreach(drivers.+=) } - def contains(submissionId: String) = + def contains(submissionId: String): Boolean = drivers.exists(d => d.submission.submissionId.equals(submissionId)) - def getNextRetry(currentTime: Date): (Option[DriverSubmission], Option[RetryState]) = { + def getNextRetry(currentTime: Date): (Option[MesosDriverDescription], Option[RetryState]) = { val retry = drivers.find(d => d.nextRetry.before(currentTime)) if (retry.isDefined) { (Some(retry.get.submission), retry) @@ -63,25 +72,22 @@ private[mesos] class SuperviseRetryList(state: ClusterPersistenceEngine) { } def retries: Iterable[RetryState] = { - drivers.collect { case d => d.copy}.toList + drivers.map(d => d.copy).toList } def remove(submissionId: String): Boolean = { - val index = - drivers.indexWhere( - s => s.submission.submissionId.equals(submissionId)) + val index = drivers.indexWhere(s => s.submission.submissionId.equals(submissionId)) if (index != -1) { drivers.remove(index) state.expunge(submissionId) - true } index != -1 } - def add(retryState: RetryState) { + def add(retryState: RetryState): Unit = { drivers += retryState - state.persist(retryState.submission.submissionId, retryState) + state.persist(retryState.submission.submissionId.get, retryState) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index 4db22bb74d0a7..7319f479862b0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.scheduler.mesos -import org.scalatest.FunSuite -import org.apache.spark.{SparkConf, LocalSparkContext} -import org.scalatest.mock.MockitoSugar -import org.apache.spark.scheduler.cluster.mesos._ import org.apache.spark.deploy.Command -import org.apache.spark.deploy.DriverDescription import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.scheduler.cluster.mesos._ +import org.apache.spark.{LocalSparkContext, SparkConf} +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar + import scala.collection.mutable class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar { @@ -36,38 +36,41 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo val conf = new SparkConf() conf.setMaster("mesos://localhost:5050") conf.setAppName("spark mesos") - val scheduler = new MesosClusterScheduler(new BlackHolePersistenceEngineFactory, conf) + val scheduler = new MesosClusterSchedulerDriver( + new BlackHoleMesosClusterPersistenceEngineFactory, conf) scheduler.recoverState val response = scheduler.submitDriver( - new MesosDriverDescription(new DriverDescription("jar", 1000, 1, true, createCommand), - new mutable.HashMap[String, String]())) + new MesosDriverDescription("jar", 1000, 1, true, + createCommand, new mutable.HashMap[String, String]())) assert(response.success) val response2 = scheduler.submitDriver(new MesosDriverDescription( - new DriverDescription("jar", 1000, 1, true, createCommand), + "jar", 1000, 1, true, createCommand, new mutable.HashMap[String, String]())) assert(response2.success) val state = scheduler.getState() - assert(state.queuedDrivers.exists(d => d.submissionId == response.id)) - assert(state.queuedDrivers.exists(d => d.submissionId == response2.id)) + assert(state.queuedDrivers.exists(d => d.submissionId == response.submissionId)) + assert(state.queuedDrivers.exists(d => d.submissionId == response2.submissionId)) } test("can kill queued drivers") { val conf = new SparkConf() conf.setMaster("mesos://localhost:5050") conf.setAppName("spark mesos") - val scheduler = new MesosClusterScheduler(new BlackHolePersistenceEngineFactory, conf) + val scheduler = new MesosClusterSchedulerDriver( + new BlackHoleMesosClusterPersistenceEngineFactory, conf) scheduler.recoverState val response = - scheduler.submitDriver(new MesosDriverDescription( - new DriverDescription("jar", 1000, 1, true, createCommand), - new mutable.HashMap[String, String]())) + scheduler.submitDriver( + new MesosDriverDescription( + "jar", 1000, 1, true, createCommand, + new mutable.HashMap[String, String]())) assert(response.success) - val killResponse = scheduler.killDriver(response.id) + val killResponse = scheduler.killDriver(response.submissionId) assert(killResponse.success) val state = scheduler.getState() diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh index 22c002f716f41..67e6a15ec7bca 100755 --- a/sbin/start-mesos-dispatcher.sh +++ b/sbin/start-mesos-dispatcher.sh @@ -16,8 +16,10 @@ # See the License for the specific language governing permissions and # limitations under the License. # - # Starts the Mesos Cluster Dispatcher on the machine this script is executed on. +# The Mesos Cluster Dispatcher is responsbile for launching the Mesos framework and +# Rest server to handle driver requests for Mesos cluster mode. +# Only one cluster dispatcher is needed per Mesos cluster. sbin="`dirname "$0"`" sbin="`cd "$sbin"; pwd`" @@ -27,7 +29,7 @@ sbin="`cd "$sbin"; pwd`" . "$SPARK_PREFIX/bin/load-spark-env.sh" if [ "$SPARK_MESOS_DISPATCHER_PORT" = "" ]; then - SPARK_MESOS_DISPATCHER_PORT=8077 + SPARK_MESOS_DISPATCHER_PORT=7077 fi if [ "$SPARK_MESOS_DISPATCHER_HOST" = "" ]; then diff --git a/sbin/stop-mesos-dispatcher.sh b/sbin/stop-mesos-dispatcher.sh index d9adfe120d269..cb65d95b5e524 100755 --- a/sbin/stop-mesos-dispatcher.sh +++ b/sbin/stop-mesos-dispatcher.sh @@ -16,8 +16,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # - -# Starts the master on the machine this script is executed on. +# Stop the Mesos Cluster dispatcher on the machine this script is executed on. sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` From df355cd83fb32bee3ce016732169f07aa93cfbd8 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 1 Apr 2015 00:21:27 -0700 Subject: [PATCH 26/36] Add metrics to mesos cluster scheduler. --- .../scheduler/cluster/mesos/DriverQueue.scala | 2 + .../cluster/mesos/LaunchedDrivers.scala | 2 + .../cluster/mesos/MesosClusterScheduler.scala | 25 ++++++++---- .../mesos/MesosClusterSchedulerSource.scala | 40 +++++++++++++++++++ .../cluster/mesos/SuperviseRetryList.scala | 2 + 5 files changed, 63 insertions(+), 8 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala index 78d6c545c41a9..fc312c8f27751 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala @@ -44,6 +44,8 @@ private[mesos] class DriverQueue(state: MesosClusterPersistenceEngine, capacity: def isFull = count >= capacity + def size: Int = count + def contains(submissionId: String): Boolean = { queue.exists(s => s.submissionId.equals(submissionId)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala index 66cca24ac61b0..170c41e71becd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala @@ -40,6 +40,8 @@ private[mesos] class LaunchedDrivers(state: MesosClusterPersistenceEngine) { } } + def size: Int = drivers.size + def get(submissionId: String): MesosClusterTaskState = drivers(submissionId) def states: Iterable[MesosClusterTaskState] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 3dd5dce309bec..b58dbad9af836 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -23,18 +23,20 @@ import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.locks.ReentrantLock import java.util.{Collections, Date, List => JList} +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + import org.apache.mesos.Protos.Environment.Variable import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} import org.apache.mesos.{Scheduler, SchedulerDriver} + import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} +import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkException, TaskState} - -import scala.collection.JavaConversions._ -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{SparkConf, SparkException, SecurityManager, TaskState} /** @@ -111,6 +113,8 @@ private[spark] class MesosClusterSchedulerDriver( var frameworkUrl: String = _ + private val metricsSystem = + MetricsSystem.createMetricsSystem("mesos_cluster", conf, new SecurityManager(conf)) private val master = conf.get("spark.master") private val appName = conf.get("spark.app.name") private val queuedCapacity = conf.getInt("spark.deploy.mesos.queuedDrivers", 200) @@ -123,13 +127,14 @@ private[spark] class MesosClusterSchedulerDriver( private var frameworkId: String = null // Stores all the launched and running drivers' states. - private var launchedDrivers: LaunchedDrivers = _ + var launchedDrivers: LaunchedDrivers = _ // A queue that stores all the submitted drivers that hasn't been launched. - private var queue: DriverQueue = _ + var queue: DriverQueue = _ // All supervised drivers that are waiting to retry after termination. - private var superviseRetryList: SuperviseRetryList = _ + var superviseRetryList: SuperviseRetryList = _ + private var masterInfo: MasterInfo = _ private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs @@ -223,11 +228,15 @@ private[spark] class MesosClusterSchedulerDriver( frameworkId = id } recoverState + metricsSystem.registerSource(new MesosClusterSchedulerSource(this)) + metricsSystem.start() startScheduler( "MesosClusterScheduler", master, MesosClusterSchedulerDriver.this, builder.build()) } def stop(): Unit = { + metricsSystem.report() + metricsSystem.stop() driver.stop(true) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala new file mode 100644 index 0000000000000..fc6601d3692fb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala @@ -0,0 +1,40 @@ +/* + * 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.scheduler.cluster.mesos + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.metrics.source.Source + +private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterSchedulerDriver) extends Source { + override def sourceName: String = "mesos_cluster" + + override def metricRegistry: MetricRegistry = new MetricRegistry() + + metricRegistry.register(MetricRegistry.name("waitingDrivers"), new Gauge[Int] { + override def getValue: Int = scheduler.queue.size + }) + + metricRegistry.register(MetricRegistry.name("launchedDrivers"), new Gauge[Int] { + override def getValue: Int = scheduler.launchedDrivers.size + }) + + metricRegistry.register(MetricRegistry.name("retryDrivers"), new Gauge[Int] { + override def getValue: Int = scheduler.superviseRetryList.size + }) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala index 0925b6969ef72..7d2cdcf6bb9de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala @@ -55,6 +55,8 @@ private[mesos] class SuperviseRetryList(state: MesosClusterPersistenceEngine) { state.fetchAll[RetryState]().foreach(drivers.+=) } + def size: Int = drivers.size + def contains(submissionId: String): Boolean = drivers.exists(d => d.submission.submissionId.equals(submissionId)) From 6ff8e5c39392ca632b4cf3e0160965bcde387e26 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Fri, 3 Apr 2015 00:32:16 -0700 Subject: [PATCH 27/36] Address comments and add logging. --- .../apache/spark/deploy/master/Master.scala | 2 +- .../deploy/mesos/MesosClusterDispatcher.scala | 6 ++--- .../deploy/mesos/MesosDriverDescription.scala | 8 +++--- .../deploy/mesos/ui/MesosClusterPage.scala | 16 ++++++----- .../apache/spark/deploy/rest/RestServer.scala | 8 +++--- .../deploy/rest/StandaloneRestServer.scala | 16 ++++++----- .../deploy/rest/mesos/MesosRestServer.scala | 19 +++++++------ .../scheduler/cluster/mesos/DriverQueue.scala | 3 ++- .../cluster/mesos/LaunchedDrivers.scala | 10 +++---- .../cluster/mesos/MesosClusterScheduler.scala | 27 ++++++++++++++++--- .../mesos/MesosClusterSchedulerSource.scala | 4 +-- .../cluster/mesos/SuperviseRetryList.scala | 4 ++- .../rest/StandaloneRestSubmitSuite.scala | 26 ++++++++---------- .../mesos/MesosClusterSchedulerSuite.scala | 11 +++----- 14 files changed, 88 insertions(+), 72 deletions(-) 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 ff2eed6dee70a..1c21c179562ac 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 @@ -130,7 +130,7 @@ private[master] class Master( private val restServer = if (restServerEnabled) { val port = conf.getInt("spark.master.rest.port", 6066) - Some(new StandaloneRestServer(host, port, self, masterUrl, conf)) + Some(new StandaloneRestServer(host, port, conf, self, masterUrl)) } else { None } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 204466eee9ad3..50d5acab6e1f3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -30,9 +30,9 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} /* * A dispatcher that is responsible for managing and launching drivers, and is intended to - * be used for Mesos cluster mode. The dispatcher ls launched by the user in the cluster, + * be used for Mesos cluster mode. The dispatcher is launched by the user in the cluster, * which it launches a [[MesosRestServer]] for listening for driver requests, and launches a - * [[MesoClusterScheduler]] to launch these drivers in the Mesos cluster. + * [[MesosClusterScheduler]] to launch these drivers in the Mesos cluster. * * A typical new driver lifecycle is the following: * @@ -141,7 +141,7 @@ private[mesos] object MesosClusterDispatcher extends spark.Logging { propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - def parse(args: List[String]): Unit = args match { + private def parse(args: List[String]): Unit = args match { case ("--host" | "-h") :: value :: tail => Utils.checkHost(value, "Please use hostname " + value) host = value diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala index 05442bb82ac5e..678597b20c0df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala @@ -19,8 +19,6 @@ package org.apache.spark.deploy.mesos import java.util.Date -import scala.collection.mutable - import org.apache.spark.deploy.Command /** @@ -34,13 +32,13 @@ import org.apache.spark.deploy.Command * @param command The command to launch the driver. * @param schedulerProperties Extra properties to pass the Mesos scheduler */ -private[spark] class MesosDriverDescription( +private[spark] case class MesosDriverDescription( val jarUrl: String, val mem: Int, - val cores: Int, + val cores: Double, val supervise: Boolean, val command: Command, - val schedulerProperties: mutable.HashMap[String, String]) + val schedulerProperties: Map[String, String]) extends Serializable { var submissionId: Option[String] = None diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala index 0189267ddc2ba..92c12859dd444 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -30,7 +30,7 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val state = parent.scheduler.getState() - val queuedHeaders = Seq("DriverID", "Submit Date", "Description") + val queuedHeaders = Seq("DriverID", "Submit Date", "Main Class", "Driver resources") val driverHeaders = queuedHeaders ++ Seq("Start Date", "Mesos Slave ID", "State") val retryHeaders = Seq("DriverID", "Submit Date", "Description") ++ @@ -61,6 +61,7 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( {submission.submissionId} {submission.submissionDate} {submission.command.mainClass} + cpus: {submission.cores}, mem: {submission.mem} } @@ -91,21 +92,22 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( return "" } val sb = new StringBuilder - sb.append(s"State: ${status.get.getState}") + val s = status.get + sb.append(s"State: ${s.getState}") if (status.get.hasMessage) { - sb.append(s", Message: ${status.get.getMessage}") + sb.append(s", Message: ${s.getMessage}") } if (status.get.hasHealthy) { - sb.append(s", Healthy: ${status.get.getHealthy}") + sb.append(s", Healthy: ${s.getHealthy}") } if (status.get.hasSource) { - sb.append(s", Source: ${status.get.getSource}") + sb.append(s", Source: ${s.getSource}") } if (status.get.hasReason) { - sb.append(s", Reason: ${status.get.getReason}") + sb.append(s", Reason: ${s.getReason}") } if (status.get.hasTimestamp) { - sb.append(s", Time: ${status.get.getTimestamp}") + sb.append(s", Time: ${s.getTimestamp}") } sb.toString() } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala index 7ee730a1f3392..dbc4ce5e85d4e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala @@ -51,14 +51,14 @@ private[spark] abstract class RestServer extends Logging { val host: String val requestedPort: Int val masterConf: SparkConf - val submitRequestServlet: SubmitRequestServlet - val killRequestServlet: KillRequestServlet - val statusRequestServlet: StatusRequestServlet + def submitRequestServlet: SubmitRequestServlet + def killRequestServlet: KillRequestServlet + def statusRequestServlet: StatusRequestServlet private var _server: Option[Server] = None // A mapping from URL prefixes to servlets that serve them. Exposed for testing. - protected val baseContext = s"/$RestServer.PROTOCOL_VERSION/submissions" + protected val baseContext = s"/${RestServer.PROTOCOL_VERSION}/submissions" protected val contextToServlet = Map[String, RestServlet]( s"$baseContext/create/*" -> submitRequestServlet, s"$baseContext/kill/*" -> killRequestServlet, diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 5a2c1afe279b8..f5618ecebb8cd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -45,21 +45,23 @@ import org.apache.spark.deploy.ClientArguments._ * * @param host the address this server should bind to * @param requestedPort the port this server will attempt to bind to + * @param masterConf the conf used by the Master * @param masterActor reference to the Master actor to which requests can be sent * @param masterUrl the URL of the Master new drivers will attempt to connect to - * @param masterConf the conf used by the Master */ private[deploy] class StandaloneRestServer( val host: String, val requestedPort: Int, + val masterConf: SparkConf, masterActor: ActorRef, - masterUrl: String, - val masterConf: SparkConf) + masterUrl: String) extends RestServer { - - val submitRequestServlet = new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) - val killRequestServlet = new StandaloneKillRequestServlet(masterActor, masterConf) - val statusRequestServlet = new StandaloneStatusRequestServlet(masterActor, masterConf) + def submitRequestServlet: SubmitRequestServlet = + new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) + def killRequestServlet: KillRequestServlet = + new StandaloneKillRequestServlet(masterActor, masterConf) + def statusRequestServlet: StatusRequestServlet = + new StandaloneStatusRequestServlet(masterActor, masterConf) } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index c4e964db667ef..e0d89ac09f014 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -20,8 +20,6 @@ package org.apache.spark.deploy.rest.mesos import java.io.File import javax.servlet.http.HttpServletResponse -import scala.collection.mutable - import org.apache.spark.deploy.Command import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest._ @@ -29,6 +27,8 @@ import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler import org.apache.spark.util.Utils import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} +import scala.collection.mutable + /** * A server that responds to requests submitted by the [[RestClient]]. @@ -44,9 +44,12 @@ private[spark] class MesosRestServer( val masterConf: SparkConf, scheduler: MesosClusterScheduler) extends RestServer { - val submitRequestServlet = new MesosSubmitRequestServlet(scheduler, masterConf) - val killRequestServlet = new MesosKillRequestServlet(scheduler, masterConf) - val statusRequestServlet = new MesosStatusRequestServlet(scheduler, masterConf) + def submitRequestServlet: SubmitRequestServlet = + new MesosSubmitRequestServlet(scheduler, masterConf) + def killRequestServlet: KillRequestServlet = + new MesosKillRequestServlet(scheduler, masterConf) + def statusRequestServlet: StatusRequestServlet = + new MesosStatusRequestServlet(scheduler, masterConf) } private[mesos] class MesosSubmitRequestServlet( @@ -56,7 +59,7 @@ private[mesos] class MesosSubmitRequestServlet( private val DEFAULT_SUPERVISE = false private val DEFAULT_MEMORY = 512 // mb - private val DEFAULT_CORES = 1 + private val DEFAULT_CORES = 1.0 /** * Build a driver description from the fields specified in the submit request. @@ -113,11 +116,11 @@ private[mesos] class MesosSubmitRequestServlet( mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) - val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES) + val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) new MesosDriverDescription( appResource, actualDriverMemory, actualDriverCores, - actualSuperviseDriver, command, schedulerProperties) + actualSuperviseDriver, command, schedulerProperties.toMap) } protected override def handleSubmit( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala index fc312c8f27751..0428bca91ce2c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala @@ -26,6 +26,7 @@ import org.apache.spark.deploy.mesos.MesosDriverDescription * This queue automatically stores the state after each pop/push * so it can be recovered later. * This queue is also bounded and rejects offers when it's full. + * This class is not thread-safe, and we expect the caller to handle synchronizing state. * @param state Mesos state abstraction to fetch persistent state. */ private[mesos] class DriverQueue(state: MesosClusterPersistenceEngine, capacity: Int) { @@ -42,7 +43,7 @@ private[mesos] class DriverQueue(state: MesosClusterPersistenceEngine, capacity: count = queue.size } - def isFull = count >= capacity + def isFull: Boolean = count >= capacity def size: Int = count diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala index 170c41e71becd..381f4a4d97da9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala @@ -22,6 +22,7 @@ import org.apache.mesos.Protos.SlaveID /** * Tracks all the launched or running drivers in the Mesos cluster scheduler. + * This class is not thread-safe, and we expect the caller to handle synchronizing state. * @param state Persistence engine to store tasks state. */ private[mesos] class LaunchedDrivers(state: MesosClusterPersistenceEngine) { @@ -51,19 +52,14 @@ private[mesos] class LaunchedDrivers(state: MesosClusterPersistenceEngine) { def contains(submissionId: String): Boolean = drivers.contains(submissionId) def remove(submissionId: String): Option[MesosClusterTaskState] = { - if (pendingRecover.contains(submissionId)) { - pendingRecover.remove(submissionId) - } - + pendingRecover.remove(submissionId) val removedState = drivers.remove(submissionId) state.expunge(submissionId) removedState } def set(submissionId: String, newState: MesosClusterTaskState): Unit = { - if (pendingRecover.contains(newState.taskId.getValue)) { - pendingRecover.remove(newState.taskId.getValue) - } + pendingRecover.remove(newState.taskId.getValue) drivers(submissionId) = newState state.persist(submissionId, newState) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index b58dbad9af836..187999bdf47da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -329,20 +329,34 @@ private[spark] class MesosClusterSchedulerDriver( options } - private class ResourceOffer(val offer: Offer, var cpu: Double, var mem: Double) + private class ResourceOffer(val offer: Offer, var cpu: Double, var mem: Double) { + override def toString(): String = { + s"Offer id: ${offer.getId.getValue}, cpu: $cpu, mem: $mem" + } + } override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { + def printOffers(offers: Iterable[ResourceOffer]): String = { + val builder = new StringBuilder() + offers.foreach { o => + builder.append(o).append("\n") + } + builder.toString() + } + val currentOffers = offers.map { o => new ResourceOffer( o, getResource(o.getResourcesList, "cpus"), getResource(o.getResourcesList, "mem")) } + logTrace(s"Received offers from Mesos: ${printOffers(currentOffers)}") val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() val currentTime = new Date() + def scheduleTasks( taskFunc: () => (Option[MesosDriverDescription], Option[RetryState]), - scheduledCallback: (String) => Unit) { + scheduledCallback: (String) => Unit): Unit = { var nextItem = taskFunc() // TODO: We should not stop scheduling at the very first task // that cannot be scheduled. Instead we should exhaust the @@ -352,11 +366,14 @@ private[spark] class MesosClusterSchedulerDriver( val (submission, retryState) = (nextItem._1.get, nextItem._2) val driverCpu = submission.cores val driverMem = submission.mem + logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem") val offerOption = currentOffers.find { o => o.cpu >= driverCpu && o.mem >= driverMem } if (offerOption.isEmpty) { + logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId.get}," + + s"cpu: $driverCpu, mem: $driverMem") return } @@ -371,9 +388,10 @@ private[spark] class MesosClusterSchedulerDriver( .setName("mem").setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() val commandInfo = buildCommand(submission) + val appName = submission.schedulerProperties("spark.app.name") val taskInfo = TaskInfo.newBuilder() .setTaskId(taskId) - .setName(s"driver for ${submission.command.mainClass}") + .setName(s"Driver for $appName") .setSlaveId(offer.offer.getSlaveId) .setCommand(commandInfo) .addResources(cpuResource) @@ -387,6 +405,9 @@ private[spark] class MesosClusterSchedulerDriver( tasks(offer.offer.getId) } queuedTasks += taskInfo + logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + + submission.submissionId.get) + launchedDrivers.set( submission.submissionId.get, new MesosClusterTaskState(submission, taskId, offer.offer.getSlaveId, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala index fc6601d3692fb..3438bda2dda9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala @@ -21,9 +21,9 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source -private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterSchedulerDriver) extends Source { +private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterSchedulerDriver) + extends Source { override def sourceName: String = "mesos_cluster" - override def metricRegistry: MetricRegistry = new MetricRegistry() metricRegistry.register(MetricRegistry.name("waitingDrivers"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala index 7d2cdcf6bb9de..d67674ab062a2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala @@ -26,6 +26,7 @@ import scala.collection.mutable.ArrayBuffer /** * Tracks the retry state of a driver, which includes the next time it should be scheduled * and necessary information to do exponential backoff. + * This class is not thread-safe, and we expect the caller to handle synchronizing state. * @param submission Driver submission. * @param lastFailureStatus Last Task status when it failed. * @param retries Number of times it has retried. @@ -38,7 +39,8 @@ private[spark] case class RetryState( retries: Int, nextRetry: Date, waitTime: Int) extends Serializable { - def copy() = new RetryState(submission, lastFailureStatus, retries, nextRetry, waitTime) + def copy(): RetryState = + new RetryState(submission, lastFailureStatus, retries, nextRetry, waitTime) } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index b58fe426c705b..5eb413e08c08c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -41,7 +41,7 @@ import org.apache.spark.deploy.master.DriverState._ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { private val client = new RestClient private var actorSystem: Option[ActorSystem] = None - private var server: Option[StandaloneRestServer] = None + private var server: Option[RestServer] = None override def afterEach() { actorSystem.foreach(_.shutdown()) @@ -400,9 +400,9 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val fakeMasterRef = _actorSystem.actorOf(Props(makeFakeMaster)) val _server = if (faulty) { - new FaultyStandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf) + new FaultyStandaloneRestServer(localhost, 0, conf, fakeMasterRef, "spark://fake:7077") } else { - new StandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf) + new StandaloneRestServer(localhost, 0, conf, fakeMasterRef, "spark://fake:7077") } val port = _server.start() // set these to clean them up after every test @@ -561,19 +561,15 @@ private class SmarterMaster extends Actor { * The purpose of this class is to test that client handles these cases gracefully. */ private class FaultyStandaloneRestServer( - host: String, - requestedPort: Int, + val host: String, + val requestedPort: Int, + val masterConf: SparkConf, masterActor: ActorRef, - masterUrl: String, - masterConf: SparkConf) - extends StandaloneRestServer(host, requestedPort, masterActor, masterUrl, masterConf) { - - protected override val contextToServlet = Map[String, RestServlet]( - s"$baseContext/create/*" -> new MalformedSubmitServlet, - s"$baseContext/kill/*" -> new InvalidKillServlet, - s"$baseContext/status/*" -> new ExplodingStatusServlet, - "/*" -> new ErrorServlet - ) + masterUrl: String) + extends RestServer { + def submitRequestServlet: SubmitRequestServlet = new MalformedSubmitServlet + def killRequestServlet: KillRequestServlet = new InvalidKillServlet + def statusRequestServlet: StatusRequestServlet = new ExplodingStatusServlet /** A faulty servlet that produces malformed responses. */ class MalformedSubmitServlet extends StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index 7319f479862b0..44fc4d7f9a3fb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -24,8 +24,6 @@ import org.apache.spark.{LocalSparkContext, SparkConf} import org.scalatest.FunSuite import org.scalatest.mock.MockitoSugar -import scala.collection.mutable - class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar { def createCommand: Command = { new Command( @@ -42,13 +40,12 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo val response = scheduler.submitDriver( new MesosDriverDescription("jar", 1000, 1, true, - createCommand, new mutable.HashMap[String, String]())) + createCommand, Map[String, String]())) assert(response.success) val response2 = scheduler.submitDriver(new MesosDriverDescription( - "jar", 1000, 1, true, createCommand, - new mutable.HashMap[String, String]())) + "jar", 1000, 1, true, createCommand, Map[String, String]())) assert(response2.success) val state = scheduler.getState() @@ -65,9 +62,7 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo scheduler.recoverState val response = scheduler.submitDriver( - new MesosDriverDescription( - "jar", 1000, 1, true, createCommand, - new mutable.HashMap[String, String]())) + new MesosDriverDescription("jar", 1000, 1, true, createCommand, Map[String, String]())) assert(response.success) val killResponse = scheduler.killDriver(response.submissionId) From 17f93a29b58cb5c43eec21187a2ac7ce0d251750 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Sat, 4 Apr 2015 21:52:20 -0700 Subject: [PATCH 28/36] Fix head of line blocking in scheduling drivers. --- .../deploy/mesos/ui/MesosClusterPage.scala | 13 +-- .../rest/SubmitRestProtocolRequest.scala | 2 +- .../scheduler/cluster/mesos/DriverQueue.scala | 23 +--- .../cluster/mesos/MesosClusterScheduler.scala | 102 ++++++++---------- .../cluster/mesos/MesosSchedulerUtils.scala | 11 +- .../cluster/mesos/SuperviseRetryList.scala | 26 +++-- .../mesos/MesosClusterSchedulerSuite.scala | 4 +- 7 files changed, 87 insertions(+), 94 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala index 92c12859dd444..5eba0ec412629 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -58,8 +58,8 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( private def queuedRow(submission: MesosDriverDescription): Seq[Node] = { - {submission.submissionId} - {submission.submissionDate} + {submission.submissionId.get} + {submission.submissionDate.get} {submission.command.mainClass} cpus: {submission.cores}, mem: {submission.mem} @@ -67,9 +67,10 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( private def driverRow(state: MesosClusterTaskState): Seq[Node] = { - {state.submission.submissionId} - {state.submission.submissionDate} + {state.submission.submissionId.get} + {state.submission.submissionDate.get} {state.submission.command.mainClass} + cpus: {state.submission.cores}, mem: {state.submission.mem} {state.startDate} {state.slaveId.getValue} {stateString(state.taskState)} @@ -78,8 +79,8 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( private def retryRow(state: RetryState): Seq[Node] = { - {state.submission.submissionId} - {state.submission.submissionDate} + {state.submission.submissionId.get} + {state.submission.submissionDate.get} {state.submission.command.mainClass} {state.lastFailureStatus} {state.nextRetry} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala index d80abdf15fb34..0d50a768942ed 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -61,7 +61,7 @@ private[rest] class CreateSubmissionRequest extends SubmitRestProtocolRequest { assertProperty[Boolean](key, "boolean", _.toBoolean) private def assertPropertyIsNumeric(key: String): Unit = - assertProperty[Int](key, "numeric", _.toInt) + assertProperty[Double](key, "numeric", _.toDouble) private def assertPropertyIsMemory(key: String): Unit = assertProperty[Int](key, "memory", Utils.memoryStringToMb) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala index 0428bca91ce2c..e308ef3a22e18 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala @@ -62,30 +62,17 @@ private[mesos] class DriverQueue(state: MesosClusterPersistenceEngine, capacity: } def remove(submissionId: String): Boolean = { - val removed = queue.dequeueFirst(d => d.submissionId.equals(submissionId)) - if (removed.isDefined) { - state.expunge(removed.get.submissionId.get) + val removed = queue.dequeueFirst { d => + d.submissionId.map(_.equals(submissionId)).getOrElse(false) } - + if (removed.isDefined) state.expunge(removed.get.submissionId.get) removed.isDefined } - def peek(): Option[MesosDriverDescription] = { - queue.headOption - } - - def poll(): Option[MesosDriverDescription] = { - if (queue.isEmpty) { - None - } else { - val item = queue.dequeue() - state.expunge(item.submissionId.get) - Some(item) - } - } + def drivers: Seq[MesosDriverDescription] = queue // Returns a copy of the queued drivers. - def drivers: Iterable[MesosDriverDescription] = { + def copyDrivers: Iterable[MesosDriverDescription] = { val buffer = new Array[MesosDriverDescription](queue.size) queue.copyToArray(buffer) buffer diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 187999bdf47da..30d22878fb8dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -76,7 +76,7 @@ private[spark] class MesosClusterTaskState( */ private[spark] class MesosClusterSchedulerState( val frameworkId: String, - val masterUrl: String, + val masterUrl: Option[String], val queuedDrivers: Iterable[MesosDriverDescription], val launchedDrivers: Iterable[MesosClusterTaskState], val finishedDrivers: Iterable[MesosClusterTaskState], @@ -135,7 +135,7 @@ private[spark] class MesosClusterSchedulerDriver( // All supervised drivers that are waiting to retry after termination. var superviseRetryList: SuperviseRetryList = _ - private var masterInfo: MasterInfo = _ + private var masterInfo: Option[MasterInfo] = None private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs private def newDriverId(submitDate: Date): String = { @@ -252,7 +252,7 @@ private[spark] class MesosClusterSchedulerDriver( markRegistered() stateLock.synchronized { - this.masterInfo = masterInfo + this.masterInfo = Some(masterInfo) if (!launchedDrivers.pendingRecover.isEmpty) { // Start task reconciliation if we need to recover. val statuses = launchedDrivers.pendingRecover.collect { @@ -350,20 +350,14 @@ private[spark] class MesosClusterSchedulerDriver( getResource(o.getResourcesList, "cpus"), getResource(o.getResourcesList, "mem")) } - logTrace(s"Received offers from Mesos: ${printOffers(currentOffers)}") + logTrace(s"Received offers from Mesos: \n${printOffers(currentOffers)}") val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() val currentTime = new Date() - def scheduleTasks( - taskFunc: () => (Option[MesosDriverDescription], Option[RetryState]), + tasksFunc: () => Seq[(MesosDriverDescription, Option[RetryState])], scheduledCallback: (String) => Unit): Unit = { - var nextItem = taskFunc() - // TODO: We should not stop scheduling at the very first task - // that cannot be scheduled. Instead we should exhaust the - // candidate list and remove drivers that cannot scheduled - // over a configurable period of time. - while (nextItem._1.isDefined) { - val (submission, retryState) = (nextItem._1.get, nextItem._2) + val candidates = tasksFunc() + for ((submission, retryState) <- candidates) { val driverCpu = submission.cores val driverMem = submission.mem logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem") @@ -374,56 +368,54 @@ private[spark] class MesosClusterSchedulerDriver( if (offerOption.isEmpty) { logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId.get}," + s"cpu: $driverCpu, mem: $driverMem") - return - } - - val offer = offerOption.get - offer.cpu -= driverCpu - offer.mem -= driverMem - val taskId = TaskID.newBuilder().setValue(submission.submissionId.get).build() - val cpuResource = Resource.newBuilder() - .setName("cpus").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() - val memResource = Resource.newBuilder() - .setName("mem").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() - val commandInfo = buildCommand(submission) - val appName = submission.schedulerProperties("spark.app.name") - val taskInfo = TaskInfo.newBuilder() - .setTaskId(taskId) - .setName(s"Driver for $appName") - .setSlaveId(offer.offer.getSlaveId) - .setCommand(commandInfo) - .addResources(cpuResource) - .addResources(memResource) - .build - val queuedTasks = if (!tasks.contains(offer.offer.getId)) { - val buffer = new ArrayBuffer[TaskInfo] - tasks(offer.offer.getId) = buffer - buffer } else { - tasks(offer.offer.getId) + val offer = offerOption.get + offer.cpu -= driverCpu + offer.mem -= driverMem + val taskId = TaskID.newBuilder().setValue(submission.submissionId.get).build() + val cpuResource = Resource.newBuilder() + .setName("cpus").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() + val memResource = Resource.newBuilder() + .setName("mem").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() + val commandInfo = buildCommand(submission) + val appName = submission.schedulerProperties("spark.app.name") + val taskInfo = TaskInfo.newBuilder() + .setTaskId(taskId) + .setName(s"Driver for $appName") + .setSlaveId(offer.offer.getSlaveId) + .setCommand(commandInfo) + .addResources(cpuResource) + .addResources(memResource) + .build + val queuedTasks = if (!tasks.contains(offer.offer.getId)) { + val buffer = new ArrayBuffer[TaskInfo] + tasks(offer.offer.getId) = buffer + buffer + } else { + tasks(offer.offer.getId) + } + queuedTasks += taskInfo + logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + + submission.submissionId.get) + + launchedDrivers.set( + submission.submissionId.get, + new MesosClusterTaskState(submission, taskId, offer.offer.getSlaveId, + None, new Date(), retryState)) + scheduledCallback(submission.submissionId.get) } - queuedTasks += taskInfo - logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + - submission.submissionId.get) - - launchedDrivers.set( - submission.submissionId.get, - new MesosClusterTaskState(submission, taskId, offer.offer.getSlaveId, - None, new Date(), retryState)) - scheduledCallback(submission.submissionId.get) - nextItem = taskFunc() } } stateLock.synchronized { scheduleTasks(() => { - superviseRetryList.getNextRetry(currentTime) + superviseRetryList.getNextRetries(currentTime) }, (id: String) => { superviseRetryList.remove(id) }) - scheduleTasks(() => (queue.peek(), None), (_) => queue.poll()) + scheduleTasks(() => queue.drivers.map(d => (d, None)), (id) => queue.remove(id)) } tasks.foreach { case (offerId, tasks) => @@ -439,8 +431,8 @@ private[spark] class MesosClusterSchedulerDriver( stateLock.synchronized { new MesosClusterSchedulerState( frameworkId, - s"http://${masterInfo.getIp}:${masterInfo.getPort}", - queue.drivers, + masterInfo.map(m => s"http://${m.getIp}:${m.getPort}"), + queue.copyDrivers, launchedDrivers.states, finishedDrivers.collect { case s => s.copy() }, superviseRetryList.retries) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 705fb40646ded..3e12da843297d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster.mesos import java.util.List import java.util.concurrent.CountDownLatch -import org.apache.mesos.Protos.{FrameworkInfo, Resource} +import org.apache.mesos.Protos.{Status, FrameworkInfo, Resource} import org.apache.mesos.{MesosSchedulerDriver, Scheduler, SchedulerDriver} import org.apache.spark.Logging @@ -64,6 +64,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { try { val ret = driver.run() logInfo("driver.run() returned with code " + ret) + onDriverExit(ret) } catch { case e: Exception => logError("driver.run() failed", e) } @@ -74,6 +75,14 @@ private[mesos] trait MesosSchedulerUtils extends Logging { } } + def onDriverExit(status: Status): Unit = { + // Exit the process when the Mesos framework driver was aborted. + // This behavior can be overriden by the scheduler. + if (status.equals(Status.DRIVER_ABORTED)) { + System.exit(1) + } + } + /** * Waits for the scheduler to be registered, which the scheduler will signal by calling * markRegistered(). diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala index d67674ab062a2..66a0443e6825d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala @@ -59,20 +59,22 @@ private[mesos] class SuperviseRetryList(state: MesosClusterPersistenceEngine) { def size: Int = drivers.size - def contains(submissionId: String): Boolean = - drivers.exists(d => d.submission.submissionId.equals(submissionId)) - - def getNextRetry(currentTime: Date): (Option[MesosDriverDescription], Option[RetryState]) = { - val retry = drivers.find(d => d.nextRetry.before(currentTime)) - if (retry.isDefined) { - (Some(retry.get.submission), retry) - } else { - (None, None) + def contains(submissionId: String): Boolean = { + drivers.exists { d => + d.submission.submissionId.map(_.equals(submissionId)).getOrElse(false) + } + } + + def getNextRetries(currentTime: Date): Seq[(MesosDriverDescription, Option[RetryState])] = { + drivers.filter(d => d.nextRetry.before(currentTime)).map { d => + (d.submission, Some(d)) } } def get(submissionId: String): Option[RetryState] = { - drivers.find(d => d.submission.submissionId.equals(submissionId)) + drivers.find { d => + d.submission.submissionId.map(_.equals(submissionId)).getOrElse(false) + } } def retries: Iterable[RetryState] = { @@ -80,7 +82,9 @@ private[mesos] class SuperviseRetryList(state: MesosClusterPersistenceEngine) { } def remove(submissionId: String): Boolean = { - val index = drivers.indexWhere(s => s.submission.submissionId.equals(submissionId)) + val index = drivers.indexWhere { s => + s.submission.submissionId.map(_.equals(submissionId)).getOrElse(false) + } if (index != -1) { drivers.remove(index) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index 44fc4d7f9a3fb..2c5dab64aa93b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -49,8 +49,8 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo assert(response2.success) val state = scheduler.getState() - assert(state.queuedDrivers.exists(d => d.submissionId == response.submissionId)) - assert(state.queuedDrivers.exists(d => d.submissionId == response2.submissionId)) + assert(state.queuedDrivers.exists(d => d.submissionId.get == response.submissionId)) + assert(state.queuedDrivers.exists(d => d.submissionId.get == response2.submissionId)) } test("can kill queued drivers") { From f7d80462ebc5787e9794ee7f593dbaaa81fd4b96 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Sun, 5 Apr 2015 21:45:41 -0700 Subject: [PATCH 29/36] Change app name to spark cluster. --- .../org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 50d5acab6e1f3..f2b5ba2971943 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -103,7 +103,7 @@ private[mesos] object MesosClusterDispatcher extends spark.Logging { val dispatcherArgs = new ClusterDispatcherArguments(args, conf) conf.setMaster(dispatcherArgs.masterUrl) - conf.setAppName("Mesos Cluster Dispatcher") + conf.setAppName("Spark Cluster") dispatcherArgs.zookeeperUrl.foreach { z => conf.set("spark.deploy.recoveryMode", "ZOOKEEPER") From c6c6b737ff987c8a8bca048eba32b42e2d2a9eec Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Mon, 6 Apr 2015 13:39:31 -0700 Subject: [PATCH 30/36] Pass spark properties to mesos cluster tasks. --- .../deploy/mesos/MesosDriverDescription.scala | 1 + .../deploy/rest/mesos/MesosRestServer.scala | 21 ++----------------- .../cluster/mesos/MesosClusterScheduler.scala | 10 ++++++--- .../mesos/MesosClusterSchedulerSuite.scala | 11 ++++------ 4 files changed, 14 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala index 678597b20c0df..de3bc7c7cd4c2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala @@ -33,6 +33,7 @@ import org.apache.spark.deploy.Command * @param schedulerProperties Extra properties to pass the Mesos scheduler */ private[spark] case class MesosDriverDescription( + val name: String, val jarUrl: String, val mem: Int, val cores: Double, diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index e0d89ac09f014..608dfe38ab38c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -27,8 +27,6 @@ import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler import org.apache.spark.util.Utils import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} -import scala.collection.mutable - /** * A server that responds to requests submitted by the [[RestClient]]. @@ -87,22 +85,6 @@ private[mesos] class MesosSubmitRequestServlet( val driverCores = sparkProperties.get("spark.driver.cores") val appArgs = request.appArgs val environmentVariables = request.environmentVariables - val schedulerProperties = new mutable.HashMap[String, String] - // Store Spark submit specific arguments here to pass to the scheduler. - schedulerProperties("spark.app.name") = sparkProperties.getOrElse("spark.app.name", mainClass) - - sparkProperties.get("spark.executor.memory").foreach { v => - schedulerProperties("spark.executor.memory") = v - } - sparkProperties.get("spark.cores.max").foreach { v => - schedulerProperties("spark.cores.max") = v - } - sparkProperties.get("spark.executor.uri").foreach { v => - schedulerProperties("spark.executor.uri") = v - } - sparkProperties.get("spark.mesos.executor.home").foreach { v => - schedulerProperties("spark.mesos.executor.home") = v - } // Construct driver description val conf = new SparkConf(false) @@ -119,8 +101,9 @@ private[mesos] class MesosSubmitRequestServlet( val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) new MesosDriverDescription( + request.sparkProperties.get("spark.app.name").getOrElse(mainClass), appResource, actualDriverMemory, actualDriverCores, - actualSuperviseDriver, command, schedulerProperties.toMap) + actualSuperviseDriver, command, request.sparkProperties) } protected override def handleSubmit( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 30d22878fb8dc..f8b6228e38a86 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -287,16 +287,19 @@ private[spark] class MesosClusterSchedulerDriver( desc.command.environment.foreach { case (k, v) => envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v).build()) } - builder.setEnvironment(envBuilder.build()) + val executorOpts = desc.schedulerProperties.map { case (k, v) => s"-D$k=$v"}.mkString(" ") + // Pass all spark properties to executor. + envBuilder.addVariables( + Variable.newBuilder().setName("SPARK_EXECUTOR_OPTS").setValue(executorOpts)) val cmdOptions = generateCmdOption(desc) val executorUri = desc.schedulerProperties.get("spark.executor.uri") .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) + val appArguments = desc.command.arguments.mkString(" ") val cmd = if (executorUri.isDefined) { builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) val folderBasename = executorUri.get.split('/').last.split('.').head val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit" val cmdJar = s"../${desc.jarUrl.split("/").last}" - val appArguments = desc.command.arguments.mkString(" ") s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar $appArguments" } else { val executorSparkHome = desc.schedulerProperties.get("spark.mesos.executor.home") @@ -307,9 +310,10 @@ private[spark] class MesosClusterSchedulerDriver( } val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getCanonicalPath val cmdJar = desc.jarUrl.split("/").last - s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar" + s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar $appArguments" } builder.setValue(cmd) + builder.setEnvironment(envBuilder.build()) builder.build } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index 2c5dab64aa93b..9aa3661973f95 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -39,15 +39,13 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo scheduler.recoverState val response = scheduler.submitDriver( - new MesosDriverDescription("jar", 1000, 1, true, + new MesosDriverDescription("d1", "jar", 1000, 1, true, createCommand, Map[String, String]())) assert(response.success) - val response2 = scheduler.submitDriver(new MesosDriverDescription( - "jar", 1000, 1, true, createCommand, Map[String, String]())) + "d1", "jar", 1000, 1, true, createCommand, Map[String, String]())) assert(response2.success) - val state = scheduler.getState() assert(state.queuedDrivers.exists(d => d.submissionId.get == response.submissionId)) assert(state.queuedDrivers.exists(d => d.submissionId.get == response2.submissionId)) @@ -62,12 +60,11 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo scheduler.recoverState val response = scheduler.submitDriver( - new MesosDriverDescription("jar", 1000, 1, true, createCommand, Map[String, String]())) + new MesosDriverDescription("d1", "jar", 1000, 1, true, + createCommand, Map[String, String]())) assert(response.success) - val killResponse = scheduler.killDriver(response.submissionId) assert(killResponse.success) - val state = scheduler.getState() assert(state.queuedDrivers.isEmpty) } From 155323031033bf14ea5bc14484e0411aa2ae8df9 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Sat, 11 Apr 2015 14:10:12 -0700 Subject: [PATCH 31/36] Address review comments. --- .../spark/deploy/DriverDescription.scala | 2 +- .../spark/deploy/FaultToleranceTest.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 4 +- .../spark/deploy/master/DriverState.scala | 2 +- .../deploy/mesos/MesosClusterDispatcher.scala | 121 +---- .../MesosClusterDispatcherArguments.scala | 101 +++++ .../deploy/mesos/MesosDriverDescription.scala | 26 +- .../deploy/mesos/ui/MesosClusterPage.scala | 33 +- .../deploy/mesos/ui/MesosClusterUI.scala | 2 +- ...lient.scala => RestSubmissionClient.scala} | 33 +- ...erver.scala => RestSubmissionServer.scala} | 37 +- .../deploy/rest/StandaloneRestServer.scala | 19 +- .../deploy/rest/mesos/MesosRestServer.scala | 53 ++- .../mesos/CoarseMesosSchedulerBackend.scala | 19 +- .../scheduler/cluster/mesos/DriverQueue.scala | 81 ---- .../cluster/mesos/LaunchedDrivers.scala | 69 --- .../cluster/mesos/MesosClusterScheduler.scala | 412 ++++++++++-------- .../mesos/MesosClusterSchedulerSource.scala | 6 +- .../cluster/mesos/MesosSchedulerUtils.scala | 51 +-- .../cluster/mesos/SuperviseRetryList.scala | 101 ----- .../rest/StandaloneRestSubmitSuite.scala | 34 +- .../mesos/MesosClusterSchedulerSuite.scala | 40 +- 22 files changed, 548 insertions(+), 700 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala rename core/src/main/scala/org/apache/spark/deploy/rest/{RestClient.scala => RestSubmissionClient.scala} (94%) rename core/src/main/scala/org/apache/spark/deploy/rest/{RestServer.scala => RestSubmissionServer.scala} (91%) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala index 458286f139b6f..659fb434a80f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -private[spark] case class DriverDescription( +private[deploy] class DriverDescription( val jarUrl: String, val mem: Int, val cores: Int, 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 82df15bedd44d..c048b78910f38 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -32,7 +32,7 @@ import org.json4s._ import org.json4s.jackson.JsonMethods import org.apache.spark.{Logging, SparkConf, SparkContext} -import org.apache.spark.deploy.master.{RecoveryState} +import org.apache.spark.deploy.master.RecoveryState import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index bf04e88faf7e3..585937afd0016 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -117,7 +117,7 @@ object SparkSubmit { * Kill an existing submission using the REST protocol. Standalone and Mesos cluster mode only. */ private def kill(args: SparkSubmitArguments): Unit = { - new RestClient() + new RestSubmissionClient() .killSubmission(args.master, args.submissionToKill) } @@ -126,7 +126,7 @@ object SparkSubmit { * Standalone and Mesos cluster mode only. */ private def requestStatus(args: SparkSubmitArguments): Unit = { - new RestClient() + new RestSubmissionClient() .requestSubmissionStatus(args.master, args.submissionToRequestStatusFor) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala index 26a68bade3c60..35ff33a61653c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object DriverState extends Enumeration { +private[deploy] object DriverState extends Enumeration { type DriverState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index f2b5ba2971943..b608e14fbaa62 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -17,25 +17,24 @@ package org.apache.spark.deploy.mesos -import java.io.File import java.util.concurrent.CountDownLatch import org.apache.spark -import org.apache.spark.deploy.mesos.MesosClusterDispatcher.ClusterDispatcherArguments import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.mesos.MesosRestServer import org.apache.spark.scheduler.cluster.mesos._ -import org.apache.spark.util.{IntParam, SignalLogger, Utils} +import org.apache.spark.util.SignalLogger import org.apache.spark.{Logging, SecurityManager, SparkConf} /* - * A dispatcher that is responsible for managing and launching drivers, and is intended to - * be used for Mesos cluster mode. The dispatcher is launched by the user in the cluster, - * which it launches a [[MesosRestServer]] for listening for driver requests, and launches a - * [[MesosClusterScheduler]] to launch these drivers in the Mesos cluster. + * A dispatcher that is responsible for managing and launching drivers, and is intended to be + * used for Mesos cluster mode. The dispatcher is a long-running process started by the user in + * the cluster independently of Spark applications. + * It contains a [[MesosRestServer]] that listens for requests to submit drivers and a + * [[MesosClusterScheduler]] that processes these requests by negotiating with the Mesos master + * for resources. * * A typical new driver lifecycle is the following: - * * - Driver submitted via spark-submit talking to the [[MesosRestServer]] * - [[MesosRestServer]] queues the driver request to [[MesosClusterScheduler]] * - [[MesosClusterScheduler]] gets resource offers and launches the drivers that are in queue @@ -43,19 +42,20 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} * This dispatcher supports both Mesos fine-grain or coarse-grain mode as the mode is configurable * per driver launched. * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as - * a daemon to launch drivers as Mesos frameworks upon request. + * a daemon to launch drivers as Mesos frameworks upon request. The dispatcher is also started and + * stopped by sbin/start-mesos-dispatcher and sbin/stop-mesos-dispatcher respectively. */ private[mesos] class MesosClusterDispatcher( - args: ClusterDispatcherArguments, + args: MesosClusterDispatcherArguments, conf: SparkConf) extends Logging { - private def publicAddress(conf: SparkConf, host: String): String = { + private def publicAddress(conf: SparkConf, defaultAddress: String): String = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") - if (envVar != null) envVar else host + if (envVar != null) envVar else defaultAddress } - private val recoveryMode = conf.get("spark.deploy.recoveryMode", "NONE").toUpperCase() + private val recoveryMode = conf.get("spark.mesos.deploy.recoveryMode", "NONE").toUpperCase() logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode) private val engineFactory = recoveryMode match { @@ -63,7 +63,7 @@ private[mesos] class MesosClusterDispatcher( case "ZOOKEEPER" => new ZookeeperMesosClusterPersistenceEngineFactory(conf) } - private val scheduler = new MesosClusterSchedulerDriver(engineFactory, conf) + private val scheduler = new MesosClusterScheduler(engineFactory, conf) private val server = new MesosRestServer(args.host, args.port, conf, scheduler) private val webUi = new MesosClusterUI( @@ -74,7 +74,6 @@ private[mesos] class MesosClusterDispatcher( scheduler) private val shutdownLatch = new CountDownLatch(1) - private val sparkHome = new File(Option(conf.getenv("SPARK_HOME")).getOrElse(".")) def start(): Unit = { webUi.bind() @@ -98,24 +97,18 @@ private[mesos] class MesosClusterDispatcher( private[mesos] object MesosClusterDispatcher extends spark.Logging { def main(args: Array[String]) { SignalLogger.register(log) - val conf = new SparkConf - val dispatcherArgs = new ClusterDispatcherArguments(args, conf) - + val dispatcherArgs = new MesosClusterDispatcherArguments(args, conf) conf.setMaster(dispatcherArgs.masterUrl) - conf.setAppName("Spark Cluster") - + conf.setAppName(dispatcherArgs.name) dispatcherArgs.zookeeperUrl.foreach { z => - conf.set("spark.deploy.recoveryMode", "ZOOKEEPER") - conf.set("spark.deploy.zookeeper.url", z) + conf.set("spark.mesos.deploy.recoveryMode", "ZOOKEEPER") + conf.set("spark.mesos.deploy.zookeeper.url", z) } - val dispatcher = new MesosClusterDispatcher( dispatcherArgs, conf) - dispatcher.start() - val shutdownHook = new Thread() { override def run() { logInfo("Shutdown hook is shutting down dispatcher") @@ -123,85 +116,7 @@ private[mesos] object MesosClusterDispatcher extends spark.Logging { dispatcher.awaitShutdown() } } - Runtime.getRuntime.addShutdownHook(shutdownHook) - dispatcher.awaitShutdown() } - - private class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) { - var host = Utils.localHostName() - var port = 7077 - var webUiPort = 8081 - var masterUrl: String = _ - var zookeeperUrl: Option[String] = None - var propertiesFile: String = _ - - parse(args.toList) - - propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - - private def parse(args: List[String]): Unit = args match { - case ("--host" | "-h") :: value :: tail => - Utils.checkHost(value, "Please use hostname " + value) - host = value - parse(tail) - - case ("--port" | "-p") :: IntParam(value) :: tail => - port = value - parse(tail) - - case ("--webui-port" | "-p") :: IntParam(value) :: tail => - webUiPort = value - parse(tail) - - case ("--zk" | "-z") :: value :: tail => - zookeeperUrl = Some(value) - parse(tail) - - case ("--master" | "-m") :: value :: tail => - if (!value.startsWith("mesos://")) { - System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") - System.exit(1) - } - masterUrl = value.stripPrefix("mesos://") - parse(tail) - - case ("--properties-file") :: value :: tail => - propertiesFile = value - parse(tail) - - case ("--help") :: tail => - printUsageAndExit(0) - - case Nil => { - if (masterUrl == null) { - System.err.println("--master is required") - System.exit(1) - } - } - - case _ => - printUsageAndExit(1) - } - - /** - * Print usage and exit JVM with the given exit code. - */ - def printUsageAndExit(exitCode: Int): Unit = { - System.err.println( - "Usage: MesosClusterDispatcher [options]\n" + - "\n" + - "Options:\n" + - " -h HOST, --host HOST Hostname to listen on\n" + - " -p PORT, --port PORT Port to listen on (default: 7077)\n" + - " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + - " -m --master MASTER URI for connecting to Mesos master\n" + - " -z --zk ZOOKEEPER Comma delimited URLs for connecting to \n" + - " Zookeeper for persistence\n" + - " --properties-file FILE Path to a custom Spark properties file.\n" + - " Default is conf/spark-defaults.conf.") - System.exit(exitCode) - } - } } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala new file mode 100644 index 0000000000000..b24b88c2b3d6d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala @@ -0,0 +1,101 @@ +/* + * 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.mesos + +import org.apache.spark.SparkConf +import org.apache.spark.util.{IntParam, Utils} + + +private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: SparkConf) { + var host = Utils.localHostName() + var port = 7077 + var name = "Spark Cluster" + var webUiPort = 8081 + var masterUrl: String = _ + var zookeeperUrl: Option[String] = None + var propertiesFile: String = _ + + parse(args.toList) + + propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) + + private def parse(args: List[String]): Unit = args match { + case ("--host" | "-h") :: value :: tail => + Utils.checkHost(value, "Please use hostname " + value) + host = value + parse(tail) + + case ("--port" | "-p") :: IntParam(value) :: tail => + port = value + parse(tail) + + case ("--webui-port" | "-p") :: IntParam(value) :: tail => + webUiPort = value + parse(tail) + + case ("--zk" | "-z") :: value :: tail => + zookeeperUrl = Some(value) + parse(tail) + + case ("--master" | "-m") :: value :: tail => + if (!value.startsWith("mesos://")) { + System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") + System.exit(1) + } + masterUrl = value.stripPrefix("mesos://") + parse(tail) + + case ("--name") :: value :: tail => + name = value + parse(tail) + + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + + case ("--help") :: tail => + printUsageAndExit(0) + + case Nil => { + if (masterUrl == null) { + System.err.println("--master is required") + System.exit(1) + } + } + + case _ => + printUsageAndExit(1) + } + + private def printUsageAndExit(exitCode: Int): Unit = { + System.err.println( + "Usage: MesosClusterDispatcher [options]\n" + + "\n" + + "Options:\n" + + " -h HOST, --host HOST Hostname to listen on\n" + + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + + " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + + " --name NAME Framework name to show in Mesos UI\n" + + " -m --master MASTER URI for connecting to Mesos master\n" + + " -z --zk ZOOKEEPER Comma delimited URLs for connecting to \n" + + " Zookeeper for persistence\n" + + " --properties-file FILE Path to a custom Spark properties file.\n" + + " Default is conf/spark-defaults.conf.") + System.exit(exitCode) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala index de3bc7c7cd4c2..7ba6be2ec52e1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.mesos import java.util.Date import org.apache.spark.deploy.Command +import org.apache.spark.scheduler.cluster.mesos.RetryState /** * Describes a Spark driver that is submitted from the @@ -32,18 +33,31 @@ import org.apache.spark.deploy.Command * @param command The command to launch the driver. * @param schedulerProperties Extra properties to pass the Mesos scheduler */ -private[spark] case class MesosDriverDescription( +private[spark] class MesosDriverDescription( val name: String, val jarUrl: String, val mem: Int, val cores: Double, val supervise: Boolean, val command: Command, - val schedulerProperties: Map[String, String]) + val schedulerProperties: Map[String, String], + val submissionId: String, + val submissionDate: Date, + val retryState: Option[RetryState] = None) extends Serializable { - - var submissionId: Option[String] = None - var submissionDate: Option[Date] = None - + def copy( + name: String = name, + jarUrl: String = jarUrl, + mem: Int = mem, + cores: Double = cores, + supervise: Boolean = supervise, + command: Command = command, + schedulerProperties: Map[String, String] = schedulerProperties, + retryState: Option[RetryState] = retryState, + submissionId: String = submissionId, + submissionDate: Date = submissionDate): MesosDriverDescription = { + new MesosDriverDescription(name, jarUrl, mem, cores, supervise, command, schedulerProperties, + submissionId, submissionDate, retryState) + } override def toString: String = s"MesosDriverDescription (${command.mainClass})" } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala index 5eba0ec412629..fb7ce892c20c8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -19,21 +19,20 @@ package org.apache.spark.deploy.mesos.ui import javax.servlet.http.HttpServletRequest -import scala.xml.Node - import org.apache.mesos.Protos.TaskStatus - import org.apache.spark.deploy.mesos.MesosDriverDescription -import org.apache.spark.scheduler.cluster.mesos.{RetryState, MesosClusterTaskState} +import org.apache.spark.scheduler.cluster.mesos.MesosClusterTaskState import org.apache.spark.ui.{UIUtils, WebUIPage} +import scala.xml.Node + private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val state = parent.scheduler.getState() - val queuedHeaders = Seq("DriverID", "Submit Date", "Main Class", "Driver resources") + val queuedHeaders = Seq("Driver ID", "Submit Date", "Main Class", "Driver Resources") val driverHeaders = queuedHeaders ++ Seq("Start Date", "Mesos Slave ID", "State") - val retryHeaders = Seq("DriverID", "Submit Date", "Description") ++ + val retryHeaders = Seq("Driver ID", "Submit Date", "Description") ++ Seq("Last Failed Status", "Next Retry Time", "Attempt Count") val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) @@ -58,8 +57,8 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( private def queuedRow(submission: MesosDriverDescription): Seq[Node] = { - {submission.submissionId.get} - {submission.submissionDate.get} + {submission.submissionId} + {submission.submissionDate} {submission.command.mainClass} cpus: {submission.cores}, mem: {submission.mem} @@ -67,8 +66,8 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( private def driverRow(state: MesosClusterTaskState): Seq[Node] = { - {state.submission.submissionId.get} - {state.submission.submissionDate.get} + {state.submission.submissionId} + {state.submission.submissionDate} {state.submission.command.mainClass} cpus: {state.submission.cores}, mem: {state.submission.mem} {state.startDate} @@ -77,14 +76,14 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( } - private def retryRow(state: RetryState): Seq[Node] = { + private def retryRow(submission: MesosDriverDescription): Seq[Node] = { - {state.submission.submissionId.get} - {state.submission.submissionDate.get} - {state.submission.command.mainClass} - {state.lastFailureStatus} - {state.nextRetry} - {state.retries} + {submission.submissionId} + {submission.submissionDate} + {submission.command.mainClass} + {submission.retryState.get.lastFailureStatus} + {submission.retryState.get.nextRetry} + {submission.retryState.get.retries} } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala index 24c64e4be1f00..7dea5304ce5c3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala @@ -17,8 +17,8 @@ package org.apache.spark.deploy.mesos.ui -import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.{SparkUI, WebUI} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/deploy/rest/RestClient.scala rename to core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 8b9239c3b093f..020d9c55dd918 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -30,11 +30,7 @@ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} import org.apache.spark.util.Utils /** - * A client that submits applications to the standalone Master and Mesos dispatcher - * using a REST protocol. - * This client is intended to communicate with the [[StandaloneRestServer]] or - * [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]] and is - * currently used for cluster mode only. + * A client that submits applications to a [[RestSubmissionServer]]. * * In protocol version v1, the REST URL takes the form http://[host:port]/v1/submissions/[action], * where [action] can be one of create, kill, or status. Each type of request is represented in @@ -55,8 +51,10 @@ import org.apache.spark.util.Utils * implementation of this client can use that information to retry using the version specified * by the server. */ -private[spark] class RestClient extends Logging { - import RestClient._ +private[spark] class RestSubmissionClient extends Logging { + import RestSubmissionClient._ + + private val supportedMasterPrefixes = Seq("spark://", "mesos://") /** * Submit an application specified by the parameters in the provided request. @@ -221,20 +219,23 @@ private[spark] class RestClient extends Logging { /** Return the base URL for communicating with the server, including the protocol version. */ private def getBaseUrl(master: String): String = { - val masterUrl = if (master.startsWith("spark://")) { - master.stripPrefix("spark://").stripSuffix("/") - } else { - master.stripPrefix("mesos://").stripSuffix("/") + var masterUrl = master + supportedMasterPrefixes.foreach { prefix => + if (master.startsWith(prefix)) { + masterUrl = master.stripPrefix(prefix) + } } - + masterUrl = masterUrl.stripSuffix("/") s"http://$masterUrl/$PROTOCOL_VERSION/submissions" } /** Throw an exception if this is not standalone mode. */ private def validateMaster(master: String): Unit = { - if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { + val valid = supportedMasterPrefixes.exists { prefix => master.startsWith(prefix) } + if (!valid) { throw new IllegalArgumentException( - "This REST client is only supported in standalone or Mesos mode.") + "This REST client only supports master URLs that start with " + + "one of the following: " + supportedMasterPrefixes.mkString(",")) } } @@ -303,7 +304,7 @@ private[spark] class RestClient extends Logging { } } -private[spark] object RestClient { +private[spark] object RestSubmissionClient { private val REPORT_DRIVER_STATUS_INTERVAL = 1000 private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" @@ -323,7 +324,7 @@ private[spark] object RestClient { } val sparkProperties = conf.getAll.toMap val environmentVariables = env.filter { case (k, _) => k.startsWith("SPARK_") } - val client = new RestClient + val client = new RestSubmissionClient val submitRequest = client.constructSubmitRequest( appResource, mainClass, appArgs, sparkProperties, environmentVariables) client.createSubmission(master, submitRequest) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala similarity index 91% rename from core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala rename to core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index dbc4ce5e85d4e..800ac1e4aee9a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -32,7 +32,7 @@ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} import org.apache.spark.util.Utils /** - * A server that responds to requests submitted by the [[RestClient]]. + * A server that responds to requests submitted by the [[RestSubmissionClient]]. * * This server responds with different HTTP codes depending on the situation: * 200 OK - Request was processed successfully @@ -45,21 +45,20 @@ import org.apache.spark.util.Utils * instead of the one expected by the client. If the construction of this error response itself * fails, the response will consist of an empty body with a response code that indicates internal * server error. - * */ -private[spark] abstract class RestServer extends Logging { - val host: String - val requestedPort: Int - val masterConf: SparkConf - def submitRequestServlet: SubmitRequestServlet - def killRequestServlet: KillRequestServlet - def statusRequestServlet: StatusRequestServlet +private[spark] abstract class RestSubmissionServer( + val host: String, + val requestedPort: Int, + val masterConf: SparkConf) extends Logging { + protected val submitRequestServlet: SubmitRequestServlet + protected val killRequestServlet: KillRequestServlet + protected val statusRequestServlet: StatusRequestServlet private var _server: Option[Server] = None // A mapping from URL prefixes to servlets that serve them. Exposed for testing. - protected val baseContext = s"/${RestServer.PROTOCOL_VERSION}/submissions" - protected val contextToServlet = Map[String, RestServlet]( + protected val baseContext = s"/${RestSubmissionServer.PROTOCOL_VERSION}/submissions" + protected lazy val contextToServlet = Map[String, RestServlet]( s"$baseContext/create/*" -> submitRequestServlet, s"$baseContext/kill/*" -> killRequestServlet, s"$baseContext/status/*" -> statusRequestServlet, @@ -99,13 +98,13 @@ private[spark] abstract class RestServer extends Logging { } } -private[rest] object RestServer { - val PROTOCOL_VERSION = RestClient.PROTOCOL_VERSION +private[rest] object RestSubmissionServer { + val PROTOCOL_VERSION = RestSubmissionClient.PROTOCOL_VERSION val SC_UNKNOWN_PROTOCOL_VERSION = 468 } /** - * An abstract servlet for handling requests passed to the [[RestServer]]. + * An abstract servlet for handling requests passed to the [[RestSubmissionServer]]. */ private[rest] abstract class RestServlet extends HttpServlet with Logging { @@ -189,7 +188,7 @@ private[rest] abstract class RestServlet extends HttpServlet with Logging { } /** - * A servlet for handling kill requests passed to the [[RestServer]]. + * A servlet for handling kill requests passed to the [[RestSubmissionServer]]. */ private[rest] abstract class KillRequestServlet extends RestServlet { @@ -213,7 +212,7 @@ private[rest] abstract class KillRequestServlet } /** - * A servlet for handling status requests passed to the [[RestServer]]. + * A servlet for handling status requests passed to the [[RestSubmissionServer]]. */ private[rest] abstract class StatusRequestServlet extends RestServlet { @@ -236,7 +235,7 @@ private[rest] abstract class StatusRequestServlet extends RestServlet { } /** - * A servlet for handling submit requests passed to the [[RestServer]]. + * A servlet for handling submit requests passed to the [[RestSubmissionServer]]. */ private[rest] abstract class SubmitRequestServlet extends RestServlet { @@ -277,7 +276,7 @@ private[rest] abstract class SubmitRequestServlet extends RestServlet { * A default servlet that handles error cases that are not captured by other servlets. */ private class ErrorServlet extends RestServlet { - private val serverVersion = RestServer.PROTOCOL_VERSION + private val serverVersion = RestSubmissionServer.PROTOCOL_VERSION /** Service a faulty request by returning an appropriate error message to the client. */ protected override def service( @@ -311,7 +310,7 @@ private class ErrorServlet extends RestServlet { // this server supports in case the client wants to retry with our version if (versionMismatch) { error.highestProtocolVersion = serverVersion - response.setStatus(RestServer.SC_UNKNOWN_PROTOCOL_VERSION) + response.setStatus(RestSubmissionServer.SC_UNKNOWN_PROTOCOL_VERSION) } else { response.setStatus(HttpServletResponse.SC_BAD_REQUEST) } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index f5618ecebb8cd..432067b72f3ec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ /** - * A server that responds to requests submitted by the [[RestClient]]. + * A server that responds to requests submitted by the [[RestSubmissionClient]]. * This is intended to be embedded in the standalone Master and used in cluster mode only. * * This server responds with different HTTP codes depending on the situation: @@ -50,18 +50,15 @@ import org.apache.spark.deploy.ClientArguments._ * @param masterUrl the URL of the Master new drivers will attempt to connect to */ private[deploy] class StandaloneRestServer( - val host: String, - val requestedPort: Int, - val masterConf: SparkConf, + host: String, + requestedPort: Int, + masterConf: SparkConf, masterActor: ActorRef, masterUrl: String) - extends RestServer { - def submitRequestServlet: SubmitRequestServlet = - new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) - def killRequestServlet: KillRequestServlet = - new StandaloneKillRequestServlet(masterActor, masterConf) - def statusRequestServlet: StatusRequestServlet = - new StandaloneStatusRequestServlet(masterActor, masterConf) + extends RestSubmissionServer(host, requestedPort, masterConf) { + val submitRequestServlet = new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) + val killRequestServlet = new StandaloneKillRequestServlet(masterActor, masterConf) + val statusRequestServlet = new StandaloneStatusRequestServlet(masterActor, masterConf) } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index 608dfe38ab38c..6fb84580dec36 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -18,6 +18,9 @@ package org.apache.spark.deploy.rest.mesos import java.io.File +import java.text.SimpleDateFormat +import java.util.Date +import java.util.concurrent.atomic.AtomicLong import javax.servlet.http.HttpServletResponse import org.apache.spark.deploy.Command @@ -29,28 +32,25 @@ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} /** - * A server that responds to requests submitted by the [[RestClient]]. - * This is intended to be used in Mesos cluster mode only, which forwards all requests to - * [[MesosClusterScheduler]]. - * + * A server that responds to requests submitted by the [[RestSubmissionClient]]. + * All requests are forwarded to + * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]]. + * This is intended to be used in Mesos cluster mode only. * For more details about the RestServer Spark protocol and status codes please refer to - * [[RestServer]] javadocs. + * [[RestSubmissionServer]] javadocs. */ private[spark] class MesosRestServer( - val host: String, - val requestedPort: Int, - val masterConf: SparkConf, + host: String, + requestedPort: Int, + masterConf: SparkConf, scheduler: MesosClusterScheduler) - extends RestServer { - def submitRequestServlet: SubmitRequestServlet = - new MesosSubmitRequestServlet(scheduler, masterConf) - def killRequestServlet: KillRequestServlet = - new MesosKillRequestServlet(scheduler, masterConf) - def statusRequestServlet: StatusRequestServlet = - new MesosStatusRequestServlet(scheduler, masterConf) + extends RestSubmissionServer(host, requestedPort, masterConf) { + protected val submitRequestServlet = new MesosSubmitRequestServlet(scheduler, masterConf) + protected val killRequestServlet = new MesosKillRequestServlet(scheduler, masterConf) + protected val statusRequestServlet = new MesosStatusRequestServlet(scheduler, masterConf) } -private[mesos] class MesosSubmitRequestServlet( +private[deploy] class MesosSubmitRequestServlet( scheduler: MesosClusterScheduler, conf: SparkConf) extends SubmitRequestServlet { @@ -59,6 +59,13 @@ private[mesos] class MesosSubmitRequestServlet( private val DEFAULT_MEMORY = 512 // mb private val DEFAULT_CORES = 1.0 + private val nextDriverNumber: AtomicLong = new AtomicLong(0) + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + private def newDriverId(submitDate: Date): String = { + "driver-%s-%04d".format( + createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) + } + /** * Build a driver description from the fields specified in the submit request. * @@ -85,7 +92,7 @@ private[mesos] class MesosSubmitRequestServlet( val driverCores = sparkProperties.get("spark.driver.cores") val appArgs = request.appArgs val environmentVariables = request.environmentVariables - + val name = request.sparkProperties.get("spark.app.name").getOrElse(mainClass) // Construct driver description val conf = new SparkConf(false) .setAll(sparkProperties) @@ -100,10 +107,12 @@ private[mesos] class MesosSubmitRequestServlet( val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) + val submitDate = new Date() + val submissionId = newDriverId(submitDate) + new MesosDriverDescription( - request.sparkProperties.get("spark.app.name").getOrElse(mainClass), - appResource, actualDriverMemory, actualDriverCores, - actualSuperviseDriver, command, request.sparkProperties) + name, appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, + command, request.sparkProperties, submissionId, submitDate) } protected override def handleSubmit( @@ -128,7 +137,7 @@ private[mesos] class MesosSubmitRequestServlet( } } -private[mesos] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) +private[deploy] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) extends KillRequestServlet { protected override def handleKill(submissionId: String): KillSubmissionResponse = { val k = scheduler.killDriver(submissionId) @@ -137,7 +146,7 @@ private[mesos] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, c } } -private[mesos] class MesosStatusRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) +private[deploy] class MesosStatusRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) extends StatusRequestServlet { protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { val d = scheduler.getStatus(submissionId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 055d2e86063f6..c1a8c00416499 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -18,20 +18,17 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File -import java.util.{List => JList} -import java.util.Collections +import java.util.{Collections, List => JList} -import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, HashSet} - -import org.apache.mesos.{Scheduler => MScheduler} -import org.apache.mesos._ -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} - -import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException, TaskState} +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} +import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskState} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, HashSet} /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala deleted file mode 100644 index e308ef3a22e18..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/DriverQueue.scala +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.scheduler.cluster.mesos - -import scala.collection.mutable - -import org.apache.spark.deploy.mesos.MesosDriverDescription - -/** - * A request queue for launching drivers in Mesos cluster mode. - * This queue automatically stores the state after each pop/push - * so it can be recovered later. - * This queue is also bounded and rejects offers when it's full. - * This class is not thread-safe, and we expect the caller to handle synchronizing state. - * @param state Mesos state abstraction to fetch persistent state. - */ -private[mesos] class DriverQueue(state: MesosClusterPersistenceEngine, capacity: Int) { - var queue: mutable.Queue[MesosDriverDescription] = new mutable.Queue[MesosDriverDescription]() - private var count = 0 - - initialize() - - def initialize(): Unit = { - state.fetchAll[MesosDriverDescription]().foreach(d => queue.enqueue(d)) - - // This size might be larger than the passed in capacity, but we allow - // this so we don't lose queued drivers. - count = queue.size - } - - def isFull: Boolean = count >= capacity - - def size: Int = count - - def contains(submissionId: String): Boolean = { - queue.exists(s => s.submissionId.equals(submissionId)) - } - - def offer(submission: MesosDriverDescription): Boolean = { - if (isFull) { - return false - } - - queue.enqueue(submission) - state.persist(submission.submissionId.get, submission) - true - } - - def remove(submissionId: String): Boolean = { - val removed = queue.dequeueFirst { d => - d.submissionId.map(_.equals(submissionId)).getOrElse(false) - } - if (removed.isDefined) state.expunge(removed.get.submissionId.get) - removed.isDefined - } - - def drivers: Seq[MesosDriverDescription] = queue - - // Returns a copy of the queued drivers. - def copyDrivers: Iterable[MesosDriverDescription] = { - val buffer = new Array[MesosDriverDescription](queue.size) - queue.copyToArray(buffer) - buffer - } - -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala deleted file mode 100644 index 381f4a4d97da9..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/LaunchedDrivers.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.scheduler.cluster.mesos - -import scala.collection.mutable -import org.apache.mesos.Protos.SlaveID - -/** - * Tracks all the launched or running drivers in the Mesos cluster scheduler. - * This class is not thread-safe, and we expect the caller to handle synchronizing state. - * @param state Persistence engine to store tasks state. - */ -private[mesos] class LaunchedDrivers(state: MesosClusterPersistenceEngine) { - private val drivers = new mutable.HashMap[String, MesosClusterTaskState] - - // Holds the list of tasks that needs to reconciliation from the master. - // All states that are loaded after failover are added here. - val pendingRecover = new mutable.HashMap[String, SlaveID] - - initialize() - - def initialize(): Unit = { - state.fetchAll[MesosClusterTaskState]().foreach { case state => - drivers(state.taskId.getValue) = state - pendingRecover(state.taskId.getValue) = state.slaveId - } - } - - def size: Int = drivers.size - - def get(submissionId: String): MesosClusterTaskState = drivers(submissionId) - - def states: Iterable[MesosClusterTaskState] = { - drivers.values.map(_.copy()).toList - } - - def contains(submissionId: String): Boolean = drivers.contains(submissionId) - - def remove(submissionId: String): Option[MesosClusterTaskState] = { - pendingRecover.remove(submissionId) - val removedState = drivers.remove(submissionId) - state.expunge(submissionId) - removedState - } - - def set(submissionId: String, newState: MesosClusterTaskState): Unit = { - pendingRecover.remove(newState.taskId.getValue) - drivers(submissionId) = newState - state.persist(submissionId, newState) - } - - def isEmpty: Boolean = drivers.isEmpty - -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index f8b6228e38a86..fe30f94ff2e28 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -18,25 +18,22 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File -import java.text.SimpleDateFormat -import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.locks.ReentrantLock import java.util.{Collections, Date, List => JList} -import scala.collection.JavaConversions._ -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - import org.apache.mesos.Protos.Environment.Variable import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} import org.apache.mesos.{Scheduler, SchedulerDriver} - import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkException, SecurityManager, TaskState} +import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState} + +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer /** @@ -47,23 +44,39 @@ import org.apache.spark.{SparkConf, SparkException, SecurityManager, TaskState} * @param slaveId Slave ID that the task is assigned to * @param taskState The last known task status update. * @param startDate The date the task was launched - * @param retryState Retry state for this task (only applicable to supervised drivers) */ private[spark] class MesosClusterTaskState( val submission: MesosDriverDescription, val taskId: TaskID, val slaveId: SlaveID, var taskState: Option[TaskStatus], - var startDate: Date, - val retryState: Option[RetryState] = None) + var startDate: Date) extends Serializable { def copy(): MesosClusterTaskState = { new MesosClusterTaskState( - submission, taskId, slaveId, taskState, startDate, retryState) + submission, taskId, slaveId, taskState, startDate) } } +/** + * Tracks the retry state of a driver, which includes the next time it should be scheduled + * and necessary information to do exponential backoff. + * This class is not thread-safe, and we expect the caller to handle synchronizing state. + * @param lastFailureStatus Last Task status when it failed. + * @param retries Number of times it has retried. + * @param nextRetry Next retry time to be scheduled. + * @param waitTime The amount of time driver is scheduled to wait until next retry. + */ +private[spark] class RetryState( + val lastFailureStatus: TaskStatus, + val retries: Int, + val nextRetry: Date, + val waitTime: Int) extends Serializable { + def copy(): RetryState = + new RetryState(lastFailureStatus, retries, nextRetry, waitTime) +} + /** * The full state of the cluster scheduler, currently being used for displaying * information on the UI. @@ -80,22 +93,7 @@ private[spark] class MesosClusterSchedulerState( val queuedDrivers: Iterable[MesosDriverDescription], val launchedDrivers: Iterable[MesosClusterTaskState], val finishedDrivers: Iterable[MesosClusterTaskState], - val retryList: Iterable[RetryState]) - -/** - * Mesos cluster scheduler for running, killing and requesting - * status of Spark drivers that will be launched in a Mesos cluster. - * This interface is mainly for [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] and - * [[org.apache.spark.deploy.mesos.ui.MesosClusterPage]] to interact with - * [[MesosClusterSchedulerDriver]] and hide all the Mesos specific methods that it doesn't - * need to care about. - */ -private[spark] trait MesosClusterScheduler { - def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse - def killDriver(submissionId: String): KillSubmissionResponse - def getStatus(submissionId: String): SubmissionStatusResponse - def getState(): MesosClusterSchedulerState -} + val retryList: Iterable[MesosDriverDescription]) /** * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode @@ -106,11 +104,10 @@ private[spark] trait MesosClusterScheduler { * This scheduler supports recovery by persisting all its state and performs task reconciliation * on recover, which gets all the latest state for all the drivers from Mesos master. */ -private[spark] class MesosClusterSchedulerDriver( +private[spark] class MesosClusterScheduler( engineFactory: MesosClusterPersistenceEngineFactory, conf: SparkConf) - extends Scheduler with MesosSchedulerUtils with MesosClusterScheduler { - + extends Scheduler with MesosSchedulerUtils { var frameworkUrl: String = _ private val metricsSystem = @@ -120,49 +117,86 @@ private[spark] class MesosClusterSchedulerDriver( private val queuedCapacity = conf.getInt("spark.deploy.mesos.queuedDrivers", 200) private val retainedDrivers = conf.getInt("spark.deploy.retainedDrivers", 200) private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute - private val state = engineFactory.createEngine("scheduler") + private val schedulerState = engineFactory.createEngine("scheduler") private val stateLock = new ReentrantLock() private val finishedDrivers = new mutable.ArrayBuffer[MesosClusterTaskState](retainedDrivers) - private val nextDriverNumber: AtomicLong = new AtomicLong(0) private var frameworkId: String = null - - // Stores all the launched and running drivers' states. - var launchedDrivers: LaunchedDrivers = _ - + val launchedDrivers = new mutable.HashMap[String, MesosClusterTaskState]() + // Holds the list of tasks that needs to reconcile with Mesos master. + // All states that are loaded after failover are added here. + private val pendingRecover = new mutable.HashMap[String, SlaveID]() // A queue that stores all the submitted drivers that hasn't been launched. - var queue: DriverQueue = _ - + val queuedDrivers = new ArrayBuffer[MesosDriverDescription]() // All supervised drivers that are waiting to retry after termination. - var superviseRetryList: SuperviseRetryList = _ - + val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]() + private val queuedDriversState = engineFactory.createEngine("driverQueue") + private val launchedDriversState = engineFactory.createEngine("launchedDrivers") + private val pendingRetryDriversState = engineFactory.createEngine("retryList") + // Flag to mark if the scheduler is ready to be called, which is until the scheduler + // is registered with Mesos master. + protected var ready = false private var masterInfo: Option[MasterInfo] = None - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - private def newDriverId(submitDate: Date): String = { - "driver-%s-%04d".format( - createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) - } + private def isQueueFull(): Boolean = launchedDrivers.size >= queuedCapacity def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { val c = new CreateSubmissionResponse - val submitDate = new Date() - desc.submissionId = Some(newDriverId(submitDate)) - desc.submissionDate = Some(submitDate) + if (!ready) { + c.success = false + c.message = "Scheduler is not ready to take requests" + return c + } stateLock.synchronized { - if (queue.isFull) { + if (isQueueFull()) { c.success = false c.message = "Already reached maximum submission size" } - c.submissionId = desc.submissionId.get - queue.offer(desc) + c.submissionId = desc.submissionId + queuedDrivers += desc c.success = true } c } + private def removeFromQueuedDrivers(id: String): Boolean = { + val index = queuedDrivers.indexWhere(_.submissionId.equals(id)) + if (index != -1) { + queuedDrivers.remove(index) + queuedDriversState.expunge(id) + true + } else { + false + } + } + + private def removeFromLaunchedDrivers(id: String): Boolean = { + if (launchedDrivers.remove(id).isDefined) { + launchedDriversState.expunge(id) + true + } else { + false + } + } + + private def removeFromPendingRetryDrivers(id: String): Boolean = { + val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(id)) + if (index != -1) { + pendingRetryDrivers.remove(index) + pendingRetryDriversState.expunge(id) + true + } else { + false + } + } + def killDriver(submissionId: String): KillSubmissionResponse = { val k = new KillSubmissionResponse + if (!ready) { + k.success = false + k.message = "Scheduler is not ready to take requests" + return k + } k.submissionId = submissionId stateLock.synchronized { // We look for the requested driver in the following places: @@ -171,17 +205,17 @@ private[spark] class MesosClusterSchedulerDriver( // 3. Check if it's in the retry list. // 4. Check if it has already completed. if (launchedDrivers.contains(submissionId)) { - val task = launchedDrivers.get(submissionId) + val task = launchedDrivers(submissionId) driver.killTask(task.taskId) k.success = true k.message = "Killing running driver" - } else if (queue.remove(submissionId)) { + } else if (removeFromQueuedDrivers(submissionId)) { k.success = true k.message = "Removed driver while it's still pending" - } else if (superviseRetryList.remove(submissionId)) { + } else if (removeFromPendingRetryDrivers(submissionId)) { k.success = true k.message = "Removed driver while it's retrying" - } else if (finishedDrivers.exists(s => s.submission.submissionId.equals(submissionId))) { + } else if (finishedDrivers.exists(_.submission.submissionId.equals(submissionId))) { k.success = false k.message = "Driver already terminated" } else { @@ -197,26 +231,36 @@ private[spark] class MesosClusterSchedulerDriver( * We still need to do task reconciliation to be up to date of the latest task states * as it might have changed while the scheduler is failing over. */ - def recoverState: Unit = { + private def recoverState(): Unit = { stateLock.synchronized { - queue = new DriverQueue(engineFactory.createEngine("driverQueue"), queuedCapacity) - launchedDrivers = new LaunchedDrivers(engineFactory.createEngine("launchedDrivers")) + launchedDriversState.fetchAll[MesosClusterTaskState]().foreach { state => + launchedDrivers(state.taskId.getValue) = state + pendingRecover(state.taskId.getValue) = state.slaveId + } + queuedDriversState.fetchAll[MesosDriverDescription]().foreach(d => queuedDrivers += d) // There is potential timing issue where a queued driver might have been launched // but the scheduler shuts down before the queued driver was able to be removed // from the queue. We try to mitigate this issue by walking through all queued drivers // and remove if they're already launched. - queue.drivers.map(_.submissionId.get).filter(launchedDrivers.contains).foreach(queue.remove) - - superviseRetryList = new SuperviseRetryList(engineFactory.createEngine("retryList")) + queuedDrivers + .filter(d => launchedDrivers.contains(d.submissionId)) + .toSeq + .foreach(d => removeFromQueuedDrivers(d.submissionId)) + pendingRetryDriversState.fetchAll[MesosDriverDescription]() + .foreach(s => pendingRetryDrivers += s) // TODO: Consider storing finished drivers so we can show them on the UI after // failover. For now we clear the history on each recovery. finishedDrivers.clear() } } + /** + * Starts the cluster scheduler and wait until the scheduler is registered. + * This also marks the scheduler to be ready for requests. + */ def start(): Unit = { // TODO: Implement leader election to make sure only one framework running in the cluster. - val fwId = state.fetch[String]("frameworkId") + val fwId = schedulerState.fetch[String]("frameworkId") val builder = FrameworkInfo.newBuilder() .setUser(Utils.getCurrentUserName()) .setName(appName) @@ -227,14 +271,16 @@ private[spark] class MesosClusterSchedulerDriver( builder.setId(FrameworkID.newBuilder().setValue(id).build()) frameworkId = id } - recoverState + recoverState() metricsSystem.registerSource(new MesosClusterSchedulerSource(this)) metricsSystem.start() startScheduler( - "MesosClusterScheduler", master, MesosClusterSchedulerDriver.this, builder.build()) + "MesosClusterScheduler", master, MesosClusterScheduler.this, builder.build()) + ready = true } def stop(): Unit = { + ready = false metricsSystem.report() metricsSystem.stop() driver.stop(true) @@ -247,22 +293,23 @@ private[spark] class MesosClusterSchedulerDriver( logInfo("Registered as framework ID " + newFrameworkId.getValue) if (newFrameworkId.getValue != frameworkId) { frameworkId = newFrameworkId.getValue - state.persist("frameworkId", frameworkId) + schedulerState.persist("frameworkId", frameworkId) } markRegistered() stateLock.synchronized { this.masterInfo = Some(masterInfo) - if (!launchedDrivers.pendingRecover.isEmpty) { + if (!pendingRecover.isEmpty) { // Start task reconciliation if we need to recover. - val statuses = launchedDrivers.pendingRecover.collect { + val statuses = pendingRecover.collect { case (taskId, slaveId) => - launchedDrivers.get(taskId).taskState.getOrElse( - TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(slaveId) - .setState(MesosTaskState.TASK_STAGING) - .build()) + val newStatus = TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(taskId).build()) + .setSlaveId(slaveId) + .setState(MesosTaskState.TASK_STAGING) + .build() + launchedDrivers.get(taskId).map(_.taskState.getOrElse(newStatus)) + .getOrElse(newStatus) } // TODO: Page the status updates to avoid trying to reconcile // a large amount of tasks at once. @@ -271,7 +318,7 @@ private[spark] class MesosClusterSchedulerDriver( } } - private def buildCommand(desc: MesosDriverDescription): CommandInfo = { + private def buildDriverCommand(desc: MesosDriverDescription): CommandInfo = { val appJar = CommandInfo.URI.newBuilder() .setValue(desc.jarUrl.stripPrefix("file:").stripPrefix("local:")).build() val builder = CommandInfo.newBuilder().addUris(appJar) @@ -287,8 +334,8 @@ private[spark] class MesosClusterSchedulerDriver( desc.command.environment.foreach { case (k, v) => envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v).build()) } - val executorOpts = desc.schedulerProperties.map { case (k, v) => s"-D$k=$v"}.mkString(" ") // Pass all spark properties to executor. + val executorOpts = desc.schedulerProperties.map { case (k, v) => s"-D$k=$v" }.mkString(" ") envBuilder.addVariables( Variable.newBuilder().setName("SPARK_EXECUTOR_OPTS").setValue(executorOpts)) val cmdOptions = generateCmdOption(desc) @@ -314,16 +361,16 @@ private[spark] class MesosClusterSchedulerDriver( } builder.setValue(cmd) builder.setEnvironment(envBuilder.build()) - builder.build + builder.build() } private def generateCmdOption(desc: MesosDriverDescription): Seq[String] = { var options = Seq( - "--name", desc.schedulerProperties("spark.app.name"), - "--class", desc.command.mainClass, - "--master", s"mesos://${conf.get("spark.master")}", - "--driver-cores", desc.cores.toString, - "--driver-memory", s"${desc.mem}M") + "--name", desc.schedulerProperties("spark.app.name"), + "--class", desc.command.mainClass, + "--master", s"mesos://${conf.get("spark.master")}", + "--driver-cores", desc.cores.toString, + "--driver-memory", s"${desc.mem}M") desc.schedulerProperties.get("spark.executor.memory").map { v => options ++= Seq("--executor-memory", v) } @@ -339,7 +386,72 @@ private[spark] class MesosClusterSchedulerDriver( } } + /** + * This method takes all the possible candidates provided by the tasksFunc + * and attempt to schedule them with Mesos offers. + * Every time a new task is scheduled, the scheduledCallback is called to + * perform post scheduled logic on each task. + */ + private def scheduleTasks( + candidates: Seq[MesosDriverDescription], + scheduledCallback: (String) => Boolean, + currentOffers: List[ResourceOffer], + tasks: mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]], + currentTime: Date): Unit = { + for (submission <- candidates) { + val driverCpu = submission.cores + val driverMem = submission.mem + logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem") + val offerOption = currentOffers.find { o => + o.cpu >= driverCpu && o.mem >= driverMem + } + if (offerOption.isEmpty) { + logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId}," + + s"cpu: $driverCpu, mem: $driverMem") + } else { + val offer = offerOption.get + offer.cpu -= driverCpu + offer.mem -= driverMem + val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() + val cpuResource = Resource.newBuilder() + .setName("cpus").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() + val memResource = Resource.newBuilder() + .setName("mem").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() + val commandInfo = buildDriverCommand(submission) + val appName = submission.schedulerProperties("spark.app.name") + val taskInfo = TaskInfo.newBuilder() + .setTaskId(taskId) + .setName(s"Driver for $appName") + .setSlaveId(offer.offer.getSlaveId) + .setCommand(commandInfo) + .addResources(cpuResource) + .addResources(memResource) + .build() + val queuedTasks = if (!tasks.contains(offer.offer.getId)) { + val buffer = new ArrayBuffer[TaskInfo] + tasks(offer.offer.getId) = buffer + buffer + } else { + tasks(offer.offer.getId) + } + queuedTasks += taskInfo + logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + + submission.submissionId) + val newState = new MesosClusterTaskState(submission, taskId, offer.offer.getSlaveId, + None, new Date()) + launchedDrivers(submission.submissionId) = newState + launchedDriversState.persist(submission.submissionId, newState) + scheduledCallback(submission.submissionId) + } + } + } + override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { + /** + * Prints the list of Mesos offers for logging purpose. + */ def printOffers(offers: Iterable[ResourceOffer]): String = { val builder = new StringBuilder() offers.foreach { o => @@ -347,120 +459,76 @@ private[spark] class MesosClusterSchedulerDriver( } builder.toString() } - val currentOffers = offers.map { o => new ResourceOffer( - o, - getResource(o.getResourcesList, "cpus"), - getResource(o.getResourcesList, "mem")) - } + o, getResource(o.getResourcesList, "cpus"), getResource(o.getResourcesList, "mem")) + }.toList logTrace(s"Received offers from Mesos: \n${printOffers(currentOffers)}") val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() val currentTime = new Date() - def scheduleTasks( - tasksFunc: () => Seq[(MesosDriverDescription, Option[RetryState])], - scheduledCallback: (String) => Unit): Unit = { - val candidates = tasksFunc() - for ((submission, retryState) <- candidates) { - val driverCpu = submission.cores - val driverMem = submission.mem - logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem") - val offerOption = currentOffers.find { o => - o.cpu >= driverCpu && o.mem >= driverMem - } - - if (offerOption.isEmpty) { - logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId.get}," + - s"cpu: $driverCpu, mem: $driverMem") - } else { - val offer = offerOption.get - offer.cpu -= driverCpu - offer.mem -= driverMem - val taskId = TaskID.newBuilder().setValue(submission.submissionId.get).build() - val cpuResource = Resource.newBuilder() - .setName("cpus").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() - val memResource = Resource.newBuilder() - .setName("mem").setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() - val commandInfo = buildCommand(submission) - val appName = submission.schedulerProperties("spark.app.name") - val taskInfo = TaskInfo.newBuilder() - .setTaskId(taskId) - .setName(s"Driver for $appName") - .setSlaveId(offer.offer.getSlaveId) - .setCommand(commandInfo) - .addResources(cpuResource) - .addResources(memResource) - .build - val queuedTasks = if (!tasks.contains(offer.offer.getId)) { - val buffer = new ArrayBuffer[TaskInfo] - tasks(offer.offer.getId) = buffer - buffer - } else { - tasks(offer.offer.getId) - } - queuedTasks += taskInfo - logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + - submission.submissionId.get) - - launchedDrivers.set( - submission.submissionId.get, - new MesosClusterTaskState(submission, taskId, offer.offer.getSlaveId, - None, new Date(), retryState)) - scheduledCallback(submission.submissionId.get) - } - } - } stateLock.synchronized { - scheduleTasks(() => { - superviseRetryList.getNextRetries(currentTime) - }, (id: String) => { - superviseRetryList.remove(id) - }) - scheduleTasks(() => queue.drivers.map(d => (d, None)), (id) => queue.remove(id)) + // We first schedule all the supervised drivers that are ready to retry. + // This list will be empty if none of the drivers are marked as supervise. + scheduleTasks(pendingRetryDrivers + .filter(d => d.retryState.get.nextRetry.before(currentTime)), + removeFromPendingRetryDrivers, + currentOffers, tasks, currentTime) + // Then we walk through the queued drivers and try to schedule them. + scheduleTasks(queuedDrivers, + removeFromQueuedDrivers, + currentOffers, tasks, currentTime) } - tasks.foreach { case (offerId, tasks) => driver.launchTasks(Collections.singleton(offerId), tasks) } - offers .filter(o => !tasks.keySet.contains(o.getId)) .foreach(o => driver.declineOffer(o.getId)) } def getState(): MesosClusterSchedulerState = { + def copyBuffer( + buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { + val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) + buffer.copyToBuffer(newBuffer) + newBuffer + } stateLock.synchronized { new MesosClusterSchedulerState( frameworkId, masterInfo.map(m => s"http://${m.getIp}:${m.getPort}"), - queue.copyDrivers, - launchedDrivers.states, - finishedDrivers.collect { case s => s.copy() }, - superviseRetryList.retries) + copyBuffer(queuedDrivers), + launchedDrivers.values.map(_.copy).toList, + finishedDrivers.map(_.copy).toList, + copyBuffer(pendingRetryDrivers)) } } def getStatus(submissionId: String): SubmissionStatusResponse = { val s = new SubmissionStatusResponse + if (!ready) { + s.success = false + s.message = "Scheduler is not ready to take requests" + return s + } s.submissionId = submissionId stateLock.synchronized { - if (queue.contains(submissionId)) { + if (queuedDrivers.exists(_.submissionId.equals(submissionId))) { s.success = true s.driverState = "Driver is queued for launch" } else if (launchedDrivers.contains(submissionId)) { s.success = true s.driverState = "Driver is running" - launchedDrivers.get(submissionId).taskState.foreach(state => s.message = state.toString) + launchedDrivers(submissionId).taskState.foreach(state => s.message = state.toString) } else if (finishedDrivers.exists(s => s.submission.submissionId.equals(submissionId))) { s.success = true s.driverState = "Driver already finished" finishedDrivers.find(d => d.submission.submissionId.equals(submissionId)).get.taskState .foreach(state => s.message = state.toString) - } else if (superviseRetryList.contains(submissionId)) { - val status = superviseRetryList.get(submissionId).get.lastFailureStatus + } else if (pendingRetryDrivers.exists(_.submissionId.equals(submissionId))) { + val status = pendingRetryDrivers.find(_.submissionId.equals(submissionId)) + .get.retryState.get.lastFailureStatus s.success = true s.driverState = "Driver failed and retrying" s.message = status.toString @@ -473,15 +541,11 @@ private[spark] class MesosClusterSchedulerDriver( } override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {} - override def disconnected(driver: SchedulerDriver): Unit = {} - override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = { logInfo(s"Framework re-registered with master ${masterInfo.getId}") } - override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {} - override def error(driver: SchedulerDriver, error: String): Unit = {} /** @@ -500,25 +564,29 @@ private[spark] class MesosClusterSchedulerDriver( stateLock.synchronized { if (launchedDrivers.contains(taskId)) { if (status.getReason == Reason.REASON_RECONCILIATION && - !launchedDrivers.pendingRecover.contains(taskId)) { + !pendingRecover.contains(taskId)) { // Task has already received update and no longer requires reconciliation. return } - val state = launchedDrivers.get(taskId) + val state = launchedDrivers(taskId) // Check if the driver is supervise enabled and can be relaunched. if (state.submission.supervise && shouldRelaunch(status.getState)) { - val (retries, waitTimeSec) = if (state.retryState.isDefined) { - (state.retryState.get.retries + 1, - Math.min(maxRetryWaitTime, state.retryState.get.waitTime * 2)) + removeFromLaunchedDrivers(taskId) + val retryState: Option[RetryState] = state.submission.retryState + val (retries, waitTimeSec) = if (retryState.isDefined) { + (retryState.get.retries + 1, + Math.min(maxRetryWaitTime, retryState.get.waitTime * 2)) } else { (1, 1) } val nextRetry = new Date(new Date().getTime + waitTimeSec * 1000L) - superviseRetryList.add( - RetryState(state.submission, status, retries, nextRetry, waitTimeSec)) - } - if (TaskState.isFinished(TaskState.fromMesos(status.getState))) { - launchedDrivers.remove(taskId) + + val newDriverDescription = state.submission.copy( + retryState = Some(new RetryState(status, retries, nextRetry, waitTimeSec))) + pendingRetryDrivers += newDriverDescription + pendingRetryDriversState.persist(taskId, newDriverDescription) + } else if (TaskState.isFinished(TaskState.fromMesos(status.getState))) { + removeFromLaunchedDrivers(taskId) if (finishedDrivers.size >= retainedDrivers) { val toRemove = math.max(retainedDrivers / 10, 1) finishedDrivers.trimStart(toRemove) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala index 3438bda2dda9b..4a9e1900bcb44 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala @@ -21,13 +21,13 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source -private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterSchedulerDriver) +private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterScheduler) extends Source { override def sourceName: String = "mesos_cluster" override def metricRegistry: MetricRegistry = new MetricRegistry() metricRegistry.register(MetricRegistry.name("waitingDrivers"), new Gauge[Int] { - override def getValue: Int = scheduler.queue.size + override def getValue: Int = scheduler.queuedDrivers.size }) metricRegistry.register(MetricRegistry.name("launchedDrivers"), new Gauge[Int] { @@ -35,6 +35,6 @@ private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterSchedule }) metricRegistry.register(MetricRegistry.name("retryDrivers"), new Gauge[Int] { - override def getValue: Int = scheduler.superviseRetryList.size + override def getValue: Int = scheduler.pendingRetryDrivers.size }) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 3e12da843297d..3e6e64cc634a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -20,22 +20,23 @@ package org.apache.spark.scheduler.cluster.mesos import java.util.List import java.util.concurrent.CountDownLatch -import org.apache.mesos.Protos.{Status, FrameworkInfo, Resource} -import org.apache.mesos.{MesosSchedulerDriver, Scheduler, SchedulerDriver} -import org.apache.spark.Logging - import scala.collection.JavaConversions._ +import org.apache.mesos.Protos.{FrameworkInfo, Resource, Status} +import org.apache.mesos.{MesosSchedulerDriver, Scheduler} +import org.apache.spark.Logging +import org.apache.spark.util.Utils + /** * Shared trait for implementing a Mesos Scheduler. This holds common state and helper * methods and Mesos scheduler will use. */ private[mesos] trait MesosSchedulerUtils extends Logging { // Lock used to wait for scheduler to be registered - final val registerLatch = new CountDownLatch(1) + private final val registerLatch = new CountDownLatch(1) // Driver for talking to Mesos - var driver: SchedulerDriver = null + protected var driver: MesosSchedulerDriver = null /** * Starts the MesosSchedulerDriver with the provided information. This method returns @@ -52,11 +53,11 @@ private[mesos] trait MesosSchedulerUtils extends Logging { fwInfo: FrameworkInfo): Unit = { synchronized { if (driver != null) { - waitForRegister() + registerLatch.await() return } - new Thread(name + " driver") { + new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") { setDaemon(true) override def run() { @@ -64,41 +65,33 @@ private[mesos] trait MesosSchedulerUtils extends Logging { try { val ret = driver.run() logInfo("driver.run() returned with code " + ret) - onDriverExit(ret) + if (ret.equals(Status.DRIVER_ABORTED)) { + System.exit(1) + } } catch { - case e: Exception => logError("driver.run() failed", e) + case e: Exception => { + logError("driver.run() failed", e) + System.exit(1) + } } } }.start() - waitForRegister() - } - } - - def onDriverExit(status: Status): Unit = { - // Exit the process when the Mesos framework driver was aborted. - // This behavior can be overriden by the scheduler. - if (status.equals(Status.DRIVER_ABORTED)) { - System.exit(1) + registerLatch.await() } } - /** - * Waits for the scheduler to be registered, which the scheduler will signal by calling - * markRegistered(). - */ - def waitForRegister(): Unit = { - registerLatch.await() - } - /** * Signal that the scheduler has registered with Mesos. */ - def markRegistered(): Unit = { + protected def markRegistered(): Unit = { registerLatch.countDown() } - def getResource(res: List[Resource], name: String): Double = { + /** + * Get the amount of resources for the specified type from the resource list + */ + protected def getResource(res: List[Resource], name: String): Double = { for (r <- res if r.getName == name) { return r.getScalar.getValue } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala deleted file mode 100644 index 66a0443e6825d..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/SuperviseRetryList.scala +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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.scheduler.cluster.mesos - -import org.apache.mesos.Protos.TaskStatus -import java.util.Date -import org.apache.spark.deploy.mesos.MesosDriverDescription - -import scala.collection.mutable.ArrayBuffer - -/** - * Tracks the retry state of a driver, which includes the next time it should be scheduled - * and necessary information to do exponential backoff. - * This class is not thread-safe, and we expect the caller to handle synchronizing state. - * @param submission Driver submission. - * @param lastFailureStatus Last Task status when it failed. - * @param retries Number of times it has retried. - * @param nextRetry Next retry time to be scheduled. - * @param waitTime The amount of time driver is scheduled to wait until next retry. - */ -private[spark] case class RetryState( - submission: MesosDriverDescription, - lastFailureStatus: TaskStatus, - retries: Int, - nextRetry: Date, - waitTime: Int) extends Serializable { - def copy(): RetryState = - new RetryState(submission, lastFailureStatus, retries, nextRetry, waitTime) -} - -/** - * Tracks all the drivers that were submitted under supervise which failed to run - * and waiting to be scheduled again. - * @param state Persistence engine to store state. - */ -private[mesos] class SuperviseRetryList(state: MesosClusterPersistenceEngine) { - val drivers = new ArrayBuffer[RetryState] - - initialize() - - def initialize(): Unit = { - state.fetchAll[RetryState]().foreach(drivers.+=) - } - - def size: Int = drivers.size - - def contains(submissionId: String): Boolean = { - drivers.exists { d => - d.submission.submissionId.map(_.equals(submissionId)).getOrElse(false) - } - } - - def getNextRetries(currentTime: Date): Seq[(MesosDriverDescription, Option[RetryState])] = { - drivers.filter(d => d.nextRetry.before(currentTime)).map { d => - (d.submission, Some(d)) - } - } - - def get(submissionId: String): Option[RetryState] = { - drivers.find { d => - d.submission.submissionId.map(_.equals(submissionId)).getOrElse(false) - } - } - - def retries: Iterable[RetryState] = { - drivers.map(d => d.copy).toList - } - - def remove(submissionId: String): Boolean = { - val index = drivers.indexWhere { s => - s.submission.submissionId.map(_.equals(submissionId)).getOrElse(false) - } - - if (index != -1) { - drivers.remove(index) - state.expunge(submissionId) - } - - index != -1 - } - - def add(retryState: RetryState): Unit = { - drivers += retryState - state.persist(retryState.submission.submissionId.get, retryState) - } -} diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 5eb413e08c08c..3d7601a2c5fda 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -39,9 +39,9 @@ import org.apache.spark.deploy.master.DriverState._ * Tests for the REST application submission protocol used in standalone cluster mode. */ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { - private val client = new RestClient + private val client = new RestSubmissionClient private var actorSystem: Option[ActorSystem] = None - private var server: Option[RestServer] = None + private var server: Option[RestSubmissionServer] = None override def afterEach() { actorSystem.foreach(_.shutdown()) @@ -89,7 +89,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { conf.set("spark.app.name", "dreamer") val appArgs = Array("one", "two", "six") // main method calls this - val response = RestClient.run("app-resource", "main-class", appArgs, conf) + val response = RestSubmissionClient.run("app-resource", "main-class", appArgs, conf) val submitResponse = getSubmitResponse(response) assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) assert(submitResponse.serverSparkVersion === SPARK_VERSION) @@ -208,7 +208,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("good request paths") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = RestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val json = constructSubmitRequest(masterUrl).toJson val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill" @@ -238,7 +238,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("good request paths, bad requests") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = RestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill" val statusRequestPath = s"$httpUrl/$v/submissions/status" @@ -276,7 +276,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("bad request paths") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = RestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val (response1, code1) = sendHttpRequestWithResponse(httpUrl, "GET") val (response2, code2) = sendHttpRequestWithResponse(s"$httpUrl/", "GET") val (response3, code3) = sendHttpRequestWithResponse(s"$httpUrl/$v", "GET") @@ -292,7 +292,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { assert(code5 === HttpServletResponse.SC_BAD_REQUEST) assert(code6 === HttpServletResponse.SC_BAD_REQUEST) assert(code7 === HttpServletResponse.SC_BAD_REQUEST) - assert(code8 === RestServer.SC_UNKNOWN_PROTOCOL_VERSION) + assert(code8 === RestSubmissionServer.SC_UNKNOWN_PROTOCOL_VERSION) // all responses should be error responses val errorResponse1 = getErrorResponse(response1) val errorResponse2 = getErrorResponse(response2) @@ -310,13 +310,13 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { assert(errorResponse5.highestProtocolVersion === null) assert(errorResponse6.highestProtocolVersion === null) assert(errorResponse7.highestProtocolVersion === null) - assert(errorResponse8.highestProtocolVersion === RestServer.PROTOCOL_VERSION) + assert(errorResponse8.highestProtocolVersion === RestSubmissionServer.PROTOCOL_VERSION) } test("server returns unknown fields") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = RestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val oldJson = constructSubmitRequest(masterUrl).toJson val oldFields = parse(oldJson).asInstanceOf[JObject].obj @@ -340,7 +340,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("client handles faulty server") { val masterUrl = startFaultyServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = RestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill/anything" val statusRequestPath = s"$httpUrl/$v/submissions/status/anything" @@ -561,15 +561,15 @@ private class SmarterMaster extends Actor { * The purpose of this class is to test that client handles these cases gracefully. */ private class FaultyStandaloneRestServer( - val host: String, - val requestedPort: Int, - val masterConf: SparkConf, + host: String, + requestedPort: Int, + masterConf: SparkConf, masterActor: ActorRef, masterUrl: String) - extends RestServer { - def submitRequestServlet: SubmitRequestServlet = new MalformedSubmitServlet - def killRequestServlet: KillRequestServlet = new InvalidKillServlet - def statusRequestServlet: StatusRequestServlet = new ExplodingStatusServlet + extends RestSubmissionServer(host, requestedPort, masterConf) { + val submitRequestServlet = new MalformedSubmitServlet + val killRequestServlet = new InvalidKillServlet + val statusRequestServlet = new ExplodingStatusServlet /** A faulty servlet that produces malformed responses. */ class MalformedSubmitServlet extends StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index 9aa3661973f95..ce790004eb377 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -17,12 +17,16 @@ package org.apache.spark.scheduler.mesos +import java.util.Date + +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar + import org.apache.spark.deploy.Command import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.scheduler.cluster.mesos._ import org.apache.spark.{LocalSparkContext, SparkConf} -import org.scalatest.FunSuite -import org.scalatest.mock.MockitoSugar + class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar { def createCommand: Command = { @@ -34,34 +38,36 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo val conf = new SparkConf() conf.setMaster("mesos://localhost:5050") conf.setAppName("spark mesos") - val scheduler = new MesosClusterSchedulerDriver( - new BlackHoleMesosClusterPersistenceEngineFactory, conf) - scheduler.recoverState - val response = - scheduler.submitDriver( + val scheduler = new MesosClusterScheduler( + new BlackHoleMesosClusterPersistenceEngineFactory, conf) { + override def start(): Unit = ready = true + } + scheduler.start() + val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", 1000, 1, true, - createCommand, Map[String, String]())) + createCommand, Map[String, String](), "s1", new Date())) assert(response.success) val response2 = scheduler.submitDriver(new MesosDriverDescription( - "d1", "jar", 1000, 1, true, createCommand, Map[String, String]())) + "d1", "jar", 1000, 1, true, createCommand, Map[String, String](), "s2", new Date())) assert(response2.success) val state = scheduler.getState() - assert(state.queuedDrivers.exists(d => d.submissionId.get == response.submissionId)) - assert(state.queuedDrivers.exists(d => d.submissionId.get == response2.submissionId)) + assert(state.queuedDrivers.exists(d => d.submissionId == response.submissionId)) + assert(state.queuedDrivers.exists(d => d.submissionId == response2.submissionId)) } test("can kill queued drivers") { val conf = new SparkConf() conf.setMaster("mesos://localhost:5050") conf.setAppName("spark mesos") - val scheduler = new MesosClusterSchedulerDriver( - new BlackHoleMesosClusterPersistenceEngineFactory, conf) - scheduler.recoverState - val response = - scheduler.submitDriver( + val scheduler = new MesosClusterScheduler( + new BlackHoleMesosClusterPersistenceEngineFactory, conf) { + override def start(): Unit = ready = true + } + scheduler.start() + val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", 1000, 1, true, - createCommand, Map[String, String]())) + createCommand, Map[String, String](), "s1", new Date())) assert(response.success) val killResponse = scheduler.killDriver(response.submissionId) assert(killResponse.success) From fd5259dddbd688d55e21f624c172d76184cd74be Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Fri, 17 Apr 2015 01:15:18 -0700 Subject: [PATCH 32/36] Address review comments. --- .../org/apache/spark/deploy/SparkSubmit.scala | 14 +- .../deploy/mesos/MesosClusterDispatcher.scala | 16 +- .../MesosClusterDispatcherArguments.scala | 2 +- .../deploy/mesos/MesosDriverDescription.scala | 12 +- .../deploy/mesos/ui/MesosClusterPage.scala | 22 +- .../deploy/mesos/ui/MesosClusterUI.scala | 4 +- .../deploy/rest/RestSubmissionClient.scala | 2 +- .../deploy/rest/RestSubmissionServer.scala | 9 +- .../deploy/rest/StandaloneRestServer.scala | 10 +- .../deploy/rest/mesos/MesosRestServer.scala | 22 +- .../mesos/CoarseMesosSchedulerBackend.scala | 15 +- .../cluster/mesos/MesosClusterScheduler.scala | 309 +++++++++--------- .../mesos/MesosClusterSchedulerSource.scala | 6 +- .../cluster/mesos/MesosSchedulerBackend.scala | 30 +- .../cluster/mesos/MesosSchedulerUtils.scala | 17 +- .../scheduler/cluster/mesos/MesosState.scala | 108 ------ .../scala/org/apache/spark/util/Utils.scala | 1 - .../spark/deploy/SparkSubmitSuite.scala | 2 +- .../rest/StandaloneRestSubmitSuite.scala | 10 +- .../mesos/MesosClusterSchedulerSuite.scala | 25 +- docs/running-on-mesos.md | 4 +- sbin/start-mesos-dispatcher.sh | 2 +- 22 files changed, 261 insertions(+), 381 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosState.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 585937afd0016..f4f572e1e256e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -22,6 +22,8 @@ import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowab import java.net.URL import java.security.PrivilegedExceptionAction +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} + import org.apache.hadoop.fs.Path import org.apache.hadoop.security.UserGroupInformation import org.apache.ivy.Ivy @@ -38,7 +40,6 @@ import org.apache.spark.SPARK_VERSION import org.apache.spark.deploy.rest._ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} -import scala.collection.mutable.{ArrayBuffer, HashMap, Map} /** * Whether to submit, kill, or request the status of an application. @@ -380,10 +381,6 @@ object SparkSubmit { OptionAssigner(args.driverExtraLibraryPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.driver.extraLibraryPath"), - // Standalone cluster only - // Do not set CL arguments here because there are multiple possibilities for the main class - OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy"), - // Yarn client only OptionAssigner(args.queue, YARN, CLIENT, sysProp = "spark.yarn.queue"), OptionAssigner(args.numExecutors, YARN, CLIENT, sysProp = "spark.executor.instances"), @@ -418,7 +415,8 @@ object SparkSubmit { OptionAssigner(args.driverCores, STANDALONE | MESOS, CLUSTER, sysProp = "spark.driver.cores"), OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, - sysProp = "spark.driver.supervise") + sysProp = "spark.driver.supervise"), + OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy") ) // In client mode, launch the application main class directly @@ -457,7 +455,7 @@ object SparkSubmit { // All Spark parameters are expected to be passed to the client through system properties. if (args.isStandaloneCluster) { if (args.useRest) { - childMainClass = "org.apache.spark.deploy.rest.RestClient" + childMainClass = "org.apache.spark.deploy.rest.RestSubmissionClient" childArgs += (args.primaryResource, args.mainClass) } else { // In legacy standalone cluster mode, use Client as a wrapper around the user class @@ -503,7 +501,7 @@ object SparkSubmit { if (isMesosCluster) { assert(args.useRest, "Mesos cluster mode is only supported through the REST submission API") - childMainClass = "org.apache.spark.deploy.rest.RestClient" + childMainClass = "org.apache.spark.deploy.rest.RestSubmissionClient" childArgs += (args.primaryResource, args.mainClass) if (args.childArgs != null) { childArgs ++= args.childArgs diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index b608e14fbaa62..5d4e5b899dfdc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy.mesos import java.util.concurrent.CountDownLatch -import org.apache.spark import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.mesos.MesosRestServer import org.apache.spark.scheduler.cluster.mesos._ @@ -50,17 +49,14 @@ private[mesos] class MesosClusterDispatcher( conf: SparkConf) extends Logging { - private def publicAddress(conf: SparkConf, defaultAddress: String): String = { - val envVar = conf.getenv("SPARK_PUBLIC_DNS") - if (envVar != null) envVar else defaultAddress - } - + private val publicAddress = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(args.host) private val recoveryMode = conf.get("spark.mesos.deploy.recoveryMode", "NONE").toUpperCase() logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode) private val engineFactory = recoveryMode match { case "NONE" => new BlackHoleMesosClusterPersistenceEngineFactory case "ZOOKEEPER" => new ZookeeperMesosClusterPersistenceEngineFactory(conf) + case _ => throw new IllegalArgumentException("Unsupported recovery mode: " + recoveryMode) } private val scheduler = new MesosClusterScheduler(engineFactory, conf) @@ -70,7 +66,7 @@ private[mesos] class MesosClusterDispatcher( new SecurityManager(conf), args.webUiPort, conf, - publicAddress(conf, args.host), + publicAddress, scheduler) private val shutdownLatch = new CountDownLatch(1) @@ -94,7 +90,7 @@ private[mesos] class MesosClusterDispatcher( } } -private[mesos] object MesosClusterDispatcher extends spark.Logging { +private[mesos] object MesosClusterDispatcher extends Logging { def main(args: Array[String]) { SignalLogger.register(log) val conf = new SparkConf @@ -105,9 +101,7 @@ private[mesos] object MesosClusterDispatcher extends spark.Logging { conf.set("spark.mesos.deploy.recoveryMode", "ZOOKEEPER") conf.set("spark.mesos.deploy.zookeeper.url", z) } - val dispatcher = new MesosClusterDispatcher( - dispatcherArgs, - conf) + val dispatcher = new MesosClusterDispatcher(dispatcherArgs, conf) dispatcher.start() val shutdownHook = new Thread() { override def run() { diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala index b24b88c2b3d6d..894cb78d8591a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala @@ -74,7 +74,7 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: case Nil => { if (masterUrl == null) { System.err.println("--master is required") - System.exit(1) + printUsageAndExit(1) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala index 7ba6be2ec52e1..1948226800afe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.mesos import java.util.Date import org.apache.spark.deploy.Command -import org.apache.spark.scheduler.cluster.mesos.RetryState +import org.apache.spark.scheduler.cluster.mesos.MesosClusterRetryState /** * Describes a Spark driver that is submitted from the @@ -28,7 +28,7 @@ import org.apache.spark.scheduler.cluster.mesos.RetryState * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]]. * @param jarUrl URL to the application jar * @param mem Amount of memory for the driver - * @param cores Amount of cores for the driver + * @param cores Number of cores for the driver * @param supervise Supervise the driver for long running app * @param command The command to launch the driver. * @param schedulerProperties Extra properties to pass the Mesos scheduler @@ -43,8 +43,9 @@ private[spark] class MesosDriverDescription( val schedulerProperties: Map[String, String], val submissionId: String, val submissionDate: Date, - val retryState: Option[RetryState] = None) + val retryState: Option[MesosClusterRetryState] = None) extends Serializable { + def copy( name: String = name, jarUrl: String = jarUrl, @@ -53,11 +54,12 @@ private[spark] class MesosDriverDescription( supervise: Boolean = supervise, command: Command = command, schedulerProperties: Map[String, String] = schedulerProperties, - retryState: Option[RetryState] = retryState, submissionId: String = submissionId, - submissionDate: Date = submissionDate): MesosDriverDescription = { + submissionDate: Date = submissionDate, + retryState: Option[MesosClusterRetryState] = retryState): MesosDriverDescription = { new MesosDriverDescription(name, jarUrl, mem, cores, supervise, command, schedulerProperties, submissionId, submissionDate, retryState) } + override def toString: String = s"MesosDriverDescription (${command.mainClass})" } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala index fb7ce892c20c8..7b2005e0f1237 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -19,16 +19,16 @@ package org.apache.spark.deploy.mesos.ui import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import org.apache.mesos.Protos.TaskStatus import org.apache.spark.deploy.mesos.MesosDriverDescription -import org.apache.spark.scheduler.cluster.mesos.MesosClusterTaskState +import org.apache.spark.scheduler.cluster.mesos.MesosClusterSubmissionState import org.apache.spark.ui.{UIUtils, WebUIPage} -import scala.xml.Node - private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { - val state = parent.scheduler.getState() + val state = parent.scheduler.getSchedulerState() val queuedHeaders = Seq("Driver ID", "Submit Date", "Main Class", "Driver Resources") val driverHeaders = queuedHeaders ++ Seq("Start Date", "Mesos Slave ID", "State") @@ -37,7 +37,7 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers) - val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.retryList) + val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.pendingRetryDrivers) val content =

Mesos Framework ID: {state.frameworkId}

@@ -64,15 +64,15 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( } - private def driverRow(state: MesosClusterTaskState): Seq[Node] = { + private def driverRow(state: MesosClusterSubmissionState): Seq[Node] = { - {state.submission.submissionId} - {state.submission.submissionDate} - {state.submission.command.mainClass} - cpus: {state.submission.cores}, mem: {state.submission.mem} + {state.driverDescription.submissionId} + {state.driverDescription.submissionDate} + {state.driverDescription.command.mainClass} + cpus: {state.driverDescription.cores}, mem: {state.driverDescription.mem} {state.startDate} {state.slaveId.getValue} - {stateString(state.taskState)} + {stateString(state.mesosTaskStatus)} } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala index 7dea5304ce5c3..4865d46dbc4ab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.{SparkUI, WebUI} /** * UI that displays driver results from the [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]] */ -private [spark] class MesosClusterUI( +private[spark] class MesosClusterUI( securityManager: SecurityManager, port: Int, conf: SparkConf, @@ -43,6 +43,6 @@ private [spark] class MesosClusterUI( } } -private[spark] object MesosClusterUI { +private object MesosClusterUI { val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 020d9c55dd918..307cebfb4bd09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -332,7 +332,7 @@ private[spark] object RestSubmissionClient { def main(args: Array[String]): Unit = { if (args.size < 2) { - sys.error("Usage: RestClient [app resource] [main class] [app args*]") + sys.error("Usage: RestSubmissionClient [app resource] [main class] [app args*]") sys.exit(1) } val appResource = args(0) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 800ac1e4aee9a..2e78d03e5c0cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -47,9 +47,9 @@ import org.apache.spark.util.Utils * server error. */ private[spark] abstract class RestSubmissionServer( - val host: String, - val requestedPort: Int, - val masterConf: SparkConf) extends Logging { + val host: String, + val requestedPort: Int, + val masterConf: SparkConf) extends Logging { protected val submitRequestServlet: SubmitRequestServlet protected val killRequestServlet: KillRequestServlet protected val statusRequestServlet: StatusRequestServlet @@ -190,8 +190,7 @@ private[rest] abstract class RestServlet extends HttpServlet with Logging { /** * A servlet for handling kill requests passed to the [[RestSubmissionServer]]. */ -private[rest] abstract class KillRequestServlet - extends RestServlet { +private[rest] abstract class KillRequestServlet extends RestServlet { /** * If a submission ID is specified in the URL, have the Master kill the corresponding diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 432067b72f3ec..1cb8433690752 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -56,9 +56,13 @@ private[deploy] class StandaloneRestServer( masterActor: ActorRef, masterUrl: String) extends RestSubmissionServer(host, requestedPort, masterConf) { - val submitRequestServlet = new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) - val killRequestServlet = new StandaloneKillRequestServlet(masterActor, masterConf) - val statusRequestServlet = new StandaloneStatusRequestServlet(masterActor, masterConf) + + protected override val submitRequestServlet = + new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) + protected override val killRequestServlet = + new StandaloneKillRequestServlet(masterActor, masterConf) + protected override val statusRequestServlet = + new StandaloneStatusRequestServlet(masterActor, masterConf) } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index 6fb84580dec36..fd17a980c9319 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -36,8 +36,7 @@ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} * All requests are forwarded to * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]]. * This is intended to be used in Mesos cluster mode only. - * For more details about the RestServer Spark protocol and status codes please refer to - * [[RestSubmissionServer]] javadocs. + * For more details about the REST submission please refer to [[RestSubmissionServer]] javadocs. */ private[spark] class MesosRestServer( host: String, @@ -45,9 +44,13 @@ private[spark] class MesosRestServer( masterConf: SparkConf, scheduler: MesosClusterScheduler) extends RestSubmissionServer(host, requestedPort, masterConf) { - protected val submitRequestServlet = new MesosSubmitRequestServlet(scheduler, masterConf) - protected val killRequestServlet = new MesosKillRequestServlet(scheduler, masterConf) - protected val statusRequestServlet = new MesosStatusRequestServlet(scheduler, masterConf) + + protected override val submitRequestServlet = + new MesosSubmitRequestServlet(scheduler, masterConf) + protected override val killRequestServlet = + new MesosKillRequestServlet(scheduler, masterConf) + protected override val statusRequestServlet = + new MesosStatusRequestServlet(scheduler, masterConf) } private[deploy] class MesosSubmitRequestServlet( @@ -59,7 +62,7 @@ private[deploy] class MesosSubmitRequestServlet( private val DEFAULT_MEMORY = 512 // mb private val DEFAULT_CORES = 1.0 - private val nextDriverNumber: AtomicLong = new AtomicLong(0) + private val nextDriverNumber = new AtomicLong(0) private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs private def newDriverId(submitDate: Date): String = { "driver-%s-%04d".format( @@ -93,9 +96,9 @@ private[deploy] class MesosSubmitRequestServlet( val appArgs = request.appArgs val environmentVariables = request.environmentVariables val name = request.sparkProperties.get("spark.app.name").getOrElse(mainClass) + // Construct driver description - val conf = new SparkConf(false) - .setAll(sparkProperties) + val conf = new SparkConf(false).setAll(sparkProperties) val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) @@ -106,7 +109,6 @@ private[deploy] class MesosSubmitRequestServlet( val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) - val submitDate = new Date() val submissionId = newDriverId(submitDate) @@ -149,7 +151,7 @@ private[deploy] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, private[deploy] class MesosStatusRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) extends StatusRequestServlet { protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { - val d = scheduler.getStatus(submissionId) + val d = scheduler.getDriverStatus(submissionId) d.serverSparkVersion = sparkVersion d } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index c1a8c00416499..26afa80b607be 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -20,6 +20,9 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File import java.util.{Collections, List => JList} +import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, HashSet} + import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.spark.scheduler.TaskSchedulerImpl @@ -27,9 +30,6 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.{AkkaUtils, Utils} import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskState} -import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, HashSet} - /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever @@ -78,7 +78,7 @@ private[spark] class CoarseMesosSchedulerBackend( override def start() { super.start() val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() - startScheduler("CoarseMesosSchedulerBackend", master, CoarseMesosSchedulerBackend.this, fwInfo) + startScheduler(master, CoarseMesosSchedulerBackend.this, fwInfo) } def createCommand(offer: Offer, numCores: Int): CommandInfo = { @@ -239,7 +239,8 @@ private[spark] class CoarseMesosSchedulerBackend( "is Spark installed on it?") } } - driver.reviveOffers() // In case we'd rejected everything before but have now lost a node + // In case we'd rejected everything before but have now lost a node + mesosDriver.reviveOffers() } } } @@ -251,8 +252,8 @@ private[spark] class CoarseMesosSchedulerBackend( override def stop() { super.stop() - if (driver != null) { - driver.stop() + if (mesosDriver != null) { + mesosDriver.stop() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index fe30f94ff2e28..1532aaba42ed5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -21,6 +21,10 @@ import java.io.File import java.util.concurrent.locks.ReentrantLock import java.util.{Collections, Date, List => JList} +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + import org.apache.mesos.Protos.Environment.Variable import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} @@ -31,31 +35,27 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.Utils import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState} -import scala.collection.JavaConversions._ -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - /** * Tracks the current state of a Mesos Task that runs a Spark driver. - * @param submission Submitted driver description from - * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] + * @param driverDescription Submitted driver description from + * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] * @param taskId Mesos TaskID generated for the task * @param slaveId Slave ID that the task is assigned to - * @param taskState The last known task status update. + * @param mesosTaskStatus The last known task status update. * @param startDate The date the task was launched */ -private[spark] class MesosClusterTaskState( - val submission: MesosDriverDescription, +private[spark] class MesosClusterSubmissionState( + val driverDescription: MesosDriverDescription, val taskId: TaskID, val slaveId: SlaveID, - var taskState: Option[TaskStatus], + var mesosTaskStatus: Option[TaskStatus], var startDate: Date) extends Serializable { - def copy(): MesosClusterTaskState = { - new MesosClusterTaskState( - submission, taskId, slaveId, taskState, startDate) + def copy(): MesosClusterSubmissionState = { + new MesosClusterSubmissionState( + driverDescription, taskId, slaveId, mesosTaskStatus, startDate) } } @@ -64,17 +64,17 @@ private[spark] class MesosClusterTaskState( * and necessary information to do exponential backoff. * This class is not thread-safe, and we expect the caller to handle synchronizing state. * @param lastFailureStatus Last Task status when it failed. - * @param retries Number of times it has retried. - * @param nextRetry Next retry time to be scheduled. + * @param retries Number of times it has been retried. + * @param nextRetry Time at which it should be retried next * @param waitTime The amount of time driver is scheduled to wait until next retry. */ -private[spark] class RetryState( +private[spark] class MesosClusterRetryState( val lastFailureStatus: TaskStatus, val retries: Int, val nextRetry: Date, val waitTime: Int) extends Serializable { - def copy(): RetryState = - new RetryState(lastFailureStatus, retries, nextRetry, waitTime) + def copy(): MesosClusterRetryState = + new MesosClusterRetryState(lastFailureStatus, retries, nextRetry, waitTime) } /** @@ -85,15 +85,15 @@ private[spark] class RetryState( * @param queuedDrivers All drivers queued to be launched * @param launchedDrivers All launched or running drivers * @param finishedDrivers All terminated drivers - * @param retryList All drivers pending to be retried + * @param pendingRetryDrivers All drivers pending to be retried */ private[spark] class MesosClusterSchedulerState( val frameworkId: String, val masterUrl: Option[String], val queuedDrivers: Iterable[MesosDriverDescription], - val launchedDrivers: Iterable[MesosClusterTaskState], - val finishedDrivers: Iterable[MesosClusterTaskState], - val retryList: Iterable[MesosDriverDescription]) + val launchedDrivers: Iterable[MesosClusterSubmissionState], + val finishedDrivers: Iterable[MesosClusterSubmissionState], + val pendingRetryDrivers: Iterable[MesosDriverDescription]) /** * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode @@ -109,36 +109,36 @@ private[spark] class MesosClusterScheduler( conf: SparkConf) extends Scheduler with MesosSchedulerUtils { var frameworkUrl: String = _ - private val metricsSystem = MetricsSystem.createMetricsSystem("mesos_cluster", conf, new SecurityManager(conf)) private val master = conf.get("spark.master") private val appName = conf.get("spark.app.name") - private val queuedCapacity = conf.getInt("spark.deploy.mesos.queuedDrivers", 200) - private val retainedDrivers = conf.getInt("spark.deploy.retainedDrivers", 200) + private val queuedCapacity = conf.getInt("spark.mesos.maxDrivers", 200) + private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200) private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute private val schedulerState = engineFactory.createEngine("scheduler") private val stateLock = new ReentrantLock() - private val finishedDrivers = new mutable.ArrayBuffer[MesosClusterTaskState](retainedDrivers) + private val finishedDrivers = + new mutable.ArrayBuffer[MesosClusterSubmissionState](retainedDrivers) private var frameworkId: String = null - val launchedDrivers = new mutable.HashMap[String, MesosClusterTaskState]() - // Holds the list of tasks that needs to reconcile with Mesos master. - // All states that are loaded after failover are added here. + // Holds all the launched drivers and current launch state, keyed by driver id. + private val launchedDrivers = new mutable.HashMap[String, MesosClusterSubmissionState]() + // Holds a map of driver id to expected slave id that is passed to Mesos for reconciliation. + // All drivers that are loaded after failover are added here, as we need get the latest + // state of the tasks from Mesos. private val pendingRecover = new mutable.HashMap[String, SlaveID]() - // A queue that stores all the submitted drivers that hasn't been launched. - val queuedDrivers = new ArrayBuffer[MesosDriverDescription]() + // Stores all the submitted drivers that hasn't been launched. + private val queuedDrivers = new ArrayBuffer[MesosDriverDescription]() // All supervised drivers that are waiting to retry after termination. - val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]() + private val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]() private val queuedDriversState = engineFactory.createEngine("driverQueue") private val launchedDriversState = engineFactory.createEngine("launchedDrivers") private val pendingRetryDriversState = engineFactory.createEngine("retryList") // Flag to mark if the scheduler is ready to be called, which is until the scheduler // is registered with Mesos master. - protected var ready = false + @volatile protected var ready = false private var masterInfo: Option[MasterInfo] = None - private def isQueueFull(): Boolean = launchedDrivers.size >= queuedCapacity - def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { val c = new CreateSubmissionResponse if (!ready) { @@ -151,6 +151,7 @@ private[spark] class MesosClusterScheduler( if (isQueueFull()) { c.success = false c.message = "Already reached maximum submission size" + return c } c.submissionId = desc.submissionId queuedDrivers += desc @@ -159,37 +160,6 @@ private[spark] class MesosClusterScheduler( c } - private def removeFromQueuedDrivers(id: String): Boolean = { - val index = queuedDrivers.indexWhere(_.submissionId.equals(id)) - if (index != -1) { - queuedDrivers.remove(index) - queuedDriversState.expunge(id) - true - } else { - false - } - } - - private def removeFromLaunchedDrivers(id: String): Boolean = { - if (launchedDrivers.remove(id).isDefined) { - launchedDriversState.expunge(id) - true - } else { - false - } - } - - private def removeFromPendingRetryDrivers(id: String): Boolean = { - val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(id)) - if (index != -1) { - pendingRetryDrivers.remove(index) - pendingRetryDriversState.expunge(id) - true - } else { - false - } - } - def killDriver(submissionId: String): KillSubmissionResponse = { val k = new KillSubmissionResponse if (!ready) { @@ -206,7 +176,7 @@ private[spark] class MesosClusterScheduler( // 4. Check if it has already completed. if (launchedDrivers.contains(submissionId)) { val task = launchedDrivers(submissionId) - driver.killTask(task.taskId) + mesosDriver.killTask(task.taskId) k.success = true k.message = "Killing running driver" } else if (removeFromQueuedDrivers(submissionId)) { @@ -214,8 +184,8 @@ private[spark] class MesosClusterScheduler( k.message = "Removed driver while it's still pending" } else if (removeFromPendingRetryDrivers(submissionId)) { k.success = true - k.message = "Removed driver while it's retrying" - } else if (finishedDrivers.exists(_.submission.submissionId.equals(submissionId))) { + k.message = "Removed driver while it's being retried" + } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) { k.success = false k.message = "Driver already terminated" } else { @@ -226,6 +196,44 @@ private[spark] class MesosClusterScheduler( k } + def getDriverStatus(submissionId: String): SubmissionStatusResponse = { + val s = new SubmissionStatusResponse + if (!ready) { + s.success = false + s.message = "Scheduler is not ready to take requests" + return s + } + s.submissionId = submissionId + stateLock.synchronized { + if (queuedDrivers.exists(_.submissionId.equals(submissionId))) { + s.success = true + s.driverState = "QUEUD" + } else if (launchedDrivers.contains(submissionId)) { + s.success = true + s.driverState = "RUNNING" + launchedDrivers(submissionId).mesosTaskStatus.foreach(state => s.message = state.toString) + } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) { + s.success = true + s.driverState = "FINISHED" + finishedDrivers + .find(d => d.driverDescription.submissionId.equals(submissionId)).get.mesosTaskStatus + .foreach(state => s.message = state.toString) + } else if (pendingRetryDrivers.exists(_.submissionId.equals(submissionId))) { + val status = pendingRetryDrivers.find(_.submissionId.equals(submissionId)) + .get.retryState.get.lastFailureStatus + s.success = true + s.driverState = "RETRYING" + s.message = status.toString + } else { + s.success = false + s.driverState = "NOT_FOUND" + } + } + s + } + + private def isQueueFull(): Boolean = launchedDrivers.size >= queuedCapacity + /** * Recover scheduler state that is persisted. * We still need to do task reconciliation to be up to date of the latest task states @@ -233,7 +241,7 @@ private[spark] class MesosClusterScheduler( */ private def recoverState(): Unit = { stateLock.synchronized { - launchedDriversState.fetchAll[MesosClusterTaskState]().foreach { state => + launchedDriversState.fetchAll[MesosClusterSubmissionState]().foreach { state => launchedDrivers(state.taskId.getValue) = state pendingRecover(state.taskId.getValue) = state.slaveId } @@ -274,8 +282,7 @@ private[spark] class MesosClusterScheduler( recoverState() metricsSystem.registerSource(new MesosClusterSchedulerSource(this)) metricsSystem.start() - startScheduler( - "MesosClusterScheduler", master, MesosClusterScheduler.this, builder.build()) + startScheduler(master, MesosClusterScheduler.this, builder.build()) ready = true } @@ -283,7 +290,7 @@ private[spark] class MesosClusterScheduler( ready = false metricsSystem.report() metricsSystem.stop() - driver.stop(true) + mesosDriver.stop(true) } override def registered( @@ -308,7 +315,7 @@ private[spark] class MesosClusterScheduler( .setSlaveId(slaveId) .setState(MesosTaskState.TASK_STAGING) .build() - launchedDrivers.get(taskId).map(_.taskState.getOrElse(newStatus)) + launchedDrivers.get(taskId).map(_.mesosTaskStatus.getOrElse(newStatus)) .getOrElse(newStatus) } // TODO: Page the status updates to avoid trying to reconcile @@ -387,17 +394,15 @@ private[spark] class MesosClusterScheduler( } /** - * This method takes all the possible candidates provided by the tasksFunc - * and attempt to schedule them with Mesos offers. - * Every time a new task is scheduled, the scheduledCallback is called to - * perform post scheduled logic on each task. + * This method takes all the possible candidates and attempt to schedule them with Mesos offers. + * Every time a new task is scheduled, the afterLaunchCallback is called to perform post scheduled + * logic on each task. */ private def scheduleTasks( candidates: Seq[MesosDriverDescription], - scheduledCallback: (String) => Boolean, + afterLaunchCallback: (String) => Boolean, currentOffers: List[ResourceOffer], - tasks: mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]], - currentTime: Date): Unit = { + tasks: mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]): Unit = { for (submission <- candidates) { val driverCpu = submission.cores val driverMem = submission.mem @@ -406,7 +411,7 @@ private[spark] class MesosClusterScheduler( o.cpu >= driverCpu && o.mem >= driverMem } if (offerOption.isEmpty) { - logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId}," + + logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId}, " + s"cpu: $driverCpu, mem: $driverMem") } else { val offer = offerOption.get @@ -429,21 +434,15 @@ private[spark] class MesosClusterScheduler( .addResources(cpuResource) .addResources(memResource) .build() - val queuedTasks = if (!tasks.contains(offer.offer.getId)) { - val buffer = new ArrayBuffer[TaskInfo] - tasks(offer.offer.getId) = buffer - buffer - } else { - tasks(offer.offer.getId) - } + val queuedTasks = tasks.getOrElseUpdate(offer.offer.getId, new ArrayBuffer[TaskInfo]) queuedTasks += taskInfo logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + submission.submissionId) - val newState = new MesosClusterTaskState(submission, taskId, offer.offer.getSlaveId, + val newState = new MesosClusterSubmissionState(submission, taskId, offer.offer.getSlaveId, None, new Date()) launchedDrivers(submission.submissionId) = newState launchedDriversState.persist(submission.submissionId, newState) - scheduledCallback(submission.submissionId) + afterLaunchCallback(submission.submissionId) } } } @@ -452,32 +451,31 @@ private[spark] class MesosClusterScheduler( /** * Prints the list of Mesos offers for logging purpose. */ - def printOffers(offers: Iterable[ResourceOffer]): String = { - val builder = new StringBuilder() - offers.foreach { o => - builder.append(o).append("\n") - } - builder.toString() - } val currentOffers = offers.map { o => new ResourceOffer( o, getResource(o.getResourcesList, "cpus"), getResource(o.getResourcesList, "mem")) }.toList - logTrace(s"Received offers from Mesos: \n${printOffers(currentOffers)}") + logTrace(s"Received offers from Mesos: \n${currentOffers.mkString("\n")}") val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() val currentTime = new Date() stateLock.synchronized { // We first schedule all the supervised drivers that are ready to retry. // This list will be empty if none of the drivers are marked as supervise. - scheduleTasks(pendingRetryDrivers - .filter(d => d.retryState.get.nextRetry.before(currentTime)), + val driversToRetry = pendingRetryDrivers.filter { d => + d.retryState.get.nextRetry.before(currentTime) + } + scheduleTasks( + driversToRetry, removeFromPendingRetryDrivers, - currentOffers, tasks, currentTime) + currentOffers, + tasks) // Then we walk through the queued drivers and try to schedule them. - scheduleTasks(queuedDrivers, + scheduleTasks( + queuedDrivers, removeFromQueuedDrivers, - currentOffers, tasks, currentTime) + currentOffers, + tasks) } tasks.foreach { case (offerId, tasks) => driver.launchTasks(Collections.singleton(offerId), tasks) @@ -487,9 +485,9 @@ private[spark] class MesosClusterScheduler( .foreach(o => driver.declineOffer(o.getId)) } - def getState(): MesosClusterSchedulerState = { + def getSchedulerState(): MesosClusterSchedulerState = { def copyBuffer( - buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { + buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) buffer.copyToBuffer(newBuffer) newBuffer @@ -499,61 +497,28 @@ private[spark] class MesosClusterScheduler( frameworkId, masterInfo.map(m => s"http://${m.getIp}:${m.getPort}"), copyBuffer(queuedDrivers), - launchedDrivers.values.map(_.copy).toList, - finishedDrivers.map(_.copy).toList, + launchedDrivers.values.map(_.copy()).toList, + finishedDrivers.map(_.copy()).toList, copyBuffer(pendingRetryDrivers)) } } - def getStatus(submissionId: String): SubmissionStatusResponse = { - val s = new SubmissionStatusResponse - if (!ready) { - s.success = false - s.message = "Scheduler is not ready to take requests" - return s - } - s.submissionId = submissionId - stateLock.synchronized { - if (queuedDrivers.exists(_.submissionId.equals(submissionId))) { - s.success = true - s.driverState = "Driver is queued for launch" - } else if (launchedDrivers.contains(submissionId)) { - s.success = true - s.driverState = "Driver is running" - launchedDrivers(submissionId).taskState.foreach(state => s.message = state.toString) - } else if (finishedDrivers.exists(s => s.submission.submissionId.equals(submissionId))) { - s.success = true - s.driverState = "Driver already finished" - finishedDrivers.find(d => d.submission.submissionId.equals(submissionId)).get.taskState - .foreach(state => s.message = state.toString) - } else if (pendingRetryDrivers.exists(_.submissionId.equals(submissionId))) { - val status = pendingRetryDrivers.find(_.submissionId.equals(submissionId)) - .get.retryState.get.lastFailureStatus - s.success = true - s.driverState = "Driver failed and retrying" - s.message = status.toString - } else { - s.success = false - s.driverState = "Driver not found" - } - } - s - } - override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {} override def disconnected(driver: SchedulerDriver): Unit = {} override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = { logInfo(s"Framework re-registered with master ${masterInfo.getId}") } override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {} - override def error(driver: SchedulerDriver, error: String): Unit = {} + override def error(driver: SchedulerDriver, error: String): Unit = { + logError("Error reveived: " + error) + } /** * Check if the task state is a recoverable state that we can relaunch the task. * Task state like TASK_ERROR are not relaunchable state since it wasn't able * to be validated by Mesos. */ - def shouldRelaunch(state: MesosTaskState): Boolean = { + private def shouldRelaunch(state: MesosTaskState): Boolean = { state == MesosTaskState.TASK_FAILED || state == MesosTaskState.TASK_KILLED || state == MesosTaskState.TASK_LOST @@ -570,19 +535,16 @@ private[spark] class MesosClusterScheduler( } val state = launchedDrivers(taskId) // Check if the driver is supervise enabled and can be relaunched. - if (state.submission.supervise && shouldRelaunch(status.getState)) { + if (state.driverDescription.supervise && shouldRelaunch(status.getState)) { removeFromLaunchedDrivers(taskId) - val retryState: Option[RetryState] = state.submission.retryState - val (retries, waitTimeSec) = if (retryState.isDefined) { - (retryState.get.retries + 1, - Math.min(maxRetryWaitTime, retryState.get.waitTime * 2)) - } else { - (1, 1) - } + val retryState: Option[MesosClusterRetryState] = state.driverDescription.retryState + val (retries, waitTimeSec) = retryState + .map { rs => (rs.retries + 1, Math.min(maxRetryWaitTime, rs.waitTime * 2)) } + .getOrElse{ (1, 1) } val nextRetry = new Date(new Date().getTime + waitTimeSec * 1000L) - val newDriverDescription = state.submission.copy( - retryState = Some(new RetryState(status, retries, nextRetry, waitTimeSec))) + val newDriverDescription = state.driverDescription.copy( + retryState = Some(new MesosClusterRetryState(status, retries, nextRetry, waitTimeSec))) pendingRetryDrivers += newDriverDescription pendingRetryDriversState.persist(taskId, newDriverDescription) } else if (TaskState.isFinished(TaskState.fromMesos(status.getState))) { @@ -593,7 +555,7 @@ private[spark] class MesosClusterScheduler( } finishedDrivers += state } - state.taskState = Option(status) + state.mesosTaskStatus = Option(status) } else { logError(s"Unable to find driver $taskId in status update") } @@ -611,4 +573,39 @@ private[spark] class MesosClusterScheduler( executorId: ExecutorID, slaveId: SlaveID, status: Int): Unit = {} + + private def removeFromQueuedDrivers(id: String): Boolean = { + val index = queuedDrivers.indexWhere(_.submissionId.equals(id)) + if (index != -1) { + queuedDrivers.remove(index) + queuedDriversState.expunge(id) + true + } else { + false + } + } + + private def removeFromLaunchedDrivers(id: String): Boolean = { + if (launchedDrivers.remove(id).isDefined) { + launchedDriversState.expunge(id) + true + } else { + false + } + } + + private def removeFromPendingRetryDrivers(id: String): Boolean = { + val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(id)) + if (index != -1) { + pendingRetryDrivers.remove(index) + pendingRetryDriversState.expunge(id) + true + } else { + false + } + } + + def getQueuedDriversSize: Int = queuedDrivers.size + def getLaunchedDriversSize: Int = launchedDrivers.size + def getPendingRetryDriversSize: Int = pendingRetryDrivers.size } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala index 4a9e1900bcb44..1fe94974c8e36 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala @@ -27,14 +27,14 @@ private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterSchedule override def metricRegistry: MetricRegistry = new MetricRegistry() metricRegistry.register(MetricRegistry.name("waitingDrivers"), new Gauge[Int] { - override def getValue: Int = scheduler.queuedDrivers.size + override def getValue: Int = scheduler.getQueuedDriversSize }) metricRegistry.register(MetricRegistry.name("launchedDrivers"), new Gauge[Int] { - override def getValue: Int = scheduler.launchedDrivers.size + override def getValue: Int = scheduler.getLaunchedDriversSize }) metricRegistry.register(MetricRegistry.name("retryDrivers"), new Gauge[Int] { - override def getValue: Int = scheduler.pendingRetryDrivers.size + override def getValue: Int = scheduler.getPendingRetryDriversSize }) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 41d77a329ce4b..5218591fba336 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -18,23 +18,19 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File -import java.util.{ArrayList => JArrayList, List => JList} -import java.util.Collections - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, HashSet} +import java.util.{ArrayList => JArrayList, Collections, List => JList} +import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} import org.apache.mesos.protobuf.ByteString -import org.apache.mesos.{Scheduler => MScheduler} -import org.apache.mesos._ -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, - ExecutorInfo => MesosExecutorInfo, _} - +import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.spark.executor.MesosExecutorBackend -import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} -import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils +import org.apache.spark.{SparkContext, SparkException, TaskState} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, HashSet} /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a @@ -68,7 +64,7 @@ private[spark] class MesosSchedulerBackend( override def start() { val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() classLoader = Thread.currentThread.getContextClassLoader - startScheduler("MesosSchedulerBackend", master, MesosSchedulerBackend.this, fwInfo) + startScheduler(master, MesosSchedulerBackend.this, fwInfo) } def createExecutorInfo(execId: String): MesosExecutorInfo = { @@ -298,13 +294,13 @@ private[spark] class MesosSchedulerBackend( } override def stop() { - if (driver != null) { - driver.stop() + if (mesosDriver != null) { + mesosDriver.stop() } } override def reviveOffers() { - driver.reviveOffers() + mesosDriver.reviveOffers() } override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} @@ -339,7 +335,7 @@ private[spark] class MesosSchedulerBackend( } override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = { - driver.killTask( + mesosDriver.killTask( TaskID.newBuilder() .setValue(taskId.toString).build() ) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 3e6e64cc634a6..d11228f3d016a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -36,23 +36,18 @@ private[mesos] trait MesosSchedulerUtils extends Logging { private final val registerLatch = new CountDownLatch(1) // Driver for talking to Mesos - protected var driver: MesosSchedulerDriver = null + protected var mesosDriver: MesosSchedulerDriver = null /** * Starts the MesosSchedulerDriver with the provided information. This method returns - * until the scheduler has registered with Mesos. - * @param name Name of the scheduler + * only after the scheduler has registered with Mesos. * @param masterUrl Mesos master connection URL * @param scheduler Scheduler object * @param fwInfo FrameworkInfo to pass to the Mesos master */ - def startScheduler( - name: String, - masterUrl: String, - scheduler: Scheduler, - fwInfo: FrameworkInfo): Unit = { + def startScheduler(masterUrl: String, scheduler: Scheduler, fwInfo: FrameworkInfo): Unit = { synchronized { - if (driver != null) { + if (mesosDriver != null) { registerLatch.await() return } @@ -61,9 +56,9 @@ private[mesos] trait MesosSchedulerUtils extends Logging { setDaemon(true) override def run() { - driver = new MesosSchedulerDriver(scheduler, fwInfo, masterUrl) + mesosDriver = new MesosSchedulerDriver(scheduler, fwInfo, masterUrl) try { - val ret = driver.run() + val ret = mesosDriver.run() logInfo("driver.run() returned with code " + ret) if (ret.equals(Status.DRIVER_ABORTED)) { System.exit(1) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosState.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosState.scala deleted file mode 100644 index 0b6b1e1cd9c7e..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosState.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.scheduler.cluster.mesos - -import org.apache.mesos.state._ -import scala.concurrent.duration.Duration -import java.util.concurrent._ -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.util.Utils -import java.lang.Boolean -import java.util -import java.util.Collections -import org.apache.mesos.MesosNativeLibrary -import org.apache.spark.deploy.SparkCuratorUtil - -/** - * An abstraction over Mesos state abstraction. This to provide automatic retries and - * serialization of objects, and translating state exceptions into Spark exceptions. - */ -private[spark] class MesosState(conf: SparkConf) { - val defaultFetchTimeoutMs = conf.getLong("spark.mesos.state.fetch.timeout.ms", 2000) - val defaultStoreTimeoutMs = conf.getLong("spark.mesos.state.store.timeout.ms", 5000) - val retries = conf.getInt("spark.mesos.state.store.retries", 3) - val quorum = conf.getInt("spark.mesos.state.quorum", 1) - val path = conf.get("spark.mesos.state.path", "/.spark_mesos_dispatcher") - private val ZK_SESSION_TIMEOUT_MILLIS = 60000 - - private val state = conf.get("spark.deploy.recoveryMode", "NONE").toUpperCase() match { - case "NONE" => new InMemoryState() - case "ZOOKEEPER" => { - val servers = conf.get("spark.deploy.zookeeper.url") - new ZooKeeperState( - servers, - ZK_SESSION_TIMEOUT_MILLIS, - TimeUnit.MILLISECONDS, - "/spark_mesos_dispatcher") - } - } - - assert(retries >= 0, s"Retries must be larger or equal than zero, retries: $retries") - - def fetch[T](name: String, timeout: Option[Duration] = None): Option[(Variable, T)] = { - val finalTimeout = - timeout.getOrElse(Duration.create(defaultFetchTimeoutMs, TimeUnit.MILLISECONDS)) - try { - val variable = state.fetch(name).get(finalTimeout.toMillis, TimeUnit.MILLISECONDS) - if (variable == null || variable.value().size == 0) { - None - } else { - Option((variable, Utils.deserialize(variable.value()).asInstanceOf[T])) - } - } catch { - case e: TimeoutException => - throw new SparkException(s"Timed out fetching $name, timeout: $finalTimeout") - case e: ExecutionException => - throw new SparkException(s"Failed to fetch $name, error: $e") - case e: CancellationException => - throw new SparkException("Fetch operation is discarded") - } - } - - def store[T]( - name: String, - variable: Variable, - value: T, - timeout: Option[Duration] = None): Variable = { - val finalTimeout = - timeout.getOrElse(Duration.create(defaultStoreTimeoutMs, TimeUnit.MILLISECONDS)) - val newVariable = variable.mutate(Utils.serialize(value)) - val future = state.store(newVariable) - var remainingRuns = retries + 1 - while (remainingRuns > 0) { - try { - future.get(finalTimeout.toMillis, TimeUnit.MILLISECONDS) - } catch { - case e: TimeoutException => - throw new SparkException(s"Timed out storing $name, timeout: $finalTimeout") - case e: ExecutionException => - throw new SparkException(s"Failed to storing $name, error: $e") - case e: CancellationException => - throw new SparkException("Store operation is discarded") - } - - val status = future.get() - if (status != null) { - status - } - remainingRuns -= 1 - } - - throw new SparkException(s"Unable to store variable $name after $retries retries") - } -} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 496d170e1e5c4..1029b0f9fce1e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -471,7 +471,6 @@ private[spark] object Utils extends Logging { in: InputStream, destFile: File, fileOverwrite: Boolean): Unit = { - logDebug(s"Creating temp file in ${destFile.getParentFile.getAbsolutePath}") val tempFile = File.createTempFile("fetchFileTemp", null, new File(destFile.getParentFile.getAbsolutePath)) logInfo(s"Fetching $url to $tempFile") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 52808cd713b4d..c4e6f06146b0a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -231,7 +231,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties val childArgsStr = childArgs.mkString(" ") if (useRest) { childArgsStr should endWith ("thejar.jar org.SomeClass arg1 arg2") - mainClass should be ("org.apache.spark.deploy.rest.RestClient") + mainClass should be ("org.apache.spark.deploy.rest.RestSubmissionClient") } else { childArgsStr should startWith ("--supervise --memory 4g --cores 5") childArgsStr should include regex "launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2" diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 3d7601a2c5fda..0a318a27ac212 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -567,12 +567,14 @@ private class FaultyStandaloneRestServer( masterActor: ActorRef, masterUrl: String) extends RestSubmissionServer(host, requestedPort, masterConf) { - val submitRequestServlet = new MalformedSubmitServlet - val killRequestServlet = new InvalidKillServlet - val statusRequestServlet = new ExplodingStatusServlet + + protected override val submitRequestServlet = new MalformedSubmitServlet + protected override val killRequestServlet = new InvalidKillServlet + protected override val statusRequestServlet = new ExplodingStatusServlet /** A faulty servlet that produces malformed responses. */ - class MalformedSubmitServlet extends StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) { + class MalformedSubmitServlet + extends StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) { protected override def sendResponse( responseMessage: SubmitRestProtocolResponse, responseServlet: HttpServletResponse): Unit = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala index ce790004eb377..f28e29e9b8d8e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -29,10 +29,8 @@ import org.apache.spark.{LocalSparkContext, SparkConf} class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar { - def createCommand: Command = { - new Command( - "mainClass", Seq("arg"), null, null, null, null) - } + + private val command = new Command("mainClass", Seq("arg"), null, null, null, null) test("can queue drivers") { val conf = new SparkConf() @@ -40,20 +38,21 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo conf.setAppName("spark mesos") val scheduler = new MesosClusterScheduler( new BlackHoleMesosClusterPersistenceEngineFactory, conf) { - override def start(): Unit = ready = true + override def start(): Unit = { ready = true } } scheduler.start() val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", 1000, 1, true, - createCommand, Map[String, String](), "s1", new Date())) + command, Map[String, String](), "s1", new Date())) assert(response.success) val response2 = scheduler.submitDriver(new MesosDriverDescription( - "d1", "jar", 1000, 1, true, createCommand, Map[String, String](), "s2", new Date())) + "d1", "jar", 1000, 1, true, command, Map[String, String](), "s2", new Date())) assert(response2.success) - val state = scheduler.getState() - assert(state.queuedDrivers.exists(d => d.submissionId == response.submissionId)) - assert(state.queuedDrivers.exists(d => d.submissionId == response2.submissionId)) + val state = scheduler.getSchedulerState() + val queuedDrivers = state.queuedDrivers.toList + assert(queuedDrivers(0).submissionId == response.submissionId) + assert(queuedDrivers(1).submissionId == response2.submissionId) } test("can kill queued drivers") { @@ -62,16 +61,16 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo conf.setAppName("spark mesos") val scheduler = new MesosClusterScheduler( new BlackHoleMesosClusterPersistenceEngineFactory, conf) { - override def start(): Unit = ready = true + override def start(): Unit = { ready = true } } scheduler.start() val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", 1000, 1, true, - createCommand, Map[String, String](), "s1", new Date())) + command, Map[String, String](), "s1", new Date())) assert(response.success) val killResponse = scheduler.killDriver(response.submissionId) assert(killResponse.success) - val state = scheduler.getState() + val state = scheduler.getSchedulerState() assert(state.queuedDrivers.isEmpty) } } diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 5a34b0baf3ad5..8f53d8201a089 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -151,10 +151,10 @@ Spark on Mesos also supports cluster mode, where the driver is launched in the c can find the results of the driver from the Mesos Web UI. To use cluster mode, you must start the MesosClusterDispatcher in your cluster via the `sbin/start-mesos-dispatcher.sh` script, -passing in the Mesos master url (ie: mesos://host:5050). +passing in the Mesos master url (e.g: mesos://host:5050). From the client, you can submit a job to Mesos cluster by running `spark-submit` and specifying the master url -to the url of the MesosClusterDispatcher (ie: mesos://dispatcher:7077). You can view driver statuses on the +to the url of the MesosClusterDispatcher (e.g: mesos://dispatcher:7077). You can view driver statuses on the Spark cluster Web UI. # Mesos Run Modes diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh index 67e6a15ec7bca..ef1fc573d5c65 100755 --- a/sbin/start-mesos-dispatcher.sh +++ b/sbin/start-mesos-dispatcher.sh @@ -17,7 +17,7 @@ # limitations under the License. # # Starts the Mesos Cluster Dispatcher on the machine this script is executed on. -# The Mesos Cluster Dispatcher is responsbile for launching the Mesos framework and +# The Mesos Cluster Dispatcher is responsible for launching the Mesos framework and # Rest server to handle driver requests for Mesos cluster mode. # Only one cluster dispatcher is needed per Mesos cluster. From e324ac1b06391f64224a6a5c85f1e6cb98079a62 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Fri, 17 Apr 2015 02:21:43 -0700 Subject: [PATCH 33/36] Fix merge. --- .../org/apache/spark/deploy/worker/DriverRunner.scala | 6 ++---- .../main/scala/org/apache/spark/deploy/worker/Worker.scala | 3 ++- .../scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 7 ++++--- .../org/apache/spark/deploy/worker/DriverRunnerTest.scala | 7 ++++--- 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e2bb42787a016..ef7a703bffe67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -44,7 +44,8 @@ private[deploy] class DriverRunner( val sparkHome: File, val driverDesc: DriverDescription, val worker: ActorRef, - val workerUrl: String) + val workerUrl: String, + val securityManager: SecurityManager) extends Logging { @volatile private var process: Option[Process] = None @@ -136,12 +137,9 @@ private[deploy] class DriverRunner( * Will throw an exception if there are errors downloading the jar. */ private def downloadUserJar(driverDir: File): String = { - val jarPath = new Path(driverDesc.jarUrl) val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val jarFileSystem = jarPath.getFileSystem(hadoopConf) - val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) val jarFileName = jarPath.getName val localJarFile = new File(driverDir, jarFileName) 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 c4c24a7866aa3..3ee2eb69e8a4e 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -436,7 +436,8 @@ private[worker] class Worker( sparkHome, driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), self, - akkaUrl) + akkaUrl, + securityMgr) drivers(driverId) = driver driver.start() 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 2071701b313db..b58d62567afe1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} class JsonProtocolSuite extends FunSuite { @@ -124,8 +124,9 @@ class JsonProtocolSuite extends FunSuite { } def createDriverRunner(): DriverRunner = { - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - createDriverDesc(), null, "akka://worker") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + createDriverDesc(), null, "akka://worker", new SecurityManager(conf)) } def assertValidJson(json: JValue) { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index aa6e4874cecde..2159fd8c16c6f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -25,7 +25,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.FunSuite -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.util.Clock @@ -33,8 +33,9 @@ class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - driverDescription, null, "akka://1.2.3.4/worker/") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + driverDescription, null, "akka://1.2.3.4/worker/", new SecurityManager(conf)) } private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = { From 390c491b8fccf913be7e3af8d7bd61d036e1b9e0 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Sat, 18 Apr 2015 22:54:58 -0700 Subject: [PATCH 34/36] Fix zk conf key for mesos zk engine. --- .../scala/org/apache/spark/deploy/SparkCuratorUtil.scala | 6 ++++-- .../cluster/mesos/MesosClusterPersistenceEngine.scala | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala index 52f4582107e85..b8d3993540220 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala @@ -32,8 +32,10 @@ private[spark] object SparkCuratorUtil extends Logging { private val RETRY_WAIT_MILLIS = 5000 private val MAX_RECONNECT_ATTEMPTS = 3 - def newClient(conf: SparkConf): CuratorFramework = { - val ZK_URL = conf.get("spark.deploy.zookeeper.url") + def newClient( + conf: SparkConf, + zkUrlConf: String = "spark.deploy.zookeeper.url"): CuratorFramework = { + val ZK_URL = conf.get(zkUrlConf) val zk = CuratorFrameworkFactory.newClient(ZK_URL, ZK_SESSION_TIMEOUT_MILLIS, ZK_CONNECTION_TIMEOUT_MILLIS, new ExponentialBackoffRetry(RETRY_WAIT_MILLIS, MAX_RECONNECT_ATTEMPTS)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala index 33ed264196b49..3efc536f1456c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -55,7 +55,7 @@ private[spark] trait MesosClusterPersistenceEngine { private[spark] class ZookeeperMesosClusterPersistenceEngineFactory(conf: SparkConf) extends MesosClusterPersistenceEngineFactory(conf) { - lazy val zk = SparkCuratorUtil.newClient(conf) + lazy val zk = SparkCuratorUtil.newClient(conf, "spark.mesos.deploy.zookeeper.url") def createEngine(path: String): MesosClusterPersistenceEngine = { new ZookeeperMesosClusterPersistenceEngine(path, zk, conf) From e24b512b413924af818637a3e27ea793684618da Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Sat, 18 Apr 2015 23:52:45 -0700 Subject: [PATCH 35/36] Persist submitted driver. --- .../spark/scheduler/cluster/mesos/MesosClusterScheduler.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 1532aaba42ed5..a6e14688dcec3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -154,6 +154,7 @@ private[spark] class MesosClusterScheduler( return c } c.submissionId = desc.submissionId + queuedDriversState.persist(desc.submissionId, desc) queuedDrivers += desc c.success = true } From 069e9463b377d1a0c7e4c35619e06213eae07fc2 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 21 Apr 2015 18:25:58 -0700 Subject: [PATCH 36/36] Fix rebase. --- .../apache/spark/deploy/rest/StandaloneRestServer.scala | 9 ++++----- .../scheduler/cluster/mesos/MesosClusterScheduler.scala | 1 - 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 1cb8433690752..502b9bb701ccf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -21,11 +21,10 @@ import java.io.File import javax.servlet.http.HttpServletResponse import akka.actor.ActorRef - -import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} -import org.apache.spark.util.{AkkaUtils, RpcUtils, Utils} -import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ +import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} +import org.apache.spark.util.{AkkaUtils, RpcUtils, Utils} +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} /** * A server that responds to requests submitted by the [[RestSubmissionClient]]. @@ -176,7 +175,7 @@ private[rest] class StandaloneSubmitRequestServlet( responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { requestMessage match { case submitRequest: CreateSubmissionRequest => - val askTimeout = AkkaUtils.askTimeout(conf) + val askTimeout = RpcUtils.askTimeout(conf) val driverDescription = buildDriverDescription(submitRequest) val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( DeployMessages.RequestSubmitDriver(driverDescription), masterActor, askTimeout) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index a6e14688dcec3..1b14b89d35008 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -253,7 +253,6 @@ private[spark] class MesosClusterScheduler( // and remove if they're already launched. queuedDrivers .filter(d => launchedDrivers.contains(d.submissionId)) - .toSeq .foreach(d => removeFromQueuedDrivers(d.submissionId)) pendingRetryDriversState.fetchAll[MesosDriverDescription]() .foreach(s => pendingRetryDrivers += s)