Changes based on review feedback.

This commit is contained in:
Patrick Wendell 2014-01-02 18:10:37 -08:00
parent 7a99702ce2
commit f236ddd1a2
7 changed files with 34 additions and 22 deletions

View file

@ -62,7 +62,7 @@ object DriverClient extends Logging {
// TODO: See if we can initialize akka so return messages are sent back using the same TCP // TODO: See if we can initialize akka so return messages are sent back using the same TCP
// flow. Else, this (sadly) requires the DriverClient be routable from the Master. // flow. Else, this (sadly) requires the DriverClient be routable from the Master.
val (actorSystem, boundPort) = AkkaUtils.createActorSystem( val (actorSystem, _) = AkkaUtils.createActorSystem(
"driverClient", Utils.localHostName(), 0) "driverClient", Utils.localHostName(), 0)
val master = driverArgs.master val master = driverArgs.master
val response = promise[(Boolean, String)] val response = promise[(Boolean, String)]

View file

@ -23,6 +23,9 @@ import scala.collection.mutable.ListBuffer
* Command-line parser for the driver client. * Command-line parser for the driver client.
*/ */
private[spark] class DriverClientArguments(args: Array[String]) { private[spark] class DriverClientArguments(args: Array[String]) {
val defaultCores = 1
val defaultMemory = 512
var cmd: String = "" // 'launch' or 'kill' var cmd: String = "" // 'launch' or 'kill'
// launch parameters // launch parameters
@ -30,8 +33,8 @@ private[spark] class DriverClientArguments(args: Array[String]) {
var jarUrl: String = "" var jarUrl: String = ""
var mainClass: String = "" var mainClass: String = ""
var supervise: Boolean = false var supervise: Boolean = false
var memory: Int = 512 var memory: Int = defaultMemory
var cores: Int = 1 var cores: Int = defaultCores
private var _driverOptions = ListBuffer[String]() private var _driverOptions = ListBuffer[String]()
def driverOptions = _driverOptions.toSeq def driverOptions = _driverOptions.toSeq
@ -78,14 +81,17 @@ private[spark] class DriverClientArguments(args: Array[String]) {
def printUsageAndExit(exitCode: Int) { def printUsageAndExit(exitCode: Int) {
// TODO: It wouldn't be too hard to allow users to submit their app and dependency jars // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars
// separately similar to in the YARN client. // separately similar to in the YARN client.
System.err.println( val usage =
"usage: DriverClient [options] launch <active-master> <jar-url> <main-class> " + s"""
"[driver options]\n" + |Usage: DriverClient [options] launch <active-master> <jar-url> <main-class> [driver options]
"usage: DriverClient kill <active-master> <driver-id>\n\n" + |Usage: DriverClient kill <active-master> <driver-id>
"Options:\n" + |
" -c CORES, --cores CORES Number of cores to request \n" + |Options:
" -m MEMORY, --memory MEMORY Megabytes of memory to request\n" + | -c CORES, --cores CORES Number of cores to request (default: $defaultCores)
" -s, --supervise Whether to restart the driver on failure\n") | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory)
| -s, --supervise Whether to restart the driver on failure
""".stripMargin
System.err.println(usage)
System.exit(exitCode) System.exit(exitCode)
} }
} }

View file

@ -21,7 +21,7 @@ object CommandUtils extends Logging {
} }
private def getEnv(key: String, command: Command): Option[String] = private def getEnv(key: String, command: Command): Option[String] =
command.environment.get(key).orElse(Option(getenv(key))) command.environment.get(key).orElse(Option(System.getenv(key)))
/** /**
* Attention: this must always be aligned with the environment variables in the run scripts and * Attention: this must always be aligned with the environment variables in the run scripts and

View file

@ -119,15 +119,14 @@ private[spark] class DriverRunner(
val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path
val jarFileSystem = jarPath.getFileSystem(emptyConf) val jarFileSystem = jarPath.getFileSystem(emptyConf)
val destPath = new Path(driverDir.getAbsolutePath()) val destPath = new File(driverDir.getAbsolutePath, jarPath.getName)
val destFileSystem = destPath.getFileSystem(emptyConf)
val jarFileName = jarPath.getName val jarFileName = jarPath.getName
val localJarFile = new File(driverDir, jarFileName) val localJarFile = new File(driverDir, jarFileName)
val localJarFilename = localJarFile.getAbsolutePath val localJarFilename = localJarFile.getAbsolutePath
if (!localJarFile.exists()) { // May already exist if running multiple workers on one node if (!localJarFile.exists()) { // May already exist if running multiple workers on one node
logInfo(s"Copying user jar $jarPath to $destPath") logInfo(s"Copying user jar $jarPath to $destPath")
FileUtil.copy(jarFileSystem, jarPath, destFileSystem, destPath, false, false, emptyConf) FileUtil.copy(jarFileSystem, jarPath, destPath, false, emptyConf)
} }
if (!localJarFile.exists()) { // Verify copy succeeded if (!localJarFile.exists()) { // Verify copy succeeded
@ -140,8 +139,12 @@ private[spark] class DriverRunner(
/** Launch the supplied command. */ /** Launch the supplied command. */
private def runCommand(command: Seq[String], envVars: Map[String, String], baseDir: File, private def runCommand(command: Seq[String], envVars: Map[String, String], baseDir: File,
supervise: Boolean) { supervise: Boolean) {
// Time to wait between submission retries. // Time to wait between submission retries.
var waitSeconds = 1 var waitSeconds = 1
// A run of this many seconds resets the exponential back-off.
val successfulRunDuration = 1
var keepTrying = !killed var keepTrying = !killed
while (keepTrying) { while (keepTrying) {
@ -161,11 +164,15 @@ private[spark] class DriverRunner(
val stderr = new File(baseDir, "stderr") val stderr = new File(baseDir, "stderr")
val header = "Launch Command: %s\n%s\n\n".format( val header = "Launch Command: %s\n%s\n\n".format(
command.mkString("\"", "\" \"", "\""), "=" * 40) command.mkString("\"", "\" \"", "\""), "=" * 40)
Files.write(header, stderr, Charsets.UTF_8) Files.append(header, stderr, Charsets.UTF_8)
CommandUtils.redirectStream(process.get.getErrorStream, stderr) CommandUtils.redirectStream(process.get.getErrorStream, stderr)
} }
val processStart = System.currentTimeMillis()
val exitCode = process.get.waitFor() val exitCode = process.get.waitFor()
if (System.currentTimeMillis() - processStart > successfulRunDuration * 1000) {
waitSeconds = 1
}
if (supervise && exitCode != 0 && !killed) { if (supervise && exitCode != 0 && !killed) {
waitSeconds = waitSeconds * 2 // exponential back-off waitSeconds = waitSeconds * 2 // exponential back-off

View file

@ -11,7 +11,7 @@ object DriverWrapper {
def main(args: Array[String]) { def main(args: Array[String]) {
args.toList match { args.toList match {
case workerUrl :: mainClass :: extraArgs => case workerUrl :: mainClass :: extraArgs =>
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("Driver", val (actorSystem, _) = AkkaUtils.createActorSystem("Driver",
Utils.localHostName(), 0) Utils.localHostName(), 0)
actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher")

View file

@ -260,8 +260,8 @@ private[spark] class Worker(
case KillDriver(driverId) => { case KillDriver(driverId) => {
logInfo(s"Asked to kill driver $driverId") logInfo(s"Asked to kill driver $driverId")
drivers.find(_._1 == driverId) match { drivers.get(driverId) match {
case Some((id, runner)) => case Some(runner) =>
runner.kill() runner.kill()
case None => case None =>
logError(s"Asked to kill unknown driver $driverId") logError(s"Asked to kill unknown driver $driverId")
@ -280,8 +280,7 @@ private[spark] class Worker(
masterLock.synchronized { masterLock.synchronized {
master ! DriverStateChanged(driverId, state, exception) master ! DriverStateChanged(driverId, state, exception)
} }
val driver = drivers(driverId) val driver = drivers.remove(driverId).get
drivers -= driverId
finishedDrivers(driverId) = driver finishedDrivers(driverId) = driver
memoryUsed -= driver.driverDesc.mem memoryUsed -= driver.driverDesc.mem
coresUsed -= driver.driverDesc.cores coresUsed -= driver.driverDesc.cores

View file

@ -47,7 +47,7 @@ private[spark] class SparkDeploySchedulerBackend(
val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
CoarseGrainedSchedulerBackend.ACTOR_NAME) CoarseGrainedSchedulerBackend.ACTOR_NAME)
val args = Seq(driverUrl, "{{WORKER_URL}}", "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}")
val command = Command( val command = Command(
"org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
val sparkHome = sc.getSparkHome().getOrElse(null) val sparkHome = sc.getSparkHome().getOrElse(null)