-
Notifications
You must be signed in to change notification settings - Fork 29k
[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
Changes from 4 commits
fa1fa80
60d97a4
2b011c9
7a881b3
5d23958
e4f4ece
979760c
9d636be
35119a0
220cb3c
a41de0b
f4f972b
76fd411
2a28aab
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,24 +28,25 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} | |
| import org.apache.spark.deploy.DeployMessages._ | ||
| import org.apache.spark.deploy.master.{DriverState, Master} | ||
| import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} | ||
| import scala.collection.mutable.HashSet | ||
|
|
||
| /** | ||
| * Proxy that relays messages to the driver. | ||
| */ | ||
| private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) | ||
| extends Actor with ActorLogReceive with Logging { | ||
|
|
||
| var masterActor: ActorSelection = _ | ||
| val mastersActor = driverArgs.masters.map { m => | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would call this |
||
| context.actorSelection(Master.toAkkaUrl(m, AkkaUtils.protocol(context.system))) | ||
| } | ||
| val lostMasters = new HashSet[Address] | ||
| var activeMasterActor: ActorSelection = null | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you make these all |
||
|
|
||
| val timeout = AkkaUtils.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 | ||
|
|
@@ -79,11 +80,15 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) | |
| driverArgs.supervise, | ||
| command) | ||
|
|
||
| for (masterActor <- mastersActor) { | ||
| masterActor ! RequestSubmitDriver(driverDescription) | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think we want to submit to all masters right? I believe HA means we want to failover to backup Masters in case the primary one fails.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Never mind, it seems that we guard against this in this line and already do something similar in client mode. Can we add a comment here (and in L89) that says something like |
||
|
|
||
| case "kill" => | ||
| val driverId = driverArgs.driverId | ||
| for (masterActor <- mastersActor) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. need to indent these |
||
| masterActor ! RequestKillDriver(driverId) | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -92,14 +97,15 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) | |
| println(s"... 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") | ||
| System.exit(-1) | ||
| statusResponse.exception.getOrElse { | ||
| println(s"ERROR: Cluster master did not recognize $driverId") | ||
| System.exit(-1) | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This changes behavior. If |
||
| case true => | ||
| println(s"State of $driverId is ${statusResponse.state.get}") | ||
| // Worker node, if present | ||
|
|
@@ -115,27 +121,50 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) | |
| System.exit(-1) | ||
| } | ||
| System.exit(0) | ||
| } | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. unindent |
||
| } | ||
|
|
||
| override def receiveWithLogging: PartialFunction[Any, Unit] = { | ||
|
|
||
| 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 (!message.contains("Can only")) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you explain what "Can only" is?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah, I see, it's from this 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 (!message.contains("Can only")) { | ||
| 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 | ||
| if (lostMasters.size >= mastersActor.size) { | ||
| println(s"No master is available, exiting.") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No need for the "s" here |
||
| System.exit(-1) | ||
| } | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
||
|
|
||
| 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 >= mastersActor.size) { | ||
| println(s"No master is available, exiting.") | ||
| System.exit(-1) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -163,7 +192,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() | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,7 +35,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 | ||
|
|
@@ -80,13 +80,13 @@ private[deploy] class ClientArguments(args: Array[String]) { | |
| } | ||
|
|
||
| jarUrl = _jarUrl | ||
| master = _master | ||
| masters = _master.stripPrefix("spark://").split(",").map("spark://" + _) | ||
| mainClass = _mainClass | ||
| _driverOptions ++= tail | ||
|
|
||
| case "kill" :: _master :: _driverId :: tail => | ||
| cmd = "kill" | ||
| master = _master | ||
| masters = _master.stripPrefix("spark://").split(",").map("spark://" + _) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you put this in a |
||
| driverId = _driverId | ||
|
|
||
| case _ => | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
please reorganize the imports