@@ -21,25 +21,29 @@ import java.util.{HashMap => JHashMap}
2121
2222import scala .collection .mutable
2323import scala .collection .JavaConversions ._
24- import scala .concurrent .Future
25- import scala .concurrent .duration ._
24+ import scala .concurrent .{ExecutionContext , Future }
2625
27- import akka .actor .{ Actor , ActorRef }
26+ import akka .actor .ActorRef
2827import akka .pattern .ask
2928
30- import org .apache .spark .{Logging , SparkConf , SparkException }
29+ import org .apache .spark .rpc .{RpcEnv , RpcCallContext , RpcEndpoint }
30+ import org .apache .spark .{Logging , SparkConf }
3131import org .apache .spark .annotation .DeveloperApi
3232import org .apache .spark .scheduler ._
3333import org .apache .spark .storage .BlockManagerMessages ._
34- import org .apache .spark .util .{ActorLogReceive , AkkaUtils , Utils }
34+ import org .apache .spark .util .{AkkaUtils , Utils }
3535
3636/**
3737 * BlockManagerMasterActor is an actor on the master node to track statuses of
3838 * all slaves' block managers.
3939 */
4040private [spark]
41- class BlockManagerMasterActor (val isLocal : Boolean , conf : SparkConf , listenerBus : LiveListenerBus )
42- extends Actor with ActorLogReceive with Logging {
41+ class BlockManagerMasterEndpoint (
42+ override val rpcEnv : RpcEnv ,
43+ val isLocal : Boolean ,
44+ conf : SparkConf ,
45+ listenerBus : LiveListenerBus )
46+ extends RpcEndpoint with Logging {
4347
4448 // Mapping from block manager id to the block manager's information.
4549 private val blockManagerInfo = new mutable.HashMap [BlockManagerId , BlockManagerInfo ]
@@ -52,66 +56,67 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
5256
5357 private val akkaTimeout = AkkaUtils .askTimeout(conf)
5458
55- override def receiveWithLogging : PartialFunction [Any , Unit ] = {
59+ private val askThreadPool = Utils .newDaemonCachedThreadPool(" block-manager-ask-thread-pool" )
60+ private implicit val askExecutionContext = ExecutionContext .fromExecutorService(askThreadPool)
61+
62+ override def receiveAndReply (context : RpcCallContext ): PartialFunction [Any , Unit ] = {
5663 case RegisterBlockManager (blockManagerId, maxMemSize, slaveActor) =>
5764 register(blockManagerId, maxMemSize, slaveActor)
58- sender ! true
65+ context.reply( true )
5966
6067 case UpdateBlockInfo (
6168 blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) =>
62- sender ! updateBlockInfo(
63- blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize)
69+ context.reply( updateBlockInfo(
70+ blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize))
6471
6572 case GetLocations (blockId) =>
66- sender ! getLocations(blockId)
73+ context.reply( getLocations(blockId) )
6774
6875 case GetLocationsMultipleBlockIds (blockIds) =>
69- sender ! getLocationsMultipleBlockIds(blockIds)
76+ context.reply( getLocationsMultipleBlockIds(blockIds) )
7077
7178 case GetPeers (blockManagerId) =>
72- sender ! getPeers(blockManagerId)
79+ context.reply( getPeers(blockManagerId) )
7380
7481 case GetActorSystemHostPortForExecutor (executorId) =>
75- sender ! getActorSystemHostPortForExecutor(executorId)
82+ context.reply( getActorSystemHostPortForExecutor(executorId) )
7683
7784 case GetMemoryStatus =>
78- sender ! memoryStatus
85+ context.reply( memoryStatus)
7986
8087 case GetStorageStatus =>
81- sender ! storageStatus
88+ context.reply( storageStatus)
8289
8390 case GetBlockStatus (blockId, askSlaves) =>
84- sender ! blockStatus(blockId, askSlaves)
91+ context.reply( blockStatus(blockId, askSlaves) )
8592
8693 case GetMatchingBlockIds (filter, askSlaves) =>
87- sender ! getMatchingBlockIds(filter, askSlaves)
94+ context.reply( getMatchingBlockIds(filter, askSlaves) )
8895
8996 case RemoveRdd (rddId) =>
90- sender ! removeRdd(rddId)
97+ context.reply( removeRdd(rddId) )
9198
9299 case RemoveShuffle (shuffleId) =>
93- sender ! removeShuffle(shuffleId)
100+ context.reply( removeShuffle(shuffleId) )
94101
95102 case RemoveBroadcast (broadcastId, removeFromDriver) =>
96- sender ! removeBroadcast(broadcastId, removeFromDriver)
103+ context.reply( removeBroadcast(broadcastId, removeFromDriver) )
97104
98105 case RemoveBlock (blockId) =>
99106 removeBlockFromWorkers(blockId)
100- sender ! true
107+ context.reply( true )
101108
102109 case RemoveExecutor (execId) =>
103110 removeExecutor(execId)
104- sender ! true
111+ context.reply( true )
105112
106113 case StopBlockManagerMaster =>
107- sender ! true
108- context. stop(self )
114+ context.reply( true )
115+ stop()
109116
110117 case BlockManagerHeartbeat (blockManagerId) =>
111- sender ! heartbeatReceived(blockManagerId)
118+ context.reply( heartbeatReceived(blockManagerId) )
112119
113- case other =>
114- logWarning(" Got unknown message: " + other)
115120 }
116121
117122 private def removeRdd (rddId : Int ): Future [Seq [Int ]] = {
@@ -129,7 +134,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
129134
130135 // Ask the slaves to remove the RDD, and put the result in a sequence of Futures.
131136 // The dispatcher is used as an implicit argument into the Future sequence construction.
132- import context .dispatcher
133137 val removeMsg = RemoveRdd (rddId)
134138 Future .sequence(
135139 blockManagerInfo.values.map { bm =>
@@ -140,7 +144,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
140144
141145 private def removeShuffle (shuffleId : Int ): Future [Seq [Boolean ]] = {
142146 // Nothing to do in the BlockManagerMasterActor data structures
143- import context .dispatcher
144147 val removeMsg = RemoveShuffle (shuffleId)
145148 Future .sequence(
146149 blockManagerInfo.values.map { bm =>
@@ -155,7 +158,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
155158 * from the executors, but not from the driver.
156159 */
157160 private def removeBroadcast (broadcastId : Long , removeFromDriver : Boolean ): Future [Seq [Int ]] = {
158- import context .dispatcher
159161 val removeMsg = RemoveBroadcast (broadcastId, removeFromDriver)
160162 val requiredBlockManagers = blockManagerInfo.values.filter { info =>
161163 removeFromDriver || ! info.blockManagerId.isDriver
@@ -247,7 +249,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
247249 private def blockStatus (
248250 blockId : BlockId ,
249251 askSlaves : Boolean ): Map [BlockManagerId , Future [Option [BlockStatus ]]] = {
250- import context .dispatcher
251252 val getBlockStatus = GetBlockStatus (blockId)
252253 /*
253254 * Rather than blocking on the block status query, master actor should simply return
@@ -276,7 +277,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
276277 private def getMatchingBlockIds (
277278 filter : BlockId => Boolean ,
278279 askSlaves : Boolean ): Future [Seq [BlockId ]] = {
279- import context .dispatcher
280280 val getMatchingBlockIds = GetMatchingBlockIds (filter)
281281 Future .sequence(
282282 blockManagerInfo.values.map { info =>
0 commit comments