@@ -34,8 +34,10 @@ import org.apache.spark.metrics.MetricsSystem
34
34
import org .apache .spark .network .BlockTransferService
35
35
import org .apache .spark .network .netty .NettyBlockTransferService
36
36
import org .apache .spark .network .nio .NioBlockTransferService
37
+ import org .apache .spark .rpc .akka .AkkaRpcEnv
38
+ import org .apache .spark .rpc .{RpcEndpointRef , RpcEndpoint , RpcEnv }
37
39
import org .apache .spark .scheduler .{OutputCommitCoordinator , LiveListenerBus }
38
- import org .apache .spark .scheduler .OutputCommitCoordinator .OutputCommitCoordinatorActor
40
+ import org .apache .spark .scheduler .OutputCommitCoordinator .OutputCommitCoordinatorEndpoint
39
41
import org .apache .spark .serializer .Serializer
40
42
import org .apache .spark .shuffle .{ShuffleMemoryManager , ShuffleManager }
41
43
import org .apache .spark .storage ._
@@ -54,7 +56,7 @@ import org.apache.spark.util.{AkkaUtils, Utils}
54
56
@ DeveloperApi
55
57
class SparkEnv (
56
58
val executorId : String ,
57
- val actorSystem : ActorSystem ,
59
+ val rpcEnv : RpcEnv ,
58
60
val serializer : Serializer ,
59
61
val closureSerializer : Serializer ,
60
62
val cacheManager : CacheManager ,
@@ -71,6 +73,9 @@ class SparkEnv (
71
73
val outputCommitCoordinator : OutputCommitCoordinator ,
72
74
val conf : SparkConf ) extends Logging {
73
75
76
+ // TODO Remove actorSystem
77
+ val actorSystem = rpcEnv.asInstanceOf [AkkaRpcEnv ].actorSystem
78
+
74
79
private [spark] var isStopped = false
75
80
private val pythonWorkers = mutable.HashMap [(String , Map [String , String ]), PythonWorkerFactory ]()
76
81
@@ -91,7 +96,8 @@ class SparkEnv (
91
96
blockManager.master.stop()
92
97
metricsSystem.stop()
93
98
outputCommitCoordinator.stop()
94
- actorSystem.shutdown()
99
+ rpcEnv.shutdown()
100
+
95
101
// Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut
96
102
// down, but let's call it anyway in case it gets fixed in a later release
97
103
// 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 {
236
242
val securityManager = new SecurityManager (conf)
237
243
238
244
// Create the ActorSystem for Akka and get the port it binds to.
239
- val (actorSystem, boundPort) = {
240
- val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName
241
- AkkaUtils .createActorSystem(actorSystemName, hostname, port, conf, securityManager)
242
- }
245
+ val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName
246
+ val rpcEnv = RpcEnv .create(actorSystemName, hostname, port, conf, securityManager)
247
+ val actorSystem = rpcEnv.asInstanceOf [AkkaRpcEnv ].actorSystem
243
248
244
249
// Figure out which port Akka actually bound to in case the original port is 0 or occupied.
245
250
if (isDriver) {
246
- conf.set(" spark.driver.port" , boundPort .toString)
251
+ conf.set(" spark.driver.port" , rpcEnv.address.port .toString)
247
252
} else {
248
- conf.set(" spark.executor.port" , boundPort .toString)
253
+ conf.set(" spark.executor.port" , rpcEnv.address.port .toString)
249
254
}
250
255
251
256
// Create an instance of the class with the given name, possibly initializing it with our conf
@@ -290,6 +295,15 @@ object SparkEnv extends Logging {
290
295
}
291
296
}
292
297
298
+ def registerOrLookupEndpoint (name : String , endpointCreator : => RpcEndpoint ): RpcEndpointRef = {
299
+ if (isDriver) {
300
+ logInfo(" Registering " + name)
301
+ rpcEnv.setupEndpoint(name, endpointCreator)
302
+ } else {
303
+ rpcEnv.setupDriverEndpointRef(name)
304
+ }
305
+ }
306
+
293
307
val mapOutputTracker = if (isDriver) {
294
308
new MapOutputTrackerMaster (conf)
295
309
} else {
@@ -377,13 +391,13 @@ object SparkEnv extends Logging {
377
391
val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse {
378
392
new OutputCommitCoordinator (conf)
379
393
}
380
- val outputCommitCoordinatorActor = registerOrLookup (" OutputCommitCoordinator" ,
381
- new OutputCommitCoordinatorActor ( outputCommitCoordinator))
382
- outputCommitCoordinator.coordinatorActor = Some (outputCommitCoordinatorActor )
394
+ val outputCommitCoordinatorRef = registerOrLookupEndpoint (" OutputCommitCoordinator" ,
395
+ new OutputCommitCoordinatorEndpoint (rpcEnv, outputCommitCoordinator))
396
+ outputCommitCoordinator.coordinatorRef = Some (outputCommitCoordinatorRef )
383
397
384
398
val envInstance = new SparkEnv (
385
399
executorId,
386
- actorSystem ,
400
+ rpcEnv ,
387
401
serializer,
388
402
closureSerializer,
389
403
cacheManager,
0 commit comments