Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/5268#discussion_r27446223
  
    --- Diff: 
core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala ---
    @@ -17,41 +17,43 @@
     
     package org.apache.spark.storage
     
    -import scala.concurrent.Future
    -
    -import akka.actor.{ActorRef, Actor}
    +import scala.concurrent.{ExecutionContext, Future}
     
    +import org.apache.spark.rpc.{RpcEnv, RpcCallContext, RpcEndpoint}
    +import org.apache.spark.util.Utils
     import org.apache.spark.{Logging, MapOutputTracker, SparkEnv}
     import org.apache.spark.storage.BlockManagerMessages._
    -import org.apache.spark.util.ActorLogReceive
     
     /**
    - * An actor to take commands from the master to execute options. For 
example,
    + * An RpcEndpoint to take commands from the master to execute options. For 
example,
      * this is used to remove blocks from the slave's BlockManager.
      */
     private[storage]
    -class BlockManagerSlaveActor(
    +class BlockManagerSlaveEndpoint(
    +    override val rpcEnv: RpcEnv,
         blockManager: BlockManager,
         mapOutputTracker: MapOutputTracker)
    -  extends Actor with ActorLogReceive with Logging {
    +  extends RpcEndpoint with Logging {
     
    -  import context.dispatcher
    +  private val asyncThreadPool =
    +    
Utils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool")
    +  private implicit val asyncExecutionContext = 
ExecutionContext.fromExecutorService(asyncThreadPool)
     
       // Operations that involve removing blocks may be slow and should be 
done asynchronously
    -  override def receiveWithLogging: PartialFunction[Any, Unit] = {
    +  override def receiveAndReply(context: RpcCallContext): 
PartialFunction[Any, Unit]  = {
         case RemoveBlock(blockId) =>
    -      doAsync[Boolean]("removing block " + blockId, sender) {
    +      doAsync[Boolean]("removing block " + blockId, context) {
             blockManager.removeBlock(blockId)
             true
           }
     
         case RemoveRdd(rddId) =>
    -      doAsync[Int]("removing RDD " + rddId, sender) {
    +      doAsync[Int]("removing RDD " + rddId, context) {
             blockManager.removeRdd(rddId)
           }
     
         case RemoveShuffle(shuffleId) =>
    -      doAsync[Boolean]("removing shuffle " + shuffleId, sender) {
    +      doAsync[Boolean]("removing shuffle " + shuffleId, context) {
    --- End diff --
    
    oh ic it is done in doAsync. Is this ok with the RPC contract that we are 
replying from a different thread?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to