2016-05-04 13:31:08 +02:00
|
|
|
/**
|
2018-01-04 17:26:29 +00:00
|
|
|
* Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com>
|
2016-05-04 13:31:08 +02:00
|
|
|
*/
|
|
|
|
|
package akka.remote.artery
|
|
|
|
|
|
2016-09-09 07:45:21 +02:00
|
|
|
import akka.util.PrettyDuration.PrettyPrintableDuration
|
2016-05-04 13:31:08 +02:00
|
|
|
import java.util.ArrayDeque
|
|
|
|
|
import scala.annotation.tailrec
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import scala.util.Failure
|
|
|
|
|
import scala.util.Success
|
|
|
|
|
import scala.util.Try
|
|
|
|
|
import akka.Done
|
2016-05-09 07:31:41 +02:00
|
|
|
import akka.remote.UniqueAddress
|
2016-05-12 08:56:28 +02:00
|
|
|
import akka.remote.artery.InboundControlJunction.ControlMessageObserver
|
2016-05-04 13:31:08 +02:00
|
|
|
import akka.stream.Attributes
|
|
|
|
|
import akka.stream.FlowShape
|
|
|
|
|
import akka.stream.Inlet
|
|
|
|
|
import akka.stream.Outlet
|
|
|
|
|
import akka.stream.stage.GraphStage
|
|
|
|
|
import akka.stream.stage.GraphStageLogic
|
|
|
|
|
import akka.stream.stage.InHandler
|
|
|
|
|
import akka.stream.stage.OutHandler
|
|
|
|
|
import akka.stream.stage.TimerGraphStageLogic
|
2016-05-25 12:28:44 +02:00
|
|
|
import akka.remote.artery.OutboundHandshake.HandshakeReq
|
2016-06-08 12:40:40 +02:00
|
|
|
import akka.actor.ActorRef
|
2016-09-08 15:01:32 +02:00
|
|
|
import akka.dispatch.sysmsg.SystemMessage
|
2016-09-09 07:45:21 +02:00
|
|
|
import scala.util.control.NoStackTrace
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] object SystemMessageDelivery {
|
2016-09-08 17:58:25 +02:00
|
|
|
final case class SystemMessageEnvelope(message: AnyRef, seqNo: Long, ackReplyTo: UniqueAddress) extends ArteryMessage
|
2016-05-09 07:31:41 +02:00
|
|
|
final case class Ack(seqNo: Long, from: UniqueAddress) extends Reply
|
|
|
|
|
final case class Nack(seqNo: Long, from: UniqueAddress) extends Reply
|
2016-05-04 13:31:08 +02:00
|
|
|
|
2016-05-13 15:34:37 +02:00
|
|
|
final case object ClearSystemMessageDelivery
|
|
|
|
|
|
2016-09-09 07:45:21 +02:00
|
|
|
final class GaveUpSystemMessageException(msg: String) extends RuntimeException(msg) with NoStackTrace
|
|
|
|
|
|
2016-05-04 13:31:08 +02:00
|
|
|
private case object ResendTick
|
2016-09-08 15:01:32 +02:00
|
|
|
|
|
|
|
|
// If other message types than SystemMesage need acked delivery they can extend this trait.
|
|
|
|
|
// Used in tests since real SystemMessage are somewhat cumbersome to create.
|
|
|
|
|
trait AckedDeliveryMessage
|
2016-09-09 07:45:21 +02:00
|
|
|
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] class SystemMessageDelivery(
|
2016-05-09 07:31:41 +02:00
|
|
|
outboundContext: OutboundContext,
|
2016-06-08 12:40:40 +02:00
|
|
|
deadLetters: ActorRef,
|
2016-06-03 11:59:00 +02:00
|
|
|
resendInterval: FiniteDuration,
|
|
|
|
|
maxBufferSize: Int)
|
2016-06-29 17:09:33 +02:00
|
|
|
extends GraphStage[FlowShape[OutboundEnvelope, OutboundEnvelope]] {
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
import SystemMessageDelivery._
|
|
|
|
|
|
2016-06-29 17:09:33 +02:00
|
|
|
val in: Inlet[OutboundEnvelope] = Inlet("SystemMessageDelivery.in")
|
|
|
|
|
val out: Outlet[OutboundEnvelope] = Outlet("SystemMessageDelivery.out")
|
|
|
|
|
override val shape: FlowShape[OutboundEnvelope, OutboundEnvelope] = FlowShape(in, out)
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
2016-05-12 08:56:28 +02:00
|
|
|
new TimerGraphStageLogic(shape) with InHandler with OutHandler with ControlMessageObserver {
|
2016-05-04 13:31:08 +02:00
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
private var replyObserverAttached = false
|
|
|
|
|
private var seqNo = 0L // sequence number for the first message will be 1
|
2016-06-29 17:09:33 +02:00
|
|
|
private val unacknowledged = new ArrayDeque[OutboundEnvelope]
|
|
|
|
|
private var resending = new ArrayDeque[OutboundEnvelope]
|
2016-05-09 07:31:41 +02:00
|
|
|
private var resendingFromSeqNo = -1L
|
|
|
|
|
private var stopping = false
|
|
|
|
|
|
2016-09-09 07:45:21 +02:00
|
|
|
private val giveUpAfterNanos = outboundContext.settings.Advanced.GiveUpSystemMessageAfter.toNanos
|
|
|
|
|
private var ackTimestamp = System.nanoTime()
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
private def localAddress = outboundContext.localAddress
|
|
|
|
|
private def remoteAddress = outboundContext.remoteAddress
|
2016-05-04 13:31:08 +02:00
|
|
|
|
|
|
|
|
override def preStart(): Unit = {
|
|
|
|
|
implicit val ec = materializer.executionContext
|
2016-05-12 08:56:28 +02:00
|
|
|
outboundContext.controlSubject.attach(this).foreach {
|
2016-05-04 13:31:08 +02:00
|
|
|
getAsyncCallback[Done] { _ ⇒
|
2016-05-09 07:31:41 +02:00
|
|
|
replyObserverAttached = true
|
2016-05-04 13:31:08 +02:00
|
|
|
if (isAvailable(out))
|
|
|
|
|
pull(in) // onPull from downstream already called
|
|
|
|
|
}.invoke
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def postStop(): Unit = {
|
2017-11-20 15:15:17 +01:00
|
|
|
val pendingCount = unacknowledged.size
|
2016-06-08 12:40:40 +02:00
|
|
|
sendUnacknowledgedToDeadLetters()
|
|
|
|
|
unacknowledged.clear()
|
2017-11-20 15:15:17 +01:00
|
|
|
if (pendingCount > 0)
|
|
|
|
|
outboundContext.quarantine(s"SystemMessageDelivery stopped with [$pendingCount] pending system messages.")
|
2016-05-12 08:56:28 +02:00
|
|
|
outboundContext.controlSubject.detach(this)
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
if (unacknowledged.isEmpty)
|
|
|
|
|
super.onUpstreamFinish()
|
|
|
|
|
else
|
|
|
|
|
stopping = true
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override protected def onTimer(timerKey: Any): Unit =
|
|
|
|
|
timerKey match {
|
|
|
|
|
case ResendTick ⇒
|
2016-09-09 07:45:21 +02:00
|
|
|
checkGiveUp()
|
2016-05-04 13:31:08 +02:00
|
|
|
if (resending.isEmpty && !unacknowledged.isEmpty) {
|
|
|
|
|
resending = unacknowledged.clone()
|
|
|
|
|
tryResend()
|
|
|
|
|
}
|
2016-05-17 14:05:50 +02:00
|
|
|
if (!unacknowledged.isEmpty)
|
|
|
|
|
scheduleOnce(ResendTick, resendInterval)
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
|
2016-05-12 08:56:28 +02:00
|
|
|
// ControlMessageObserver, external call
|
|
|
|
|
override def notify(inboundEnvelope: InboundEnvelope): Unit = {
|
2016-05-09 07:31:41 +02:00
|
|
|
inboundEnvelope.message match {
|
|
|
|
|
case ack: Ack ⇒ if (ack.from.address == remoteAddress) ackCallback.invoke(ack)
|
|
|
|
|
case nack: Nack ⇒ if (nack.from.address == remoteAddress) nackCallback.invoke(nack)
|
|
|
|
|
case _ ⇒ // not interested
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
}
|
2017-11-20 15:15:17 +01:00
|
|
|
|
|
|
|
|
// ControlMessageObserver, external call
|
|
|
|
|
override def controlSubjectCompleted(signal: Try[Done]): Unit = {
|
|
|
|
|
getAsyncCallback[Try[Done]] {
|
|
|
|
|
case Success(_) ⇒ completeStage()
|
|
|
|
|
case Failure(cause) ⇒ failStage(cause)
|
|
|
|
|
}.invoke(signal)
|
|
|
|
|
}
|
2016-05-04 13:31:08 +02:00
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
private val ackCallback = getAsyncCallback[Ack] { reply ⇒
|
|
|
|
|
ack(reply.seqNo)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private val nackCallback = getAsyncCallback[Nack] { reply ⇒
|
2016-05-13 15:34:37 +02:00
|
|
|
if (reply.seqNo <= seqNo) {
|
|
|
|
|
ack(reply.seqNo)
|
|
|
|
|
if (reply.seqNo > resendingFromSeqNo)
|
|
|
|
|
resending = unacknowledged.clone()
|
|
|
|
|
tryResend()
|
|
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
}
|
|
|
|
|
|
2016-05-04 13:31:08 +02:00
|
|
|
private def ack(n: Long): Unit = {
|
2016-09-09 07:45:21 +02:00
|
|
|
ackTimestamp = System.nanoTime()
|
2016-05-13 15:34:37 +02:00
|
|
|
if (n <= seqNo)
|
|
|
|
|
clearUnacknowledged(n)
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@tailrec private def clearUnacknowledged(ackedSeqNo: Long): Unit = {
|
|
|
|
|
if (!unacknowledged.isEmpty &&
|
|
|
|
|
unacknowledged.peek().message.asInstanceOf[SystemMessageEnvelope].seqNo <= ackedSeqNo) {
|
|
|
|
|
unacknowledged.removeFirst()
|
2016-05-17 14:05:50 +02:00
|
|
|
if (unacknowledged.isEmpty)
|
|
|
|
|
cancelTimer(resendInterval)
|
|
|
|
|
|
2016-05-04 13:31:08 +02:00
|
|
|
if (stopping && unacknowledged.isEmpty)
|
|
|
|
|
completeStage()
|
|
|
|
|
else
|
|
|
|
|
clearUnacknowledged(ackedSeqNo)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def tryResend(): Unit = {
|
|
|
|
|
if (isAvailable(out) && !resending.isEmpty)
|
2016-06-29 17:09:33 +02:00
|
|
|
pushCopy(resending.poll())
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// important to not send the buffered instance, since it's mutable
|
|
|
|
|
private def pushCopy(outboundEnvelope: OutboundEnvelope): Unit = {
|
|
|
|
|
push(out, outboundEnvelope.copy())
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// InHandler
|
|
|
|
|
override def onPush(): Unit = {
|
2016-06-29 17:09:33 +02:00
|
|
|
val outboundEnvelope = grab(in)
|
|
|
|
|
outboundEnvelope.message match {
|
2016-09-08 15:01:32 +02:00
|
|
|
case msg @ (_: SystemMessage | _: AckedDeliveryMessage) ⇒
|
2016-05-13 15:34:37 +02:00
|
|
|
if (unacknowledged.size < maxBufferSize) {
|
|
|
|
|
seqNo += 1
|
2016-09-09 07:45:21 +02:00
|
|
|
if (unacknowledged.isEmpty)
|
|
|
|
|
ackTimestamp = System.nanoTime()
|
|
|
|
|
else
|
|
|
|
|
checkGiveUp()
|
2016-06-29 17:09:33 +02:00
|
|
|
val sendEnvelope = outboundEnvelope.withMessage(SystemMessageEnvelope(msg, seqNo, localAddress))
|
|
|
|
|
unacknowledged.offer(sendEnvelope)
|
2016-05-17 14:05:50 +02:00
|
|
|
scheduleOnce(ResendTick, resendInterval)
|
2016-05-13 15:34:37 +02:00
|
|
|
if (resending.isEmpty && isAvailable(out))
|
2016-06-29 17:09:33 +02:00
|
|
|
pushCopy(sendEnvelope)
|
2016-05-13 15:34:37 +02:00
|
|
|
else {
|
2016-06-29 17:09:33 +02:00
|
|
|
resending.offer(sendEnvelope)
|
2016-05-13 15:34:37 +02:00
|
|
|
tryResend()
|
|
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
// buffer overflow
|
|
|
|
|
outboundContext.quarantine(reason = s"System message delivery buffer overflow, size [$maxBufferSize]")
|
2016-06-29 17:09:33 +02:00
|
|
|
deadLetters ! outboundEnvelope
|
2016-05-13 15:34:37 +02:00
|
|
|
pull(in)
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
2016-09-08 15:01:32 +02:00
|
|
|
case _: HandshakeReq ⇒
|
|
|
|
|
// pass on HandshakeReq
|
|
|
|
|
if (isAvailable(out))
|
|
|
|
|
pushCopy(outboundEnvelope)
|
|
|
|
|
case ClearSystemMessageDelivery ⇒
|
|
|
|
|
clear()
|
|
|
|
|
pull(in)
|
|
|
|
|
case _ ⇒
|
|
|
|
|
// e.g. ActorSystemTerminating or ActorSelectionMessage with PriorityMessage, no need for acked delivery
|
|
|
|
|
if (resending.isEmpty && isAvailable(out))
|
|
|
|
|
push(out, outboundEnvelope)
|
|
|
|
|
else {
|
|
|
|
|
resending.offer(outboundEnvelope)
|
|
|
|
|
tryResend()
|
|
|
|
|
}
|
2016-05-04 13:31:08 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-09-09 07:45:21 +02:00
|
|
|
private def checkGiveUp(): Unit = {
|
|
|
|
|
if (!unacknowledged.isEmpty && (System.nanoTime() - ackTimestamp > giveUpAfterNanos))
|
|
|
|
|
throw new GaveUpSystemMessageException(
|
|
|
|
|
s"Gave up sending system message to [${outboundContext.remoteAddress}] after " +
|
|
|
|
|
s"${outboundContext.settings.Advanced.GiveUpSystemMessageAfter.pretty}.")
|
|
|
|
|
}
|
|
|
|
|
|
2016-05-13 15:34:37 +02:00
|
|
|
private def clear(): Unit = {
|
2016-06-08 12:40:40 +02:00
|
|
|
sendUnacknowledgedToDeadLetters()
|
2016-05-13 15:34:37 +02:00
|
|
|
seqNo = 0L // sequence number for the first message will be 1
|
|
|
|
|
unacknowledged.clear()
|
|
|
|
|
resending.clear()
|
|
|
|
|
resendingFromSeqNo = -1L
|
2016-05-17 14:05:50 +02:00
|
|
|
cancelTimer(resendInterval)
|
2016-05-13 15:34:37 +02:00
|
|
|
}
|
|
|
|
|
|
2016-06-08 12:40:40 +02:00
|
|
|
private def sendUnacknowledgedToDeadLetters(): Unit = {
|
|
|
|
|
val iter = unacknowledged.iterator
|
|
|
|
|
while (iter.hasNext()) {
|
|
|
|
|
deadLetters ! iter.next()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-05-04 13:31:08 +02:00
|
|
|
// OutHandler
|
|
|
|
|
override def onPull(): Unit = {
|
2016-05-09 07:31:41 +02:00
|
|
|
if (replyObserverAttached) { // otherwise it will be pulled after attached
|
2016-05-04 13:31:08 +02:00
|
|
|
if (resending.isEmpty && !hasBeenPulled(in) && !stopping)
|
|
|
|
|
pull(in)
|
|
|
|
|
else
|
|
|
|
|
tryResend()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
setHandlers(in, out, this)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] class SystemMessageAcker(inboundContext: InboundContext) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] {
|
2016-05-04 13:31:08 +02:00
|
|
|
import SystemMessageDelivery._
|
|
|
|
|
|
|
|
|
|
val in: Inlet[InboundEnvelope] = Inlet("SystemMessageAcker.in")
|
|
|
|
|
val out: Outlet[InboundEnvelope] = Outlet("SystemMessageAcker.out")
|
|
|
|
|
override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out)
|
|
|
|
|
|
|
|
|
|
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
|
|
|
|
|
new GraphStageLogic(shape) with InHandler with OutHandler {
|
|
|
|
|
|
2016-06-23 16:36:55 +02:00
|
|
|
// TODO we might need have to prune old unused entries
|
|
|
|
|
var sequenceNumbers = Map.empty[UniqueAddress, Long]
|
2016-05-04 13:31:08 +02:00
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
def localAddress = inboundContext.localAddress
|
|
|
|
|
|
2016-05-04 13:31:08 +02:00
|
|
|
// InHandler
|
|
|
|
|
override def onPush(): Unit = {
|
2016-06-06 08:26:15 +02:00
|
|
|
val env = grab(in)
|
|
|
|
|
env.message match {
|
|
|
|
|
case sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo) ⇒
|
2016-06-23 16:36:55 +02:00
|
|
|
val expectedSeqNo = sequenceNumbers.get(ackReplyTo) match {
|
|
|
|
|
case None ⇒ 1L
|
|
|
|
|
case Some(seqNo) ⇒ seqNo
|
|
|
|
|
}
|
|
|
|
|
if (n == expectedSeqNo) {
|
2016-05-12 08:56:28 +02:00
|
|
|
inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress))
|
2016-06-23 16:36:55 +02:00
|
|
|
sequenceNumbers = sequenceNumbers.updated(ackReplyTo, n + 1)
|
2016-06-06 08:26:15 +02:00
|
|
|
val unwrapped = env.withMessage(sysEnv.message)
|
2016-05-04 13:31:08 +02:00
|
|
|
push(out, unwrapped)
|
2016-06-23 16:36:55 +02:00
|
|
|
} else if (n < expectedSeqNo) {
|
|
|
|
|
inboundContext.sendControl(ackReplyTo.address, Ack(expectedSeqNo - 1, localAddress))
|
2016-05-04 13:31:08 +02:00
|
|
|
pull(in)
|
|
|
|
|
} else {
|
2016-06-23 16:36:55 +02:00
|
|
|
inboundContext.sendControl(ackReplyTo.address, Nack(expectedSeqNo - 1, localAddress))
|
2016-05-04 13:31:08 +02:00
|
|
|
pull(in)
|
|
|
|
|
}
|
2016-06-06 08:26:15 +02:00
|
|
|
case _ ⇒
|
2016-05-04 13:31:08 +02:00
|
|
|
// messages that don't need acking
|
|
|
|
|
push(out, env)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// OutHandler
|
|
|
|
|
override def onPull(): Unit = pull(in)
|
|
|
|
|
|
|
|
|
|
setHandlers(in, out, this)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|