2012-09-12 11:18:42 +02:00
|
|
|
package akka.remote
|
|
|
|
|
|
|
|
|
|
import akka.AkkaException
|
|
|
|
|
import akka.actor._
|
|
|
|
|
import akka.dispatch.SystemMessage
|
|
|
|
|
import akka.event.LoggingAdapter
|
|
|
|
|
import akka.pattern.pipe
|
|
|
|
|
import akka.remote.EndpointManager.Send
|
|
|
|
|
import akka.remote.RemoteProtocol.MessageProtocol
|
|
|
|
|
import akka.remote.transport.AkkaPduCodec._
|
|
|
|
|
import akka.remote.transport.AssociationHandle._
|
|
|
|
|
import akka.remote.transport.{ AkkaPduCodec, Transport, AssociationHandle }
|
|
|
|
|
import akka.serialization.Serialization
|
|
|
|
|
import akka.util.ByteString
|
|
|
|
|
import scala.util.control.NonFatal
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
/**
|
|
|
|
|
* Internal API
|
|
|
|
|
*/
|
|
|
|
|
private[remote] trait InboundMessageDispatcher {
|
2012-09-12 11:18:42 +02:00
|
|
|
def dispatch(recipient: InternalActorRef,
|
|
|
|
|
recipientAddress: Address,
|
|
|
|
|
serializedMessage: MessageProtocol,
|
|
|
|
|
senderOption: Option[ActorRef]): Unit
|
|
|
|
|
}
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
private[remote] class DefaultMessageDispatcher(private val system: ExtendedActorSystem,
|
|
|
|
|
private val provider: RemoteActorRefProvider,
|
|
|
|
|
private val log: LoggingAdapter) extends InboundMessageDispatcher {
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
private val remoteDaemon = provider.remoteDaemon
|
|
|
|
|
|
|
|
|
|
override def dispatch(recipient: InternalActorRef,
|
|
|
|
|
recipientAddress: Address,
|
|
|
|
|
serializedMessage: MessageProtocol,
|
|
|
|
|
senderOption: Option[ActorRef]): Unit = {
|
|
|
|
|
|
|
|
|
|
import provider.remoteSettings._
|
|
|
|
|
|
|
|
|
|
lazy val payload: AnyRef = MessageSerializer.deserialize(system, serializedMessage)
|
2012-12-07 16:03:04 +01:00
|
|
|
def payloadClass: Class[_] = if (payload eq null) null else payload.getClass
|
2012-09-12 11:18:42 +02:00
|
|
|
val sender: ActorRef = senderOption.getOrElse(system.deadLetters)
|
|
|
|
|
val originalReceiver = recipient.path
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
def msgLog = s"RemoteMessage: [$payload] to [$recipient]<+[$originalReceiver] from [$sender]"
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
recipient match {
|
|
|
|
|
|
|
|
|
|
case `remoteDaemon` ⇒
|
2012-12-12 15:04:44 +01:00
|
|
|
if (UntrustedMode) log.debug("dropping daemon message in untrusted mode")
|
|
|
|
|
else {
|
2012-11-23 10:15:19 +01:00
|
|
|
if (LogReceive) log.debug("received daemon message {}", msgLog)
|
|
|
|
|
payload match {
|
|
|
|
|
case m @ (_: DaemonMsg | _: Terminated) ⇒
|
|
|
|
|
try remoteDaemon ! m catch {
|
|
|
|
|
case NonFatal(e) ⇒ log.error(e, "exception while processing remote command {} from {}", m, sender)
|
|
|
|
|
}
|
|
|
|
|
case x ⇒ log.debug("remoteDaemon received illegal message {} from {}", x, sender)
|
|
|
|
|
}
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case l @ (_: LocalRef | _: RepointableRef) if l.isLocal ⇒
|
|
|
|
|
if (LogReceive) log.debug("received local message {}", msgLog)
|
|
|
|
|
payload match {
|
|
|
|
|
case msg: PossiblyHarmful if UntrustedMode ⇒
|
|
|
|
|
log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass)
|
|
|
|
|
case msg: SystemMessage ⇒ l.sendSystemMessage(msg)
|
|
|
|
|
case msg ⇒ l.!(msg)(sender)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !UntrustedMode ⇒
|
|
|
|
|
if (LogReceive) log.debug("received remote-destined message {}", msgLog)
|
|
|
|
|
if (provider.transport.addresses(recipientAddress))
|
|
|
|
|
// if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed)
|
|
|
|
|
r.!(payload)(sender)
|
|
|
|
|
else
|
|
|
|
|
log.error("dropping message {} for non-local recipient {} arriving at {} inbound addresses are {}",
|
|
|
|
|
payloadClass, r, recipientAddress, provider.transport.addresses)
|
|
|
|
|
|
|
|
|
|
case r ⇒ log.error("dropping message {} for unknown recipient {} arriving at {} inbound addresses are {}",
|
|
|
|
|
payloadClass, r, recipientAddress, provider.transport.addresses)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
/**
|
|
|
|
|
* Internal API
|
|
|
|
|
*/
|
|
|
|
|
private[remote] object EndpointWriter {
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* This message signals that the current association maintained by the local EndpointWriter and EndpointReader is
|
|
|
|
|
* to be overridden by a new inbound association. This is needed to avoid parallel inbound associations from the
|
2012-12-11 13:08:36 +01:00
|
|
|
* same remote endpoint: when a parallel inbound association is detected, the old one is removed and the new one is
|
|
|
|
|
* used instead.
|
|
|
|
|
* @param handle Handle of the new inbound association.
|
2012-12-07 16:03:04 +01:00
|
|
|
*/
|
2012-11-21 16:39:04 +01:00
|
|
|
case class TakeOver(handle: AssociationHandle)
|
2012-09-12 11:18:42 +02:00
|
|
|
case object BackoffTimer
|
|
|
|
|
|
|
|
|
|
sealed trait State
|
|
|
|
|
case object Initializing extends State
|
|
|
|
|
case object Buffering extends State
|
|
|
|
|
case object Writing extends State
|
|
|
|
|
}
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
private[remote] class EndpointException(msg: String, cause: Throwable) extends AkkaException(msg, cause)
|
|
|
|
|
private[remote] case class InvalidAssociation(localAddress: Address, remoteAddress: Address, cause: Throwable)
|
2012-09-12 11:18:42 +02:00
|
|
|
extends EndpointException("Invalid address: " + remoteAddress, cause)
|
|
|
|
|
|
|
|
|
|
private[remote] class EndpointWriter(
|
|
|
|
|
handleOrActive: Option[AssociationHandle],
|
|
|
|
|
val localAddress: Address,
|
|
|
|
|
val remoteAddress: Address,
|
|
|
|
|
val transport: Transport,
|
|
|
|
|
val settings: RemotingSettings,
|
|
|
|
|
val codec: AkkaPduCodec) extends Actor with Stash with FSM[EndpointWriter.State, Unit] {
|
|
|
|
|
|
|
|
|
|
import EndpointWriter._
|
|
|
|
|
import context.dispatcher
|
|
|
|
|
|
|
|
|
|
val extendedSystem: ExtendedActorSystem = context.system.asInstanceOf[ExtendedActorSystem]
|
2012-11-21 16:39:04 +01:00
|
|
|
val eventPublisher = new EventPublisher(context.system, log, settings.LogLifecycleEvents)
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
var reader: Option[ActorRef] = None
|
|
|
|
|
var handle: Option[AssociationHandle] = handleOrActive // FIXME: refactor into state data
|
|
|
|
|
val readerId = Iterator from 0
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-11-22 13:33:48 +01:00
|
|
|
override val supervisorStrategy = OneForOneStrategy() { case NonFatal(e) ⇒ publishAndThrow(e) }
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
val msgDispatch =
|
2012-12-11 13:08:36 +01:00
|
|
|
new DefaultMessageDispatcher(extendedSystem, RARP(extendedSystem).provider, log)
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
def inbound = handle.isDefined
|
2012-09-12 11:18:42 +02:00
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
private def publishAndThrow(reason: Throwable): Nothing =
|
|
|
|
|
try
|
|
|
|
|
// FIXME: Casting seems very evil here...
|
|
|
|
|
eventPublisher.notifyListeners(AssociationErrorEvent(reason, localAddress, remoteAddress, inbound)).asInstanceOf[Nothing]
|
|
|
|
|
finally throw reason
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
override def postRestart(reason: Throwable): Unit = {
|
2012-12-07 16:03:04 +01:00
|
|
|
handle = None // Wipe out the possibly injected handle
|
2012-09-12 11:18:42 +02:00
|
|
|
preStart()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def preStart(): Unit = {
|
2012-12-07 16:03:04 +01:00
|
|
|
if (!inbound) {
|
2012-09-12 11:18:42 +02:00
|
|
|
transport.associate(remoteAddress) pipeTo self
|
|
|
|
|
startWith(Initializing, ())
|
|
|
|
|
} else {
|
|
|
|
|
startReadEndpoint()
|
|
|
|
|
startWith(Writing, ())
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
when(Initializing) {
|
|
|
|
|
case Event(Send(msg, senderOption, recipient), _) ⇒
|
|
|
|
|
stash()
|
2012-12-07 16:03:04 +01:00
|
|
|
stay()
|
2012-09-12 11:18:42 +02:00
|
|
|
case Event(Transport.Invalid(e), _) ⇒
|
2012-11-22 13:33:48 +01:00
|
|
|
log.error(e, "Tried to associate with invalid remote address [{}]. " +
|
|
|
|
|
"Address is now quarantined, all messages to this address will be delivered to dead letters.", remoteAddress)
|
2012-09-12 11:18:42 +02:00
|
|
|
publishAndThrow(new InvalidAssociation(localAddress, remoteAddress, e))
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
case Event(Transport.Fail(e), _) ⇒
|
|
|
|
|
publishAndThrow(new EndpointException(s"Association failed with [$remoteAddress]", e))
|
2012-09-12 11:18:42 +02:00
|
|
|
case Event(Transport.Ready(inboundHandle), _) ⇒
|
2012-12-07 16:03:04 +01:00
|
|
|
handle = Some(inboundHandle)
|
2012-09-12 11:18:42 +02:00
|
|
|
startReadEndpoint()
|
|
|
|
|
goto(Writing)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
when(Buffering) {
|
|
|
|
|
case Event(Send(msg, senderOption, recipient), _) ⇒
|
|
|
|
|
stash()
|
2012-12-07 16:03:04 +01:00
|
|
|
stay()
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
case Event(BackoffTimer, _) ⇒ goto(Writing)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
when(Writing) {
|
|
|
|
|
case Event(Send(msg, senderOption, recipient), _) ⇒
|
2012-11-21 15:58:01 +01:00
|
|
|
val pdu = codec.constructMessage(recipient.localAddressToUse, recipient, serializeMessage(msg), senderOption)
|
2012-12-07 16:03:04 +01:00
|
|
|
val success = try handle match {
|
|
|
|
|
case Some(h) ⇒ h.write(pdu)
|
|
|
|
|
case None ⇒ throw new EndpointException("Internal error: Endpoint is in state Writing, but no association" +
|
|
|
|
|
"handle is present.", null)
|
|
|
|
|
} catch {
|
|
|
|
|
case NonFatal(e) ⇒ publishAndThrow(new EndpointException("Failed to write message to the transport", e))
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
2012-12-07 16:03:04 +01:00
|
|
|
if (success) stay() else {
|
2012-11-21 16:39:04 +01:00
|
|
|
stash()
|
2012-09-12 11:18:42 +02:00
|
|
|
goto(Buffering)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
whenUnhandled {
|
2012-12-07 16:03:04 +01:00
|
|
|
case Event(Terminated(r), _) if r == reader ⇒ publishAndThrow(new EndpointException("Disassociated", null))
|
2012-11-21 16:39:04 +01:00
|
|
|
case Event(TakeOver(newHandle), _) ⇒
|
|
|
|
|
// Shutdown old reader
|
2012-12-07 16:03:04 +01:00
|
|
|
handle foreach { _.disassociate() }
|
|
|
|
|
reader match {
|
|
|
|
|
case Some(r) ⇒
|
|
|
|
|
context.unwatch(r)
|
|
|
|
|
context.stop(r)
|
|
|
|
|
case None ⇒
|
2012-11-21 16:39:04 +01:00
|
|
|
}
|
2012-12-07 16:03:04 +01:00
|
|
|
handle = Some(newHandle)
|
2012-11-21 16:39:04 +01:00
|
|
|
startReadEndpoint()
|
|
|
|
|
unstashAll()
|
|
|
|
|
goto(Writing)
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
onTransition {
|
|
|
|
|
case Initializing -> Writing ⇒
|
|
|
|
|
unstashAll()
|
|
|
|
|
eventPublisher.notifyListeners(AssociatedEvent(localAddress, remoteAddress, inbound))
|
2012-12-07 16:03:04 +01:00
|
|
|
case Writing -> Buffering ⇒
|
|
|
|
|
setTimer("backoff-timer", BackoffTimer, settings.BackoffPeriod, repeat = false)
|
2012-09-12 11:18:42 +02:00
|
|
|
case Buffering -> Writing ⇒
|
|
|
|
|
unstashAll()
|
|
|
|
|
cancelTimer("backoff-timer")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
onTermination {
|
|
|
|
|
case StopEvent(_, _, _) ⇒ if (handle ne null) {
|
2012-12-07 16:03:04 +01:00
|
|
|
// FIXME: Add a test case for this
|
|
|
|
|
// It is important to call unstashAll() for the stash to work properly and maintain messages during restart.
|
|
|
|
|
// As the FSM trait does not call super.postStop(), this call is needed
|
2012-11-21 16:39:04 +01:00
|
|
|
unstashAll()
|
2012-12-07 16:03:04 +01:00
|
|
|
handle foreach { _.disassociate() }
|
2012-09-12 11:18:42 +02:00
|
|
|
eventPublisher.notifyListeners(DisassociatedEvent(localAddress, remoteAddress, inbound))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
private def startReadEndpoint(): Unit = handle match {
|
|
|
|
|
case Some(h) ⇒
|
2012-12-11 13:08:36 +01:00
|
|
|
val readerLocalAddress = h.localAddress
|
|
|
|
|
val readerCodec = codec
|
|
|
|
|
val readerDispatcher = msgDispatch
|
|
|
|
|
reader = Some(
|
|
|
|
|
context.watch(context.actorOf(Props(new EndpointReader(readerCodec, readerLocalAddress, readerDispatcher)),
|
2012-12-12 14:49:38 +01:00
|
|
|
"endpointReader-" + AddressUrlEncoder(remoteAddress) + "-" + readerId.next())))
|
2012-12-07 16:03:04 +01:00
|
|
|
h.readHandlerPromise.success(reader.get)
|
|
|
|
|
case None ⇒ throw new EndpointException("Internal error: No handle was present during creation of the endpoint" +
|
|
|
|
|
"reader.", null)
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
2012-12-07 16:03:04 +01:00
|
|
|
private def serializeMessage(msg: Any): MessageProtocol = handle match {
|
|
|
|
|
case Some(h) ⇒
|
|
|
|
|
Serialization.currentTransportAddress.withValue(h.localAddress) {
|
|
|
|
|
(MessageSerializer.serialize(extendedSystem, msg.asInstanceOf[AnyRef]))
|
|
|
|
|
}
|
|
|
|
|
case None ⇒ throw new EndpointException("Internal error: No handle was present during serialization of" +
|
|
|
|
|
"outbound message.", null)
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private[remote] class EndpointReader(
|
|
|
|
|
val codec: AkkaPduCodec,
|
2012-11-21 15:58:01 +01:00
|
|
|
val localAddress: Address,
|
2012-09-12 11:18:42 +02:00
|
|
|
val msgDispatch: InboundMessageDispatcher) extends Actor {
|
|
|
|
|
|
2012-12-11 13:08:36 +01:00
|
|
|
val provider = RARP(context.system).provider
|
2012-09-12 11:18:42 +02:00
|
|
|
|
|
|
|
|
override def receive: Receive = {
|
|
|
|
|
case Disassociated ⇒ context.stop(self)
|
|
|
|
|
|
2012-11-21 15:58:01 +01:00
|
|
|
case InboundPayload(p) ⇒
|
|
|
|
|
val msg = decodePdu(p)
|
|
|
|
|
msgDispatch.dispatch(msg.recipient, msg.recipientAddress, msg.serializedMessage, msg.senderOption)
|
2012-09-12 11:18:42 +02:00
|
|
|
}
|
|
|
|
|
|
2012-11-21 15:58:01 +01:00
|
|
|
private def decodePdu(pdu: ByteString): Message = try {
|
|
|
|
|
codec.decodeMessage(pdu, provider, localAddress)
|
2012-09-12 11:18:42 +02:00
|
|
|
} catch {
|
|
|
|
|
case NonFatal(e) ⇒ throw new EndpointException("Error while decoding incoming Akka PDU", e)
|
|
|
|
|
}
|
|
|
|
|
}
|