Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ private[spark] class ApplicationDescription(
// short name of compression codec used when writing event logs, if any (e.g. lzf)
val eventLogCodec: Option[String] = None,
val coresPerExecutor: Option[Int] = None)
extends Serializable {
extends Description with Serializable {

val user = System.getProperty("user.name", "<unknown>")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import org.apache.spark.deploy.ExecutorState.ExecutorState
import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
import org.apache.spark.deploy.master.DriverState.DriverState
import org.apache.spark.deploy.master.RecoveryState.MasterState
import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
import org.apache.spark.deploy.worker.{DriverRunnerInfo, ExecutorRunnerInfo}
import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -188,8 +188,8 @@ private[deploy] object DeployMessages {
// Worker to WorkerWebUI

case class WorkerStateResponse(host: String, port: Int, workerId: String,
executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner],
drivers: List[DriverRunner], finishedDrivers: List[DriverRunner], masterUrl: String,
executors: List[ExecutorRunnerInfo], finishedExecutors: List[ExecutorRunnerInfo],
drivers: List[DriverRunnerInfo], finishedDrivers: List[DriverRunnerInfo], masterUrl: String,
cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) {

Utils.checkHost(host, "Required hostname")
Expand Down
22 changes: 22 additions & 0 deletions core/src/main/scala/org/apache/spark/deploy/Description.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
/*
* 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

trait Description {
def command: Command
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,21 +17,12 @@

package org.apache.spark.deploy

private[deploy] class DriverDescription(
val jarUrl: String,
val mem: Int,
val cores: Int,
val supervise: Boolean,
val command: Command)
extends Serializable {

def copy(
jarUrl: String = jarUrl,
mem: Int = mem,
cores: Int = cores,
supervise: Boolean = supervise,
command: Command = command): DriverDescription =
new DriverDescription(jarUrl, mem, cores, supervise, command)
private[deploy] case class DriverDescription(
jarUrl: String,
mem: Int,
cores: Int,
supervise: Boolean,
command: Command) extends Description {

override def toString: String = s"DriverDescription (${command.mainClass})"
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,11 @@ package org.apache.spark.deploy
* This state is sufficient for the Master to reconstruct its internal data structures during
* failover.
*/
private[deploy] class ExecutorDescription(
val appId: String,
val execId: Int,
val cores: Int,
val state: ExecutorState.Value)
extends Serializable {
private[deploy] case class ExecutorDescription(
appId: String,
execId: Int,
cores: Int,
state: ExecutorState.Value) {

override def toString: String =
"ExecutorState(appId=%s, execId=%d, cores=%d, state=%s)".format(appId, execId, cores, state)
Expand Down
10 changes: 5 additions & 5 deletions core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.json4s.JsonDSL._

import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
import org.apache.spark.deploy.worker.ExecutorRunner
import org.apache.spark.deploy.worker.ExecutorRunnerInfo

private[deploy] object JsonProtocol {
def writeWorkerInfo(obj: WorkerInfo): JObject = {
Expand Down Expand Up @@ -60,11 +60,11 @@ private[deploy] object JsonProtocol {
("command" -> obj.command.toString)
}

def writeExecutorRunner(obj: ExecutorRunner): JObject = {
("id" -> obj.execId) ~
("memory" -> obj.memory) ~
def writeExecutorRunner(obj: ExecutorRunnerInfo): JObject = {
("id" -> obj.setup.id) ~
("memory" -> obj.setup.memory) ~
("appid" -> obj.appId) ~
("appdesc" -> writeApplicationDescription(obj.appDesc))
("appdesc" -> writeApplicationDescription(obj.setup.description))
}

def writeDriverInfo(obj: DriverInfo): JObject = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ class LocalSparkCluster(

/* Start the Workers */
for (workerNum <- 1 to numWorkers) {
val workerEnv = Worker.startRpcEnvAndEndpoint(localHostname, 0, 0, coresPerWorker,
val (workerEnv, _) = Worker.startRpcEnvAndEndpoint(localHostname, 0, 0, coresPerWorker,
memoryPerWorker, masters, null, Some(workerNum), _conf)
workerRpcEnvs += workerEnv
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* 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.worker

import org.apache.spark.deploy.{DriverDescription, Description, ApplicationDescription, ExecutorState}

private[deploy] trait ChildRunnerFactory[D <: Description, T <: ChildRunnerInfo[D]] {
def createRunner(
appId: Option[String],
processSetup: ChildProcessCommonSetup[D],
workerSetup: WorkerSetup,
stateChangeListener: StateChangeListener[D, T],
localDirs: Seq[String]): ChildProcessRunner[D, T]
}

private[deploy] object DriverRunnerFactoryImpl
extends ChildRunnerFactory[DriverDescription, DriverRunnerInfo] {

override def createRunner(
appId: Option[String],
processSetup: ChildProcessCommonSetup[DriverDescription],
workerSetup: WorkerSetup,
stateChangeListener: StateChangeListener[DriverDescription, DriverRunnerInfo],
localDirs: Seq[String]): DriverRunner = {

val manager = new DriverRunnerImpl(
processSetup,
workerSetup,
stateChangeListener)

manager
}
}

private[deploy] object ExecutorRunnerFactoryImpl
extends ChildRunnerFactory[ApplicationDescription, ExecutorRunnerInfo] {

override def createRunner(
appId: Option[String],
processSetup: ChildProcessCommonSetup[ApplicationDescription],
workerSetup: WorkerSetup,
stateChangeListener:
StateChangeListener[ApplicationDescription, ExecutorRunnerInfo],
localDirs: Seq[String]): ExecutorRunner = {

val manager = new ExecutorRunnerImpl(
processSetup,
workerSetup,
stateChangeListener,
appId.get,
localDirs,
ExecutorState.LOADING)

manager
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -24,29 +24,28 @@ import scala.collection.JavaConverters._
import com.google.common.base.Charsets.UTF_8
import com.google.common.io.Files
import org.apache.hadoop.fs.Path
import org.apache.spark.Logging

import org.apache.spark.{Logging, SparkConf, SecurityManager}
import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil}
import org.apache.spark.deploy.DeployMessages.DriverStateChanged
import org.apache.spark.deploy.master.DriverState
import org.apache.spark.deploy.master.DriverState.DriverState
import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.util.{Utils, Clock, SystemClock}
import org.apache.spark.util.{Clock, SystemClock, Utils}

/**
* Manages the execution of one driver, including automatically restarting the driver on failure.
* This is currently only used in standalone cluster deploy mode.
*/
private[deploy] class DriverRunner(
conf: SparkConf,
val driverId: String,
val workDir: File,
val sparkHome: File,
val driverDesc: DriverDescription,
val worker: RpcEndpointRef,
val workerUrl: String,
val securityManager: SecurityManager)
extends Logging {
private[deploy] class DriverRunnerImpl(
processSetup: ChildProcessCommonSetup[DriverDescription],
workerSetup: WorkerSetup,
stateChangeListener: StateChangeListener[DriverDescription, DriverRunnerInfo])
extends ChildProcessRunner[DriverDescription, DriverRunnerInfo]
with DriverRunnerInfo with Logging { self =>

override def state: DriverState = finalState.getOrElse(DriverState.RUNNING)
override def info: DriverRunnerImpl = this
override def exception: Option[Exception] = finalException
override def setup: ChildProcessCommonSetup[DriverDescription] = processSetup

@volatile private var process: Option[Process] = None
@volatile private var killed = false
Expand All @@ -71,23 +70,26 @@ private[deploy] class DriverRunner(
}

/** Starts a thread to run and manage the driver. */
private[worker] def start() = {
new Thread("DriverRunner for " + driverId) {
def start(): Unit = {
new Thread("DriverRunner for " + processSetup.id) {
override def run() {
try {
val driverDir = createWorkingDirectory()
val localJarFilename = downloadUserJar(driverDir)
val localJarFilename = downloadUserJar(processSetup.workDir)

def substituteVariables(argument: String): String = argument match {
case "{{WORKER_URL}}" => workerUrl
case "{{WORKER_URL}}" => workerSetup.workerUri
case "{{USER_JAR}}" => localJarFilename
case other => other
}

// TODO: If we add ability to submit multiple jars they should also be added here
val builder = CommandUtils.buildProcessBuilder(driverDesc.command, securityManager,
driverDesc.mem, sparkHome.getAbsolutePath, substituteVariables)
launchDriver(builder, driverDir, driverDesc.supervise)
val builder = CommandUtils.buildProcessBuilder(
processSetup.description.command,
workerSetup.securityManager,
processSetup.memory,
workerSetup.sparkHome.getAbsolutePath,
substituteVariables)
launchDriver(builder, processSetup.workDir, processSetup.description.supervise)
}
catch {
case e: Exception => finalException = Some(e)
Expand All @@ -107,39 +109,28 @@ private[deploy] class DriverRunner(

finalState = Some(state)

worker.send(DriverStateChanged(driverId, state, finalException))
stateChangeListener(self, None,
finalException orElse finalExitCode.filter(_ != 0).map(new NonZeroExitCodeException(_)))
}
}.start()
}

/** Terminate this driver (or prevent it from ever starting if not yet started) */
private[worker] def kill() {
def kill(): Unit = {
synchronized {
process.foreach(p => p.destroy())
killed = true
}
}

/**
* Creates the working directory for this driver.
* Will throw an exception if there are errors preparing the directory.
*/
private def createWorkingDirectory(): File = {
val driverDir = new File(workDir, driverId)
if (!driverDir.exists() && !driverDir.mkdirs()) {
throw new IOException("Failed to create directory " + driverDir)
}
driverDir
}

/**
* Download the user jar into the supplied directory and return its local path.
* Will throw an exception if there are errors downloading the jar.
*/
private def downloadUserJar(driverDir: File): String = {
val jarPath = new Path(driverDesc.jarUrl)
val jarPath = new Path(processSetup.description.jarUrl)

val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)
val hadoopConf = SparkHadoopUtil.get.newConfiguration(workerSetup.conf)
val destPath = new File(driverDir.getAbsolutePath, jarPath.getName)
val jarFileName = jarPath.getName
val localJarFile = new File(driverDir, jarFileName)
Expand All @@ -148,10 +139,10 @@ private[deploy] class DriverRunner(
if (!localJarFile.exists()) { // May already exist if running multiple workers on one node
logInfo(s"Copying user jar $jarPath to $destPath")
Utils.fetchFile(
driverDesc.jarUrl,
processSetup.description.jarUrl,
driverDir,
conf,
securityManager,
workerSetup.conf,
workerSetup.securityManager,
hadoopConf,
System.currentTimeMillis(),
useCache = false)
Expand Down Expand Up @@ -180,7 +171,7 @@ private[deploy] class DriverRunner(
runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise)
}

def runCommandWithRetry(
private[spark] def runCommandWithRetry(
command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean): Unit = {
// Time to wait between submission retries.
var waitSeconds = 1
Expand Down Expand Up @@ -232,3 +223,7 @@ private[deploy] object ProcessBuilderLike {
override def command: Seq[String] = processBuilder.command().asScala
}
}

private[deploy] trait DriverRunnerInfo extends ChildRunnerInfo[DriverDescription] {
def state: DriverState
}
Loading