Skip to content

Commit da20270

Browse files
committed
Merge pull request #1 from aarondav/driver
Refactor DriverClient to be more Actor-based
2 parents a97ad55 + 61372b1 commit da20270

File tree

1 file changed

+31
-62
lines changed

1 file changed

+31
-62
lines changed

core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala

+31-62
Original file line numberDiff line numberDiff line change
@@ -17,86 +17,43 @@
1717

1818
package org.apache.spark.deploy.client
1919

20-
import java.util.concurrent.TimeUnit
21-
22-
import scala.concurrent.Await
23-
import scala.concurrent.duration.{Duration, FiniteDuration}
20+
import scala.concurrent._
2421

2522
import akka.actor._
26-
import akka.actor.Actor.emptyBehavior
27-
import akka.pattern.ask
28-
import akka.remote.RemotingLifecycleEvent
2923

3024
import org.apache.spark.Logging
31-
import org.apache.spark.deploy.{DeployMessage, DriverDescription}
25+
import org.apache.spark.deploy.DriverDescription
3226
import org.apache.spark.deploy.DeployMessages._
3327
import org.apache.spark.deploy.master.Master
3428
import org.apache.spark.util.{AkkaUtils, Utils}
3529

3630
/**
37-
* Actor that sends a single message to the standalone master and then shuts down.
31+
* Actor that sends a single message to the standalone master and returns the response in the
32+
* given promise.
3833
*/
39-
private[spark] abstract class SingleMessageClient(
40-
actorSystem: ActorSystem, master: String, message: DeployMessage)
41-
extends Logging {
42-
43-
// Concrete child classes must implement
44-
def handleResponse(response: Any)
45-
46-
var actor: ActorRef = actorSystem.actorOf(Props(new DriverActor()))
47-
48-
class DriverActor extends Actor with Logging {
49-
override def preStart() {
50-
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
51-
logInfo("Sending message to master " + master + "...")
52-
val masterActor = context.actorSelection(Master.toAkkaUrl(master))
53-
val timeoutDuration: FiniteDuration = Duration.create(
54-
System.getProperty("spark.akka.askTimeout", "10").toLong, TimeUnit.SECONDS)
55-
val submitFuture = masterActor.ask(message)(timeoutDuration)
56-
handleResponse(Await.result(submitFuture, timeoutDuration))
57-
actorSystem.stop(actor)
58-
actorSystem.shutdown()
34+
class DriverActor(master: String, response: Promise[(Boolean, String)]) extends Actor with Logging {
35+
override def receive = {
36+
case SubmitDriverResponse(success, message) => {
37+
response.success((success, message))
5938
}
6039

61-
override def receive = emptyBehavior
62-
}
63-
}
64-
65-
/**
66-
* Submits a driver to the master.
67-
*/
68-
private[spark] class SubmissionClient(actorSystem: ActorSystem, master: String,
69-
driverDescription: DriverDescription)
70-
extends SingleMessageClient(actorSystem, master, RequestSubmitDriver(driverDescription)) {
71-
72-
override def handleResponse(response: Any) {
73-
val resp = response.asInstanceOf[SubmitDriverResponse]
74-
if (!resp.success) {
75-
logError(s"Error submitting driver to $master")
76-
logError(resp.message)
40+
case KillDriverResponse(success, message) => {
41+
response.success((success, message))
7742
}
78-
}
79-
}
8043

81-
/**
82-
* Terminates a client at the master.
83-
*/
84-
private[spark] class TerminationClient(actorSystem: ActorSystem, master: String, driverId: String)
85-
extends SingleMessageClient(actorSystem, master, RequestKillDriver(driverId)) {
86-
87-
override def handleResponse(response: Any) {
88-
val resp = response.asInstanceOf[KillDriverResponse]
89-
if (!resp.success) {
90-
logError(s"Error terminating $driverId at $master")
91-
logError(resp.message)
44+
// Relay all other messages to the server.
45+
case message => {
46+
logInfo(s"Sending message to master $master...")
47+
val masterActor = context.actorSelection(Master.toAkkaUrl(master))
48+
masterActor ! message
9249
}
9350
}
9451
}
9552

9653
/**
9754
* Executable utility for starting and terminating drivers inside of a standalone cluster.
9855
*/
99-
object DriverClient {
56+
object DriverClient extends Logging {
10057

10158
def main(args: Array[String]) {
10259
val driverArgs = new DriverClientArguments(args)
@@ -105,6 +62,9 @@ object DriverClient {
10562
// flow. Else, this (sadly) requires the DriverClient be routable from the Master.
10663
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
10764
"driverClient", Utils.localHostName(), 0)
65+
val master = driverArgs.master
66+
val response = promise[(Boolean, String)]
67+
val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response)))
10868

10969
driverArgs.cmd match {
11070
case "launch" =>
@@ -116,13 +76,22 @@ object DriverClient {
11676
driverArgs.driverOptions,
11777
driverArgs.driverJavaOptions,
11878
driverArgs.driverEnvVars)
119-
val client = new SubmissionClient(actorSystem, driverArgs.master, driverDescription)
79+
driver ! RequestSubmitDriver(driverDescription)
12080

12181
case "kill" =>
122-
val master = driverArgs.master
12382
val driverId = driverArgs.driverId
124-
val client = new TerminationClient(actorSystem, master, driverId)
83+
driver ! RequestKillDriver(driverId)
12584
}
85+
86+
val (success, message) =
87+
try {
88+
Await.result(response.future, AkkaUtils.askTimeout)
89+
} catch {
90+
case e: TimeoutException => (false, s"Master $master failed to respond in time")
91+
}
92+
if (success) logInfo(message) else logError(message)
93+
actorSystem.stop(driver)
94+
actorSystem.shutdown()
12695
actorSystem.awaitTermination()
12796
}
12897
}

0 commit comments

Comments
 (0)