Skip to content
Snippets Groups Projects
Commit 4f5a24d7 authored by Shixiong Zhu's avatar Shixiong Zhu Committed by Reynold Xin
Browse files

[SPARK-7995][SPARK-6280][CORE] Remove AkkaRpcEnv and remove systemName from setupEndpointRef

### Remove AkkaRpcEnv

Keep `SparkEnv.actorSystem` because Streaming still uses it. Will remove it and AkkaUtils after refactoring Streaming actorStream API.

### Remove systemName
There are 2 places using `systemName`:
* `RpcEnvConfig.name`. Actually, although it's used as `systemName` in `AkkaRpcEnv`, `NettyRpcEnv` uses it as the service name to output the log `Successfully started service *** on port ***`. Since the service name in log is useful, I keep `RpcEnvConfig.name`.
* `def setupEndpointRef(systemName: String, address: RpcAddress, endpointName: String)`. Each `ActorSystem` has a `systemName`. Akka requires `systemName` in its URI and will refuse a connection if `systemName` is not matched. However, `NettyRpcEnv` doesn't use it. So we can remove `systemName` from `setupEndpointRef` since we are removing `AkkaRpcEnv`.

### Remove RpcEnv.uriOf

`uriOf` exists because Akka uses different URI formats for with and without authentication, e.g., `akka.ssl.tcp...` and `akka.tcp://...`. But `NettyRpcEnv` uses the same format. So it's not necessary after removing `AkkaRpcEnv`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10459 from zsxwing/remove-akka-rpc-env.
parent e6c77874
No related branches found
No related tags found
No related merge requests found
Showing
with 52 additions and 655 deletions
......@@ -544,7 +544,8 @@ private[spark] object SparkConf extends Logging {
DeprecatedConfig("spark.kryoserializer.buffer.mb", "1.4",
"Please use spark.kryoserializer.buffer instead. The default value for " +
"spark.kryoserializer.buffer.mb was previously specified as '0.064'. Fractional values " +
"are no longer accepted. To specify the equivalent now, one may use '64k'.")
"are no longer accepted. To specify the equivalent now, one may use '64k'."),
DeprecatedConfig("spark.rpc", "2.0", "Not used any more.")
)
Map(configs.map { cfg => (cfg.key -> cfg) } : _*)
......
......@@ -34,7 +34,6 @@ import org.apache.spark.memory.{MemoryManager, StaticMemoryManager, UnifiedMemor
import org.apache.spark.network.BlockTransferService
import org.apache.spark.network.netty.NettyBlockTransferService
import org.apache.spark.rpc.{RpcEndpointRef, RpcEndpoint, RpcEnv}
import org.apache.spark.rpc.akka.AkkaRpcEnv
import org.apache.spark.scheduler.{OutputCommitCoordinator, LiveListenerBus}
import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorEndpoint
import org.apache.spark.serializer.Serializer
......@@ -97,9 +96,7 @@ class SparkEnv (
blockManager.master.stop()
metricsSystem.stop()
outputCommitCoordinator.stop()
if (!rpcEnv.isInstanceOf[AkkaRpcEnv]) {
actorSystem.shutdown()
}
actorSystem.shutdown()
rpcEnv.shutdown()
// Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut
......@@ -248,14 +245,11 @@ object SparkEnv extends Logging {
val securityManager = new SecurityManager(conf)
// Create the ActorSystem for Akka and get the port it binds to.
val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName
// Create the ActorSystem for Akka and get the port it binds to.
val rpcEnv = RpcEnv.create(actorSystemName, hostname, port, conf, securityManager,
clientMode = !isDriver)
val actorSystem: ActorSystem =
if (rpcEnv.isInstanceOf[AkkaRpcEnv]) {
rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem
} else {
val actorSystem: ActorSystem = {
val actorSystemPort =
if (port == 0 || rpcEnv.address == null) {
port
......
......@@ -230,7 +230,7 @@ object Client {
RpcEnv.create("driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf))
val masterEndpoints = driverArgs.masters.map(RpcAddress.fromSparkURL).
map(rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, _, Master.ENDPOINT_NAME))
map(rpcEnv.setupEndpointRef(_, Master.ENDPOINT_NAME))
rpcEnv.setupEndpoint("client", new ClientEndpoint(rpcEnv, driverArgs, masterEndpoints, conf))
rpcEnv.awaitTermination()
......
......@@ -104,8 +104,7 @@ private[spark] class AppClient(
return
}
logInfo("Connecting to master " + masterAddress.toSparkURL + "...")
val masterRef =
rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, masterAddress, Master.ENDPOINT_NAME)
val masterRef = rpcEnv.setupEndpointRef(masterAddress, Master.ENDPOINT_NAME)
masterRef.send(RegisterApplication(appDescription, self))
} catch {
case ie: InterruptedException => // Cancelled
......
......@@ -45,7 +45,6 @@ private[deploy] class Worker(
cores: Int,
memory: Int,
masterRpcAddresses: Array[RpcAddress],
systemName: String,
endpointName: String,
workDirPath: String = null,
val conf: SparkConf,
......@@ -101,7 +100,7 @@ private[deploy] class Worker(
private var master: Option[RpcEndpointRef] = None
private var activeMasterUrl: String = ""
private[worker] var activeMasterWebUiUrl : String = ""
private val workerUri = rpcEnv.uriOf(systemName, rpcEnv.address, endpointName)
private val workerUri = RpcEndpointAddress(rpcEnv.address, endpointName).toString
private var registered = false
private var connected = false
private val workerId = generateWorkerId()
......@@ -209,8 +208,7 @@ private[deploy] class Worker(
override def run(): Unit = {
try {
logInfo("Connecting to master " + masterAddress + "...")
val masterEndpoint =
rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, masterAddress, Master.ENDPOINT_NAME)
val masterEndpoint = rpcEnv.setupEndpointRef(masterAddress, Master.ENDPOINT_NAME)
registerWithMaster(masterEndpoint)
} catch {
case ie: InterruptedException => // Cancelled
......@@ -266,8 +264,7 @@ private[deploy] class Worker(
override def run(): Unit = {
try {
logInfo("Connecting to master " + masterAddress + "...")
val masterEndpoint =
rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, masterAddress, Master.ENDPOINT_NAME)
val masterEndpoint = rpcEnv.setupEndpointRef(masterAddress, Master.ENDPOINT_NAME)
registerWithMaster(masterEndpoint)
} catch {
case ie: InterruptedException => // Cancelled
......@@ -711,7 +708,7 @@ private[deploy] object Worker extends Logging {
val rpcEnv = RpcEnv.create(systemName, host, port, conf, securityMgr)
val masterAddresses = masterUrls.map(RpcAddress.fromSparkURL(_))
rpcEnv.setupEndpoint(ENDPOINT_NAME, new Worker(rpcEnv, webUiPort, cores, memory,
masterAddresses, systemName, ENDPOINT_NAME, workDir, conf, securityMgr))
masterAddresses, ENDPOINT_NAME, workDir, conf, securityMgr))
rpcEnv
}
......
......@@ -15,10 +15,9 @@
* limitations under the License.
*/
package org.apache.spark.rpc.netty
package org.apache.spark.rpc
import org.apache.spark.SparkException
import org.apache.spark.rpc.RpcAddress
/**
* An address identifier for an RPC endpoint.
......@@ -29,7 +28,7 @@ import org.apache.spark.rpc.RpcAddress
* @param rpcAddress The socket address of the endpoint.
* @param name Name of the endpoint.
*/
private[netty] case class RpcEndpointAddress(val rpcAddress: RpcAddress, val name: String) {
private[spark] case class RpcEndpointAddress(val rpcAddress: RpcAddress, val name: String) {
require(name != null, "RpcEndpoint name must be provided.")
......@@ -44,7 +43,11 @@ private[netty] case class RpcEndpointAddress(val rpcAddress: RpcAddress, val nam
}
}
private[netty] object RpcEndpointAddress {
private[spark] object RpcEndpointAddress {
def apply(host: String, port: Int, name: String): RpcEndpointAddress = {
new RpcEndpointAddress(host, port, name)
}
def apply(sparkUrl: String): RpcEndpointAddress = {
try {
......
......@@ -23,7 +23,8 @@ import java.nio.channels.ReadableByteChannel
import scala.concurrent.Future
import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.util.{RpcUtils, Utils}
import org.apache.spark.rpc.netty.NettyRpcEnvFactory
import org.apache.spark.util.RpcUtils
/**
......@@ -32,15 +33,6 @@ import org.apache.spark.util.{RpcUtils, Utils}
*/
private[spark] object RpcEnv {
private def getRpcEnvFactory(conf: SparkConf): RpcEnvFactory = {
val rpcEnvNames = Map(
"akka" -> "org.apache.spark.rpc.akka.AkkaRpcEnvFactory",
"netty" -> "org.apache.spark.rpc.netty.NettyRpcEnvFactory")
val rpcEnvName = conf.get("spark.rpc", "netty")
val rpcEnvFactoryClassName = rpcEnvNames.getOrElse(rpcEnvName.toLowerCase, rpcEnvName)
Utils.classForName(rpcEnvFactoryClassName).newInstance().asInstanceOf[RpcEnvFactory]
}
def create(
name: String,
host: String,
......@@ -48,9 +40,8 @@ private[spark] object RpcEnv {
conf: SparkConf,
securityManager: SecurityManager,
clientMode: Boolean = false): RpcEnv = {
// Using Reflection to create the RpcEnv to avoid to depend on Akka directly
val config = RpcEnvConfig(conf, name, host, port, securityManager, clientMode)
getRpcEnvFactory(conf).create(config)
new NettyRpcEnvFactory().create(config)
}
}
......@@ -98,12 +89,11 @@ private[spark] abstract class RpcEnv(conf: SparkConf) {
}
/**
* Retrieve the [[RpcEndpointRef]] represented by `systemName`, `address` and `endpointName`.
* Retrieve the [[RpcEndpointRef]] represented by `address` and `endpointName`.
* This is a blocking action.
*/
def setupEndpointRef(
systemName: String, address: RpcAddress, endpointName: String): RpcEndpointRef = {
setupEndpointRefByURI(uriOf(systemName, address, endpointName))
def setupEndpointRef(address: RpcAddress, endpointName: String): RpcEndpointRef = {
setupEndpointRefByURI(RpcEndpointAddress(address, endpointName).toString)
}
/**
......@@ -124,12 +114,6 @@ private[spark] abstract class RpcEnv(conf: SparkConf) {
*/
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
/**
* [[RpcEndpointRef]] cannot be deserialized without [[RpcEnv]]. So when deserializing any object
* that contains [[RpcEndpointRef]]s, the deserialization codes should be wrapped by this method.
......
/*
* 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.io.File
import java.nio.channels.ReadableByteChannel
import java.util.concurrent.ConcurrentHashMap
import scala.concurrent.Future
import scala.language.postfixOps
import scala.reflect.ClassTag
import scala.util.control.NonFatal
import akka.actor.{ActorSystem, ExtendedActorSystem, Actor, ActorRef, Props, Address}
import akka.event.Logging.Error
import akka.pattern.{ask => akkaAsk}
import akka.remote.{AssociationEvent, AssociatedEvent, DisassociatedEvent, AssociationErrorEvent}
import akka.serialization.JavaSerializer
import org.apache.spark.{HttpFileServer, Logging, SecurityManager, SparkConf, SparkException}
import org.apache.spark.rpc._
import org.apache.spark.util.{ActorLogReceive, AkkaUtils, ThreadUtils}
/**
* 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.
*/
private[spark] class AkkaRpcEnv private[akka] (
val actorSystem: ActorSystem,
val securityManager: SecurityManager,
conf: SparkConf,
boundPort: Int)
extends RpcEnv(conf) 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 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]()
/**
* Need this map to remove `RpcEndpoint` from `endpointToRef` via a `RpcEndpointRef`
*/
private val refToEndpoint = new ConcurrentHashMap[RpcEndpointRef, RpcEndpoint]()
private val _fileServer = new AkkaFileServer(conf, securityManager)
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 = endpointToRef.get(endpoint)
override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = {
@volatile var endpointRef: AkkaRpcEndpointRef = null
// Use defered function because the Actor needs to use `endpointRef`.
// So `actorRef` should be created after assigning `endpointRef`.
val actorRef = () => actorSystem.actorOf(Props(new Actor with ActorLogReceive with Logging {
assert(endpointRef != null)
override def preStart(): Unit = {
// Listen for remote client network events
context.system.eventStream.subscribe(self, classOf[AssociationEvent])
safelyCall(endpoint) {
endpoint.onStart()
}
}
override def receiveWithLogging: Receive = {
case AssociatedEvent(_, remoteAddress, _) =>
safelyCall(endpoint) {
endpoint.onConnected(akkaAddressToRpcAddress(remoteAddress))
}
case DisassociatedEvent(_, remoteAddress, _) =>
safelyCall(endpoint) {
endpoint.onDisconnected(akkaAddressToRpcAddress(remoteAddress))
}
case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) =>
safelyCall(endpoint) {
endpoint.onNetworkError(cause, akkaAddressToRpcAddress(remoteAddress))
}
case e: AssociationEvent =>
// TODO ignore?
case m: AkkaMessage =>
logDebug(s"Received RPC message: $m")
safelyCall(endpoint) {
processMessage(endpoint, m, sender)
}
case AkkaFailure(e) =>
safelyCall(endpoint) {
throw e
}
case message: Any => {
logWarning(s"Unknown message: $message")
}
}
override def postStop(): Unit = {
unregisterEndpoint(endpoint.self)
safelyCall(endpoint) {
endpoint.onStop()
}
}
}), name = name)
endpointRef = new AkkaRpcEndpointRef(defaultAddress, actorRef, conf, initInConstructor = false)
registerEndpoint(endpoint, endpointRef)
// Now actorRef can be created safely
endpointRef.init()
endpointRef
}
private def processMessage(endpoint: RpcEndpoint, m: AkkaMessage, _sender: ActorRef): Unit = {
val message = m.message
val needReply = m.needReply
val pf: PartialFunction[Any, Unit] =
if (needReply) {
endpoint.receiveAndReply(new RpcCallContext {
override def sendFailure(e: Throwable): Unit = {
_sender ! AkkaFailure(e)
}
override def reply(response: Any): Unit = {
_sender ! AkkaMessage(response, false)
}
// Use "lazy" because most of RpcEndpoints don't need "senderAddress"
override lazy val senderAddress: RpcAddress =
new AkkaRpcEndpointRef(defaultAddress, _sender, conf).address
})
} else {
endpoint.receive
}
try {
pf.applyOrElse[Any, Unit](message, { message =>
throw new SparkException(s"Unmatched message $message from ${_sender}")
})
} catch {
case NonFatal(e) =>
_sender ! AkkaFailure(e)
if (!needReply) {
// If the sender does not require a reply, it may not handle the exception. So we rethrow
// "e" to make sure it will be processed.
throw e
}
}
}
/**
* Run `action` safely to avoid to crash the thread. If any non-fatal exception happens, it will
* call `endpoint.onError`. If `endpoint.onError` throws any non-fatal exception, just log it.
*/
private def safelyCall(endpoint: RpcEndpoint)(action: => Unit): Unit = {
try {
action
} catch {
case NonFatal(e) => {
try {
endpoint.onError(e)
} catch {
case NonFatal(e) => logError(s"Ignore error: ${e.getMessage}", e)
}
}
}
}
private def akkaAddressToRpcAddress(address: Address): RpcAddress = {
RpcAddress(address.host.getOrElse(defaultAddress.host),
address.port.getOrElse(defaultAddress.port))
}
override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = {
import actorSystem.dispatcher
actorSystem.actorSelection(uri).resolveOne(defaultLookupTimeout.duration).
map(new AkkaRpcEndpointRef(defaultAddress, _, conf)).
// this is just in case there is a timeout from creating the future in resolveOne, we want the
// exception to indicate the conf that determines the timeout
recover(defaultLookupTimeout.addMessageIfTimeout)
}
override def uriOf(systemName: String, address: RpcAddress, endpointName: String): String = {
AkkaUtils.address(
AkkaUtils.protocol(actorSystem), systemName, address.host, address.port, endpointName)
}
override def shutdown(): Unit = {
actorSystem.shutdown()
_fileServer.shutdown()
}
override def stop(endpoint: RpcEndpointRef): Unit = {
require(endpoint.isInstanceOf[AkkaRpcEndpointRef])
actorSystem.stop(endpoint.asInstanceOf[AkkaRpcEndpointRef].actorRef)
}
override def awaitTermination(): Unit = {
actorSystem.awaitTermination()
}
override def toString: String = s"${getClass.getSimpleName}($actorSystem)"
override def deserialize[T](deserializationAction: () => T): T = {
JavaSerializer.currentSystem.withValue(actorSystem.asInstanceOf[ExtendedActorSystem]) {
deserializationAction()
}
}
override def openChannel(uri: String): ReadableByteChannel = {
throw new UnsupportedOperationException(
"AkkaRpcEnv's files should be retrieved using an HTTP client.")
}
override def fileServer: RpcEnvFileServer = _fileServer
}
private[akka] class AkkaFileServer(
conf: SparkConf,
securityManager: SecurityManager) extends RpcEnvFileServer {
@volatile private var httpFileServer: HttpFileServer = _
override def addFile(file: File): String = {
getFileServer().addFile(file)
}
override def addJar(file: File): String = {
getFileServer().addJar(file)
}
override def addDirectory(baseUri: String, path: File): String = {
val fixedBaseUri = validateDirectoryUri(baseUri)
getFileServer().addDirectory(fixedBaseUri, path.getAbsolutePath())
}
def shutdown(): Unit = {
if (httpFileServer != null) {
httpFileServer.stop()
}
}
private def getFileServer(): HttpFileServer = {
if (httpFileServer == null) synchronized {
if (httpFileServer == null) {
httpFileServer = startFileServer()
}
}
httpFileServer
}
private def startFileServer(): HttpFileServer = {
val fileServerPort = conf.getInt("spark.fileserver.port", 0)
val server = new HttpFileServer(conf, securityManager, fileServerPort)
server.initialize()
server
}
}
private[spark] class AkkaRpcEnvFactory extends RpcEnvFactory {
def create(config: RpcEnvConfig): RpcEnv = {
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
config.name, config.host, config.port, config.conf, config.securityManager)
actorSystem.actorOf(Props(classOf[ErrorMonitor]), "ErrorMonitor")
new AkkaRpcEnv(actorSystem, config.securityManager, config.conf, boundPort)
}
}
/**
* Monitor errors reported by Akka and log them.
*/
private[akka] class ErrorMonitor extends Actor with ActorLogReceive with Logging {
override def preStart(): Unit = {
context.system.eventStream.subscribe(self, classOf[Error])
}
override def receiveWithLogging: Actor.Receive = {
case Error(cause: Throwable, _, _, message: String) => logDebug(message, cause)
}
}
private[akka] class AkkaRpcEndpointRef(
@transient private val defaultAddress: RpcAddress,
@transient private val _actorRef: () => ActorRef,
conf: SparkConf,
initInConstructor: Boolean)
extends RpcEndpointRef(conf) with Logging {
def this(
defaultAddress: RpcAddress,
_actorRef: ActorRef,
conf: SparkConf) = {
this(defaultAddress, () => _actorRef, conf, true)
}
lazy val actorRef = _actorRef()
override lazy val address: RpcAddress = {
val akkaAddress = actorRef.path.address
RpcAddress(akkaAddress.host.getOrElse(defaultAddress.host),
akkaAddress.port.getOrElse(defaultAddress.port))
}
override lazy val name: String = actorRef.path.name
private[akka] def init(): Unit = {
// Initialize the lazy vals
actorRef
address
name
}
if (initInConstructor) {
init()
}
override def send(message: Any): Unit = {
actorRef ! AkkaMessage(message, false)
}
override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = {
actorRef.ask(AkkaMessage(message, true))(timeout.duration).flatMap {
// The function will run in the calling thread, so it should be short and never block.
case msg @ AkkaMessage(message, reply) =>
if (reply) {
logError(s"Receive $msg but the sender cannot reply")
Future.failed(new SparkException(s"Receive $msg but the sender cannot reply"))
} else {
Future.successful(message)
}
case AkkaFailure(e) =>
Future.failed(e)
}(ThreadUtils.sameThread).mapTo[T].
recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread)
}
override def toString: String = s"${getClass.getSimpleName}($actorRef)"
final override def equals(that: Any): Boolean = that match {
case other: AkkaRpcEndpointRef => actorRef == other.actorRef
case _ => false
}
final override def hashCode(): Int = if (actorRef == null) 0 else actorRef.hashCode()
}
/**
* A wrapper to `message` so that the receiver knows if the sender expects a reply.
* @param message
* @param needReply if the sender expects a reply message
*/
private[akka] case class AkkaMessage(message: Any, needReply: Boolean)
/**
* A reply with the failure error from the receiver to the sender
*/
private[akka] case class AkkaFailure(e: Throwable)
......@@ -257,9 +257,6 @@ private[netty] class NettyRpcEnv(
dispatcher.getRpcEndpointRef(endpoint)
}
override def uriOf(systemName: String, address: RpcAddress, endpointName: String): String =
new RpcEndpointAddress(address, endpointName).toString
override def shutdown(): Unit = {
cleanup()
}
......@@ -427,7 +424,7 @@ private[netty] object NettyRpcEnv extends Logging {
}
private[netty] class NettyRpcEnvFactory extends RpcEnvFactory with Logging {
private[rpc] class NettyRpcEnvFactory extends RpcEnvFactory with Logging {
def create(config: RpcEnvConfig): RpcEnv = {
val sparkConf = config.conf
......
......@@ -19,9 +19,9 @@ package org.apache.spark.scheduler.cluster
import org.apache.hadoop.fs.{Path, FileSystem}
import org.apache.spark.rpc.RpcAddress
import org.apache.spark.{Logging, SparkContext, SparkEnv}
import org.apache.spark.{Logging, SparkContext}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.rpc.RpcEndpointAddress
import org.apache.spark.scheduler.TaskSchedulerImpl
private[spark] class SimrSchedulerBackend(
......@@ -39,9 +39,10 @@ private[spark] class SimrSchedulerBackend(
override def start() {
super.start()
val driverUrl = rpcEnv.uriOf(SparkEnv.driverActorSystemName,
RpcAddress(sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port").toInt),
CoarseGrainedSchedulerBackend.ENDPOINT_NAME)
val driverUrl = RpcEndpointAddress(
sc.conf.get("spark.driver.host"),
sc.conf.get("spark.driver.port").toInt,
CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
val conf = SparkHadoopUtil.get.newConfiguration(sc.conf)
val fs = FileSystem.get(conf)
......
......@@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster
import java.util.concurrent.Semaphore
import org.apache.spark.rpc.RpcAddress
import org.apache.spark.rpc.{RpcEndpointAddress, RpcAddress}
import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv}
import org.apache.spark.deploy.{ApplicationDescription, Command}
import org.apache.spark.deploy.client.{AppClient, AppClientListener}
......@@ -54,9 +54,10 @@ private[spark] class SparkDeploySchedulerBackend(
launcherBackend.connect()
// The endpoint for executors to talk to us
val driverUrl = rpcEnv.uriOf(SparkEnv.driverActorSystemName,
RpcAddress(sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port").toInt),
CoarseGrainedSchedulerBackend.ENDPOINT_NAME)
val driverUrl = RpcEndpointAddress(
sc.conf.get("spark.driver.host"),
sc.conf.get("spark.driver.port").toInt,
CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
val args = Seq(
"--driver-url", driverUrl,
"--executor-id", "{{EXECUTOR_ID}}",
......
......@@ -31,7 +31,7 @@ import org.apache.mesos.{Scheduler => MScheduler, SchedulerDriver}
import org.apache.spark.{SecurityManager, SparkContext, SparkEnv, SparkException, TaskState}
import org.apache.spark.network.netty.SparkTransportConf
import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient
import org.apache.spark.rpc.RpcAddress
import org.apache.spark.rpc.{RpcEndpointAddress, RpcAddress}
import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl}
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
import org.apache.spark.util.Utils
......@@ -215,10 +215,10 @@ private[spark] class CoarseMesosSchedulerBackend(
if (conf.contains("spark.testing")) {
"driverURL"
} else {
sc.env.rpcEnv.uriOf(
SparkEnv.driverActorSystemName,
RpcAddress(conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt),
CoarseGrainedSchedulerBackend.ENDPOINT_NAME)
RpcEndpointAddress(
conf.get("spark.driver.host"),
conf.get("spark.driver.port").toInt,
CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
}
}
......
/*
* 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.util
import akka.actor.Actor
import org.slf4j.Logger
/**
* A trait to enable logging all Akka actor messages. Here's an example of using this:
*
* {{{
* class BlockManagerMasterActor extends Actor with ActorLogReceive with Logging {
* ...
* override def receiveWithLogging = {
* case GetLocations(blockId) =>
* sender ! getLocations(blockId)
* ...
* }
* ...
* }
* }}}
*
*/
private[spark] trait ActorLogReceive {
self: Actor =>
override def receive: Actor.Receive = new Actor.Receive {
private val _receiveWithLogging = receiveWithLogging
override def isDefinedAt(o: Any): Boolean = {
val handled = _receiveWithLogging.isDefinedAt(o)
if (!handled) {
log.debug(s"Received unexpected actor system event: $o")
}
handled
}
override def apply(o: Any): Unit = {
if (log.isDebugEnabled) {
log.debug(s"[actor] received message $o from ${self.sender}")
}
val start = System.nanoTime
_receiveWithLogging.apply(o)
val timeTaken = (System.nanoTime - start).toDouble / 1000000
if (log.isDebugEnabled) {
log.debug(s"[actor] handled message ($timeTaken ms) $o from ${self.sender}")
}
}
}
def receiveWithLogging: Actor.Receive
protected def log: Logger
}
......@@ -19,14 +19,11 @@ package org.apache.spark.util
import scala.collection.JavaConverters._
import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem}
import akka.pattern.ask
import akka.actor.{ActorSystem, ExtendedActorSystem}
import com.typesafe.config.ConfigFactory
import org.apache.log4j.{Level, Logger}
import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException}
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.{Logging, SecurityManager, SparkConf}
/**
* Various utility classes for working with Akka.
......@@ -139,104 +136,4 @@ private[spark] object AkkaUtils extends Logging {
/** Space reserved for extra data in an Akka message besides serialized task or task result. */
val reservedSizeBytes = 200 * 1024
/**
* Send a message to the given actor and get its result within a default timeout, or
* throw a SparkException if this fails.
*/
def askWithReply[T](
message: Any,
actor: ActorRef,
timeout: RpcTimeout): T = {
askWithReply[T](message, actor, maxAttempts = 1, retryInterval = Int.MaxValue, timeout)
}
/**
* Send a message to the given actor and get its result within a default timeout, or
* throw a SparkException if this fails even after the specified number of retries.
*/
def askWithReply[T](
message: Any,
actor: ActorRef,
maxAttempts: Int,
retryInterval: Long,
timeout: RpcTimeout): T = {
// TODO: Consider removing multiple attempts
if (actor == null) {
throw new SparkException(s"Error sending message [message = $message]" +
" as actor is null ")
}
var attempts = 0
var lastException: Exception = null
while (attempts < maxAttempts) {
attempts += 1
try {
val future = actor.ask(message)(timeout.duration)
val result = timeout.awaitResult(future)
if (result == null) {
throw new SparkException("Actor returned null")
}
return result.asInstanceOf[T]
} catch {
case ie: InterruptedException => throw ie
case e: Exception =>
lastException = e
logWarning(s"Error sending message [message = $message] in $attempts attempts", e)
}
if (attempts < maxAttempts) {
Thread.sleep(retryInterval)
}
}
throw new SparkException(
s"Error sending message [message = $message]", lastException)
}
def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = {
val driverActorSystemName = SparkEnv.driverActorSystemName
val driverHost: String = conf.get("spark.driver.host", "localhost")
val driverPort: Int = conf.getInt("spark.driver.port", 7077)
Utils.checkHost(driverHost, "Expected hostname")
val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name)
val timeout = RpcUtils.lookupRpcTimeout(conf)
logInfo(s"Connecting to $name: $url")
timeout.awaitResult(actorSystem.actorSelection(url).resolveOne(timeout.duration))
}
def makeExecutorRef(
name: String,
conf: SparkConf,
host: String,
port: Int,
actorSystem: ActorSystem): ActorRef = {
val executorActorSystemName = SparkEnv.executorActorSystemName
Utils.checkHost(host, "Expected hostname")
val url = address(protocol(actorSystem), executorActorSystemName, host, port, name)
val timeout = RpcUtils.lookupRpcTimeout(conf)
logInfo(s"Connecting to $name: $url")
timeout.awaitResult(actorSystem.actorSelection(url).resolveOne(timeout.duration))
}
def protocol(actorSystem: ActorSystem): String = {
val akkaConf = actorSystem.settings.config
val sslProp = "akka.remote.netty.tcp.enable-ssl"
protocol(akkaConf.hasPath(sslProp) && akkaConf.getBoolean(sslProp))
}
def protocol(ssl: Boolean = false): String = {
if (ssl) {
"akka.ssl.tcp"
} else {
"akka.tcp"
}
}
def address(
protocol: String,
systemName: String,
host: String,
port: Int,
actorName: String): String = {
s"$protocol://$systemName@$host:$port/user/$actorName"
}
}
......@@ -20,20 +20,19 @@ package org.apache.spark.util
import scala.concurrent.duration.FiniteDuration
import scala.language.postfixOps
import org.apache.spark.{SparkEnv, SparkConf}
import org.apache.spark.SparkConf
import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, RpcTimeout}
object RpcUtils {
private[spark] object RpcUtils {
/**
* Retrieve a [[RpcEndpointRef]] which is located in the driver via its name.
*/
def makeDriverRef(name: String, conf: SparkConf, rpcEnv: RpcEnv): RpcEndpointRef = {
val driverActorSystemName = SparkEnv.driverActorSystemName
val driverHost: String = conf.get("spark.driver.host", "localhost")
val driverPort: Int = conf.getInt("spark.driver.port", 7077)
Utils.checkHost(driverHost, "Expected hostname")
rpcEnv.setupEndpointRef(driverActorSystemName, RpcAddress(driverHost, driverPort), name)
rpcEnv.setupEndpointRef(RpcAddress(driverHost, driverPort), name)
}
/** Returns the configured number of times to retry connecting */
......@@ -47,7 +46,7 @@ object RpcUtils {
}
/** Returns the default Spark timeout to use for RPC ask operations. */
private[spark] def askRpcTimeout(conf: SparkConf): RpcTimeout = {
def askRpcTimeout(conf: SparkConf): RpcTimeout = {
RpcTimeout(conf, Seq("spark.rpc.askTimeout", "spark.network.timeout"), "120s")
}
......@@ -57,7 +56,7 @@ object RpcUtils {
}
/** Returns the default Spark timeout to use for RPC remote endpoint lookup. */
private[spark] def lookupRpcTimeout(conf: SparkConf): RpcTimeout = {
def lookupRpcTimeout(conf: SparkConf): RpcTimeout = {
RpcTimeout(conf, Seq("spark.rpc.lookupTimeout", "spark.network.timeout"), "120s")
}
......
......@@ -125,7 +125,7 @@ class MapOutputTrackerSuite extends SparkFunSuite {
val slaveRpcEnv = createRpcEnv("spark-slave", hostname, 0, new SecurityManager(conf))
val slaveTracker = new MapOutputTrackerWorker(conf)
slaveTracker.trackerEndpoint =
slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME)
slaveRpcEnv.setupEndpointRef(rpcEnv.address, MapOutputTracker.ENDPOINT_NAME)
masterTracker.registerShuffle(10, 1)
masterTracker.incrementEpoch()
......
......@@ -41,7 +41,6 @@ object SSLSampleConfigs {
def sparkSSLConfig(): SparkConf = {
val conf = new SparkConf(loadDefaults = false)
conf.set("spark.rpc", "akka")
conf.set("spark.ssl.enabled", "true")
conf.set("spark.ssl.keyStore", keyStorePath)
conf.set("spark.ssl.keyStorePassword", "password")
......@@ -55,7 +54,6 @@ object SSLSampleConfigs {
def sparkSSLConfigUntrusted(): SparkConf = {
val conf = new SparkConf(loadDefaults = false)
conf.set("spark.rpc", "akka")
conf.set("spark.ssl.enabled", "true")
conf.set("spark.ssl.keyStore", untrustedKeyStorePath)
conf.set("spark.ssl.keyStorePassword", "password")
......
......@@ -474,7 +474,7 @@ class StandaloneDynamicAllocationSuite
(0 until numWorkers).map { i =>
val rpcEnv = workerRpcEnvs(i)
val worker = new Worker(rpcEnv, 0, cores, memory, Array(masterRpcEnv.address),
Worker.SYSTEM_NAME + i, Worker.ENDPOINT_NAME, null, conf, securityManager)
Worker.ENDPOINT_NAME, null, conf, securityManager)
rpcEnv.setupEndpoint(Worker.ENDPOINT_NAME, worker)
worker
}
......
......@@ -147,7 +147,7 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd
(0 until numWorkers).map { i =>
val rpcEnv = workerRpcEnvs(i)
val worker = new Worker(rpcEnv, 0, cores, memory, Array(masterRpcEnv.address),
Worker.SYSTEM_NAME + i, Worker.ENDPOINT_NAME, null, conf, securityManager)
Worker.ENDPOINT_NAME, null, conf, securityManager)
rpcEnv.setupEndpoint(Worker.ENDPOINT_NAME, worker)
worker
}
......
......@@ -98,7 +98,7 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva
Master.startRpcEnvAndEndpoint("127.0.0.1", 0, 0, conf)
try {
rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, rpcEnv.address, Master.ENDPOINT_NAME)
rpcEnv.setupEndpointRef(rpcEnv.address, Master.ENDPOINT_NAME)
CustomPersistenceEngine.lastInstance.isDefined shouldBe true
val persistenceEngine = CustomPersistenceEngine.lastInstance.get
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment