2013-01-09 12:21:31 +01:00
/* *
2018-01-04 17:26:29 +00:00
* Copyright ( C ) 2009 - 2018 Lightbend Inc . < https : //www.lightbend.com>
2013-01-09 12:21:31 +01:00
*/
2018-03-13 23:45:55 +09:00
2012-09-12 11:18:42 +02:00
package akka.remote
2013-06-19 14:09:14 +02:00
import akka.actor.OneForOneStrategy
import akka.actor.SupervisorStrategy._
import akka.actor.Terminated
2012-09-12 11:18:42 +02:00
import akka.actor._
2013-03-05 16:19:54 +01:00
import akka.dispatch.sysmsg.SystemMessage
2018-03-16 19:08:29 +08:00
import akka.event. { LogMarker , Logging , MarkerLoggingAdapter }
2012-09-12 11:18:42 +02:00
import akka.pattern.pipe
2016-10-28 14:52:17 +02:00
import akka.remote.EndpointManager. { Link , ResendState , Send }
import akka.remote.EndpointWriter. { FlushAndStop , StoppedReading }
2013-03-27 17:47:56 +01:00
import akka.remote.WireFormats.SerializedMessage
2013-06-19 14:09:14 +02:00
import akka.remote.transport.AkkaPduCodec.Message
2016-10-28 14:52:17 +02:00
import akka.remote.transport.AssociationHandle. { ActorHandleEventListener , DisassociateInfo , Disassociated , InboundPayload }
2013-03-27 17:47:56 +01:00
import akka.remote.transport.Transport.InvalidAssociationException
2013-06-19 14:09:14 +02:00
import akka.remote.transport._
2013-03-26 11:25:09 +01:00
import akka.serialization.Serialization
2012-09-12 11:18:42 +02:00
import akka.util.ByteString
2016-10-28 14:52:17 +02:00
import akka. { AkkaException , OnlyCauseStackTrace }
2013-03-20 20:38:49 +13:00
import java.io.NotSerializableException
2016-10-28 14:52:17 +02:00
import java.util.concurrent. { ConcurrentHashMap , TimeUnit , TimeoutException }
2013-06-19 14:09:14 +02:00
import scala.annotation.tailrec
2016-10-28 14:52:17 +02:00
import scala.concurrent.duration.Deadline
2013-03-27 17:47:56 +01:00
import scala.util.control.NonFatal
2014-04-03 15:56:11 +02:00
import java.util.concurrent.locks.LockSupport
2016-10-28 14:52:17 +02:00
2014-04-03 15:56:11 +02:00
import scala.concurrent.Future
2016-06-05 15:40:06 +02:00
import akka.util.OptionVal
import akka.util.OptionVal
2012-09-12 11:18:42 +02:00
2012-12-07 16:03:04 +01:00
/* *
2013-02-08 13:13:52 +01:00
* INTERNAL API
2012-12-07 16:03:04 +01:00
*/
private [ remote ] trait InboundMessageDispatcher {
2016-06-02 14:06:57 +02:00
def dispatch (
recipient : InternalActorRef ,
recipientAddress : Address ,
serializedMessage : SerializedMessage ,
2016-06-05 15:40:06 +02:00
senderOption : OptionVal [ ActorRef ] ) : Unit
2012-09-12 11:18:42 +02:00
}
2013-02-08 13:13:52 +01:00
/* *
* INTERNAL API
*/
2016-06-02 14:06:57 +02:00
private [ remote ] class DefaultMessageDispatcher (
private val system : ExtendedActorSystem ,
private val provider : RemoteActorRefProvider ,
2016-10-28 14:52:17 +02:00
private val log : MarkerLoggingAdapter ) extends InboundMessageDispatcher {
2012-09-12 11:18:42 +02:00
private val remoteDaemon = provider . remoteDaemon
2016-06-02 14:06:57 +02:00
override def dispatch (
recipient : InternalActorRef ,
recipientAddress : Address ,
serializedMessage : SerializedMessage ,
2016-06-05 15:40:06 +02:00
senderOption : OptionVal [ ActorRef ] ) : Unit = {
2012-09-12 11:18:42 +02:00
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
2014-01-16 15:16:35 +01:00
def msgLog = s" RemoteMessage: [ $payload ] to [ $recipient ]<+[ $originalReceiver ] from [ $sender ()] "
2012-09-12 11:18:42 +02:00
recipient match {
case `remoteDaemon` ⇒
2016-10-28 14:52:17 +02:00
if ( UntrustedMode ) log . debug ( LogMarker . Security , "dropping daemon message in untrusted mode" )
2012-12-12 15:04:44 +01:00
else {
2012-11-23 10:15:19 +01:00
if ( LogReceive ) log . debug ( "received daemon message {}" , msgLog )
2013-03-26 18:17:50 +01:00
remoteDaemon ! payload
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 {
2013-11-06 14:34:07 +01:00
case sel : ActorSelectionMessage ⇒
if ( UntrustedMode && ( ! TrustedSelectionPaths . contains ( sel . elements . mkString ( "/" , "/" , "" ) ) ||
sel . msg . isInstanceOf [ PossiblyHarmful ] || l != provider . rootGuardian ) )
2016-06-02 14:06:57 +02:00
log . debug (
2016-10-28 14:52:17 +02:00
LogMarker . Security ,
2016-06-02 14:06:57 +02:00
"operating in UntrustedMode, dropping inbound actor selection to [{}], " +
"allow it by adding the path to 'akka.remote.trusted-selection-paths' configuration" ,
2013-11-06 14:34:07 +01:00
sel . elements . mkString ( "/" , "/" , "" ) )
else
// run the receive logic for ActorSelectionMessage here to make sure it is not stuck on busy user actor
ActorSelection . deliverSelection ( l , sender , sel )
2012-09-12 11:18:42 +02:00
case msg : PossiblyHarmful if UntrustedMode ⇒
2016-10-28 14:52:17 +02:00
log . debug ( LogMarker . Security , "operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]" , msg . getClass . getName )
2012-09-12 11:18:42 +02:00
case msg : SystemMessage ⇒ l . sendSystemMessage ( msg )
2013-11-06 14:34:07 +01:00
case msg ⇒ l . ! ( msg ) ( sender )
2012-09-12 11:18:42 +02:00
}
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
2016-06-02 14:06:57 +02:00
log . error (
"dropping message [{}] for non-local recipient [{}] arriving at [{}] inbound addresses are [{}]" ,
2013-05-30 21:37:57 +02:00
payloadClass , r , recipientAddress , provider . transport . addresses . mkString ( ", " ) )
2012-09-12 11:18:42 +02:00
2016-06-02 14:06:57 +02:00
case r ⇒ log . error (
"dropping message [{}] for unknown recipient [{}] arriving at [{}] inbound addresses are [{}]" ,
2013-05-30 21:37:57 +02:00
payloadClass , r , recipientAddress , provider . transport . addresses . mkString ( ", " ) )
2012-09-12 11:18:42 +02:00
}
}
}
2013-02-08 13:13:52 +01:00
/* *
* INTERNAL API
*/
@SerialVersionUID ( 1L )
2012-12-20 12:59:48 +01:00
private [ remote ] class EndpointException ( msg : String , cause : Throwable ) extends AkkaException ( msg , cause ) with OnlyCauseStackTrace {
2012-12-18 13:46:10 +01:00
def this ( msg : String ) = this ( msg , null )
}
2013-10-09 11:29:02 +02:00
/* *
* INTERNAL API
*/
private [ remote ] trait AssociationProblem
2013-02-08 13:13:52 +01:00
/* *
* INTERNAL API
*/
2013-03-07 18:08:07 +01:00
@SerialVersionUID ( 1L )
2014-03-07 13:20:01 +01:00
private [ remote ] final case class ShutDownAssociation ( localAddress : Address , remoteAddress : Address , cause : Throwable )
2013-10-09 11:29:02 +02:00
extends EndpointException ( "Shut down address: " + remoteAddress , cause ) with AssociationProblem
2012-09-12 11:18:42 +02:00
2013-03-27 17:47:56 +01:00
/* *
* INTERNAL API
*/
2015-12-06 20:14:44 -05:00
@SerialVersionUID ( 2L )
2016-06-02 14:06:57 +02:00
private [ remote ] final case class InvalidAssociation (
localAddress : Address ,
remoteAddress : Address ,
cause : Throwable ,
disassociationInfo : Option [ DisassociateInfo ] = None )
2013-10-09 11:29:02 +02:00
extends EndpointException ( "Invalid address: " + remoteAddress , cause ) with AssociationProblem
2013-03-27 17:47:56 +01:00
2013-03-07 18:08:07 +01:00
/* *
* INTERNAL API
*/
@SerialVersionUID ( 1L )
2014-03-07 13:20:01 +01:00
private [ remote ] final case class HopelessAssociation ( localAddress : Address , remoteAddress : Address , uid : Option [ Int ] , cause : Throwable )
2013-10-09 11:29:02 +02:00
extends EndpointException ( "Catastrophic association error." ) with AssociationProblem
2013-03-07 18:08:07 +01:00
/* *
* INTERNAL API
*/
@SerialVersionUID ( 1L )
2013-10-09 11:29:02 +02:00
private [ remote ] class EndpointDisassociatedException ( msg : String ) extends EndpointException ( msg )
2013-03-07 18:08:07 +01:00
2013-04-18 17:35:43 +02:00
/* *
* INTERNAL API
*/
@SerialVersionUID ( 1L )
2013-10-09 11:29:02 +02:00
private [ remote ] class EndpointAssociationException ( msg : String , cause : Throwable ) extends EndpointException ( msg , cause )
2013-04-18 17:35:43 +02:00
2013-02-08 13:13:52 +01:00
/* *
* INTERNAL API
*/
2013-03-20 20:38:49 +13:00
@SerialVersionUID ( 1L )
private [ remote ] class OversizedPayloadException ( msg : String ) extends EndpointException ( msg )
2013-03-27 17:47:56 +01:00
/* *
* INTERNAL API
*/
private [ remote ] object ReliableDeliverySupervisor {
case object Ungate
2013-06-19 14:09:14 +02:00
case object AttemptSysMsgRedelivery
2014-05-12 13:51:05 +02:00
final case class GotUid ( uid : Int , remoteAddres : Address )
2013-03-27 17:47:56 +01:00
2015-02-17 13:40:48 +01:00
case object IsIdle
case object Idle
2015-09-16 15:26:24 +02:00
case object TooLongIdle
2015-02-17 13:40:48 +01:00
2013-05-30 14:03:35 +02:00
def props (
2013-03-27 17:47:56 +01:00
handleOrActive : Option [ AkkaProtocolHandle ] ,
2016-06-02 14:06:57 +02:00
localAddress : Address ,
remoteAddress : Address ,
refuseUid : Option [ Int ] ,
transport : AkkaProtocolTransport ,
settings : RemoteSettings ,
codec : AkkaPduCodec ,
2013-08-19 15:34:24 +02:00
receiveBuffers : ConcurrentHashMap [ Link , ResendState ] ) : Props =
2013-11-26 15:25:05 +01:00
Props ( classOf [ ReliableDeliverySupervisor ] , handleOrActive , localAddress , remoteAddress , refuseUid , transport , settings ,
2013-05-24 14:01:29 +02:00
codec , receiveBuffers )
2013-03-27 17:47:56 +01:00
}
/* *
* INTERNAL API
*/
private [ remote ] class ReliableDeliverySupervisor (
2016-06-02 14:06:57 +02:00
handleOrActive : Option [ AkkaProtocolHandle ] ,
val localAddress : Address ,
val remoteAddress : Address ,
val refuseUid : Option [ Int ] ,
val transport : AkkaProtocolTransport ,
val settings : RemoteSettings ,
val codec : AkkaPduCodec ,
2013-12-12 13:05:59 +01:00
val receiveBuffers : ConcurrentHashMap [ Link , ResendState ] ) extends Actor with ActorLogging {
2013-03-27 17:47:56 +01:00
import ReliableDeliverySupervisor._
2013-06-19 14:09:14 +02:00
import context.dispatcher
2013-03-27 17:47:56 +01:00
2015-01-19 10:03:40 +01:00
val autoResendTimer = context . system . scheduler . schedule (
settings . SysResendTimeout , settings . SysResendTimeout , self , AttemptSysMsgRedelivery )
2013-10-08 09:56:59 +02:00
2013-12-12 13:05:59 +01:00
override val supervisorStrategy = OneForOneStrategy ( loggingEnabled = false ) {
2013-10-09 11:29:02 +02:00
case e @ ( _ : AssociationProblem ) ⇒ Escalate
2013-03-27 17:47:56 +01:00
case NonFatal ( e ) ⇒
2014-11-11 08:15:37 +01:00
val causedBy = if ( e . getCause == null ) "" else s" Caused by: [ ${ e . getCause . getMessage } ] "
2016-06-02 14:06:57 +02:00
log . warning (
"Association with remote system [{}] has failed, address is now gated for [{}] ms. Reason: [{}] {}" ,
2014-11-11 08:15:37 +01:00
remoteAddress , settings . RetryGateClosedFor . toMillis , e . getMessage , causedBy )
2013-09-17 14:17:31 +02:00
uidConfirmed = false // Need confirmation of UID again
2017-06-09 08:10:44 +02:00
if ( ( resendBuffer . nacked . nonEmpty || resendBuffer . nonAcked . nonEmpty ) && bailoutAt . isEmpty )
bailoutAt = Some ( Deadline . now + settings . InitialSysMsgDeliveryTimeout )
context . become ( gated ( writerTerminated = false , earlyUngateRequested = false ) )
currentHandle = None
context . parent ! StoppedReading ( self )
Stop
2013-03-27 17:47:56 +01:00
}
var currentHandle : Option [ AkkaProtocolHandle ] = handleOrActive
2013-08-19 15:34:24 +02:00
var resendBuffer : AckedSendBuffer [ Send ] = _
var seqCounter : Long = _
2016-03-15 13:29:33 +01:00
def reset ( ) : Unit = {
2013-08-19 15:34:24 +02:00
resendBuffer = new AckedSendBuffer [ Send ] ( settings . SysMsgBufferSize )
seqCounter = 0L
2015-01-19 10:03:40 +01:00
bailoutAt = None
2013-08-19 15:34:24 +02:00
}
reset ( )
def nextSeq ( ) : SeqNo = {
val tmp = seqCounter
seqCounter += 1
SeqNo ( tmp )
2013-03-27 17:47:56 +01:00
}
var writer : ActorRef = createWriter ( )
var uid : Option [ Int ] = handleOrActive map { _ . handshakeInfo . uid }
2015-01-19 10:03:40 +01:00
var bailoutAt : Option [ Deadline ] = None
2015-09-16 15:26:24 +02:00
var maxSilenceTimer : Option [ Cancellable ] = None
2013-09-17 14:17:31 +02:00
// Processing of Acks has to be delayed until the UID after a reconnect is discovered. Depending whether the
// UID matches the expected one, pending Acks can be processed, or must be dropped. It is guaranteed that for
// any inbound connections (calling createWriter()) the first message from that connection is GotUid() therefore
// it serves a separator.
// If we already have an inbound handle then UID is initially confirmed.
// (This actor is never restarted)
2017-09-11 10:10:04 +02:00
var uidConfirmed : Boolean = uid . isDefined && ( uid != refuseUid )
if ( uid . isDefined && ( uid == refuseUid ) )
throw new HopelessAssociation ( localAddress , remoteAddress , uid ,
new IllegalStateException (
s" The remote system [ $remoteAddress ] has a UID [ ${ uid . get } ] that has been quarantined. Association aborted. " ) )
2013-09-17 14:17:31 +02:00
2013-03-27 17:47:56 +01:00
override def postStop ( ) : Unit = {
// All remaining messages in the buffer has to be delivered to dead letters. It is important to clear the sequence
// number otherwise deadLetters will ignore it to avoid reporting system messages as dead letters while they are
// still possibly retransmitted.
// Such a situation may arise when the EndpointWriter is shut down, and all of its mailbox contents are delivered
// to dead letters. These messages should be ignored, as they still live in resendBuffer and might be delivered to
// the remote system later.
( resendBuffer . nacked ++ resendBuffer . nonAcked ) foreach { s ⇒ context . system . deadLetters ! s . copy ( seqOpt = None ) }
receiveBuffers . remove ( Link ( localAddress , remoteAddress ) )
2015-01-19 10:03:40 +01:00
autoResendTimer . cancel ( )
2015-09-16 15:26:24 +02:00
maxSilenceTimer . foreach ( _ . cancel ( ) )
2013-03-27 17:47:56 +01:00
}
override def postRestart ( reason : Throwable ) : Unit = {
throw new IllegalStateException (
"BUG: ReliableDeliverySupervisor has been attempted to be restarted. This must not happen." )
}
override def receive : Receive = {
case FlushAndStop ⇒
// Trying to serve until our last breath
resendAll ( )
writer ! FlushAndStop
context . become ( flushWait )
2015-02-17 13:40:48 +01:00
case IsIdle ⇒ // Do not reply, we will Terminate soon, or send a GotUid
2013-03-27 17:47:56 +01:00
case s : Send ⇒
handleSend ( s )
case ack : Ack ⇒
2015-01-19 10:03:40 +01:00
// If we are not sure about the UID just ignore the ack. Ignoring is fine.
if ( uidConfirmed ) {
2013-09-17 14:17:31 +02:00
try resendBuffer = resendBuffer . acknowledge ( ack )
catch {
case NonFatal ( e ) ⇒
2015-01-19 10:03:40 +01:00
throw new HopelessAssociation ( localAddress , remoteAddress , uid ,
new IllegalStateException ( s" Error encountered while processing system message " +
s" acknowledgement buffer: $resendBuffer ack: $ack " , e ) )
2013-09-17 14:17:31 +02:00
}
2013-08-19 15:34:24 +02:00
2013-09-17 14:17:31 +02:00
resendNacked ( )
2013-03-27 17:47:56 +01:00
}
2013-10-08 09:56:59 +02:00
case AttemptSysMsgRedelivery ⇒
if ( uidConfirmed ) resendAll ( )
2013-03-27 17:47:56 +01:00
case Terminated ( _ ) ⇒
currentHandle = None
2013-06-19 14:09:14 +02:00
context . parent ! StoppedReading ( self )
if ( resendBuffer . nonAcked . nonEmpty || resendBuffer . nacked . nonEmpty )
context . system . scheduler . scheduleOnce ( settings . SysResendTimeout , self , AttemptSysMsgRedelivery )
2015-09-16 15:26:24 +02:00
goToIdle ( )
2014-05-12 13:51:05 +02:00
case g @ GotUid ( receivedUid , _ ) ⇒
2015-01-19 10:03:40 +01:00
bailoutAt = None
2014-03-21 20:22:16 +01:00
context . parent ! g
2013-08-19 15:34:24 +02:00
// New system that has the same address as the old - need to start from fresh state
2013-09-17 14:17:31 +02:00
uidConfirmed = true
if ( uid . exists ( _ != receivedUid ) ) reset ( )
uid = Some ( receivedUid )
resendAll ( )
2014-04-03 15:56:11 +02:00
case s : EndpointWriter . StopReading ⇒
writer forward s
2017-09-11 10:10:04 +02:00
case Ungate ⇒ // ok, not gated
2013-03-27 17:47:56 +01:00
}
2015-12-14 20:33:36 +01:00
def gated ( writerTerminated : Boolean , earlyUngateRequested : Boolean ) : Receive = {
case Terminated ( _ ) if ! writerTerminated ⇒
if ( earlyUngateRequested )
self ! Ungate
else
context . system . scheduler . scheduleOnce ( settings . RetryGateClosedFor , self , Ungate )
context . become ( gated ( writerTerminated = true , earlyUngateRequested ) )
2015-02-17 13:40:48 +01:00
case IsIdle ⇒ sender ( ) ! Idle
2013-03-27 17:47:56 +01:00
case Ungate ⇒
2015-12-14 20:33:36 +01:00
if ( ! writerTerminated ) {
// Ungate was sent from EndpointManager, but we must wait for Terminated first.
context . become ( gated ( writerTerminated = false , earlyUngateRequested = true ) )
} else if ( resendBuffer . nonAcked . nonEmpty || resendBuffer . nacked . nonEmpty ) {
2014-02-19 11:42:41 +01:00
// If we talk to a system we have not talked to before (or has given up talking to in the past) stop
// system delivery attempts after the specified time. This act will drop the pending system messages and gate the
// remote address at the EndpointManager level stopping this actor. In case the remote system becomes reachable
// again it will be immediately quarantined due to out-of-sync system message buffer and becomes quarantined.
// In other words, this action is safe.
2015-01-19 10:03:40 +01:00
if ( bailoutAt . exists ( _ . isOverdue ( ) ) )
throw new HopelessAssociation ( localAddress , remoteAddress , uid ,
2014-02-19 11:42:41 +01:00
new java . util . concurrent . TimeoutException ( "Delivery of system messages timed out and they were dropped." ) )
2013-03-27 17:47:56 +01:00
writer = createWriter ( )
2013-09-17 14:17:31 +02:00
// Resending will be triggered by the incoming GotUid message after the connection finished
2015-09-16 15:26:24 +02:00
goToActive ( )
} else goToIdle ( )
2015-01-19 10:03:40 +01:00
case AttemptSysMsgRedelivery ⇒ // Ignore
2013-03-27 17:47:56 +01:00
case s @ Send ( msg : SystemMessage , _ , _ , _ ) ⇒ tryBuffer ( s . copy ( seqOpt = Some ( nextSeq ( ) ) ) )
case s : Send ⇒ context . system . deadLetters ! s
2013-06-03 11:35:37 +02:00
case EndpointWriter . FlushAndStop ⇒ context . stop ( self )
2014-04-03 15:56:11 +02:00
case EndpointWriter . StopReading ( w , replyTo ) ⇒
replyTo ! EndpointWriter . StoppedReading ( w )
sender ( ) ! EndpointWriter . StoppedReading ( w )
2013-03-27 17:47:56 +01:00
}
def idle : Receive = {
2015-02-17 13:40:48 +01:00
case IsIdle ⇒ sender ( ) ! Idle
2013-03-27 17:47:56 +01:00
case s : Send ⇒
writer = createWriter ( )
2013-09-17 14:17:31 +02:00
// Resending will be triggered by the incoming GotUid message after the connection finished
2013-03-27 17:47:56 +01:00
handleSend ( s )
2015-09-16 15:26:24 +02:00
goToActive ( )
2013-06-19 14:09:14 +02:00
case AttemptSysMsgRedelivery ⇒
2015-01-19 10:03:40 +01:00
if ( resendBuffer . nacked . nonEmpty || resendBuffer . nonAcked . nonEmpty ) {
writer = createWriter ( )
// Resending will be triggered by the incoming GotUid message after the connection finished
2015-09-16 15:26:24 +02:00
goToActive ( )
2015-01-19 10:03:40 +01:00
}
2015-09-16 15:26:24 +02:00
case TooLongIdle ⇒
throw new HopelessAssociation ( localAddress , remoteAddress , uid ,
new TimeoutException ( "Remote system has been silent for too long. " +
s" (more than ${ settings . QuarantineSilentSystemTimeout . toUnit ( TimeUnit . HOURS ) } hours) " ) )
2014-04-03 15:56:11 +02:00
case EndpointWriter . FlushAndStop ⇒ context . stop ( self )
case EndpointWriter . StopReading ( w , replyTo ) ⇒
replyTo ! EndpointWriter . StoppedReading ( w )
2017-09-11 10:10:04 +02:00
case Ungate ⇒ // ok, not gated
2013-03-27 17:47:56 +01:00
}
2015-09-16 15:26:24 +02:00
private def goToIdle ( ) : Unit = {
2017-06-09 08:10:44 +02:00
if ( maxSilenceTimer . isEmpty )
2015-09-16 15:26:24 +02:00
maxSilenceTimer = Some ( context . system . scheduler . scheduleOnce ( settings . QuarantineSilentSystemTimeout , self , TooLongIdle ) )
context . become ( idle )
}
private def goToActive ( ) : Unit = {
maxSilenceTimer . foreach ( _ . cancel ( ) )
maxSilenceTimer = None
context . become ( receive )
}
2013-03-27 17:47:56 +01:00
def flushWait : Receive = {
2015-02-17 13:40:48 +01:00
case IsIdle ⇒ // Do not reply, we will Terminate soon, which will do the inbound connection unstashing
2013-03-27 17:47:56 +01:00
case Terminated ( _ ) ⇒
// Clear buffer to prevent sending system messages to dead letters -- at this point we are shutting down
// and don't really know if they were properly delivered or not.
resendBuffer = new AckedSendBuffer [ Send ] ( 0 )
context . stop ( self )
case _ ⇒ // Ignore
}
private def handleSend ( send : Send ) : Unit =
if ( send . message . isInstanceOf [ SystemMessage ] ) {
val sequencedSend = send . copy ( seqOpt = Some ( nextSeq ( ) ) )
tryBuffer ( sequencedSend )
2013-09-17 14:17:31 +02:00
// If we have not confirmed the remote UID we cannot transfer the system message at this point just buffer it.
2015-05-27 14:22:16 +02:00
// GotUid will kick resendAll() causing the messages to be properly written.
// Flow control by not sending more when we already have many outstanding.
if ( uidConfirmed && resendBuffer . nonAcked . size <= settings . SysResendLimit )
2015-01-19 10:03:40 +01:00
writer ! sequencedSend
2013-03-27 17:47:56 +01:00
} else writer ! send
private def resendNacked ( ) : Unit = resendBuffer . nacked foreach { writer ! _ }
private def resendAll ( ) : Unit = {
resendNacked ( )
2015-05-27 14:22:16 +02:00
resendBuffer . nonAcked . take ( settings . SysResendLimit ) foreach { writer ! _ }
2013-03-27 17:47:56 +01:00
}
private def tryBuffer ( s : Send ) : Unit =
try {
resendBuffer = resendBuffer buffer s
} catch {
case NonFatal ( e ) ⇒ throw new HopelessAssociation ( localAddress , remoteAddress , uid , e )
}
private def createWriter ( ) : ActorRef = {
2013-08-19 12:06:07 +02:00
context . watch ( context . actorOf ( RARP ( context . system ) . configureDispatcher ( EndpointWriter . props (
2013-03-27 17:47:56 +01:00
handleOrActive = currentHandle ,
localAddress = localAddress ,
remoteAddress = remoteAddress ,
2013-11-26 15:25:05 +01:00
refuseUid ,
2013-03-27 17:47:56 +01:00
transport = transport ,
settings = settings ,
AkkaPduProtobufCodec ,
receiveBuffers = receiveBuffers ,
2013-08-19 12:06:07 +02:00
reliableDeliverySupervisor = Some ( self ) ) ) . withDeploy ( Deploy . local ) , "endpointWriter" ) )
2013-03-27 17:47:56 +01:00
}
}
/* *
* INTERNAL API
*/
2013-03-20 20:38:49 +13:00
private [ remote ] abstract class EndpointActor (
2016-06-02 14:06:57 +02:00
val localAddress : Address ,
2012-09-12 11:18:42 +02:00
val remoteAddress : Address ,
2016-06-02 14:06:57 +02:00
val transport : Transport ,
val settings : RemoteSettings ,
val codec : AkkaPduCodec ) extends Actor with ActorLogging {
2013-03-20 20:38:49 +13:00
def inbound : Boolean
2013-08-26 15:41:05 +02:00
val eventPublisher = new EventPublisher ( context . system , log , settings . RemoteLifecycleEventsLogLevel )
2013-03-20 20:38:49 +13:00
2013-08-26 15:41:05 +02:00
def publishError ( reason : Throwable , logLevel : Logging . LogLevel ) : Unit =
tryPublish ( AssociationErrorEvent ( reason , localAddress , remoteAddress , inbound , logLevel ) )
def publishDisassociated ( ) : Unit = tryPublish ( DisassociatedEvent ( localAddress , remoteAddress , inbound ) )
private def tryPublish ( ev : AssociationEvent ) : Unit = try
eventPublisher . notifyListeners ( ev )
catch { case NonFatal ( e ) ⇒ log . error ( e , "Unable to publish error event to EventStream." ) }
2013-03-20 20:38:49 +13:00
}
2013-03-27 17:47:56 +01:00
/* *
* INTERNAL API
*/
private [ remote ] object EndpointWriter {
2013-05-30 14:03:35 +02:00
def props (
2016-06-02 14:06:57 +02:00
handleOrActive : Option [ AkkaProtocolHandle ] ,
localAddress : Address ,
remoteAddress : Address ,
refuseUid : Option [ Int ] ,
transport : AkkaProtocolTransport ,
settings : RemoteSettings ,
codec : AkkaPduCodec ,
receiveBuffers : ConcurrentHashMap [ Link , ResendState ] ,
2013-03-27 17:47:56 +01:00
reliableDeliverySupervisor : Option [ ActorRef ] ) : Props =
2013-11-26 15:25:05 +01:00
Props ( classOf [ EndpointWriter ] , handleOrActive , localAddress , remoteAddress , refuseUid , transport , settings , codec ,
2013-05-24 14:01:29 +02:00
receiveBuffers , reliableDeliverySupervisor )
2013-03-27 17:47:56 +01:00
/* *
* 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
* 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 .
*/
2014-04-03 15:56:11 +02:00
final case class TakeOver ( handle : AkkaProtocolHandle , replyTo : ActorRef ) extends NoSerializationVerificationNeeded
2014-03-07 13:20:01 +01:00
final case class TookOver ( writer : ActorRef , handle : AkkaProtocolHandle ) extends NoSerializationVerificationNeeded
2013-03-27 17:47:56 +01:00
case object BackoffTimer
case object FlushAndStop
2014-04-03 15:56:11 +02:00
private case object FlushAndStopTimeout
2013-03-27 17:47:56 +01:00
case object AckIdleCheckTimer
2014-04-03 15:56:11 +02:00
final case class StopReading ( writer : ActorRef , replyTo : ActorRef )
2014-03-07 13:20:01 +01:00
final case class StoppedReading ( writer : ActorRef )
2013-03-27 17:47:56 +01:00
2014-03-07 13:20:01 +01:00
final case class Handle ( handle : AkkaProtocolHandle ) extends NoSerializationVerificationNeeded
2013-08-23 14:39:21 +02:00
2014-03-07 13:20:01 +01:00
final case class OutboundAck ( ack : Ack )
2013-03-27 17:47:56 +01:00
2014-11-11 08:15:37 +01:00
// These settings are not configurable because wrong configuration will break the auto-tuning
2014-04-03 15:56:11 +02:00
private val SendBufferBatchSize = 5
private val MinAdaptiveBackoffNanos = 300000L // 0.3 ms
private val MaxAdaptiveBackoffNanos = 2000000L // 2 ms
private val LogBufferSizeInterval = 5000000000L // 5 s, in nanoseconds
private val MaxWriteCount = 50
2013-03-27 17:47:56 +01:00
}
2013-03-20 20:38:49 +13:00
/* *
* INTERNAL API
*/
private [ remote ] class EndpointWriter (
2016-06-02 14:06:57 +02:00
handleOrActive : Option [ AkkaProtocolHandle ] ,
localAddress : Address ,
remoteAddress : Address ,
refuseUid : Option [ Int ] ,
transport : AkkaProtocolTransport ,
settings : RemoteSettings ,
codec : AkkaPduCodec ,
val receiveBuffers : ConcurrentHashMap [ Link , ResendState ] ,
2013-04-26 12:18:01 +02:00
val reliableDeliverySupervisor : Option [ ActorRef ] )
2014-04-03 15:56:11 +02:00
extends EndpointActor ( localAddress , remoteAddress , transport , settings , codec ) {
2012-09-12 11:18:42 +02:00
import EndpointWriter._
import context.dispatcher
2016-10-28 14:52:17 +02:00
private val markLog = Logging . withMarker ( this )
2012-09-12 11:18:42 +02:00
val extendedSystem : ExtendedActorSystem = context . system . asInstanceOf [ ExtendedActorSystem ]
2013-05-20 14:35:51 +02:00
val remoteMetrics = RemoteMetricsExtension ( extendedSystem )
2014-04-03 15:56:11 +02:00
val backoffDispatcher = context . system . dispatchers . lookup ( "akka.remote.backoff-remote-dispatcher" )
2012-11-21 16:39:04 +01:00
2012-12-07 16:03:04 +01:00
var reader : Option [ ActorRef ] = None
2014-04-03 15:56:11 +02:00
var handle : Option [ AkkaProtocolHandle ] = handleOrActive
2012-12-07 16:03:04 +01:00
val readerId = Iterator from 0
2012-09-12 11:18:42 +02:00
2013-03-27 17:47:56 +01:00
def newAckDeadline : Deadline = Deadline . now + settings . SysMsgAckTimeout
var ackDeadline : Deadline = newAckDeadline
var lastAck : Option [ Ack ] = None
2013-08-26 15:41:05 +02:00
override val supervisorStrategy = OneForOneStrategy ( loggingEnabled = false ) {
case NonFatal ( e ) ⇒ publishAndThrow ( e , Logging . ErrorLevel )
}
2012-09-12 11:18:42 +02:00
2013-05-20 14:35:51 +02:00
val provider = RARP ( extendedSystem ) . provider
2016-10-28 14:52:17 +02:00
val msgDispatch = new DefaultMessageDispatcher ( extendedSystem , provider , markLog )
2012-09-12 11:18:42 +02:00
2014-04-03 15:56:11 +02:00
val inbound = handle . isDefined
2013-06-19 14:09:14 +02:00
var stopReason : DisassociateInfo = AssociationHandle . Unknown
2012-09-12 11:18:42 +02:00
2014-04-03 15:56:11 +02:00
// Use an internal buffer instead of Stash for efficiency
// stash/unstashAll is slow when many messages are stashed
// IMPORTANT: sender is not stored, so sender() and forward must not be used in EndpointWriter
val buffer = new java . util . LinkedList [ AnyRef ]
val prioBuffer = new java . util . LinkedList [ Send ]
var largeBufferLogTimestamp = System . nanoTime ( )
2013-08-26 15:41:05 +02:00
private def publishAndThrow ( reason : Throwable , logLevel : Logging . LogLevel ) : Nothing = {
reason match {
case _ : EndpointDisassociatedException ⇒ publishDisassociated ( )
case _ ⇒ publishError ( reason , logLevel )
}
2012-12-14 13:45:55 +01:00
throw reason
}
2012-09-12 11:18:42 +02:00
2014-04-03 15:56:11 +02:00
val ackIdleTimer = {
val interval = settings . SysMsgAckTimeout / 2
context . system . scheduler . schedule ( interval , interval , self , AckIdleCheckTimer )
2012-09-12 11:18:42 +02:00
}
2013-03-27 17:47:56 +01:00
override def preStart ( ) : Unit = {
2014-04-03 15:56:11 +02:00
handle match {
case Some ( h ) ⇒
reader = startReadEndpoint ( h )
case None ⇒
transport . associate ( remoteAddress , refuseUid ) . map ( Handle ( _ ) ) pipeTo self
}
}
2013-03-27 17:47:56 +01:00
2014-04-03 15:56:11 +02:00
override def postRestart ( reason : Throwable ) : Unit =
throw new IllegalStateException ( "EndpointWriter must not be restarted" )
2013-03-27 17:47:56 +01:00
2014-04-03 15:56:11 +02:00
override def postStop ( ) : Unit = {
ackIdleTimer . cancel ( )
while ( ! prioBuffer . isEmpty )
extendedSystem . deadLetters ! prioBuffer . poll
while ( ! buffer . isEmpty )
extendedSystem . deadLetters ! buffer . poll
handle foreach { _ . disassociate ( stopReason ) }
eventPublisher . notifyListeners ( DisassociatedEvent ( localAddress , remoteAddress , inbound ) )
2013-03-27 17:47:56 +01:00
}
2012-09-12 11:18:42 +02:00
2014-04-03 15:56:11 +02:00
def receive = if ( handle . isEmpty ) initializing else writing
def initializing : Receive = {
case s : Send ⇒
enqueueInBuffer ( s )
case Status . Failure ( e : InvalidAssociationException ) ⇒
2013-08-26 15:41:05 +02:00
publishAndThrow ( new InvalidAssociation ( localAddress , remoteAddress , e ) , Logging . WarningLevel )
2014-04-03 15:56:11 +02:00
case Status . Failure ( e ) ⇒
2013-08-26 15:41:05 +02:00
publishAndThrow ( new EndpointAssociationException ( s" Association failed with [ $remoteAddress ] " , e ) , Logging . DebugLevel )
2014-04-03 15:56:11 +02:00
case Handle ( inboundHandle ) ⇒
2012-12-21 18:11:56 +01:00
// Assert handle == None?
2014-05-12 13:51:05 +02:00
context . parent ! ReliableDeliverySupervisor . GotUid ( inboundHandle . handshakeInfo . uid , remoteAddress )
2012-12-07 16:03:04 +01:00
handle = Some ( inboundHandle )
2012-12-21 18:11:56 +01:00
reader = startReadEndpoint ( inboundHandle )
2014-04-03 15:56:11 +02:00
eventPublisher . notifyListeners ( AssociatedEvent ( localAddress , remoteAddress , inbound ) )
becomeWritingOrSendBufferedMessages ( )
2012-09-12 11:18:42 +02:00
}
2014-04-03 15:56:11 +02:00
def enqueueInBuffer ( msg : AnyRef ) : Unit = msg match {
case s @ Send ( _ : PriorityMessage , _ , _ , _ ) ⇒ prioBuffer offer s
2014-06-04 15:26:33 +02:00
case s @ Send ( ActorSelectionMessage ( _ : PriorityMessage , _ , _ ) , _ , _ , _ ) ⇒ prioBuffer offer s
2014-04-03 15:56:11 +02:00
case _ ⇒ buffer offer msg
}
2012-09-12 11:18:42 +02:00
2014-04-03 15:56:11 +02:00
val buffering : Receive = {
case s : Send ⇒ enqueueInBuffer ( s )
case BackoffTimer ⇒ sendBufferedMessages ( )
case FlushAndStop ⇒
// Flushing is postponed after the pending writes
buffer offer FlushAndStop
context . system . scheduler . scheduleOnce ( settings . FlushWait , self , FlushAndStopTimeout )
case FlushAndStopTimeout ⇒
// enough
flushAndStop ( )
}
def becomeWritingOrSendBufferedMessages ( ) : Unit =
if ( buffer . isEmpty )
context . become ( writing )
else {
context . become ( buffering )
sendBufferedMessages ( )
}
2013-02-25 12:11:39 +01:00
2014-04-03 15:56:11 +02:00
var writeCount = 0
var maxWriteCount = MaxWriteCount
var adaptiveBackoffNanos = 1000000L // 1 ms
var fullBackoff = false
// FIXME remove these counters when tuning/testing is completed
var fullBackoffCount = 1
var smallBackoffCount = 0
var noBackoffCount = 0
def adjustAdaptiveBackup ( ) : Unit = {
maxWriteCount = math . max ( writeCount , maxWriteCount )
if ( writeCount <= SendBufferBatchSize ) {
fullBackoff = true
adaptiveBackoffNanos = math . min ( ( adaptiveBackoffNanos * 1.2 ) . toLong , MaxAdaptiveBackoffNanos )
} else if ( writeCount >= maxWriteCount * 0.6 )
adaptiveBackoffNanos = math . max ( ( adaptiveBackoffNanos * 0.9 ) . toLong , MinAdaptiveBackoffNanos )
else if ( writeCount <= maxWriteCount * 0.2 )
adaptiveBackoffNanos = math . min ( ( adaptiveBackoffNanos * 1.1 ) . toLong , MaxAdaptiveBackoffNanos )
writeCount = 0
}
def sendBufferedMessages ( ) : Unit = {
def delegate ( msg : Any ) : Boolean = msg match {
case s : Send ⇒
writeSend ( s )
case FlushAndStop ⇒
flushAndStop ( )
false
case s @ StopReading ( _ , replyTo ) ⇒
reader . foreach ( _ . tell ( s , replyTo ) )
true
}
2012-09-12 11:18:42 +02:00
2014-04-03 15:56:11 +02:00
@tailrec def writeLoop ( count : Int ) : Boolean =
if ( count > 0 && ! buffer . isEmpty )
if ( delegate ( buffer . peek ) ) {
buffer . removeFirst ( )
writeCount += 1
writeLoop ( count - 1 )
} else false
else true
@tailrec def writePrioLoop ( ) : Boolean =
if ( prioBuffer . isEmpty ) true
else writeSend ( prioBuffer . peek ) && { prioBuffer . removeFirst ( ) ; writePrioLoop ( ) }
val size = buffer . size
val ok = writePrioLoop ( ) && writeLoop ( SendBufferBatchSize )
if ( buffer . isEmpty && prioBuffer . isEmpty ) {
// FIXME remove this when testing/tuning is completed
if ( log . isDebugEnabled )
log . debug ( s" Drained buffer with maxWriteCount: $maxWriteCount , fullBackoffCount: $fullBackoffCount " +
s" , smallBackoffCount: $smallBackoffCount , noBackoffCount: $noBackoffCount " +
s" , adaptiveBackoff: ${ adaptiveBackoffNanos / 1000 } " )
fullBackoffCount = 1
smallBackoffCount = 0
noBackoffCount = 0
writeCount = 0
maxWriteCount = MaxWriteCount
context . become ( writing )
} else if ( ok ) {
noBackoffCount += 1
self ! BackoffTimer
} else {
if ( size > settings . LogBufferSizeExceeding ) {
val now = System . nanoTime ( )
if ( now - largeBufferLogTimestamp >= LogBufferSizeInterval ) {
2016-06-02 14:06:57 +02:00
log . warning (
"[{}] buffered messages in EndpointWriter for [{}]. " +
"You should probably implement flow control to avoid flooding the remote connection." ,
2014-04-03 15:56:11 +02:00
size , remoteAddress )
largeBufferLogTimestamp = now
}
}
2013-05-20 14:35:51 +02:00
2014-04-03 15:56:11 +02:00
adjustAdaptiveBackup ( )
scheduleBackoffTimer ( )
}
2013-03-27 17:47:56 +01:00
2014-04-03 15:56:11 +02:00
}
2013-10-08 09:56:59 +02:00
2014-04-03 15:56:11 +02:00
def scheduleBackoffTimer ( ) : Unit = {
if ( fullBackoff ) {
fullBackoffCount += 1
fullBackoff = false
context . system . scheduler . scheduleOnce ( settings . BackoffPeriod , self , BackoffTimer )
} else {
smallBackoffCount += 1
val s = self
val backoffDeadlinelineNanoTime = System . nanoTime + adaptiveBackoffNanos
Future {
@tailrec def backoff ( ) : Unit = {
val backoffNanos = backoffDeadlinelineNanoTime - System . nanoTime
if ( backoffNanos > 0 ) {
LockSupport . parkNanos ( backoffNanos )
2015-06-02 21:01:00 -07:00
// parkNanos allows for spurious wake-up, check again
2014-04-03 15:56:11 +02:00
backoff ( )
}
2012-12-14 13:45:55 +01:00
}
2014-04-03 15:56:11 +02:00
backoff ( )
s . tell ( BackoffTimer , ActorRef . noSender )
} ( backoffDispatcher )
}
}
val writing : Receive = {
case s : Send ⇒
if ( ! writeSend ( s ) ) {
2015-05-27 14:22:16 +02:00
enqueueInBuffer ( s )
2014-04-03 15:56:11 +02:00
scheduleBackoffTimer ( )
context . become ( buffering )
2012-09-12 11:18:42 +02:00
}
2013-02-25 12:11:39 +01:00
2014-04-03 15:56:11 +02:00
// We are in Writing state, so buffer is empty, safe to stop here
case FlushAndStop ⇒
flushAndStop ( )
2013-03-27 17:47:56 +01:00
2014-04-03 15:56:11 +02:00
case AckIdleCheckTimer if ackDeadline . isOverdue ( ) ⇒
2013-03-27 17:47:56 +01:00
trySendPureAck ( )
2012-09-12 11:18:42 +02:00
}
2014-04-03 15:56:11 +02:00
def writeSend ( s : Send ) : Boolean = try {
handle match {
case Some ( h ) ⇒
if ( provider . remoteSettings . LogSend ) {
def msgLog = s" RemoteMessage: [ ${ s . message } ] to [ ${ s . recipient } ]<+[ ${ s . recipient . path } ] from [ ${ s . senderOption . getOrElse ( extendedSystem . deadLetters ) } ] "
log . debug ( "sending message {}" , msgLog )
}
2013-01-14 10:04:49 +01:00
2014-04-03 15:56:11 +02:00
val pdu = codec . constructMessage (
s . recipient . localAddressToUse ,
s . recipient ,
serializeMessage ( s . message ) ,
s . senderOption ,
seqOption = s . seqOpt ,
ackOption = lastAck )
val pduSize = pdu . size
remoteMetrics . logPayloadBytes ( s . message , pduSize )
if ( pduSize > transport . maximumPayloadBytes ) {
val reason = new OversizedPayloadException ( s" Discarding oversized payload sent to ${ s . recipient } : max allowed size ${ transport . maximumPayloadBytes } bytes, actual size of encoded ${ s . message . getClass } was ${ pdu . size } bytes. " )
log . error ( reason , "Transient association error (association remains live)" )
true
} else {
val ok = h . write ( pdu )
if ( ok ) {
ackDeadline = newAckDeadline
lastAck = None
}
ok
}
case None ⇒
throw new EndpointException ( "Internal error: Endpoint is in state Writing, but no association handle is present." )
}
} catch {
case e : NotSerializableException ⇒
2017-09-29 04:09:57 +02:00
log . error ( e , "Serializer not defined for message type [{}]. Transient association error (association remains live)" , s . message . getClass )
2016-09-02 11:05:00 +02:00
true
case e : MessageSerializer . SerializationException ⇒
log . error ( e , "{} Transient association error (association remains live)" , e . getMessage )
2014-04-03 15:56:11 +02:00
true
case e : EndpointException ⇒
publishAndThrow ( e , Logging . ErrorLevel )
case NonFatal ( e ) ⇒
publishAndThrow ( new EndpointException ( "Failed to write message to the transport" , e ) , Logging . ErrorLevel )
}
def handoff : Receive = {
case Terminated ( _ ) ⇒
reader = startReadEndpoint ( handle . get )
becomeWritingOrSendBufferedMessages ( )
2013-01-14 10:04:49 +01:00
2014-04-03 15:56:11 +02:00
case s : Send ⇒
enqueueInBuffer ( s )
2016-03-21 08:41:11 +01:00
case OutboundAck ( _ ) ⇒
// Ignore outgoing acks during take over, since we might have
// replaced the handle with a connection to a new, restarted, system
// and the ack might be targeted to the old incarnation.
// See issue #19780
2013-01-14 10:04:49 +01:00
}
2014-04-03 15:56:11 +02:00
override def unhandled ( message : Any ) : Unit = message match {
case Terminated ( r ) if r == reader . orNull ⇒
2013-08-26 15:41:05 +02:00
publishAndThrow ( new EndpointDisassociatedException ( "Disassociated" ) , Logging . DebugLevel )
2014-04-03 15:56:11 +02:00
case s @ StopReading ( _ , replyTo ) ⇒
2013-06-03 11:35:37 +02:00
reader match {
2014-04-03 15:56:11 +02:00
case Some ( r ) ⇒
r . tell ( s , replyTo )
case None ⇒
2015-06-02 21:01:00 -07:00
// initializing, buffer and take care of it later when buffer is sent
2014-04-03 15:56:11 +02:00
enqueueInBuffer ( s )
2013-06-03 11:35:37 +02:00
}
2014-04-03 15:56:11 +02:00
case TakeOver ( newHandle , replyTo ) ⇒
2012-11-21 16:39:04 +01:00
// Shutdown old reader
2017-05-30 10:03:10 +02:00
handle foreach { _ . disassociate ( "the association was replaced by a new one" , log ) }
2012-12-07 16:03:04 +01:00
handle = Some ( newHandle )
2014-04-03 15:56:11 +02:00
replyTo ! TookOver ( self , newHandle )
context . become ( handoff )
case FlushAndStop ⇒
2013-06-19 14:09:14 +02:00
stopReason = AssociationHandle . Shutdown
2014-04-03 15:56:11 +02:00
context . stop ( self )
case OutboundAck ( ack ) ⇒
2013-03-27 17:47:56 +01:00
lastAck = Some ( ack )
2015-01-19 10:03:40 +01:00
if ( ackDeadline . isOverdue ( ) )
trySendPureAck ( )
2014-04-03 15:56:11 +02:00
case AckIdleCheckTimer ⇒ // Ignore
case FlushAndStopTimeout ⇒ // ignore
case BackoffTimer ⇒ // ignore
case other ⇒ super . unhandled ( other )
2012-09-12 11:18:42 +02:00
}
2014-04-03 15:56:11 +02:00
def flushAndStop ( ) : Unit = {
// Try to send a last Ack message
trySendPureAck ( )
stopReason = AssociationHandle . Shutdown
context . stop ( self )
}
private def trySendPureAck ( ) : Unit =
2017-09-11 10:10:04 +02:00
for ( h ← handle ; ack ← lastAck ) {
2014-04-03 15:56:11 +02:00
if ( h . write ( codec . constructPureAck ( ack ) ) ) {
ackDeadline = newAckDeadline
lastAck = None
}
2017-09-11 10:10:04 +02:00
}
2013-03-27 17:47:56 +01:00
private def startReadEndpoint ( handle : AkkaProtocolHandle ) : Some [ ActorRef ] = {
2012-12-21 18:11:56 +01:00
val newReader =
2013-03-20 20:38:49 +13:00
context . watch ( context . actorOf (
2013-08-19 12:06:07 +02:00
RARP ( context . system ) . configureDispatcher ( EndpointReader . props ( localAddress , remoteAddress , transport , settings , codec ,
2013-08-19 15:34:24 +02:00
msgDispatch , inbound , handle . handshakeInfo . uid , reliableDeliverySupervisor , receiveBuffers ) ) . withDeploy ( Deploy . local ) ,
2012-12-21 18:11:56 +01:00
"endpointReader-" + AddressUrlEncoder ( remoteAddress ) + "-" + readerId . next ( ) ) )
handle . readHandlerPromise . success ( ActorHandleEventListener ( newReader ) )
Some ( newReader )
2012-09-12 11:18:42 +02:00
}
2013-03-27 17:47:56 +01:00
private def serializeMessage ( msg : Any ) : SerializedMessage = handle match {
2012-12-07 16:03:04 +01:00
case Some ( h ) ⇒
2013-05-20 14:35:51 +02:00
Serialization . currentTransportInformation . withValue ( Serialization . Information ( h . localAddress , extendedSystem ) ) {
2012-12-07 16:03:04 +01:00
( MessageSerializer . serialize ( extendedSystem , msg . asInstanceOf [ AnyRef ] ) )
}
2013-05-30 21:37:57 +02:00
case None ⇒
throw new EndpointException ( "Internal error: No handle was present during serialization of outbound message." )
2012-09-12 11:18:42 +02:00
}
}
2013-03-27 17:47:56 +01:00
/* *
* INTERNAL API
*/
private [ remote ] object EndpointReader {
2013-05-30 14:03:35 +02:00
def props (
2016-06-02 14:06:57 +02:00
localAddress : Address ,
remoteAddress : Address ,
transport : Transport ,
settings : RemoteSettings ,
codec : AkkaPduCodec ,
msgDispatch : InboundMessageDispatcher ,
inbound : Boolean ,
uid : Int ,
2013-03-27 17:47:56 +01:00
reliableDeliverySupervisor : Option [ ActorRef ] ,
2016-06-02 14:06:57 +02:00
receiveBuffers : ConcurrentHashMap [ Link , ResendState ] ) : Props =
2013-03-27 17:47:56 +01:00
Props ( classOf [ EndpointReader ] , localAddress , remoteAddress , transport , settings , codec , msgDispatch , inbound ,
2013-08-19 15:34:24 +02:00
uid , reliableDeliverySupervisor , receiveBuffers )
2013-03-27 17:47:56 +01:00
}
2013-02-08 13:13:52 +01:00
/* *
* INTERNAL API
*/
2012-09-12 11:18:42 +02:00
private [ remote ] class EndpointReader (
2016-06-02 14:06:57 +02:00
localAddress : Address ,
remoteAddress : Address ,
transport : Transport ,
settings : RemoteSettings ,
codec : AkkaPduCodec ,
msgDispatch : InboundMessageDispatcher ,
val inbound : Boolean ,
val uid : Int ,
2013-03-27 17:47:56 +01:00
val reliableDeliverySupervisor : Option [ ActorRef ] ,
2016-06-02 14:06:57 +02:00
val receiveBuffers : ConcurrentHashMap [ Link , ResendState ] ) extends EndpointActor ( localAddress , remoteAddress , transport , settings , codec ) {
2013-03-27 17:47:56 +01:00
2013-06-03 11:35:37 +02:00
import EndpointWriter. { OutboundAck , StopReading , StoppedReading }
2012-09-12 11:18:42 +02:00
2012-12-11 13:08:36 +01:00
val provider = RARP ( context . system ) . provider
2013-03-27 17:47:56 +01:00
var ackedReceiveBuffer = new AckedReceiveBuffer [ Message ]
override def preStart ( ) : Unit = {
receiveBuffers . get ( Link ( localAddress , remoteAddress ) ) match {
case null ⇒
2013-08-19 15:34:24 +02:00
case ResendState ( `uid` , buffer ) ⇒
ackedReceiveBuffer = buffer
2013-03-27 17:47:56 +01:00
deliverAndAck ( )
2013-08-19 15:34:24 +02:00
case _ ⇒
2013-03-27 17:47:56 +01:00
}
}
2013-06-03 11:35:37 +02:00
override def postStop ( ) : Unit = saveState ( )
2013-03-27 17:47:56 +01:00
2013-06-03 11:35:37 +02:00
def saveState ( ) : Unit = {
2013-08-19 15:34:24 +02:00
def merge ( currentState : ResendState , oldState : ResendState ) : ResendState =
if ( currentState . uid == oldState . uid ) ResendState ( uid , oldState . buffer . mergeFrom ( currentState . buffer ) )
else currentState
2013-03-27 17:47:56 +01:00
@tailrec
2013-08-19 15:34:24 +02:00
def updateSavedState ( key : Link , expectedState : ResendState ) : Unit = {
2013-03-27 17:47:56 +01:00
if ( expectedState eq null ) {
2013-08-19 15:34:24 +02:00
if ( receiveBuffers . putIfAbsent ( key , ResendState ( uid , ackedReceiveBuffer ) ) ne null )
updateSavedState ( key , receiveBuffers . get ( key ) )
} else if ( ! receiveBuffers . replace ( key , expectedState , merge ( ResendState ( uid , ackedReceiveBuffer ) , expectedState ) ) )
2013-03-27 17:47:56 +01:00
updateSavedState ( key , receiveBuffers . get ( key ) )
}
val key = Link ( localAddress , remoteAddress )
updateSavedState ( key , receiveBuffers . get ( key ) )
}
2012-09-12 11:18:42 +02:00
override def receive : Receive = {
2013-06-19 14:09:14 +02:00
case Disassociated ( info ) ⇒ handleDisassociated ( info )
2012-09-12 11:18:42 +02:00
2013-03-27 17:47:56 +01:00
case InboundPayload ( p ) if p . size <= transport . maximumPayloadBytes ⇒
val ( ackOption , msgOption ) = tryDecodeMessageAndAck ( p )
for ( ack ← ackOption ; reliableDelivery ← reliableDeliverySupervisor ) reliableDelivery ! ack
msgOption match {
case Some ( msg ) ⇒
if ( msg . reliableDeliveryEnabled ) {
ackedReceiveBuffer = ackedReceiveBuffer . receive ( msg )
deliverAndAck ( )
2016-12-16 11:36:04 +01:00
} else try
msgDispatch . dispatch ( msg . recipient , msg . recipientAddress , msg . serializedMessage , msg . senderOption )
catch {
case e : NotSerializableException ⇒
val sm = msg . serializedMessage
log . warning (
"Serializer not defined for message with serializer id [{}] and manifest [{}]. " +
"Transient association error (association remains live). {}" ,
sm . getSerializerId ,
if ( sm . hasMessageManifest ) sm . getMessageManifest . toStringUtf8 else "" ,
e . getMessage )
}
2013-03-27 17:47:56 +01:00
case None ⇒
2013-03-20 20:38:49 +13:00
}
2013-03-27 17:47:56 +01:00
case InboundPayload ( oversized ) ⇒
2016-06-02 14:06:57 +02:00
log . error (
new OversizedPayloadException ( s" Discarding oversized payload received: " +
s" max allowed size [ ${ transport . maximumPayloadBytes } ] bytes, actual size [ ${ oversized . size } ] bytes. " ) ,
2013-07-01 12:51:15 +02:00
"Transient error while reading from association (association remains live)" )
2013-03-27 17:47:56 +01:00
2014-04-03 15:56:11 +02:00
case StopReading ( writer , replyTo ) ⇒
2013-06-03 11:35:37 +02:00
saveState ( )
context . become ( notReading )
2014-04-03 15:56:11 +02:00
replyTo ! StoppedReading ( writer )
2013-06-03 11:35:37 +02:00
}
def notReading : Receive = {
2013-06-19 14:09:14 +02:00
case Disassociated ( info ) ⇒ handleDisassociated ( info )
2013-06-03 11:35:37 +02:00
2014-04-03 15:56:11 +02:00
case StopReading ( writer , replyTo ) ⇒
replyTo ! StoppedReading ( writer )
2013-06-03 11:35:37 +02:00
case InboundPayload ( p ) ⇒
val ( ackOption , _ ) = tryDecodeMessageAndAck ( p )
for ( ack ← ackOption ; reliableDelivery ← reliableDeliverySupervisor ) reliableDelivery ! ack
case _ ⇒
2013-03-27 17:47:56 +01:00
}
2013-06-19 14:09:14 +02:00
private def handleDisassociated ( info : DisassociateInfo ) : Unit = info match {
case AssociationHandle . Unknown ⇒
context . stop ( self )
case AssociationHandle . Shutdown ⇒
2013-10-09 11:29:02 +02:00
throw ShutDownAssociation (
2013-06-19 14:09:14 +02:00
localAddress ,
remoteAddress ,
InvalidAssociationException ( "The remote system terminated the association because it is shutting down." ) )
case AssociationHandle . Quarantined ⇒
throw InvalidAssociation (
localAddress ,
remoteAddress ,
InvalidAssociationException ( "The remote system has quarantined this system. No further associations " +
2015-12-06 20:14:44 -05:00
"to the remote system are possible until this system is restarted." ) ,
Some ( AssociationHandle . Quarantined ) )
2013-06-19 14:09:14 +02:00
}
2013-03-27 17:47:56 +01:00
private def deliverAndAck ( ) : Unit = {
val ( updatedBuffer , deliver , ack ) = ackedReceiveBuffer . extractDeliverable
ackedReceiveBuffer = updatedBuffer
2013-10-08 09:56:59 +02:00
2013-03-27 17:47:56 +01:00
// Notify writer that some messages can be acked
context . parent ! OutboundAck ( ack )
deliver foreach { m ⇒
msgDispatch . dispatch ( m . recipient , m . recipientAddress , m . serializedMessage , m . senderOption )
}
2012-09-12 11:18:42 +02:00
}
2013-03-27 17:47:56 +01:00
private def tryDecodeMessageAndAck ( pdu : ByteString ) : ( Option [ Ack ] , Option [ Message ] ) = try {
2012-11-21 15:58:01 +01:00
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 )
}
}