@@ -23,18 +23,15 @@ import scala.collection.mutable
2323import scala .collection .JavaConversions ._
2424import scala .concurrent .{ExecutionContext , Future }
2525
26- import akka .actor .ActorRef
27- import akka .pattern .ask
28-
29- import org .apache .spark .rpc .{RpcEnv , RpcCallContext , RpcEndpoint }
26+ import org .apache .spark .rpc .{RpcEndpointRef , RpcEnv , RpcCallContext , RpcEndpoint }
3027import org .apache .spark .{Logging , SparkConf }
3128import org .apache .spark .annotation .DeveloperApi
3229import org .apache .spark .scheduler ._
3330import org .apache .spark .storage .BlockManagerMessages ._
34- import org .apache .spark .util .{ AkkaUtils , Utils }
31+ import org .apache .spark .util .Utils
3532
3633/**
37- * BlockManagerMasterActor is an actor on the master node to track statuses of
34+ * BlockManagerMasterEndpoint is an [[ RpcEndpoint ]] on the master node to track statuses of
3835 * all slaves' block managers.
3936 */
4037private [spark]
@@ -54,14 +51,12 @@ class BlockManagerMasterEndpoint(
5451 // Mapping from block id to the set of block managers that have the block.
5552 private val blockLocations = new JHashMap [BlockId , mutable.HashSet [BlockManagerId ]]
5653
57- private val akkaTimeout = AkkaUtils .askTimeout(conf)
58-
5954 private val askThreadPool = Utils .newDaemonCachedThreadPool(" block-manager-ask-thread-pool" )
6055 private implicit val askExecutionContext = ExecutionContext .fromExecutorService(askThreadPool)
6156
6257 override def receiveAndReply (context : RpcCallContext ): PartialFunction [Any , Unit ] = {
63- case RegisterBlockManager (blockManagerId, maxMemSize, slaveActor ) =>
64- register(blockManagerId, maxMemSize, slaveActor )
58+ case RegisterBlockManager (blockManagerId, maxMemSize, slaveEndpoint ) =>
59+ register(blockManagerId, maxMemSize, slaveEndpoint )
6560 context.reply(true )
6661
6762 case UpdateBlockInfo (
@@ -78,8 +73,8 @@ class BlockManagerMasterEndpoint(
7873 case GetPeers (blockManagerId) =>
7974 context.reply(getPeers(blockManagerId))
8075
81- case GetActorSystemHostPortForExecutor (executorId) =>
82- context.reply(getActorSystemHostPortForExecutor (executorId))
76+ case GetRpcHostPortForExecutor (executorId) =>
77+ context.reply(getRpcHostPortForExecutor (executorId))
8378
8479 case GetMemoryStatus =>
8580 context.reply(memoryStatus)
@@ -137,17 +132,17 @@ class BlockManagerMasterEndpoint(
137132 val removeMsg = RemoveRdd (rddId)
138133 Future .sequence(
139134 blockManagerInfo.values.map { bm =>
140- bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo [Int ]
135+ bm.slaveEndpoint.sendWithReply [Int ](removeMsg)
141136 }.toSeq
142137 )
143138 }
144139
145140 private def removeShuffle (shuffleId : Int ): Future [Seq [Boolean ]] = {
146- // Nothing to do in the BlockManagerMasterActor data structures
141+ // Nothing to do in the BlockManagerMasterEndpoint data structures
147142 val removeMsg = RemoveShuffle (shuffleId)
148143 Future .sequence(
149144 blockManagerInfo.values.map { bm =>
150- bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo [Boolean ]
145+ bm.slaveEndpoint.sendWithReply [Boolean ](removeMsg)
151146 }.toSeq
152147 )
153148 }
@@ -164,7 +159,7 @@ class BlockManagerMasterEndpoint(
164159 }
165160 Future .sequence(
166161 requiredBlockManagers.map { bm =>
167- bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo [Int ]
162+ bm.slaveEndpoint.sendWithReply [Int ](removeMsg)
168163 }.toSeq
169164 )
170165 }
@@ -219,7 +214,7 @@ class BlockManagerMasterEndpoint(
219214 // Remove the block from the slave's BlockManager.
220215 // Doesn't actually wait for a confirmation and the message might get lost.
221216 // If message loss becomes frequent, we should add retry logic here.
222- blockManager.get.slaveActor.ask (RemoveBlock (blockId))(akkaTimeout )
217+ blockManager.get.slaveEndpoint.sendWithReply[ Boolean ] (RemoveBlock (blockId))
223218 }
224219 }
225220 }
@@ -251,14 +246,14 @@ class BlockManagerMasterEndpoint(
251246 askSlaves : Boolean ): Map [BlockManagerId , Future [Option [BlockStatus ]]] = {
252247 val getBlockStatus = GetBlockStatus (blockId)
253248 /*
254- * Rather than blocking on the block status query, master actor should simply return
249+ * Rather than blocking on the block status query, master endpoint should simply return
255250 * Futures to avoid potential deadlocks. This can arise if there exists a block manager
256- * that is also waiting for this master actor 's response to a previous message.
251+ * that is also waiting for this master endpoint 's response to a previous message.
257252 */
258253 blockManagerInfo.values.map { info =>
259254 val blockStatusFuture =
260255 if (askSlaves) {
261- info.slaveActor.ask(getBlockStatus)(akkaTimeout).mapTo [Option [BlockStatus ]]
256+ info.slaveEndpoint.sendWithReply [Option [BlockStatus ]](getBlockStatus)
262257 } else {
263258 Future { info.getStatus(blockId) }
264259 }
@@ -282,7 +277,7 @@ class BlockManagerMasterEndpoint(
282277 blockManagerInfo.values.map { info =>
283278 val future =
284279 if (askSlaves) {
285- info.slaveActor.ask(getMatchingBlockIds)(akkaTimeout).mapTo [Seq [BlockId ]]
280+ info.slaveEndpoint.sendWithReply [Seq [BlockId ]](getMatchingBlockIds)
286281 } else {
287282 Future { info.blocks.keys.filter(filter).toSeq }
288283 }
@@ -291,7 +286,7 @@ class BlockManagerMasterEndpoint(
291286 ).map(_.flatten.toSeq)
292287 }
293288
294- private def register (id : BlockManagerId , maxMemSize : Long , slaveActor : ActorRef ) {
289+ private def register (id : BlockManagerId , maxMemSize : Long , slaveEndpoint : RpcEndpointRef ) {
295290 val time = System .currentTimeMillis()
296291 if (! blockManagerInfo.contains(id)) {
297292 blockManagerIdByExecutor.get(id.executorId) match {
@@ -308,7 +303,7 @@ class BlockManagerMasterEndpoint(
308303 blockManagerIdByExecutor(id.executorId) = id
309304
310305 blockManagerInfo(id) = new BlockManagerInfo (
311- id, System .currentTimeMillis(), maxMemSize, slaveActor )
306+ id, System .currentTimeMillis(), maxMemSize, slaveEndpoint )
312307 }
313308 listenerBus.post(SparkListenerBlockManagerAdded (time, id, maxMemSize))
314309 }
@@ -379,17 +374,15 @@ class BlockManagerMasterEndpoint(
379374 }
380375
381376 /**
382- * Returns the hostname and port of an executor's actor system , based on the Akka address of its
383- * BlockManagerSlaveActor .
377+ * Returns the hostname and port of an executor, based on the [[ RpcEnv ]] address of its
378+ * [[ BlockManagerSlaveEndpoint ]] .
384379 */
385- private def getActorSystemHostPortForExecutor (executorId : String ): Option [(String , Int )] = {
380+ private def getRpcHostPortForExecutor (executorId : String ): Option [(String , Int )] = {
386381 for (
387382 blockManagerId <- blockManagerIdByExecutor.get(executorId);
388- info <- blockManagerInfo.get(blockManagerId);
389- host <- info.slaveActor.path.address.host;
390- port <- info.slaveActor.path.address.port
383+ info <- blockManagerInfo.get(blockManagerId)
391384 ) yield {
392- (host, port)
385+ (info.slaveEndpoint.address. host, info.slaveEndpoint.address. port)
393386 }
394387 }
395388}
@@ -412,7 +405,7 @@ private[spark] class BlockManagerInfo(
412405 val blockManagerId : BlockManagerId ,
413406 timeMs : Long ,
414407 val maxMem : Long ,
415- val slaveActor : ActorRef )
408+ val slaveEndpoint : RpcEndpointRef )
416409 extends Logging {
417410
418411 private var _lastSeenMs : Long = timeMs
0 commit comments