2016-05-09 07:31:41 +02:00
|
|
|
/**
|
2018-01-04 17:26:29 +00:00
|
|
|
* Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com>
|
2016-05-09 07:31:41 +02:00
|
|
|
*/
|
|
|
|
|
package akka.remote.artery
|
|
|
|
|
|
2016-05-13 08:06:13 +02:00
|
|
|
import java.util.ArrayDeque
|
2016-10-28 16:05:56 +02:00
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
import scala.concurrent.Future
|
|
|
|
|
import scala.concurrent.Promise
|
2017-11-20 15:15:17 +01:00
|
|
|
import scala.util.Try
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
import akka.Done
|
|
|
|
|
import akka.stream.Attributes
|
|
|
|
|
import akka.stream.FlowShape
|
|
|
|
|
import akka.stream.Inlet
|
|
|
|
|
import akka.stream.Outlet
|
2016-10-28 16:05:56 +02:00
|
|
|
import akka.stream.stage._
|
2016-05-13 15:34:37 +02:00
|
|
|
import akka.remote.UniqueAddress
|
2016-06-05 15:40:06 +02:00
|
|
|
import akka.util.OptionVal
|
2016-09-19 11:17:41 +02:00
|
|
|
import akka.event.Logging
|
2016-05-09 07:31:41 +02:00
|
|
|
|
2016-09-08 17:58:25 +02:00
|
|
|
/** INTERNAL API: marker trait for protobuf-serializable artery messages */
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] trait ArteryMessage extends Serializable
|
2016-09-08 17:58:25 +02:00
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
/**
|
2016-05-13 15:34:37 +02:00
|
|
|
* INTERNAL API: Marker trait for reply messages
|
2016-05-09 07:31:41 +02:00
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] trait Reply extends ControlMessage
|
2016-05-09 07:31:41 +02:00
|
|
|
|
|
|
|
|
/**
|
2016-05-13 15:34:37 +02:00
|
|
|
* INTERNAL API
|
2016-05-09 07:31:41 +02:00
|
|
|
* Marker trait for control messages that can be sent via the system message sub-channel
|
|
|
|
|
* but don't need full reliable delivery. E.g. `HandshakeReq` and `Reply`.
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] trait ControlMessage extends ArteryMessage
|
2016-05-13 15:34:37 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] final case class Quarantined(from: UniqueAddress, to: UniqueAddress) extends ControlMessage
|
2016-05-09 07:31:41 +02:00
|
|
|
|
2016-06-23 18:11:56 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] case class ActorSystemTerminating(from: UniqueAddress) extends ControlMessage
|
2016-06-23 18:11:56 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] case class ActorSystemTerminatingAck(from: UniqueAddress) extends ArteryMessage
|
2016-06-23 18:11:56 +02:00
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] object InboundControlJunction {
|
2016-05-12 08:56:28 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Observer subject for inbound control messages.
|
|
|
|
|
* Interested observers can attach themselves to the
|
|
|
|
|
* subject to get notification of incoming control
|
|
|
|
|
* messages.
|
|
|
|
|
*/
|
2016-09-07 16:07:29 +02:00
|
|
|
private[remote] trait ControlMessageSubject {
|
2016-05-12 08:56:28 +02:00
|
|
|
def attach(observer: ControlMessageObserver): Future[Done]
|
|
|
|
|
def detach(observer: ControlMessageObserver): Unit
|
2016-05-09 07:31:41 +02:00
|
|
|
}
|
|
|
|
|
|
2016-09-07 16:07:29 +02:00
|
|
|
private[remote] trait ControlMessageObserver {
|
2016-05-12 08:56:28 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Notification of incoming control message. The message
|
|
|
|
|
* of the envelope is always a `ControlMessage`.
|
|
|
|
|
*/
|
|
|
|
|
def notify(inboundEnvelope: InboundEnvelope): Unit
|
2017-11-20 15:15:17 +01:00
|
|
|
|
|
|
|
|
def controlSubjectCompleted(signal: Try[Done]): Unit
|
2016-05-09 07:31:41 +02:00
|
|
|
}
|
2016-05-11 15:55:06 +02:00
|
|
|
|
2017-06-17 22:51:34 +03:00
|
|
|
// messages for the stream callback
|
2016-05-12 08:56:28 +02:00
|
|
|
private[InboundControlJunction] sealed trait CallbackMessage
|
|
|
|
|
private[InboundControlJunction] final case class Attach(observer: ControlMessageObserver, done: Promise[Done])
|
2016-05-11 15:55:06 +02:00
|
|
|
extends CallbackMessage
|
2016-05-12 08:56:28 +02:00
|
|
|
private[InboundControlJunction] final case class Dettach(observer: ControlMessageObserver) extends CallbackMessage
|
2016-05-09 07:31:41 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] class InboundControlJunction
|
2016-05-12 08:56:28 +02:00
|
|
|
extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], InboundControlJunction.ControlMessageSubject] {
|
|
|
|
|
import InboundControlJunction._
|
2016-05-09 07:31:41 +02:00
|
|
|
|
2016-05-12 08:56:28 +02:00
|
|
|
val in: Inlet[InboundEnvelope] = Inlet("InboundControlJunction.in")
|
|
|
|
|
val out: Outlet[InboundEnvelope] = Outlet("InboundControlJunction.out")
|
2016-05-09 07:31:41 +02:00
|
|
|
override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out)
|
|
|
|
|
|
|
|
|
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
2017-06-17 22:51:34 +03:00
|
|
|
val logic = new GraphStageLogic(shape) with InHandler with OutHandler with ControlMessageSubject {
|
2016-05-09 07:31:41 +02:00
|
|
|
|
2016-05-12 08:56:28 +02:00
|
|
|
private var observers: Vector[ControlMessageObserver] = Vector.empty
|
2016-05-11 15:55:06 +02:00
|
|
|
|
|
|
|
|
private val callback = getAsyncCallback[CallbackMessage] {
|
|
|
|
|
case Attach(observer, done) ⇒
|
2016-05-12 08:56:28 +02:00
|
|
|
observers :+= observer
|
2016-05-11 15:55:06 +02:00
|
|
|
done.success(Done)
|
|
|
|
|
case Dettach(observer) ⇒
|
2016-05-12 08:56:28 +02:00
|
|
|
observers = observers.filterNot(_ == observer)
|
2016-05-11 15:55:06 +02:00
|
|
|
}
|
|
|
|
|
|
2017-11-20 15:15:17 +01:00
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
observers.foreach(_.controlSubjectCompleted(Try(Done)))
|
|
|
|
|
observers = Vector.empty
|
|
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
|
|
|
|
|
// InHandler
|
|
|
|
|
override def onPush(): Unit = {
|
|
|
|
|
grab(in) match {
|
2016-06-06 08:26:15 +02:00
|
|
|
case env: InboundEnvelope if env.message.isInstanceOf[ControlMessage] ⇒
|
2016-05-12 08:56:28 +02:00
|
|
|
observers.foreach(_.notify(env))
|
2016-05-09 07:31:41 +02:00
|
|
|
pull(in)
|
|
|
|
|
case env ⇒
|
|
|
|
|
push(out, env)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// OutHandler
|
|
|
|
|
override def onPull(): Unit = pull(in)
|
|
|
|
|
|
2016-05-11 15:55:06 +02:00
|
|
|
setHandlers(in, out, this)
|
|
|
|
|
|
2017-06-17 22:51:34 +03:00
|
|
|
// ControlMessageSubject impl
|
2016-05-12 08:56:28 +02:00
|
|
|
override def attach(observer: ControlMessageObserver): Future[Done] = {
|
2016-05-09 07:31:41 +02:00
|
|
|
val p = Promise[Done]()
|
2017-06-17 22:51:34 +03:00
|
|
|
callback.invoke(Attach(observer, p))
|
2016-05-09 07:31:41 +02:00
|
|
|
p.future
|
|
|
|
|
}
|
|
|
|
|
|
2016-05-12 08:56:28 +02:00
|
|
|
override def detach(observer: ControlMessageObserver): Unit =
|
2017-06-17 22:51:34 +03:00
|
|
|
callback.invoke(Dettach(observer))
|
2016-05-11 15:55:06 +02:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2017-06-17 22:51:34 +03:00
|
|
|
(logic, logic)
|
2016-05-11 15:55:06 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] object OutboundControlJunction {
|
|
|
|
|
private[remote] trait OutboundControlIngress {
|
2016-05-11 15:55:06 +02:00
|
|
|
def sendControlMessage(message: ControlMessage): Unit
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2016-09-29 10:50:37 +02:00
|
|
|
private[remote] class OutboundControlJunction(
|
2016-06-29 17:09:33 +02:00
|
|
|
outboundContext: OutboundContext, outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope])
|
|
|
|
|
extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, OutboundEnvelope], OutboundControlJunction.OutboundControlIngress] {
|
2016-05-12 08:56:28 +02:00
|
|
|
import OutboundControlJunction._
|
2016-06-29 17:09:33 +02:00
|
|
|
val in: Inlet[OutboundEnvelope] = Inlet("OutboundControlJunction.in")
|
|
|
|
|
val out: Outlet[OutboundEnvelope] = Outlet("OutboundControlJunction.out")
|
|
|
|
|
override val shape: FlowShape[OutboundEnvelope, OutboundEnvelope] = FlowShape(in, out)
|
2016-05-11 15:55:06 +02:00
|
|
|
|
|
|
|
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
2017-06-17 22:51:34 +03:00
|
|
|
|
|
|
|
|
val logic = new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging with OutboundControlIngress {
|
2016-05-12 08:56:28 +02:00
|
|
|
import OutboundControlJunction._
|
2016-05-11 15:55:06 +02:00
|
|
|
|
2017-06-17 22:51:34 +03:00
|
|
|
val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage)
|
2016-09-07 10:41:36 +02:00
|
|
|
private val maxControlMessageBufferSize: Int = outboundContext.settings.Advanced.OutboundControlQueueSize
|
2016-06-29 17:09:33 +02:00
|
|
|
private val buffer = new ArrayDeque[OutboundEnvelope]
|
2016-05-11 15:55:06 +02:00
|
|
|
|
|
|
|
|
// InHandler
|
|
|
|
|
override def onPush(): Unit = {
|
|
|
|
|
if (buffer.isEmpty && isAvailable(out))
|
|
|
|
|
push(out, grab(in))
|
|
|
|
|
else
|
|
|
|
|
buffer.offer(grab(in))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// OutHandler
|
|
|
|
|
override def onPull(): Unit = {
|
|
|
|
|
if (buffer.isEmpty && !hasBeenPulled(in))
|
|
|
|
|
pull(in)
|
|
|
|
|
else if (!buffer.isEmpty)
|
|
|
|
|
push(out, buffer.poll())
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def internalSendControlMessage(message: ControlMessage): Unit = {
|
|
|
|
|
if (buffer.isEmpty && isAvailable(out))
|
|
|
|
|
push(out, wrap(message))
|
2016-05-12 13:19:57 +02:00
|
|
|
else if (buffer.size < maxControlMessageBufferSize)
|
2016-05-11 15:55:06 +02:00
|
|
|
buffer.offer(wrap(message))
|
2016-05-12 13:19:57 +02:00
|
|
|
else {
|
|
|
|
|
// it's alright to drop control messages
|
2016-09-19 11:17:41 +02:00
|
|
|
log.debug("Dropping control message [{}] due to full buffer.", Logging.messageClassName(message))
|
2016-05-12 13:19:57 +02:00
|
|
|
}
|
2016-05-11 15:55:06 +02:00
|
|
|
}
|
|
|
|
|
|
2016-06-29 17:09:33 +02:00
|
|
|
private def wrap(message: ControlMessage): OutboundEnvelope =
|
|
|
|
|
outboundEnvelopePool.acquire().init(
|
|
|
|
|
recipient = OptionVal.None, message = message, sender = OptionVal.None)
|
2016-05-09 07:31:41 +02:00
|
|
|
|
2016-05-11 15:55:06 +02:00
|
|
|
override def sendControlMessage(message: ControlMessage): Unit =
|
2017-06-17 22:51:34 +03:00
|
|
|
sendControlMessageCallback.invoke(message)
|
|
|
|
|
|
|
|
|
|
setHandlers(in, out, this)
|
2016-05-11 15:55:06 +02:00
|
|
|
}
|
|
|
|
|
|
2017-06-17 22:51:34 +03:00
|
|
|
(logic, logic)
|
2016-05-09 07:31:41 +02:00
|
|
|
}
|
2016-05-11 15:55:06 +02:00
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
}
|