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

    https://github.com/apache/spark/pull/4588#discussion_r26957574
  
    --- Diff: core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala ---
    @@ -0,0 +1,412 @@
    +/*
    + * 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
    +
    +import java.net.URI
    +
    +import scala.concurrent.{Await, Future}
    +import scala.concurrent.duration._
    +import scala.language.postfixOps
    +import scala.reflect.ClassTag
    +
    +import org.apache.spark.{Logging, SparkException, SecurityManager, 
SparkConf}
    +import org.apache.spark.util.{AkkaUtils, Utils}
    +
    +/**
    + * An RPC environment. [[RpcEndpoint]]s need to register itself with a 
name to [[RpcEnv]] to
    + * receives messages. Then [[RpcEnv]] will process messages sent from 
[[RpcEndpointRef]] or remote
    + * nodes, and deliver them to corresponding [[RpcEndpoint]]s.
    + *
    + * [[RpcEnv]] also provides some methods to retrieve [[RpcEndpointRef]]s 
given name or uri.
    + */
    +private[spark] abstract class RpcEnv(conf: SparkConf) {
    +
    +  private[spark] val defaultLookupTimeout = AkkaUtils.lookupTimeout(conf)
    +
    +  /**
    +   * Return RpcEndpointRef of the registered [[RpcEndpoint]]. Will be used 
to implement
    +   * [[RpcEndpoint.self]].
    +   */
    +  private[rpc] def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef
    +
    +  /**
    +   * Return the address that [[RpcEnv]] is listening to.
    +   */
    +  def address: RpcAddress
    +
    +  /**
    +   * Register a [[RpcEndpoint]] with a name and return its 
[[RpcEndpointRef]]. [[RpcEnv]] does not
    +   * guarantee thread-safety.
    +   */
    +  def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef
    +
    +  /**
    +   * Register a [[RpcEndpoint]] with a name and return its 
[[RpcEndpointRef]]. [[RpcEnv]] should
    +   * make sure thread-safely sending messages to [[RpcEndpoint]].
    +   *
    +   * Thread-safety means processing of one message happens before 
processing of the next message by
    +   * the same [[RpcEndpoint]]. In the other words, changes to internal 
fields of a [[RpcEndpoint]]
    +   * are visible when processing the next message, and fields in the 
[[RpcEndpoint]] need not be
    +   * volatile or equivalent.
    +   *
    +   * However, there is no guarantee that the same thread will be executing 
the same [[RpcEndpoint]]
    +   * for different messages.
    +   */
    +  def setupThreadSafeEndpoint(name: String, endpoint: RpcEndpoint): 
RpcEndpointRef
    +
    +  /**
    +   * Retrieve the [[RpcEndpointRef]] represented by `url` asynchronously.
    +   */
    +  def asyncSetupEndpointRefByUrl(url: String): Future[RpcEndpointRef]
    +
    +  /**
    +   * Retrieve the [[RpcEndpointRef]] represented by `url`. This is a 
blocking action.
    +   */
    +  def setupEndpointRefByUrl(url: String): RpcEndpointRef = {
    +    Await.result(asyncSetupEndpointRefByUrl(url), defaultLookupTimeout)
    +  }
    +
    +  /**
    +   * Retrieve the [[RpcEndpointRef]] represented by `systemName`, 
`address` and `endpointName`
    +   * asynchronously.
    +   */
    +  def asyncSetupEndpointRef(
    +      systemName: String, address: RpcAddress, endpointName: String): 
Future[RpcEndpointRef] = {
    +    asyncSetupEndpointRefByUrl(uriOf(systemName, address, endpointName))
    +  }
    +
    +  /**
    +   * Retrieve the [[RpcEndpointRef]] represented by `systemName`, 
`address` and `endpointName`.
    +   * This is a blocking action.
    +   */
    +  def setupEndpointRef(
    +      systemName: String, address: RpcAddress, endpointName: String): 
RpcEndpointRef = {
    +    setupEndpointRefByUrl(uriOf(systemName, address, endpointName))
    +  }
    +
    +  /**
    +   * Stop [[RpcEndpoint]] specified by `endpoint`.
    +   */
    +  def stop(endpoint: RpcEndpointRef): Unit
    +
    +  /**
    +   * Shutdown this [[RpcEnv]] asynchronously. If need to make sure 
[[RpcEnv]] exits successfully,
    +   * call [[awaitTermination()]] straight after [[shutdown()]].
    +   */
    +  def shutdown(): Unit
    +
    +  /**
    +   * Wait until [[RpcEnv]] exits.
    +   *
    +   * TODO do we need a timeout parameter?
    +   */
    +  def awaitTermination(): Unit
    +
    +  /**
    +   * Create a URI used to create a [[RpcEndpointRef]]. Use this one to 
create the URI instead of
    +   * creating it manually because different [[RpcEnv]] may have different 
formats.
    +   */
    +  def uriOf(systemName: String, address: RpcAddress, endpointName: 
String): String
    +}
    +
    +private[spark] case class RpcEnvConfig(
    +    conf: SparkConf,
    +    name: String,
    +    host: String,
    +    port: Int,
    +    securityManager: SecurityManager)
    +
    +/**
    + * A RpcEnv implementation must have a [[RpcEnvFactory]] implementation 
with an empty constructor
    + * so that it can be created via Reflection.
    + */
    +private[spark] object RpcEnv {
    +
    +  private def getRpcEnvFactory(conf: SparkConf): RpcEnvFactory = {
    +    // Add more RpcEnv implementations here
    +    val rpcEnvNames = Map("akka" -> 
"org.apache.spark.rpc.akka.AkkaRpcEnvFactory")
    --- End diff --
    
    You could use `classOf[AkkaRpcEnvFactory].getName()` to be safer, but not a 
big deal.


---
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