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, 10 Mar 2015 22:37:44 GMT
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4588#discussion_r26173836
  
    --- Diff: core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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.Future
    +import scala.concurrent.duration.FiniteDuration
    +import scala.reflect.ClassTag
    +
    +import org.apache.spark.{SparkException, SecurityManager, SparkConf}
    +import org.apache.spark.util.Utils
    +
    +/**
    + * An RPC environment.
    + */
    +private[spark] trait RpcEnv {
    +
    +  /**
    +   * Return RpcEndpointRef of the registered [[RpcEndpoint]]. Will be used to implement
    +   * [[RpcEndpoint.self]].
    +   */
    +  private[rpc] def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef
    +
    +  /**
    +   * Return an ActionScheduler for the caller to run long-time actions out of the current
thread.
    +   */
    +  def scheduler: ActionScheduler
    +
    +  /**
    +   * 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]].
    +   */
    +  def setupThreadSafeEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef
    +
    +  /**
    +   * Retrieve a [[RpcEndpointRef]] which is located in the driver via its name.
    +   */
    +  def setupDriverEndpointRef(name: String): RpcEndpointRef
    +
    +  /**
    +   * Retrieve the [[RpcEndpointRef]] represented by `url`.
    +   */
    +  def setupEndpointRefByUrl(url: String): RpcEndpointRef
    +
    +  /**
    +   * Retrieve the [[RpcEndpointRef]] represented by `systemName`, `address` and `endpointName`
    +   */
    +  def setupEndpointRef(
    +      systemName: String, address: RpcAddress, endpointName: String): RpcEndpointRef
    +
    +  /**
    +   * 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 companion object with an
    + * `apply(config: RpcEnvConfig): RpcEnv` method so that it can be created via Reflection.
    + *
    + * {{{
    + * object MyCustomRpcEnv {
    + *   def apply(config: RpcEnvConfig): RpcEnv = {
    + *     ...
    + *   }
    + * }
    + * }}}
    + */
    +private[spark] object RpcEnv {
    +
    +  private def getRpcEnvCompanion(conf: SparkConf): AnyRef = {
    +    // Add more RpcEnv implementations here
    +    val rpcEnvNames = Map("akka" -> "org.apache.spark.rpc.akka.AkkaRpcEnv")
    +    val rpcEnvName = conf.get("spark.rpc", "akka")
    +    val rpcEnvClassName = rpcEnvNames.getOrElse(rpcEnvName.toLowerCase, rpcEnvName)
    +    val companion = Class.forName(
    +      rpcEnvClassName + "$", true, Utils.getContextOrSparkClassLoader).getField("MODULE$").get(null)
    +    companion
    +  }
    +
    +  def create(
    +      name: String,
    +      host: String,
    +      port: Int,
    +      conf: SparkConf,
    +      securityManager: SecurityManager): RpcEnv = {
    +    // Using Reflection to create the RpcEnv to avoid to depend on Akka directly
    +    val config = RpcEnvConfig(conf, name, host, port, securityManager)
    +    val companion = getRpcEnvCompanion(conf)
    +    companion.getClass.getMethod("apply", classOf[RpcEnvConfig]).
    +      invoke(companion, config).asInstanceOf[RpcEnv]
    +  }
    +
    +}
    +
    +/**
    + * An end point for the RPC that defines what functions to trigger given a message.
    + *
    + * RpcEndpoint will be guaranteed that `onStart`, `receive` and `onStop` will
    + * be called in sequence.
    + *
    + * The lift-cycle will be:
    + *
    + * constructor onStart receive* onStop
    + *
    + * Note: `receive` can be called concurrently. If you want `receive` is thread-safe,
please use
    + * [[RpcEnv.setupThreadSafeEndpoint]]
    + *
    + * If any error is thrown from one of [[RpcEndpoint]] methods except `onError`, `onError`
will be
    + * invoked with the cause. If `onError` throws an error, [[RpcEnv]] will ignore it.
    + */
    +private[spark] trait RpcEndpoint {
    +
    +  /**
    +   * The [[RpcEnv]] that this [[RpcEndpoint]] is registered to.
    +   */
    +  val rpcEnv: RpcEnv
    +
    +  /**
    +   * The [[RpcEndpointRef]] of this [[RpcEndpoint]]. `self` will become valid when `onStart`
is
    +   * called.
    +   *
    +   * Note: Because before `onStart`, [[RpcEndpoint]] has not yet been registered and
there is not
    +   * valid [[RpcEndpointRef]] for it. So don't call `self` before `onStart` is called.
    +   */
    +  final def self: RpcEndpointRef = {
    +    require(rpcEnv != null, "rpcEnv has not been initialized")
    +    rpcEnv.endpointRef(this)
    +  }
    +
    +  /**
    +   * Process messages from [[RpcEndpointRef.send]] or [[RpcResponse.reply)]]
    +   */
    +  def receive: PartialFunction[Any, Unit] = {
    +    case _ =>
    +      // network events will be passed here by default, so do nothing by default to avoid
noise.
    +  }
    +
    +  /**
    +   * Process messages from [[RpcEndpointRef.sendWithReply]] or [[RpcResponse.replyWithSender)]]
    +   */
    +  def receiveAndReply(response: RpcResponse): PartialFunction[Any, Unit] = {
    +    case _ => response.fail(new SparkException(self + " won't reply anything"))
    +  }
    +
    +  /**
    +   * Call onError when any exception is thrown during handling messages.
    +   *
    +   * @param cause
    +   */
    +  def onError(cause: Throwable): Unit = {
    +    // By default, throw e and let RpcEnv handle it
    +    throw cause
    +  }
    +
    +  /**
    +   * Invoked before [[RpcEndpoint]] starts to handle any message.
    +   */
    +  def onStart(): Unit = {
    +    // By default, do nothing.
    +  }
    +
    +  /**
    +   * Invoked when [[RpcEndpoint]] is stopping.
    +   */
    +  def onStop(): Unit = {
    +    // By default, do nothing.
    +  }
    +
    +  /**
    +   * An convenient method to stop [[RpcEndpoint]].
    +   */
    +  final def stop(): Unit = {
    +    val _self = self
    +    if (_self != null) {
    +      rpcEnv.stop(self)
    +    }
    +  }
    +}
    +
    +/**
    + * A reference for a remote [[RpcEndpoint]]. [[RpcEndpointRef]] is thread-safe.
    + */
    +private[spark] trait RpcEndpointRef {
    +
    +  /**
    +   * return the address for the [[RpcEndpointRef]]
    +   */
    +  def address: RpcAddress
    +
    +  def name: String
    +
    +  /**
    +   * Send a message to the corresponding [[RpcEndpoint]] and get its result within a
default
    +   * timeout, or throw a SparkException if this fails even after the default number of
retries.
    +   *
    +   * Note: this is a blocking action which may cost a lot of time,  so don't call it
in an message
    +   * loop of [[RpcEndpoint]].
    +   *
    +   * @param message the message to send
    +   * @tparam T type of the reply message
    +   * @return the reply message from the corresponding [[RpcEndpoint]]
    +   */
    +  def askWithReply[T: ClassTag](message: Any): T
    +
    +  /**
    +   * Send a message to the corresponding [[RpcEndpoint.receive]] and get its result within
a
    +   * specified timeout, throw a SparkException if this fails even after the specified
number of
    +   * retries.
    +   *
    +   * Note: this is a blocking action which may cost a lot of time, so don't call it in
an message
    +   * loop of [[RpcEndpoint]].
    +   *
    +   * @param message the message to send
    +   * @param timeout the timeout duration
    +   * @tparam T type of the reply message
    +   * @return the reply message from the corresponding [[RpcEndpoint]]
    +   */
    +  def askWithReply[T: ClassTag](message: Any, timeout: FiniteDuration): T
    --- End diff --
    
    (My question was hidden by github now because the line I commented on no longer exists)
I think the answer at the time was because we have a lot of uses of this, and every instance
of it would need to get the retry config value from SparkConf, and thus it might be simpler
to just have this here for now.


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