rename reply to control

This commit is contained in:
Patrik Nordwall 2016-05-12 08:56:28 +02:00
parent 87386e18cf
commit 8a04b6d05a
7 changed files with 130 additions and 121 deletions

View file

@ -23,7 +23,7 @@ import akka.remote.RemoteActorRef
import akka.remote.RemoteActorRefProvider import akka.remote.RemoteActorRefProvider
import akka.remote.RemoteTransport import akka.remote.RemoteTransport
import akka.remote.UniqueAddress import akka.remote.UniqueAddress
import akka.remote.artery.InboundReplyJunction.ReplySubject import akka.remote.artery.InboundControlJunction.ControlMessageSubject
import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduCodec
import akka.remote.transport.AkkaPduProtobufCodec import akka.remote.transport.AkkaPduProtobufCodec
import akka.serialization.Serialization import akka.serialization.Serialization
@ -47,7 +47,7 @@ import io.aeron.exceptions.ConductorServiceTimeoutException
import org.agrona.ErrorHandler import org.agrona.ErrorHandler
import org.agrona.IoUtil import org.agrona.IoUtil
import java.io.File import java.io.File
import akka.remote.artery.OutboundReplyJunction.OutboundReplyIngress import akka.remote.artery.OutboundControlJunction.OutboundControlIngress
/** /**
* INTERNAL API * INTERNAL API
@ -70,10 +70,10 @@ private[akka] trait InboundContext {
def localAddress: UniqueAddress def localAddress: UniqueAddress
/** /**
* An inbound stage can send reply message to the origin * An inbound stage can send control message, e.g. a reply, to the origin
* address with this method. * address with this method.
*/ */
def sendReply(to: Address, message: ControlMessage): Unit // FIXME rename to sendControl def sendControl(to: Address, message: ControlMessage): Unit
/** /**
* Lookup the outbound association for a given address. * Lookup the outbound association for a given address.
@ -110,10 +110,10 @@ private[akka] trait OutboundContext {
def completeRemoteAddress(a: UniqueAddress): Unit def completeRemoteAddress(a: UniqueAddress): Unit
/** /**
* An outbound stage can listen to reply messages * An outbound stage can listen to control messages
* via this observer subject. * via this observer subject.
*/ */
def replySubject: ReplySubject // FIXME rename to controlSubject def controlSubject: ControlMessageSubject
// FIXME we should be able to Send without a recipient ActorRef // FIXME we should be able to Send without a recipient ActorRef
def dummyRecipient: RemoteActorRef def dummyRecipient: RemoteActorRef
@ -130,7 +130,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
@volatile private[this] var _localAddress: UniqueAddress = _ @volatile private[this] var _localAddress: UniqueAddress = _
override def localAddress: UniqueAddress = _localAddress override def localAddress: UniqueAddress = _localAddress
@volatile private[this] var materializer: Materializer = _ @volatile private[this] var materializer: Materializer = _
@volatile private[this] var replySubject: ReplySubject = _ @volatile private[this] var controlSubject: ControlMessageSubject = _
@volatile private[this] var messageDispatcher: MessageDispatcher = _ @volatile private[this] var messageDispatcher: MessageDispatcher = _
@volatile private[this] var driver: MediaDriver = _ @volatile private[this] var driver: MediaDriver = _
@volatile private[this] var aeron: Aeron = _ @volatile private[this] var aeron: Aeron = _
@ -147,7 +147,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
// TODO support port 0 // TODO support port 0
private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}"
private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}"
private val systemMessageStreamId = 1 // FIXME rename to controlStreamId private val controlStreamId = 1
private val ordinaryStreamId = 3 private val ordinaryStreamId = 3
private val taskRunner = new TaskRunner(system) private val taskRunner = new TaskRunner(system)
@ -215,10 +215,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
} }
private def runInboundFlows(): Unit = { private def runInboundFlows(): Unit = {
replySubject = Source.fromGraph(new AeronSource(inboundChannel, systemMessageStreamId, aeron, taskRunner)) controlSubject = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner))
.async // FIXME measure .async // FIXME measure
.map(ByteString.apply) // TODO we should use ByteString all the way .map(ByteString.apply) // TODO we should use ByteString all the way
.viaMat(inboundSystemMessageFlow)(Keep.right) .viaMat(inboundControlFlow)(Keep.right)
.to(Sink.ignore) .to(Sink.ignore)
.run()(materializer) .run()(materializer)
@ -242,8 +242,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
} }
// InboundContext // InboundContext
override def sendReply(to: Address, message: ControlMessage) = override def sendControl(to: Address, message: ControlMessage) =
association(to).outboundReplyIngress.sendControlMessage(message) association(to).outboundControlIngress.sendControlMessage(message)
override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = {
val cached = recipient.cachedAssociation val cached = recipient.cachedAssociation
@ -262,7 +262,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
else { else {
associations.computeIfAbsent(remoteAddress, new JFunction[Address, Association] { associations.computeIfAbsent(remoteAddress, new JFunction[Address, Association] {
override def apply(remoteAddress: Address): Association = { override def apply(remoteAddress: Address): Association = {
val newAssociation = new Association(ArteryTransport.this, materializer, remoteAddress, replySubject) val newAssociation = new Association(ArteryTransport.this, materializer, remoteAddress, controlSubject)
newAssociation.associate() // This is a bit costly for this blocking method :( newAssociation.associate() // This is a bit costly for this blocking method :(
newAssociation newAssociation
} }
@ -282,14 +282,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
.to(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner)) .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner))
} }
def outboundSystemMessage(outboundContext: OutboundContext): Sink[Send, OutboundReplyIngress] = { def outboundControl(outboundContext: OutboundContext): Sink[Send, OutboundControlIngress] = {
Flow.fromGraph(killSwitch.flow[Send]) Flow.fromGraph(killSwitch.flow[Send])
.via(new OutboundHandshake(outboundContext)) .via(new OutboundHandshake(outboundContext))
.via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval)) .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval))
.viaMat(new OutboundReplyJunction(outboundContext))(Keep.right) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right)
.via(encoder) .via(encoder)
.map(_.toArray) // TODO we should use ByteString all the way .map(_.toArray) // TODO we should use ByteString all the way
.to(new AeronSink(outboundChannel(outboundContext.remoteAddress), systemMessageStreamId, aeron, taskRunner)) .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner))
} }
// TODO: Try out parallelized serialization (mapAsync) for performance // TODO: Try out parallelized serialization (mapAsync) for performance
@ -339,14 +339,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
Source.maybe[ByteString].via(killSwitch.flow)) Source.maybe[ByteString].via(killSwitch.flow))
} }
// FIXME rename to controlFlow val inboundControlFlow: Flow[ByteString, ByteString, ControlMessageSubject] = {
val inboundSystemMessageFlow: Flow[ByteString, ByteString, ReplySubject] = {
Flow.fromSinkAndSourceMat( Flow.fromSinkAndSourceMat(
decoder decoder
.via(deserializer) .via(deserializer)
.via(new InboundHandshake(this)) .via(new InboundHandshake(this))
.via(new SystemMessageAcker(this)) .via(new SystemMessageAcker(this))
.viaMat(new InboundReplyJunction)(Keep.right) .viaMat(new InboundControlJunction)(Keep.right)
.to(messageDispatcherSink), .to(messageDispatcherSink),
Source.maybe[ByteString].via(killSwitch.flow))((a, b) a) Source.maybe[ByteString].via(killSwitch.flow))((a, b) a)
} }

View file

@ -12,12 +12,12 @@ import akka.dispatch.sysmsg.SystemMessage
import akka.remote.EndpointManager.Send import akka.remote.EndpointManager.Send
import akka.remote.RemoteActorRef import akka.remote.RemoteActorRef
import akka.remote.UniqueAddress import akka.remote.UniqueAddress
import akka.remote.artery.InboundReplyJunction.ReplySubject import akka.remote.artery.InboundControlJunction.ControlMessageSubject
import akka.stream.Materializer import akka.stream.Materializer
import akka.stream.OverflowStrategy import akka.stream.OverflowStrategy
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.scaladsl.SourceQueueWithComplete import akka.stream.scaladsl.SourceQueueWithComplete
import akka.remote.artery.OutboundReplyJunction.OutboundReplyIngress import akka.remote.artery.OutboundControlJunction.OutboundControlIngress
import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Keep
/** /**
@ -30,16 +30,16 @@ private[akka] class Association(
val transport: ArteryTransport, val transport: ArteryTransport,
val materializer: Materializer, val materializer: Materializer,
override val remoteAddress: Address, override val remoteAddress: Address,
override val replySubject: ReplySubject) extends OutboundContext { override val controlSubject: ControlMessageSubject) extends OutboundContext {
@volatile private[this] var queue: SourceQueueWithComplete[Send] = _ @volatile private[this] var queue: SourceQueueWithComplete[Send] = _
@volatile private[this] var systemMessageQueue: SourceQueueWithComplete[Send] = _ @volatile private[this] var systemMessageQueue: SourceQueueWithComplete[Send] = _
@volatile private[this] var _outboundReplyIngress: OutboundReplyIngress = _ @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _
def outboundReplyIngress: OutboundReplyIngress = { def outboundControlIngress: OutboundControlIngress = {
if (_outboundReplyIngress eq null) if (_outboundControlIngress eq null)
throw new IllegalStateException("outboundReplyIngress not initialized yet") throw new IllegalStateException("outboundControlIngress not initialized yet")
_outboundReplyIngress _outboundControlIngress
} }
override def localAddress: UniqueAddress = transport.localAddress override def localAddress: UniqueAddress = transport.localAddress
@ -82,10 +82,10 @@ private[akka] class Association(
.to(transport.outbound(this)).run()(materializer) .to(transport.outbound(this)).run()(materializer)
if (systemMessageQueue eq null) { if (systemMessageQueue eq null) {
val (q, control) = Source.queue(256, OverflowStrategy.dropBuffer) val (q, control) = Source.queue(256, OverflowStrategy.dropBuffer)
.toMat(transport.outboundSystemMessage(this))(Keep.both) .toMat(transport.outboundControl(this))(Keep.both)
.run()(materializer) .run()(materializer)
systemMessageQueue = q systemMessageQueue = q
_outboundReplyIngress = control _outboundControlIngress = control
} }
} }
} }

View file

@ -32,35 +32,45 @@ trait ControlMessage
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] object InboundReplyJunction { private[akka] object InboundControlJunction {
// FIXME rename all Reply stuff to Control or ControlMessage /**
* Observer subject for inbound control messages.
private[akka] trait ReplySubject { * Interested observers can attach themselves to the
def attach(observer: ReplyObserver): Future[Done] * subject to get notification of incoming control
def detach(observer: ReplyObserver): Unit * messages.
*/
private[akka] trait ControlMessageSubject {
def attach(observer: ControlMessageObserver): Future[Done]
def detach(observer: ControlMessageObserver): Unit
def stopped: Future[Done] def stopped: Future[Done]
} }
private[akka] trait ReplyObserver { private[akka] trait ControlMessageObserver {
def reply(inboundEnvelope: InboundEnvelope): Unit
/**
* Notification of incoming control message. The message
* of the envelope is always a `ControlMessage`.
*/
def notify(inboundEnvelope: InboundEnvelope): Unit
} }
private[InboundReplyJunction] sealed trait CallbackMessage // messages for the CallbackWrapper
private[InboundReplyJunction] final case class Attach(observer: ReplyObserver, done: Promise[Done]) private[InboundControlJunction] sealed trait CallbackMessage
private[InboundControlJunction] final case class Attach(observer: ControlMessageObserver, done: Promise[Done])
extends CallbackMessage extends CallbackMessage
private[InboundReplyJunction] final case class Dettach(observer: ReplyObserver) extends CallbackMessage private[InboundControlJunction] final case class Dettach(observer: ControlMessageObserver) extends CallbackMessage
} }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] class InboundReplyJunction private[akka] class InboundControlJunction
extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], InboundReplyJunction.ReplySubject] { extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], InboundControlJunction.ControlMessageSubject] {
import InboundReplyJunction._ import InboundControlJunction._
val in: Inlet[InboundEnvelope] = Inlet("InboundReplyJunction.in") val in: Inlet[InboundEnvelope] = Inlet("InboundControlJunction.in")
val out: Outlet[InboundEnvelope] = Outlet("InboundReplyJunction.out") val out: Outlet[InboundEnvelope] = Outlet("InboundControlJunction.out")
override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out)
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
@ -68,14 +78,14 @@ private[akka] class InboundReplyJunction
// FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way
val logic = new GraphStageLogic(shape) with CallbackWrapper[CallbackMessage] with InHandler with OutHandler { val logic = new GraphStageLogic(shape) with CallbackWrapper[CallbackMessage] with InHandler with OutHandler {
private var replyObservers: Vector[ReplyObserver] = Vector.empty private var observers: Vector[ControlMessageObserver] = Vector.empty
private val callback = getAsyncCallback[CallbackMessage] { private val callback = getAsyncCallback[CallbackMessage] {
case Attach(observer, done) case Attach(observer, done)
replyObservers :+= observer observers :+= observer
done.success(Done) done.success(Done)
case Dettach(observer) case Dettach(observer)
replyObservers = replyObservers.filterNot(_ == observer) observers = observers.filterNot(_ == observer)
} }
override def preStart(): Unit = { override def preStart(): Unit = {
@ -87,8 +97,8 @@ private[akka] class InboundReplyJunction
// InHandler // InHandler
override def onPush(): Unit = { override def onPush(): Unit = {
grab(in) match { grab(in) match {
case env @ InboundEnvelope(_, _, reply: Reply, _) case env @ InboundEnvelope(_, _, _: ControlMessage, _)
replyObservers.foreach(_.reply(env)) observers.foreach(_.notify(env))
pull(in) pull(in)
case env case env
push(out, env) push(out, env)
@ -102,29 +112,29 @@ private[akka] class InboundReplyJunction
} }
// materialized value // materialized value
val replySubject: ReplySubject = new ReplySubject { val controlSubject: ControlMessageSubject = new ControlMessageSubject {
override def attach(observer: ReplyObserver): Future[Done] = { override def attach(observer: ControlMessageObserver): Future[Done] = {
val p = Promise[Done]() val p = Promise[Done]()
logic.invoke(Attach(observer, p)) logic.invoke(Attach(observer, p))
p.future p.future
} }
override def detach(observer: ReplyObserver): Unit = override def detach(observer: ControlMessageObserver): Unit =
logic.invoke(Dettach(observer)) logic.invoke(Dettach(observer))
override def stopped: Future[Done] = override def stopped: Future[Done] =
stoppedPromise.future stoppedPromise.future
} }
(logic, replySubject) (logic, controlSubject)
} }
} }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] object OutboundReplyJunction { private[akka] object OutboundControlJunction {
trait OutboundReplyIngress { trait OutboundControlIngress {
def sendControlMessage(message: ControlMessage): Unit def sendControlMessage(message: ControlMessage): Unit
} }
} }
@ -132,17 +142,17 @@ private[akka] object OutboundReplyJunction {
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[akka] class OutboundReplyJunction(outboundContext: OutboundContext) private[akka] class OutboundControlJunction(outboundContext: OutboundContext)
extends GraphStageWithMaterializedValue[FlowShape[Send, Send], OutboundReplyJunction.OutboundReplyIngress] { extends GraphStageWithMaterializedValue[FlowShape[Send, Send], OutboundControlJunction.OutboundControlIngress] {
import OutboundReplyJunction._ import OutboundControlJunction._
val in: Inlet[Send] = Inlet("OutboundReplyJunction.in") val in: Inlet[Send] = Inlet("OutboundControlJunction.in")
val out: Outlet[Send] = Outlet("OutboundReplyJunction.out") val out: Outlet[Send] = Outlet("OutboundControlJunction.out")
override val shape: FlowShape[Send, Send] = FlowShape(in, out) override val shape: FlowShape[Send, Send] = FlowShape(in, out)
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
// FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way
val logic = new GraphStageLogic(shape) with CallbackWrapper[ControlMessage] with InHandler with OutHandler { val logic = new GraphStageLogic(shape) with CallbackWrapper[ControlMessage] with InHandler with OutHandler {
import OutboundReplyJunction._ import OutboundControlJunction._
private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage) private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage)
private val buffer = new ArrayDeque[Send] private val buffer = new ArrayDeque[Send]
@ -181,12 +191,12 @@ private[akka] class OutboundReplyJunction(outboundContext: OutboundContext)
} }
// materialized value // materialized value
val outboundReplyIngress = new OutboundReplyIngress { val outboundControlIngress = new OutboundControlIngress {
override def sendControlMessage(message: ControlMessage): Unit = override def sendControlMessage(message: ControlMessage): Unit =
logic.invoke(message) logic.invoke(message)
} }
(logic, outboundReplyIngress) (logic, outboundControlIngress)
} }
} }

View file

@ -8,7 +8,7 @@ import scala.concurrent.duration._
import akka.Done import akka.Done
import akka.remote.EndpointManager.Send import akka.remote.EndpointManager.Send
import akka.remote.UniqueAddress import akka.remote.UniqueAddress
import akka.remote.artery.InboundReplyJunction.ReplyObserver import akka.remote.artery.InboundControlJunction.ControlMessageObserver
import akka.stream.Attributes import akka.stream.Attributes
import akka.stream.FlowShape import akka.stream.FlowShape
import akka.stream.Inlet import akka.stream.Inlet
@ -29,7 +29,7 @@ private[akka] object OutboundHandshake {
private sealed trait HandshakeState private sealed trait HandshakeState
private case object Start extends HandshakeState private case object Start extends HandshakeState
private case object ReplyObserverAttached extends HandshakeState private case object ControlMessageObserverAttached extends HandshakeState
private case object ReqInProgress extends HandshakeState private case object ReqInProgress extends HandshakeState
private case object Completed extends HandshakeState private case object Completed extends HandshakeState
@ -46,7 +46,7 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends
override val shape: FlowShape[Send, Send] = FlowShape(in, out) override val shape: FlowShape[Send, Send] = FlowShape(in, out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new TimerGraphStageLogic(shape) with InHandler with OutHandler with ReplyObserver { new TimerGraphStageLogic(shape) with InHandler with OutHandler with ControlMessageObserver {
import OutboundHandshake._ import OutboundHandshake._
private val timeout: FiniteDuration = 10.seconds // FIXME config private val timeout: FiniteDuration = 10.seconds // FIXME config
@ -59,13 +59,13 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends
handshakeState = Completed handshakeState = Completed
} else { } else {
implicit val ec = materializer.executionContext implicit val ec = materializer.executionContext
outboundContext.replySubject.attach(this).foreach { outboundContext.controlSubject.attach(this).foreach {
getAsyncCallback[Done] { _ getAsyncCallback[Done] { _
if (handshakeState != Completed) { if (handshakeState != Completed) {
if (isAvailable(out)) if (isAvailable(out))
pushHandshakeReq() pushHandshakeReq()
else else
handshakeState = ReplyObserverAttached handshakeState = ControlMessageObserverAttached
} }
}.invoke }.invoke
} }
@ -85,7 +85,7 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends
} }
override def postStop(): Unit = { override def postStop(): Unit = {
outboundContext.replySubject.detach(this) outboundContext.controlSubject.detach(this)
} }
// InHandler // InHandler
@ -99,9 +99,9 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends
override def onPull(): Unit = { override def onPull(): Unit = {
handshakeState match { handshakeState match {
case Completed pull(in) case Completed pull(in)
case ReplyObserverAttached case ControlMessageObserverAttached
pushHandshakeReq() pushHandshakeReq()
case Start // will push HandshakeReq when ReplyObserver is attached case Start // will push HandshakeReq when ControlMessageObserver is attached
case ReqInProgress // will pull when handshake reply is received case ReqInProgress // will pull when handshake reply is received
} }
} }
@ -115,7 +115,7 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends
private def handshakeCompleted(): Unit = { private def handshakeCompleted(): Unit = {
handshakeState = Completed handshakeState = Completed
cancelTimer(HandshakeTimeout) cancelTimer(HandshakeTimeout)
outboundContext.replySubject.detach(this) outboundContext.controlSubject.detach(this)
} }
override protected def onTimer(timerKey: Any): Unit = override protected def onTimer(timerKey: Any): Unit =
@ -125,8 +125,8 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends
s"Handshake with [$remoteAddress] did not complete within ${timeout.toMillis} ms")) s"Handshake with [$remoteAddress] did not complete within ${timeout.toMillis} ms"))
} }
// ReplyObserver, external call // ControlMessageObserver, external call
override def reply(inboundEnvelope: InboundEnvelope): Unit = { override def notify(inboundEnvelope: InboundEnvelope): Unit = {
inboundEnvelope.message match { inboundEnvelope.message match {
case rsp: HandshakeRsp case rsp: HandshakeRsp
if (rsp.from.address == remoteAddress) { if (rsp.from.address == remoteAddress) {
@ -165,7 +165,7 @@ private[akka] class InboundHandshake(inboundContext: InboundContext) extends Gra
grab(in) match { grab(in) match {
case InboundEnvelope(_, _, HandshakeReq(from), _) case InboundEnvelope(_, _, HandshakeReq(from), _)
inboundContext.association(from.address).completeRemoteAddress(from) inboundContext.association(from.address).completeRemoteAddress(from)
inboundContext.sendReply(from.address, HandshakeRsp(inboundContext.localAddress)) inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress))
pull(in) pull(in)
case other case other
push(out, other) push(out, other)

View file

@ -14,7 +14,7 @@ import scala.util.Try
import akka.Done import akka.Done
import akka.remote.EndpointManager.Send import akka.remote.EndpointManager.Send
import akka.remote.UniqueAddress import akka.remote.UniqueAddress
import akka.remote.artery.InboundReplyJunction.ReplyObserver import akka.remote.artery.InboundControlJunction.ControlMessageObserver
import akka.stream.Attributes import akka.stream.Attributes
import akka.stream.FlowShape import akka.stream.FlowShape
import akka.stream.Inlet import akka.stream.Inlet
@ -53,7 +53,7 @@ private[akka] class SystemMessageDelivery(
override val shape: FlowShape[Send, Send] = FlowShape(in, out) override val shape: FlowShape[Send, Send] = FlowShape(in, out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new TimerGraphStageLogic(shape) with InHandler with OutHandler with ReplyObserver { new TimerGraphStageLogic(shape) with InHandler with OutHandler with ControlMessageObserver {
private var replyObserverAttached = false private var replyObserverAttached = false
private var seqNo = 0L // sequence number for the first message will be 1 private var seqNo = 0L // sequence number for the first message will be 1
@ -69,7 +69,7 @@ private[akka] class SystemMessageDelivery(
this.schedulePeriodically(ResendTick, resendInterval) this.schedulePeriodically(ResendTick, resendInterval)
implicit val ec = materializer.executionContext implicit val ec = materializer.executionContext
outboundContext.replySubject.attach(this).foreach { outboundContext.controlSubject.attach(this).foreach {
getAsyncCallback[Done] { _ getAsyncCallback[Done] { _
replyObserverAttached = true replyObserverAttached = true
if (isAvailable(out)) if (isAvailable(out))
@ -77,7 +77,7 @@ private[akka] class SystemMessageDelivery(
}.invoke }.invoke
} }
outboundContext.replySubject.stopped.onComplete { outboundContext.controlSubject.stopped.onComplete {
getAsyncCallback[Try[Done]] { getAsyncCallback[Try[Done]] {
// FIXME quarantine // FIXME quarantine
case Success(_) completeStage() case Success(_) completeStage()
@ -87,7 +87,7 @@ private[akka] class SystemMessageDelivery(
} }
override def postStop(): Unit = { override def postStop(): Unit = {
outboundContext.replySubject.detach(this) outboundContext.controlSubject.detach(this)
} }
override def onUpstreamFinish(): Unit = { override def onUpstreamFinish(): Unit = {
@ -106,8 +106,8 @@ private[akka] class SystemMessageDelivery(
} }
} }
// ReplyObserver, external call // ControlMessageObserver, external call
override def reply(inboundEnvelope: InboundEnvelope): Unit = { override def notify(inboundEnvelope: InboundEnvelope): Unit = {
inboundEnvelope.message match { inboundEnvelope.message match {
case ack: Ack if (ack.from.address == remoteAddress) ackCallback.invoke(ack) case ack: Ack if (ack.from.address == remoteAddress) ackCallback.invoke(ack)
case nack: Nack if (nack.from.address == remoteAddress) nackCallback.invoke(nack) case nack: Nack if (nack.from.address == remoteAddress) nackCallback.invoke(nack)
@ -201,15 +201,15 @@ private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends G
grab(in) match { grab(in) match {
case env @ InboundEnvelope(_, _, sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo), _) case env @ InboundEnvelope(_, _, sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo), _)
if (n == seqNo) { if (n == seqNo) {
inboundContext.sendReply(ackReplyTo.address, Ack(n, localAddress)) inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress))
seqNo += 1 seqNo += 1
val unwrapped = env.copy(message = sysEnv.message) val unwrapped = env.copy(message = sysEnv.message)
push(out, unwrapped) push(out, unwrapped)
} else if (n < seqNo) { } else if (n < seqNo) {
inboundContext.sendReply(ackReplyTo.address, Ack(n, localAddress)) inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress))
pull(in) pull(in)
} else { } else {
inboundContext.sendReply(ackReplyTo.address, Nack(seqNo - 1, localAddress)) inboundContext.sendControl(ackReplyTo.address, Nack(seqNo - 1, localAddress))
pull(in) pull(in)
} }
case env case env

View file

@ -58,17 +58,17 @@ object SystemMessageDeliverySpec {
class ManualReplyInboundContext( class ManualReplyInboundContext(
replyProbe: ActorRef, replyProbe: ActorRef,
localAddress: UniqueAddress, localAddress: UniqueAddress,
replySubject: TestReplySubject) extends TestInboundContext(localAddress, replySubject) { controlSubject: TestControlMessageSubject) extends TestInboundContext(localAddress, controlSubject) {
private var lastReply: Option[(Address, ControlMessage)] = None private var lastReply: Option[(Address, ControlMessage)] = None
override def sendReply(to: Address, message: ControlMessage) = { override def sendControl(to: Address, message: ControlMessage) = {
lastReply = Some((to, message)) lastReply = Some((to, message))
replyProbe ! message replyProbe ! message
} }
def deliverLastReply(): Unit = { def deliverLastReply(): Unit = {
lastReply.foreach { case (to, message) super.sendReply(to, message) } lastReply.foreach { case (to, message) super.sendControl(to, message) }
lastReply = None lastReply = None
} }
} }
@ -147,9 +147,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo
"be resent when some in the middle are lost" in { "be resent when some in the middle are lost" in {
val replyProbe = TestProbe() val replyProbe = TestProbe()
val replySubject = new TestReplySubject val controlSubject = new TestControlMessageSubject
val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject)
val inboundContextA = new TestInboundContext(addressB, replySubject) val inboundContextA = new TestInboundContext(addressB, controlSubject)
val outboundContextA = inboundContextA.association(addressB.address) val outboundContextA = inboundContextA.association(addressB.address)
val sink = send(sendCount = 5, resendInterval = 60.seconds, outboundContextA) val sink = send(sendCount = 5, resendInterval = 60.seconds, outboundContextA)
@ -181,9 +181,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo
"be resent when first is lost" in { "be resent when first is lost" in {
val replyProbe = TestProbe() val replyProbe = TestProbe()
val replySubject = new TestReplySubject val controlSubject = new TestControlMessageSubject
val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject)
val inboundContextA = new TestInboundContext(addressB, replySubject) val inboundContextA = new TestInboundContext(addressB, controlSubject)
val outboundContextA = inboundContextA.association(addressB.address) val outboundContextA = inboundContextA.association(addressB.address)
val sink = send(sendCount = 3, resendInterval = 60.seconds, outboundContextA) val sink = send(sendCount = 3, resendInterval = 60.seconds, outboundContextA)
@ -210,9 +210,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo
"be resent when last is lost" in { "be resent when last is lost" in {
val replyProbe = TestProbe() val replyProbe = TestProbe()
val replySubject = new TestReplySubject val controlSubject = new TestControlMessageSubject
val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject)
val inboundContextA = new TestInboundContext(addressB, replySubject) val inboundContextA = new TestInboundContext(addressB, controlSubject)
val outboundContextA = inboundContextA.association(addressB.address) val outboundContextA = inboundContextA.association(addressB.address)
val sink = send(sendCount = 3, resendInterval = 1.seconds, outboundContextA) val sink = send(sendCount = 3, resendInterval = 1.seconds, outboundContextA)
@ -239,9 +239,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo
"deliver all during stress and random dropping" in { "deliver all during stress and random dropping" in {
val N = 10000 val N = 10000
val dropRate = 0.1 val dropRate = 0.1
val replySubject = new TestReplySubject val controlSubject = new TestControlMessageSubject
val inboundContextB = new TestInboundContext(addressB, replySubject, replyDropRate = dropRate) val inboundContextB = new TestInboundContext(addressB, controlSubject, replyDropRate = dropRate)
val inboundContextA = new TestInboundContext(addressB, replySubject) val inboundContextA = new TestInboundContext(addressB, controlSubject)
val outboundContextA = inboundContextA.association(addressB.address) val outboundContextA = inboundContextA.association(addressB.address)
val output = val output =
@ -257,9 +257,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo
"deliver all during throttling and random dropping" in { "deliver all during throttling and random dropping" in {
val N = 500 val N = 500
val dropRate = 0.1 val dropRate = 0.1
val replySubject = new TestReplySubject val controlSubject = new TestControlMessageSubject
val inboundContextB = new TestInboundContext(addressB, replySubject, replyDropRate = dropRate) val inboundContextB = new TestInboundContext(addressB, controlSubject, replyDropRate = dropRate)
val inboundContextA = new TestInboundContext(addressB, replySubject) val inboundContextA = new TestInboundContext(addressB, controlSubject)
val outboundContextA = inboundContextA.association(addressB.address) val outboundContextA = inboundContextA.association(addressB.address)
val output = val output =

View file

@ -6,31 +6,31 @@ package akka.remote.artery
import akka.remote.UniqueAddress import akka.remote.UniqueAddress
import akka.actor.Address import akka.actor.Address
import scala.concurrent.Future import scala.concurrent.Future
import akka.remote.artery.InboundReplyJunction.ReplySubject import akka.remote.artery.InboundControlJunction.ControlMessageSubject
import akka.remote.RemoteActorRef import akka.remote.RemoteActorRef
import scala.concurrent.Promise import scala.concurrent.Promise
import akka.Done import akka.Done
import akka.remote.artery.InboundReplyJunction.ReplyObserver import akka.remote.artery.InboundControlJunction.ControlMessageObserver
import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.CopyOnWriteArrayList
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
private[akka] class TestInboundContext( private[akka] class TestInboundContext(
override val localAddress: UniqueAddress, override val localAddress: UniqueAddress,
val replySubject: TestReplySubject = new TestReplySubject, val controlSubject: TestControlMessageSubject = new TestControlMessageSubject,
replyDropRate: Double = 0.0) extends InboundContext { replyDropRate: Double = 0.0) extends InboundContext {
private val associations = new ConcurrentHashMap[Address, OutboundContext] private val associations = new ConcurrentHashMap[Address, OutboundContext]
def sendReply(to: Address, message: ControlMessage) = { def sendControl(to: Address, message: ControlMessage) = {
if (ThreadLocalRandom.current().nextDouble() >= replyDropRate) if (ThreadLocalRandom.current().nextDouble() >= replyDropRate)
replySubject.sendReply(InboundEnvelope(null, to, message, None)) controlSubject.sendControl(InboundEnvelope(null, to, message, None))
} }
def association(remoteAddress: Address): OutboundContext = def association(remoteAddress: Address): OutboundContext =
associations.get(remoteAddress) match { associations.get(remoteAddress) match {
case null case null
val a = new TestOutboundContext(localAddress, remoteAddress, replySubject) val a = new TestOutboundContext(localAddress, remoteAddress, controlSubject)
associations.putIfAbsent(remoteAddress, a) match { associations.putIfAbsent(remoteAddress, a) match {
case null a case null a
case existing existing case existing existing
@ -39,13 +39,13 @@ private[akka] class TestInboundContext(
} }
protected def createAssociation(remoteAddress: Address): OutboundContext = protected def createAssociation(remoteAddress: Address): OutboundContext =
new TestOutboundContext(localAddress, remoteAddress, replySubject) new TestOutboundContext(localAddress, remoteAddress, controlSubject)
} }
private[akka] class TestOutboundContext( private[akka] class TestOutboundContext(
override val localAddress: UniqueAddress, override val localAddress: UniqueAddress,
override val remoteAddress: Address, override val remoteAddress: Address,
override val replySubject: TestReplySubject) extends OutboundContext { override val controlSubject: TestControlMessageSubject) extends OutboundContext {
private val _uniqueRemoteAddress = Promise[UniqueAddress]() private val _uniqueRemoteAddress = Promise[UniqueAddress]()
def uniqueRemoteAddress: Future[UniqueAddress] = _uniqueRemoteAddress.future def uniqueRemoteAddress: Future[UniqueAddress] = _uniqueRemoteAddress.future
@ -56,24 +56,24 @@ private[akka] class TestOutboundContext(
} }
private[akka] class TestReplySubject extends ReplySubject { private[akka] class TestControlMessageSubject extends ControlMessageSubject {
private var replyObservers = new CopyOnWriteArrayList[ReplyObserver] private var observers = new CopyOnWriteArrayList[ControlMessageObserver]
override def attach(observer: ReplyObserver): Future[Done] = { override def attach(observer: ControlMessageObserver): Future[Done] = {
replyObservers.add(observer) observers.add(observer)
Future.successful(Done) Future.successful(Done)
} }
override def detach(observer: ReplyObserver): Unit = { override def detach(observer: ControlMessageObserver): Unit = {
replyObservers.remove(observer) observers.remove(observer)
} }
override def stopped: Future[Done] = Promise[Done]().future override def stopped: Future[Done] = Promise[Done]().future
def sendReply(env: InboundEnvelope): Unit = { def sendControl(env: InboundEnvelope): Unit = {
val iter = replyObservers.iterator() val iter = observers.iterator()
while (iter.hasNext()) while (iter.hasNext())
iter.next().reply(env) iter.next().notify(env)
} }
} }