rename reply to control
This commit is contained in:
parent
87386e18cf
commit
8a04b6d05a
7 changed files with 130 additions and 121 deletions
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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 ⇒
|
||||||
|
|
|
||||||
|
|
@ -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 =
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue