spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From rxin <...@git.apache.org>
Subject [GitHub] spark pull request: [SPARK-5124][Core] A standard RPC interface an...
Date Tue, 24 Feb 2015 22:05:34 GMT
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4588#discussion_r25297762
  
    --- Diff: core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.rpc.akka
    +
    +import java.net.URI
    +import java.util.concurrent.{ConcurrentHashMap, CountDownLatch}
    +
    +import scala.concurrent.Await
    +import scala.concurrent.duration._
    +import scala.concurrent.Future
    +import scala.language.postfixOps
    +import scala.reflect.ClassTag
    +import scala.util.control.NonFatal
    +
    +import _root_.akka.actor.{ActorSystem, ExtendedActorSystem, Actor, ActorRef, Props, Address}
    +import akka.pattern.{ask => akkaAsk}
    +import akka.remote._
    +
    +import org.apache.spark.{Logging, SparkConf}
    +import org.apache.spark.rpc._
    +import org.apache.spark.util.{SparkUncaughtExceptionHandler, ActorLogReceive, AkkaUtils}
    +
    +/**
    + * A RpcEnv implementation based on Akka.
    + *
    + * TODO Once we remove all usages of Akka in other place, we can move this file to a
new project and
    + * remove Akka from the dependencies.
    + *
    + * @param actorSystem
    + * @param conf
    + * @param boundPort
    + */
    +private[spark] class AkkaRpcEnv private (
    +    val actorSystem: ActorSystem, conf: SparkConf, boundPort: Int) extends RpcEnv with
Logging {
    +
    +  private val defaultAddress: RpcAddress = {
    +    val address = actorSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
    +    // In some test case, ActorSystem doesn't bind to any address.
    +    // So just use some default value since they are only some unit tests
    +    RpcAddress(address.host.getOrElse("localhost"), address.port.getOrElse(boundPort))
    +  }
    +
    +  override val systemName: String = actorSystem.name
    +
    +  override val address: RpcAddress = defaultAddress
    +
    +  /**
    +   * A lookup table to search a [[RpcEndpointRef]] for a [[RpcEndpoint]]. We need it
to make
    +   * [[RpcEndpoint.self]] work.
    +   */
    +  private val endpointToRef = new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef]()
    +
    +  override val scheduler = new ActionSchedulerImpl(conf)
    +
    +  /**
    +   * Need this map to remove `RpcEndpoint` from `endpointToRef` via a `RpcEndpointRef`
    +   */
    +  private val refToEndpoint = new ConcurrentHashMap[RpcEndpointRef, RpcEndpoint]()
    +
    +  private def registerEndpoint(endpoint: RpcEndpoint, endpointRef: RpcEndpointRef): Unit
= {
    +    endpointToRef.put(endpoint, endpointRef)
    +    refToEndpoint.put(endpointRef, endpoint)
    +  }
    +
    +  private def unregisterEndpoint(endpointRef: RpcEndpointRef): Unit = {
    +    val endpoint = refToEndpoint.remove(endpointRef)
    +    if (endpoint != null) {
    +      endpointToRef.remove(endpoint)
    +    }
    +  }
    +
    +  /**
    +   * Retrieve the [[RpcEndpointRef]] of `endpoint`.
    +   */
    +  override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = {
    +    val endpointRef = endpointToRef.get(endpoint)
    +    require(endpointRef != null, s"Cannot find RpcEndpointRef of ${endpoint} in ${this}")
    +    endpointRef
    +  }
    +
    +  override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = {
    +    val latch = new CountDownLatch(1)
    +    try {
    +      @volatile var endpointRef: AkkaRpcEndpointRef = null
    +      val actorRef = actorSystem.actorOf(Props(new Actor with ActorLogReceive with Logging
{
    +
    +        // Wait until `endpointRef` is set. TODO better solution?
    +        latch.await()
    +        require(endpointRef != null)
    +        registerEndpoint(endpoint, endpointRef)
    +
    +        var isNetworkRpcEndpoint = false
    +
    +        override def preStart(): Unit = {
    +          if (endpoint.isInstanceOf[NetworkRpcEndpoint]) {
    +            isNetworkRpcEndpoint = true
    +            // Listen for remote client network events only when it's `NetworkRpcEndpoint`
    +            context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
    +          }
    +          safelyCall(endpoint) {
    +            endpoint.onStart()
    +          }
    +        }
    +
    +        override def receiveWithLogging: Receive = if (isNetworkRpcEndpoint) {
    +          case AssociatedEvent(_, remoteAddress, _) =>
    +            safelyCall(endpoint) {
    +              endpoint.asInstanceOf[NetworkRpcEndpoint].
    +                onConnected(akkaAddressToRpcAddress(remoteAddress))
    +            }
    +
    +          case DisassociatedEvent(_, remoteAddress, _) =>
    +            safelyCall(endpoint) {
    +              endpoint.asInstanceOf[NetworkRpcEndpoint].
    +                onDisconnected(akkaAddressToRpcAddress(remoteAddress))
    +            }
    +
    +          case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _)
=>
    +            safelyCall(endpoint) {
    +              endpoint.asInstanceOf[NetworkRpcEndpoint].
    +                onNetworkError(cause, akkaAddressToRpcAddress(remoteAddress))
    +            }
    +          case e: RemotingLifecycleEvent =>
    +          // TODO ignore?
    +
    +          case message: Any =>
    +            logDebug("Received RPC message: " + message)
    +            safelyCall(endpoint) {
    +              val pf = endpoint.receive(new AkkaRpcEndpointRef(defaultAddress, sender(),
conf))
    +              if (pf.isDefinedAt(message)) {
    +                pf.apply(message)
    +              }
    +            }
    +        } else {
    +          case message: Any =>
    +            logDebug("Received RPC message: " + message)
    +            safelyCall(endpoint) {
    +              val pf = endpoint.receive(new AkkaRpcEndpointRef(defaultAddress, sender(),
conf))
    +              if (pf.isDefinedAt(message)) {
    +                pf.apply(message)
    +              }
    +            }
    +        }
    +
    +        override def postStop(): Unit = {
    +          unregisterEndpoint(endpoint.self)
    +          safelyCall(endpoint) {
    +            endpoint.onStop()
    +          }
    +        }
    +
    +        }), name = name)
    +      endpointRef = new AkkaRpcEndpointRef(defaultAddress, actorRef, conf)
    +      endpointRef
    +    } finally {
    +      latch.countDown()
    +    }
    +  }
    +
    +  private def safelyCall(endpoint: RpcEndpoint)(action: => Unit): Unit = {
    --- End diff --
    
    document this


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


Mime
View raw message