Skip to content

[SPARK-6443][Spark Submit]Could not submit app in standalone cluster mode when HA is enabled #5116

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 14 commits into from
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
73 changes: 55 additions & 18 deletions core/src/main/scala/org/apache/spark/deploy/Client.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.deploy

import scala.collection.mutable.HashSet
import scala.concurrent._

import akka.actor._
Expand All @@ -31,21 +32,24 @@ import org.apache.spark.util.{ActorLogReceive, AkkaUtils, RpcUtils, Utils}

/**
* Proxy that relays messages to the driver.
*
* We currently don't support retry if submission fails. In HA mode, client will submit request to
* all masters and see which one could handle it.
*/
private class ClientActor(driverArgs: ClientArguments, conf: SparkConf)
extends Actor with ActorLogReceive with Logging {

var masterActor: ActorSelection = _
private val masterActors = driverArgs.masters.map { m =>
context.actorSelection(Master.toAkkaUrl(m, AkkaUtils.protocol(context.system)))
}
private val lostMasters = new HashSet[Address]
private var activeMasterActor: ActorSelection = null

val timeout = RpcUtils.askTimeout(conf)

override def preStart(): Unit = {
masterActor = context.actorSelection(
Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(context.system)))

context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])

println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}")

driverArgs.cmd match {
case "launch" =>
// TODO: We could add an env variable here and intercept it in `sc.addJar` that would
Expand Down Expand Up @@ -79,11 +83,17 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf)
driverArgs.supervise,
command)

masterActor ! RequestSubmitDriver(driverDescription)
// This assumes only one Master is active at a time
for (masterActor <- masterActors) {
masterActor ! RequestSubmitDriver(driverDescription)
}

case "kill" =>
val driverId = driverArgs.driverId
masterActor ! RequestKillDriver(driverId)
// This assumes only one Master is active at a time
for (masterActor <- masterActors) {
masterActor ! RequestKillDriver(driverId)
}
}
}

Expand All @@ -92,10 +102,9 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf)
println("... waiting before polling master for driver state")
Thread.sleep(5000)
println("... polling master for driver state")
val statusFuture = (masterActor ? RequestDriverStatus(driverId))(timeout)
val statusFuture = (activeMasterActor ? RequestDriverStatus(driverId))(timeout)
.mapTo[DriverStatusResponse]
val statusResponse = Await.result(statusFuture, timeout)

statusResponse.found match {
case false =>
println(s"ERROR: Cluster master did not recognize $driverId")
Expand All @@ -122,20 +131,46 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf)

case SubmitDriverResponse(success, driverId, message) =>
println(message)
if (success) pollAndReportStatus(driverId.get) else System.exit(-1)
if (success) {
activeMasterActor = context.actorSelection(sender.path)
pollAndReportStatus(driverId.get)
} else if (!Utils.responseFromBackup(message)) {
System.exit(-1)
}


case KillDriverResponse(driverId, success, message) =>
println(message)
if (success) pollAndReportStatus(driverId) else System.exit(-1)
if (success) {
activeMasterActor = context.actorSelection(sender.path)
pollAndReportStatus(driverId)
} else if (!Utils.responseFromBackup(message)) {
System.exit(-1)
}

case DisassociatedEvent(_, remoteAddress, _) =>
println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.")
System.exit(-1)
if (!lostMasters.contains(remoteAddress)) {
println(s"Error connecting to master $remoteAddress.")
lostMasters += remoteAddress
// Note that this heuristic does not account for the fact that a Master can recover within
// the lifetime of this client. Thus, once a Master is lost it is lost to us forever. This
// is not currently a concern, however, because this client does not retry submissions.
if (lostMasters.size >= masterActors.size) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This line needs a big comment:

// Note that this heuristic does not account for the fact that a Master can recover within
// the lifetime of this client. Thus, once a Master is lost it is lost to us forever. This
// is not currently a concern, however, because this client does not retry submissions.

println("No master is available, exiting.")
System.exit(-1)
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure if this is safe to do. A failed master can come back up, but according to this code it will still be in lostMasters. We need to somehow detect when a failed master has recovered.


case AssociationErrorEvent(cause, _, remoteAddress, _, _) =>
println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.")
println(s"Cause was: $cause")
System.exit(-1)
if (!lostMasters.contains(remoteAddress)) {
println(s"Error connecting to master ($remoteAddress).")
println(s"Cause was: $cause")
lostMasters += remoteAddress
if (lostMasters.size >= masterActors.size) {
println("No master is available, exiting.")
System.exit(-1)
}
}
}
}

Expand Down Expand Up @@ -163,7 +198,9 @@ object Client {
"driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf))

// Verify driverArgs.master is a valid url so that we can use it in ClientActor safely
Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(actorSystem))
for (m <- driverArgs.masters) {
Master.toAkkaUrl(m, AkkaUtils.protocol(actorSystem))
}
actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf))

actorSystem.awaitTermination()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,7 @@ import java.net.{URI, URISyntaxException}
import scala.collection.mutable.ListBuffer

import org.apache.log4j.Level

import org.apache.spark.util.{IntParam, MemoryParam}
import org.apache.spark.util.{IntParam, MemoryParam, Utils}

/**
* Command-line parser for the driver client.
Expand All @@ -35,7 +34,7 @@ private[deploy] class ClientArguments(args: Array[String]) {
var logLevel = Level.WARN

// launch parameters
var master: String = ""
var masters: Array[String] = null
var jarUrl: String = ""
var mainClass: String = ""
var supervise: Boolean = DEFAULT_SUPERVISE
Expand Down Expand Up @@ -80,13 +79,13 @@ private[deploy] class ClientArguments(args: Array[String]) {
}

jarUrl = _jarUrl
master = _master
masters = Utils.parseStandaloneMasterUrls(_master)
mainClass = _mainClass
_driverOptions ++= tail

case "kill" :: _master :: _driverId :: tail =>
cmd = "kill"
master = _master
masters = Utils.parseStandaloneMasterUrls(_master)
driverId = _driverId

case _ =>
Expand Down
8 changes: 4 additions & 4 deletions core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
Original file line number Diff line number Diff line change
Expand Up @@ -118,17 +118,17 @@ object SparkSubmit {
* Kill an existing submission using the REST protocol. Standalone and Mesos cluster mode only.
*/
private def kill(args: SparkSubmitArguments): Unit = {
new RestSubmissionClient()
.killSubmission(args.master, args.submissionToKill)
new RestSubmissionClient(args.master)
.killSubmission(args.submissionToKill)
}

/**
* Request the status of an existing submission using the REST protocol.
* Standalone and Mesos cluster mode only.
*/
private def requestStatus(args: SparkSubmitArguments): Unit = {
new RestSubmissionClient()
.requestSubmissionStatus(args.master, args.submissionToRequestStatusFor)
new RestSubmissionClient(args.master)
.requestSubmissionStatus(args.submissionToRequestStatusFor)
}

/**
Expand Down
24 changes: 16 additions & 8 deletions core/src/main/scala/org/apache/spark/deploy/master/Master.scala
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,8 @@ private[master] class Master(

case RequestSubmitDriver(description) => {
if (state != RecoveryState.ALIVE) {
val msg = s"Can only accept driver submissions in ALIVE state. Current state: $state."
val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " +
"Can only accept driver submissions in ALIVE state."
sender ! SubmitDriverResponse(false, None, msg)
} else {
logInfo("Driver submitted " + description.command.mainClass)
Expand All @@ -274,7 +275,8 @@ private[master] class Master(

case RequestKillDriver(driverId) => {
if (state != RecoveryState.ALIVE) {
val msg = s"Can only kill drivers in ALIVE state. Current state: $state."
val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " +
s"Can only kill drivers in ALIVE state."
sender ! KillDriverResponse(driverId, success = false, msg)
} else {
logInfo("Asked to kill driver " + driverId)
Expand Down Expand Up @@ -305,12 +307,18 @@ private[master] class Master(
}

case RequestDriverStatus(driverId) => {
(drivers ++ completedDrivers).find(_.id == driverId) match {
case Some(driver) =>
sender ! DriverStatusResponse(found = true, Some(driver.state),
driver.worker.map(_.id), driver.worker.map(_.hostPort), driver.exception)
case None =>
sender ! DriverStatusResponse(found = false, None, None, None, None)
if (state != RecoveryState.ALIVE) {
val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " +
"Can only request driver status in ALIVE state."
sender ! DriverStatusResponse(found = false, None, None, None, Some(new Exception(msg)))
} else {
(drivers ++ completedDrivers).find(_.id == driverId) match {
case Some(driver) =>
sender ! DriverStatusResponse(found = true, Some(driver.state),
driver.worker.map(_.id), driver.worker.map(_.hostPort), driver.exception)
case None =>
sender ! DriverStatusResponse(found = false, None, None, None, None)
}
}
}

Expand Down
Loading