-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-5124][Core] A standard RPC interface and an Akka implementation #4588
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
45b2317
155b987
2a579f4
04a106e
7b9e0c9
fe7d1ff
3751c97
28e6d0f
ffc1280
51e6667
7cdd95e
4d34191
07f128f
3e56123
5f87700
c425022
3007c09
9288406
7fc95e1
ec7c5b0
e5df4ca
08564ae
e8dfec3
2cc3f78
385b9c3
78a1733
9ffa997
15cfd7b
b221398
f459380
8bd1097
f6f3287
fe3df4c
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 |
|---|---|---|
|
|
@@ -34,8 +34,10 @@ import org.apache.spark.metrics.MetricsSystem | |
| import org.apache.spark.network.BlockTransferService | ||
| import org.apache.spark.network.netty.NettyBlockTransferService | ||
| import org.apache.spark.network.nio.NioBlockTransferService | ||
| import org.apache.spark.rpc.akka.AkkaRpcEnv | ||
| import org.apache.spark.rpc.{RpcEndpointRef, RpcEndpoint, RpcEnv} | ||
| import org.apache.spark.scheduler.{OutputCommitCoordinator, LiveListenerBus} | ||
| import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorActor | ||
| import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorEndpoint | ||
| import org.apache.spark.serializer.Serializer | ||
| import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} | ||
| import org.apache.spark.storage._ | ||
|
|
@@ -54,7 +56,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} | |
| @DeveloperApi | ||
| class SparkEnv ( | ||
| val executorId: String, | ||
| val actorSystem: ActorSystem, | ||
| val rpcEnv: RpcEnv, | ||
|
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. just curious, why changing the constructor's signature didn't break MIMA checking?
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. it's a developer api
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. BTW, not really related to this change; but I believe that while
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. Oh, and if code outside of Spark is not supposed to use |
||
| val serializer: Serializer, | ||
| val closureSerializer: Serializer, | ||
| val cacheManager: CacheManager, | ||
|
|
@@ -71,6 +73,9 @@ class SparkEnv ( | |
| val outputCommitCoordinator: OutputCommitCoordinator, | ||
| val conf: SparkConf) extends Logging { | ||
|
|
||
| // TODO Remove actorSystem | ||
| val actorSystem = rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem | ||
|
|
||
| private[spark] var isStopped = false | ||
| private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() | ||
|
|
||
|
|
@@ -91,7 +96,8 @@ class SparkEnv ( | |
| blockManager.master.stop() | ||
| metricsSystem.stop() | ||
| outputCommitCoordinator.stop() | ||
| actorSystem.shutdown() | ||
| rpcEnv.shutdown() | ||
|
|
||
| // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut | ||
| // down, but let's call it anyway in case it gets fixed in a later release | ||
| // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. | ||
|
|
@@ -236,16 +242,15 @@ object SparkEnv extends Logging { | |
| val securityManager = new SecurityManager(conf) | ||
|
|
||
| // Create the ActorSystem for Akka and get the port it binds to. | ||
| val (actorSystem, boundPort) = { | ||
| val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName | ||
| AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) | ||
| } | ||
| val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName | ||
|
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. Seems like this is still needed because the
Member
Author
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.
|
||
| val rpcEnv = RpcEnv.create(actorSystemName, hostname, port, conf, securityManager) | ||
| val actorSystem = rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem | ||
|
|
||
| // Figure out which port Akka actually bound to in case the original port is 0 or occupied. | ||
| if (isDriver) { | ||
| conf.set("spark.driver.port", boundPort.toString) | ||
| conf.set("spark.driver.port", rpcEnv.address.port.toString) | ||
| } else { | ||
| conf.set("spark.executor.port", boundPort.toString) | ||
| conf.set("spark.executor.port", rpcEnv.address.port.toString) | ||
| } | ||
|
|
||
| // Create an instance of the class with the given name, possibly initializing it with our conf | ||
|
|
@@ -290,6 +295,15 @@ object SparkEnv extends Logging { | |
| } | ||
| } | ||
|
|
||
| def registerOrLookupEndpoint(name: String, endpointCreator: => RpcEndpoint): RpcEndpointRef = { | ||
| if (isDriver) { | ||
| logInfo("Registering " + name) | ||
| rpcEnv.setupEndpoint(name, endpointCreator) | ||
| } else { | ||
| rpcEnv.setupDriverEndpointRef(name) | ||
| } | ||
| } | ||
|
|
||
| val mapOutputTracker = if (isDriver) { | ||
| new MapOutputTrackerMaster(conf) | ||
| } else { | ||
|
|
@@ -377,13 +391,13 @@ object SparkEnv extends Logging { | |
| val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse { | ||
| new OutputCommitCoordinator(conf) | ||
| } | ||
| val outputCommitCoordinatorActor = registerOrLookup("OutputCommitCoordinator", | ||
| new OutputCommitCoordinatorActor(outputCommitCoordinator)) | ||
| outputCommitCoordinator.coordinatorActor = Some(outputCommitCoordinatorActor) | ||
| val outputCommitCoordinatorRef = registerOrLookupEndpoint("OutputCommitCoordinator", | ||
| new OutputCommitCoordinatorEndpoint(rpcEnv, outputCommitCoordinator)) | ||
| outputCommitCoordinator.coordinatorRef = Some(outputCommitCoordinatorRef) | ||
|
|
||
| val envInstance = new SparkEnv( | ||
| executorId, | ||
| actorSystem, | ||
| rpcEnv, | ||
| serializer, | ||
| closureSerializer, | ||
| cacheManager, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import java.io.File | |
|
|
||
| import akka.actor._ | ||
|
|
||
| import org.apache.spark.rpc.RpcEnv | ||
|
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. nit: import order |
||
| import org.apache.spark.{SecurityManager, SparkConf} | ||
| import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLClassLoader, Utils} | ||
|
|
||
|
|
@@ -32,9 +33,9 @@ object DriverWrapper { | |
| args.toList match { | ||
| case workerUrl :: userJar :: mainClass :: extraArgs => | ||
| val conf = new SparkConf() | ||
| val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", | ||
| val rpcEnv = RpcEnv.create("Driver", | ||
| Utils.localHostName(), 0, conf, new SecurityManager(conf)) | ||
| actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") | ||
| rpcEnv.setupEndpoint("workerWatcher", new WorkerWatcher(rpcEnv, workerUrl)) | ||
|
|
||
| val currentLoader = Thread.currentThread.getContextClassLoader | ||
| val userJarUrl = new File(userJar).toURI().toURL() | ||
|
|
@@ -51,7 +52,7 @@ object DriverWrapper { | |
| val mainMethod = clazz.getMethod("main", classOf[Array[String]]) | ||
| mainMethod.invoke(null, extraArgs.toArray[String]) | ||
|
|
||
| actorSystem.shutdown() | ||
| rpcEnv.shutdown() | ||
|
|
||
| case _ => | ||
| System.err.println("Usage: DriverWrapper <workerUrl> <userJar> <driverMainClass> [options]") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,58 +17,66 @@ | |
|
|
||
| package org.apache.spark.deploy.worker | ||
|
|
||
| import akka.actor.{Actor, Address, AddressFromURIString} | ||
| import akka.remote.{AssociatedEvent, AssociationErrorEvent, AssociationEvent, DisassociatedEvent, RemotingLifecycleEvent} | ||
|
|
||
| import org.apache.spark.Logging | ||
| import org.apache.spark.deploy.DeployMessages.SendHeartbeat | ||
| import org.apache.spark.util.ActorLogReceive | ||
| import org.apache.spark.rpc._ | ||
|
|
||
| /** | ||
| * Actor which connects to a worker process and terminates the JVM if the connection is severed. | ||
| * Provides fate sharing between a worker and its associated child processes. | ||
| */ | ||
| private[spark] class WorkerWatcher(workerUrl: String) | ||
| extends Actor with ActorLogReceive with Logging { | ||
|
|
||
| override def preStart() { | ||
| context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) | ||
| private[spark] class WorkerWatcher(override val rpcEnv: RpcEnv, workerUrl: String) | ||
| extends NetworkRpcEndpoint with Logging { | ||
|
|
||
| override def onStart() { | ||
| logInfo(s"Connecting to worker $workerUrl") | ||
| val worker = context.actorSelection(workerUrl) | ||
| worker ! SendHeartbeat // need to send a message here to initiate connection | ||
| if (!isTesting) { | ||
| val worker = rpcEnv.setupEndpointRefByUrl(workerUrl) | ||
| worker.send(SendHeartbeat) // need to send a message here to initiate connection | ||
| } | ||
| } | ||
|
|
||
| // Used to avoid shutting down JVM during tests | ||
| // In the normal case, exitNonZero will call `System.exit(-1)` to shutdown the JVM. In the unit | ||
| // test, the user should call `setTesting(true)` so that `exitNonZero` will set `isShutDown` to | ||
| // true rather than calling `System.exit`. The user can check `isShutDown` to know if | ||
| // `exitNonZero` is called. | ||
| private[deploy] var isShutDown = false | ||
| private[deploy] def setTesting(testing: Boolean) = isTesting = testing | ||
| private var isTesting = false | ||
|
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 u document what this does?
Member
Author
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. Added some docs to explain it. |
||
|
|
||
| // Lets us filter events only from the worker's actor system | ||
| private val expectedHostPort = AddressFromURIString(workerUrl).hostPort | ||
| private def isWorker(address: Address) = address.hostPort == expectedHostPort | ||
| private val expectedHostPort = new java.net.URI(workerUrl) | ||
|
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. Perhaps RpcAddress.fromUriString(workerUrl) and use of the == method?
Member
Author
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. Updated to use |
||
| private def isWorker(address: RpcAddress) = { | ||
| expectedHostPort.getHost == address.host && expectedHostPort.getPort == address.port | ||
| } | ||
|
|
||
| def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) | ||
|
|
||
| override def receiveWithLogging = { | ||
| case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => | ||
| logInfo(s"Successfully connected to $workerUrl") | ||
| override def receive(sender: RpcEndpointRef) = { | ||
| case e => logWarning(s"Received unexpected actor system event: $e") | ||
|
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. nit: "actor system" is very akka-specific.
Member
Author
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. Fixed |
||
| } | ||
|
|
||
| case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) | ||
| if isWorker(remoteAddress) => | ||
| // These logs may not be seen if the worker (and associated pipe) has died | ||
| logError(s"Could not initialize connection to worker $workerUrl. Exiting.") | ||
| logError(s"Error was: $cause") | ||
| exitNonZero() | ||
| override def onConnected(remoteAddress: RpcAddress): Unit = { | ||
| if (isWorker(remoteAddress)) { | ||
| logInfo(s"Successfully connected to $workerUrl") | ||
| } | ||
| } | ||
|
|
||
| case DisassociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => | ||
| override def onDisconnected(remoteAddress: RpcAddress): Unit = { | ||
| if (isWorker(remoteAddress)) { | ||
| // This log message will never be seen | ||
| logError(s"Lost connection to worker actor $workerUrl. Exiting.") | ||
| exitNonZero() | ||
| } | ||
| } | ||
|
|
||
| case e: AssociationEvent => | ||
| // pass through association events relating to other remote actor systems | ||
|
|
||
| case e => logWarning(s"Received unexpected actor system event: $e") | ||
| override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { | ||
| if (isWorker(remoteAddress)) { | ||
| // These logs may not be seen if the worker (and associated pipe) has died | ||
| logError(s"Could not initialize connection to worker $workerUrl. Exiting.") | ||
| logError(s"Error was: $cause") | ||
| exitNonZero() | ||
| } | ||
| } | ||
| } | ||
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.
super nit: order. swap with import above.