2012-09-12 11:18:42 +02:00
|
|
|
package akka.remote
|
|
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
import scala.language.postfixOps
|
2012-09-12 11:18:42 +02:00
|
|
|
import akka.actor.SupervisorStrategy._
|
|
|
|
|
import akka.actor._
|
|
|
|
|
import akka.event.{ Logging, LoggingAdapter }
|
2012-12-12 15:04:44 +01:00
|
|
|
import akka.pattern.{ gracefulStop, pipe }
|
2012-11-23 10:15:19 +01:00
|
|
|
import akka.remote.EndpointManager.{ StartupFinished, ManagementCommand, Listen, Send }
|
|
|
|
|
import akka.remote.transport.Transport.{ AssociationEventListener, InboundAssociation }
|
2012-09-12 11:18:42 +02:00
|
|
|
import akka.remote.transport._
|
|
|
|
|
import akka.util.Timeout
|
2012-11-23 10:15:19 +01:00
|
|
|
import com.typesafe.config.{ ConfigFactory, Config }
|
2012-09-12 11:18:42 +02:00
|
|
|
import scala.collection.immutable.{ Seq, HashMap }
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import scala.concurrent.{ Promise, Await, Future }
|
|
|
|
|
import scala.util.control.NonFatal
|
|
|
|
|
import java.net.URLEncoder
|
|
|
|
|
import java.util.concurrent.TimeoutException
|
|
|
|
|
import scala.util.{ Failure, Success }
|
2012-11-21 14:18:24 +01:00
|
|
|
import scala.collection.immutable
|
|
|
|
|
import akka.japi.Util.immutableSeq
|
2012-12-11 13:08:36 +01:00
|
|
|
import akka.remote.Remoting.{ TransportSupervisor, RegisterTransportActor }
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
class RemotingSettings(val config: Config) {
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
import config._
|
|
|
|
|
import scala.collection.JavaConverters._
|
|
|
|
|
|
|
|
|
|
val LogLifecycleEvents: Boolean = getBoolean("akka.remoting.log-remote-lifecycle-events")
|
|
|
|
|
|
|
|
|
|
val ShutdownTimeout: FiniteDuration = Duration(getMilliseconds("akka.remoting.shutdown-timeout"), MILLISECONDS)
|
|
|
|
|
|
|
|
|
|
val StartupTimeout: FiniteDuration = Duration(getMilliseconds("akka.remoting.startup-timeout"), MILLISECONDS)
|
|
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
val RetryGateClosedFor: Long = getNanoseconds("akka.remoting.retry-gate-closed-for")
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
val UsePassiveConnections: Boolean = getBoolean("akka.remoting.use-passive-connections")
|
|
|
|
|
|
|
|
|
|
val MaximumRetriesInWindow: Int = getInt("akka.remoting.maximum-retries-in-window")
|
|
|
|
|
|
|
|
|
|
val RetryWindow: FiniteDuration = Duration(getMilliseconds("akka.remoting.retry-window"), MILLISECONDS)
|
|
|
|
|
|
|
|
|
|
val BackoffPeriod: FiniteDuration =
|
|
|
|
|
Duration(getMilliseconds("akka.remoting.backoff-interval"), MILLISECONDS)
|
|
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
val Transports: Seq[(String, Seq[String], Config)] = transportNames.map { name ⇒
|
|
|
|
|
val transportConfig = transportConfigFor(name)
|
|
|
|
|
(transportConfig.getString("transport-class"),
|
|
|
|
|
immutableSeq(transportConfig.getStringList("applied-adapters")),
|
|
|
|
|
transportConfig)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val Adapters: Map[String, String] = configToMap(getConfig("akka.remoting.adapters"))
|
|
|
|
|
|
|
|
|
|
private def transportNames: Seq[String] = immutableSeq(getStringList("akka.remoting.enabled-transports"))
|
|
|
|
|
|
|
|
|
|
private def transportConfigFor(transportName: String): Config = getConfig("akka.remoting.transports." + transportName)
|
|
|
|
|
|
|
|
|
|
private def configToMap(cfg: Config): Map[String, String] =
|
|
|
|
|
cfg.root.unwrapped.asScala.toMap.map { case (k, v) ⇒ (k, v.toString) }
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
2012-12-12 14:49:38 +01:00
|
|
|
private[remote] object AddressUrlEncoder {
|
|
|
|
|
def apply(address: Address): String = URLEncoder.encode(address.toString, "utf-8")
|
|
|
|
|
}
|
|
|
|
|
|
2012-12-11 13:08:36 +01:00
|
|
|
private[remote] case class RARP(provider: RemoteActorRefProvider) extends Extension
|
|
|
|
|
private[remote] object RARP extends ExtensionId[RARP] with ExtensionIdProvider {
|
|
|
|
|
|
|
|
|
|
override def lookup() = RARP
|
|
|
|
|
|
|
|
|
|
override def createExtension(system: ExtendedActorSystem) = RARP(system.provider.asInstanceOf[RemoteActorRefProvider])
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
private[remote] object Remoting {
|
|
|
|
|
|
2012-11-22 13:33:48 +01:00
|
|
|
final val EndpointManagerName = "endpointManager"
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
def localAddressForRemote(transportMapping: Map[String, Set[(Transport, Address)]], remote: Address): Address = {
|
|
|
|
|
|
|
|
|
|
transportMapping.get(remote.protocol) match {
|
|
|
|
|
case Some(transports) ⇒
|
2012-11-22 13:33:48 +01:00
|
|
|
val responsibleTransports = transports.filter { case (t, _) ⇒ t.isResponsibleFor(remote) }
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
responsibleTransports.size match {
|
|
|
|
|
case 0 ⇒
|
|
|
|
|
throw new RemoteTransportException(
|
2012-11-23 10:15:19 +01:00
|
|
|
s"No transport is responsible for address: ${remote} although protocol ${remote.protocol} is available." +
|
2012-09-12 11:18:42 +02:00
|
|
|
" Make sure at least one transport is configured to be responsible for the address.",
|
|
|
|
|
null)
|
|
|
|
|
|
|
|
|
|
case 1 ⇒
|
|
|
|
|
responsibleTransports.head._2
|
|
|
|
|
|
|
|
|
|
case _ ⇒
|
|
|
|
|
throw new RemoteTransportException(
|
2012-11-22 13:33:48 +01:00
|
|
|
s"Multiple transports are available for [${remote}]: [${responsibleTransports.mkString(",")}]. " +
|
2012-09-12 11:18:42 +02:00
|
|
|
"Remoting cannot decide which transport to use to reach the remote system. Change your configuration " +
|
|
|
|
|
"so that only one transport is responsible for the address.",
|
|
|
|
|
null)
|
|
|
|
|
}
|
2012-11-23 10:15:19 +01:00
|
|
|
case None ⇒ throw new RemoteTransportException(
|
|
|
|
|
s"No transport is loaded for protocol: ${remote.protocol}, available protocols: ${transportMapping.keys.mkString}", null)
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
case class RegisterTransportActor(props: Props, name: String)
|
|
|
|
|
|
2012-12-11 13:08:36 +01:00
|
|
|
private[Remoting] class TransportSupervisor extends Actor {
|
|
|
|
|
override def supervisorStrategy = OneForOneStrategy() {
|
|
|
|
|
case NonFatal(e) ⇒ Restart
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receive = {
|
|
|
|
|
case RegisterTransportActor(props, name) ⇒ sender ! context.actorOf(props, name)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) {
|
|
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
@volatile private var endpointManager: Option[ActorRef] = None
|
2012-11-21 14:18:24 +01:00
|
|
|
@volatile private var transportMapping: Map[String, Set[(Transport, Address)]] = _
|
2012-12-12 12:14:43 +01:00
|
|
|
// This is effectively a write-once variable similar to a lazy val. The reason for not using a lazy val is exception
|
|
|
|
|
// handling.
|
2012-09-12 11:18:42 +02:00
|
|
|
@volatile var addresses: Set[Address] = _
|
2012-12-12 15:04:44 +01:00
|
|
|
// This variable has the same semantics as the addresses variable, in the sense it is written once, and emulates
|
|
|
|
|
// a lazy val
|
|
|
|
|
@volatile var defaultAddress: Address = _
|
2012-11-21 14:18:24 +01:00
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
private val settings = new RemotingSettings(provider.remoteSettings.config)
|
|
|
|
|
|
2012-12-11 13:08:36 +01:00
|
|
|
val transportSupervisor = system.asInstanceOf[ActorSystemImpl].systemActorOf(Props[TransportSupervisor], "transports")
|
2012-11-23 10:15:19 +01:00
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
override def localAddressForRemote(remote: Address): Address = Remoting.localAddressForRemote(transportMapping, remote)
|
|
|
|
|
|
|
|
|
|
val log: LoggingAdapter = Logging(system.eventStream, "Remoting")
|
|
|
|
|
val eventPublisher = new EventPublisher(system, log, settings.LogLifecycleEvents)
|
|
|
|
|
|
|
|
|
|
private def notifyError(msg: String, cause: Throwable): Unit =
|
|
|
|
|
eventPublisher.notifyListeners(RemotingErrorEvent(new RemoteTransportException(msg, cause)))
|
|
|
|
|
|
|
|
|
|
override def shutdown(): Unit = {
|
2012-12-12 12:14:43 +01:00
|
|
|
endpointManager match {
|
|
|
|
|
case Some(manager) ⇒
|
|
|
|
|
try {
|
|
|
|
|
val stopped: Future[Boolean] = gracefulStop(manager, settings.ShutdownTimeout)(system)
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
if (Await.result(stopped, settings.ShutdownTimeout)) {
|
|
|
|
|
eventPublisher.notifyListeners(RemotingShutdownEvent)
|
|
|
|
|
}
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
} catch {
|
|
|
|
|
case e: TimeoutException ⇒ notifyError("Shutdown timed out.", e)
|
|
|
|
|
case NonFatal(e) ⇒ notifyError("Shutdown failed.", e)
|
|
|
|
|
} finally endpointManager = None
|
|
|
|
|
case None ⇒ log.warning("Remoting is not running. Ignoring shutdown attempt.")
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Start assumes that it cannot be followed by another start() without having a shutdown() first
|
|
|
|
|
override def start(): Unit = {
|
2012-12-12 12:14:43 +01:00
|
|
|
endpointManager match {
|
|
|
|
|
case None ⇒
|
|
|
|
|
log.info("Starting remoting")
|
|
|
|
|
val manager: ActorRef = system.asInstanceOf[ActorSystemImpl].systemActorOf(
|
|
|
|
|
Props(new EndpointManager(provider.remoteSettings.config, log)), Remoting.EndpointManagerName)
|
|
|
|
|
endpointManager = Some(manager)
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
implicit val timeout = new Timeout(settings.StartupTimeout)
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
try {
|
2012-12-12 15:04:44 +01:00
|
|
|
val addressesPromise: Promise[Seq[(Transport, Address)]] = Promise()
|
2012-12-12 12:14:43 +01:00
|
|
|
manager ! Listen(addressesPromise)
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 15:04:44 +01:00
|
|
|
val transports: Seq[(Transport, Address)] = Await.result(addressesPromise.future, timeout.duration)
|
|
|
|
|
if (transports.isEmpty) throw new RemoteTransportException("No transport drivers were loaded.", null)
|
|
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
transportMapping = transports.groupBy { case (transport, _) ⇒ transport.schemeIdentifier }.mapValues {
|
|
|
|
|
_.toSet
|
|
|
|
|
}
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 15:04:44 +01:00
|
|
|
defaultAddress = transports.head._2
|
2012-12-12 12:14:43 +01:00
|
|
|
addresses = transports.map { _._2 }.toSet
|
2012-12-07 16:03:04 +01:00
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
manager ! StartupFinished
|
|
|
|
|
eventPublisher.notifyListeners(RemotingListenEvent(addresses))
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
} catch {
|
|
|
|
|
case e: TimeoutException ⇒ notifyError("Startup timed out", e)
|
|
|
|
|
case NonFatal(e) ⇒ notifyError("Startup failed", e)
|
|
|
|
|
}
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
case Some(_) ⇒
|
|
|
|
|
log.warning("Remoting was already started. Ignoring start attempt.")
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// TODO: this is called in RemoteActorRefProvider to handle the lifecycle of connections (clients)
|
|
|
|
|
// which is not how things work in the new remoting
|
|
|
|
|
override def shutdownClientConnection(address: Address): Unit = {
|
|
|
|
|
// Ignore
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// TODO: this is never called anywhere, should be taken out from RemoteTransport API
|
|
|
|
|
override def restartClientConnection(address: Address): Unit = {
|
|
|
|
|
// Ignore
|
|
|
|
|
}
|
|
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = endpointManager match {
|
|
|
|
|
case Some(manager) ⇒ manager.tell(Send(message, senderOption, recipient), sender = senderOption getOrElse Actor.noSender)
|
|
|
|
|
case None ⇒ throw new IllegalStateException("Attempted to send remote message but Remoting is not running.")
|
|
|
|
|
}
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
override def managementCommand(cmd: Any): Future[Boolean] = endpointManager match {
|
|
|
|
|
case Some(manager) ⇒
|
|
|
|
|
val statusPromise = Promise[Boolean]()
|
|
|
|
|
manager.tell(ManagementCommand(cmd, statusPromise), sender = Actor.noSender)
|
|
|
|
|
statusPromise.future
|
|
|
|
|
case None ⇒ throw new IllegalStateException("Attempted to send management command but Remoting is not running.")
|
2012-11-23 10:15:19 +01:00
|
|
|
}
|
|
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
// Not used anywhere only to keep compatibility with RemoteTransport interface
|
|
|
|
|
protected def useUntrustedMode: Boolean = provider.remoteSettings.UntrustedMode
|
|
|
|
|
|
|
|
|
|
// Not used anywhere only to keep compatibility with RemoteTransport interface
|
|
|
|
|
protected def logRemoteLifeCycleEvents: Boolean = provider.remoteSettings.LogRemoteLifeCycleEvents
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private[remote] object EndpointManager {
|
|
|
|
|
|
|
|
|
|
sealed trait RemotingCommand
|
2012-12-12 15:04:44 +01:00
|
|
|
case class Listen(addressesPromise: Promise[Seq[(Transport, Address)]]) extends RemotingCommand
|
2012-11-22 13:33:48 +01:00
|
|
|
case object StartupFinished extends RemotingCommand
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
case class Send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef) extends RemotingCommand {
|
|
|
|
|
override def toString = s"Remote message $senderOption -> $recipient"
|
|
|
|
|
}
|
|
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
case class ManagementCommand(cmd: Any, statusPromise: Promise[Boolean]) extends RemotingCommand
|
|
|
|
|
|
2012-12-12 15:04:44 +01:00
|
|
|
case class ListensResult(addressesPromise: Promise[Seq[(Transport, Address)]],
|
|
|
|
|
results: Seq[(Transport, Address, Promise[AssociationEventListener])])
|
|
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
sealed trait EndpointPolicy
|
|
|
|
|
case class Pass(endpoint: ActorRef) extends EndpointPolicy
|
2012-11-22 13:33:48 +01:00
|
|
|
case class Gated(timeOfFailure: Long) extends EndpointPolicy
|
2012-09-12 11:18:42 +02:00
|
|
|
case class Quarantined(reason: Throwable) extends EndpointPolicy
|
|
|
|
|
|
|
|
|
|
case object Prune
|
|
|
|
|
|
|
|
|
|
// Not threadsafe -- only to be used in HeadActor
|
|
|
|
|
private[EndpointManager] class EndpointRegistry {
|
2012-11-21 16:39:04 +01:00
|
|
|
private var addressToEndpointAndPolicy = HashMap[Address, EndpointPolicy]()
|
|
|
|
|
private var endpointToAddress = HashMap[ActorRef, Address]()
|
|
|
|
|
private var addressToPassive = HashMap[Address, ActorRef]()
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
def getEndpointWithPolicy(address: Address): Option[EndpointPolicy] = addressToEndpointAndPolicy.get(address)
|
|
|
|
|
|
2012-11-21 16:39:04 +01:00
|
|
|
def hasActiveEndpointFor(address: Address): Boolean = addressToEndpointAndPolicy.get(address) match {
|
|
|
|
|
case Some(Pass(_)) ⇒ true
|
|
|
|
|
case _ ⇒ false
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def passiveEndpointFor(address: Address): Option[ActorRef] = addressToPassive.get(address)
|
|
|
|
|
|
2012-11-22 13:33:48 +01:00
|
|
|
def isQuarantined(address: Address): Boolean = addressToEndpointAndPolicy.get(address) match {
|
|
|
|
|
case Some(Quarantined(_)) ⇒ true
|
|
|
|
|
case _ ⇒ false
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
def prune(pruneAge: Long): Unit = {
|
|
|
|
|
addressToEndpointAndPolicy = addressToEndpointAndPolicy.filter {
|
2012-11-22 13:33:48 +01:00
|
|
|
case (_, Gated(timeOfFailure)) ⇒ timeOfFailure + pruneAge > System.nanoTime()
|
|
|
|
|
case _ ⇒ true
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-11-21 16:39:04 +01:00
|
|
|
def registerActiveEndpoint(address: Address, endpoint: ActorRef): ActorRef = {
|
2012-09-12 11:18:42 +02:00
|
|
|
addressToEndpointAndPolicy = addressToEndpointAndPolicy + (address -> Pass(endpoint))
|
|
|
|
|
endpointToAddress = endpointToAddress + (endpoint -> address)
|
|
|
|
|
endpoint
|
|
|
|
|
}
|
|
|
|
|
|
2012-11-21 16:39:04 +01:00
|
|
|
def registerPassiveEndpoint(address: Address, endpoint: ActorRef): ActorRef = {
|
|
|
|
|
addressToPassive = addressToPassive + (address -> endpoint)
|
|
|
|
|
endpointToAddress = endpointToAddress + (endpoint -> address)
|
|
|
|
|
endpoint
|
|
|
|
|
}
|
|
|
|
|
|
2012-12-12 12:29:36 +01:00
|
|
|
def isPassive(endpoint: ActorRef): Boolean = addressToPassive.contains(endpointToAddress(endpoint))
|
|
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
def markFailed(endpoint: ActorRef, timeOfFailure: Long): Unit = {
|
2012-11-22 13:33:48 +01:00
|
|
|
addressToEndpointAndPolicy += endpointToAddress(endpoint) -> Gated(timeOfFailure)
|
2012-12-12 12:29:36 +01:00
|
|
|
if (!isPassive(endpoint)) endpointToAddress = endpointToAddress - endpoint
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def markQuarantine(address: Address, reason: Throwable): Unit =
|
|
|
|
|
addressToEndpointAndPolicy += address -> Quarantined(reason)
|
|
|
|
|
|
2012-11-22 13:33:48 +01:00
|
|
|
def removeIfNotGated(endpoint: ActorRef): Unit = {
|
2012-09-12 11:18:42 +02:00
|
|
|
endpointToAddress.get(endpoint) foreach { address ⇒
|
2012-12-12 13:39:57 +01:00
|
|
|
addressToEndpointAndPolicy.get(address) foreach {
|
2012-12-12 12:29:36 +01:00
|
|
|
case Pass(_) ⇒ addressToEndpointAndPolicy = addressToEndpointAndPolicy - address
|
|
|
|
|
case _ ⇒
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
2012-12-12 12:29:36 +01:00
|
|
|
|
|
|
|
|
endpointToAddress = endpointToAddress - endpoint
|
|
|
|
|
addressToPassive = addressToPassive - address
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends Actor {
|
|
|
|
|
|
|
|
|
|
import EndpointManager._
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
|
|
|
|
|
val settings = new RemotingSettings(conf)
|
|
|
|
|
val extendedSystem = context.system.asInstanceOf[ExtendedActorSystem]
|
2012-11-22 13:33:48 +01:00
|
|
|
val endpointId: Iterator[Int] = Iterator from 0
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
val eventPublisher = new EventPublisher(context.system, log, settings.LogLifecycleEvents)
|
|
|
|
|
|
|
|
|
|
// Mapping between addresses and endpoint actors. If passive connections are turned off, incoming connections
|
|
|
|
|
// will be not part of this map!
|
|
|
|
|
val endpoints = new EndpointRegistry
|
|
|
|
|
// Mapping between transports and the local addresses they listen to
|
|
|
|
|
var transportMapping: Map[Address, Transport] = Map()
|
|
|
|
|
|
2012-12-12 12:14:43 +01:00
|
|
|
def retryGateEnabled = settings.RetryGateClosedFor > 0L
|
2012-11-22 13:33:48 +01:00
|
|
|
val pruneInterval: Long = if (retryGateEnabled) settings.RetryGateClosedFor * 2L else 0L
|
|
|
|
|
val pruneTimerCancellable: Option[Cancellable] = if (retryGateEnabled)
|
2012-09-12 11:18:42 +02:00
|
|
|
Some(context.system.scheduler.schedule(pruneInterval milliseconds, pruneInterval milliseconds, self, Prune))
|
|
|
|
|
else None
|
|
|
|
|
|
|
|
|
|
override val supervisorStrategy = OneForOneStrategy(settings.MaximumRetriesInWindow, settings.RetryWindow) {
|
|
|
|
|
case InvalidAssociation(localAddress, remoteAddress, e) ⇒
|
|
|
|
|
endpoints.markQuarantine(remoteAddress, e)
|
|
|
|
|
Stop
|
|
|
|
|
|
|
|
|
|
case NonFatal(e) ⇒
|
2012-11-22 13:33:48 +01:00
|
|
|
if (!retryGateEnabled)
|
2012-09-12 11:18:42 +02:00
|
|
|
// This strategy keeps all the messages in the stash of the endpoint so restart will transfer the queue
|
|
|
|
|
// to the restarted endpoint -- thus no messages are lost
|
|
|
|
|
Restart
|
|
|
|
|
else {
|
|
|
|
|
// This strategy throws away all the messages enqueued in the endpoint (in its stash), registers the time of failure,
|
2012-11-22 13:33:48 +01:00
|
|
|
// keeps throwing away messages until the retry gate becomes open (time specified in RetryGateClosedFor)
|
2012-09-12 11:18:42 +02:00
|
|
|
endpoints.markFailed(sender, System.nanoTime())
|
|
|
|
|
Stop
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def receive = {
|
2012-12-12 12:14:43 +01:00
|
|
|
case Listen(addressesPromise) ⇒
|
2012-12-12 15:04:44 +01:00
|
|
|
listens map { ListensResult(addressesPromise, _) } pipeTo self
|
|
|
|
|
|
|
|
|
|
case ListensResult(addressesPromise, results) ⇒
|
|
|
|
|
transportMapping = results.groupBy {
|
|
|
|
|
case (_, transportAddress, _) ⇒ transportAddress
|
|
|
|
|
} map {
|
|
|
|
|
case (a, t) if t.size > 1 ⇒
|
|
|
|
|
throw new RemoteTransportException(s"There are more than one transports listening on local address [$a]", null)
|
|
|
|
|
case (a, t) ⇒ a -> t.head._1
|
|
|
|
|
}
|
|
|
|
|
// Register to each transport as listener and collect mapping to addresses
|
|
|
|
|
val transportsAndAddresses = results map {
|
|
|
|
|
case (transport, address, promise) ⇒
|
|
|
|
|
promise.success(self)
|
|
|
|
|
transport -> address
|
2012-12-12 12:14:43 +01:00
|
|
|
}
|
2012-12-12 15:04:44 +01:00
|
|
|
addressesPromise.success(transportsAndAddresses)
|
2012-11-22 13:33:48 +01:00
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
case ManagementCommand(_, statusPromise) ⇒ statusPromise.success(false)
|
|
|
|
|
|
|
|
|
|
case StartupFinished ⇒ context.become(accepting)
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val accepting: Receive = {
|
2012-11-23 10:15:19 +01:00
|
|
|
case ManagementCommand(cmd, statusPromise) ⇒
|
|
|
|
|
transportMapping.values foreach { _.managementCommand(cmd, statusPromise) }
|
|
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
case s @ Send(message, senderOption, recipientRef) ⇒
|
|
|
|
|
val recipientAddress = recipientRef.path.address
|
|
|
|
|
|
|
|
|
|
endpoints.getEndpointWithPolicy(recipientAddress) match {
|
|
|
|
|
case Some(Pass(endpoint)) ⇒ endpoint ! s
|
2012-11-22 13:33:48 +01:00
|
|
|
case Some(Gated(timeOfFailure)) ⇒ if (retryGateOpen(timeOfFailure)) {
|
2012-12-12 12:14:43 +01:00
|
|
|
val endpoint = createEndpoint(
|
|
|
|
|
recipientAddress,
|
|
|
|
|
recipientRef.localAddressToUse,
|
|
|
|
|
transportMapping(recipientRef.localAddressToUse),
|
|
|
|
|
settings,
|
|
|
|
|
None)
|
2012-11-21 16:39:04 +01:00
|
|
|
endpoints.registerActiveEndpoint(recipientAddress, endpoint)
|
|
|
|
|
endpoint ! s
|
2012-12-12 12:14:43 +01:00
|
|
|
} else extendedSystem.deadLetters forward message
|
|
|
|
|
case Some(Quarantined(_)) ⇒ extendedSystem.deadLetters forward message
|
2012-11-21 16:39:04 +01:00
|
|
|
case None ⇒
|
2012-12-12 12:14:43 +01:00
|
|
|
val endpoint = createEndpoint(
|
|
|
|
|
recipientAddress,
|
|
|
|
|
recipientRef.localAddressToUse,
|
|
|
|
|
transportMapping(recipientRef.localAddressToUse),
|
|
|
|
|
settings,
|
|
|
|
|
None)
|
2012-11-21 16:39:04 +01:00
|
|
|
endpoints.registerActiveEndpoint(recipientAddress, endpoint)
|
|
|
|
|
endpoint ! s
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2012-11-21 16:39:04 +01:00
|
|
|
case InboundAssociation(handle) ⇒ endpoints.passiveEndpointFor(handle.remoteAddress) match {
|
|
|
|
|
case Some(endpoint) ⇒ endpoint ! EndpointWriter.TakeOver(handle)
|
|
|
|
|
case None ⇒
|
2012-12-12 12:14:43 +01:00
|
|
|
val endpoint = createEndpoint(
|
|
|
|
|
handle.remoteAddress,
|
|
|
|
|
handle.localAddress,
|
|
|
|
|
transportMapping(handle.localAddress),
|
|
|
|
|
settings,
|
|
|
|
|
Some(handle))
|
2012-11-21 16:39:04 +01:00
|
|
|
eventPublisher.notifyListeners(AssociatedEvent(handle.localAddress, handle.remoteAddress, true))
|
|
|
|
|
if (settings.UsePassiveConnections && !endpoints.hasActiveEndpointFor(handle.remoteAddress)) {
|
|
|
|
|
endpoints.registerActiveEndpoint(handle.remoteAddress, endpoint)
|
2012-11-22 13:33:48 +01:00
|
|
|
} else if (!endpoints.isQuarantined(handle.remoteAddress))
|
|
|
|
|
endpoints.registerPassiveEndpoint(handle.remoteAddress, endpoint)
|
|
|
|
|
else handle.disassociate()
|
2012-11-21 16:39:04 +01:00
|
|
|
}
|
2012-12-07 16:03:04 +01:00
|
|
|
case Terminated(endpoint) ⇒ endpoints.removeIfNotGated(endpoint)
|
2012-11-22 13:33:48 +01:00
|
|
|
case Prune ⇒ endpoints.prune(settings.RetryGateClosedFor)
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
2012-12-12 15:04:44 +01:00
|
|
|
private def listens: Future[Seq[(Transport, Address, Promise[AssociationEventListener])]] = {
|
2012-11-23 10:15:19 +01:00
|
|
|
val transports = for ((fqn, adapters, config) ← settings.Transports) yield {
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
val args = Seq(classOf[ExtendedActorSystem] -> context.system, classOf[Config] -> config)
|
|
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
val driver = extendedSystem.dynamicAccess
|
2012-09-12 11:18:42 +02:00
|
|
|
.createInstanceFor[Transport](fqn, args).recover({
|
|
|
|
|
|
|
|
|
|
case exception ⇒ throw new IllegalArgumentException(
|
|
|
|
|
(s"Cannot instantiate transport [$fqn]. " +
|
|
|
|
|
"Make sure it extends [akka.remote.transport.Transport] and has constructor with " +
|
|
|
|
|
"[akka.actor.ExtendedActorSystem] and [com.typesafe.config.Config] parameters"), exception)
|
|
|
|
|
|
|
|
|
|
}).get
|
|
|
|
|
|
2012-11-23 10:15:19 +01:00
|
|
|
val wrappedTransport =
|
|
|
|
|
adapters.map { TransportAdaptersExtension.get(context.system).getAdapterProvider(_) }.foldLeft(driver) {
|
|
|
|
|
(t: Transport, provider: TransportAdapterProvider) ⇒
|
|
|
|
|
provider(t, context.system.asInstanceOf[ExtendedActorSystem])
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-12 11:18:42 +02:00
|
|
|
new AkkaProtocolTransport(wrappedTransport, context.system, new AkkaProtocolSettings(conf), AkkaPduProtobufCodec)
|
|
|
|
|
}
|
|
|
|
|
|
2012-12-12 15:04:44 +01:00
|
|
|
// Collect all transports, listen addresses and listener promises in one future
|
|
|
|
|
Future.sequence(transports.map { transport ⇒
|
|
|
|
|
transport.listen map { case (address, listenerPromise) ⇒ (transport, address, listenerPromise) }
|
|
|
|
|
})
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def createEndpoint(remoteAddress: Address,
|
|
|
|
|
localAddress: Address,
|
2012-12-12 12:14:43 +01:00
|
|
|
transport: Transport,
|
|
|
|
|
endpointSettings: RemotingSettings,
|
2012-09-12 11:18:42 +02:00
|
|
|
handleOption: Option[AssociationHandle]): ActorRef = {
|
2012-12-07 16:03:04 +01:00
|
|
|
assert(transportMapping contains localAddress)
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
context.watch(context.actorOf(Props(
|
2012-09-12 11:18:42 +02:00
|
|
|
new EndpointWriter(
|
|
|
|
|
handleOption,
|
|
|
|
|
localAddress,
|
|
|
|
|
remoteAddress,
|
2012-12-12 12:14:43 +01:00
|
|
|
transport,
|
|
|
|
|
endpointSettings,
|
2012-09-12 11:18:42 +02:00
|
|
|
AkkaPduProtobufCodec))
|
|
|
|
|
.withDispatcher("akka.remoting.writer-dispatcher"),
|
2012-12-12 14:49:38 +01:00
|
|
|
"endpointWriter-" + AddressUrlEncoder(remoteAddress) + "-" + endpointId.next()))
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
2012-11-22 13:33:48 +01:00
|
|
|
private def retryGateOpen(timeOfFailure: Long): Boolean = (timeOfFailure + settings.RetryGateClosedFor) < System.nanoTime()
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
pruneTimerCancellable.foreach { _.cancel() }
|
|
|
|
|
transportMapping.values foreach { transport ⇒
|
2012-11-22 13:33:48 +01:00
|
|
|
try transport.shutdown() catch {
|
2012-09-12 11:18:42 +02:00
|
|
|
case NonFatal(e) ⇒
|
2012-12-07 16:03:04 +01:00
|
|
|
log.error(e, s"Unable to shut down the underlying transport: [$transport]")
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|