[WIP] Large message stream for Artery (#20545)
* First stab at separate large message channel for Artery * Full actor paths, no implicit "/user/" part * Various small fixes after review * Fixes to make it work after rebasing * Use a separate EnvelopeBufferPool for the large message stream * Docs for actorSelection not sending through large message stream
This commit is contained in:
parent
a4b996546e
commit
cd71643a91
6 changed files with 266 additions and 14 deletions
|
|
@ -12,8 +12,7 @@ import scala.concurrent.Promise
|
|||
import scala.concurrent.duration._
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.util.Success
|
||||
|
||||
import akka.Done
|
||||
import akka.{ Done, NotUsed }
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ActorSelectionMessage
|
||||
import akka.actor.Address
|
||||
|
|
@ -21,8 +20,7 @@ import akka.actor.RootActorPath
|
|||
import akka.dispatch.sysmsg.SystemMessage
|
||||
import akka.event.Logging
|
||||
import akka.remote.EndpointManager.Send
|
||||
import akka.remote.RemoteActorRef
|
||||
import akka.remote.UniqueAddress
|
||||
import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef, UniqueAddress }
|
||||
import akka.remote.artery.InboundControlJunction.ControlMessageSubject
|
||||
import akka.remote.artery.OutboundControlJunction.OutboundControlIngress
|
||||
import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException
|
||||
|
|
@ -33,7 +31,7 @@ import akka.stream.OverflowStrategy
|
|||
import akka.stream.scaladsl.Keep
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.SourceQueueWithComplete
|
||||
import akka.util.Unsafe
|
||||
import akka.util.{ Unsafe, WildcardTree }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -45,7 +43,8 @@ private[akka] class Association(
|
|||
val transport: ArteryTransport,
|
||||
val materializer: Materializer,
|
||||
override val remoteAddress: Address,
|
||||
override val controlSubject: ControlMessageSubject)
|
||||
override val controlSubject: ControlMessageSubject,
|
||||
largeMessageDestinations: WildcardTree[NotUsed])
|
||||
extends AbstractAssociation with OutboundContext {
|
||||
|
||||
private val log = Logging(transport.system, getClass.getName)
|
||||
|
|
@ -54,8 +53,10 @@ private[akka] class Association(
|
|||
private val restartTimeout: FiniteDuration = 5.seconds // FIXME config
|
||||
private val maxRestarts = 5 // FIXME config
|
||||
private val restartCounter = new RestartCounter(maxRestarts, restartTimeout)
|
||||
private val largeMessageChannelEnabled = largeMessageDestinations.children.nonEmpty
|
||||
|
||||
@volatile private[this] var queue: SourceQueueWithComplete[Send] = _
|
||||
@volatile private[this] var largeQueue: SourceQueueWithComplete[Send] = _
|
||||
@volatile private[this] var controlQueue: SourceQueueWithComplete[Send] = _
|
||||
@volatile private[this] var _outboundControlIngress: OutboundControlIngress = _
|
||||
@volatile private[this] var materializing = new CountDownLatch(1)
|
||||
|
|
@ -136,12 +137,32 @@ private[akka] class Association(
|
|||
quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]")
|
||||
}
|
||||
case _ ⇒
|
||||
queue.offer(Send(message, senderOption, recipient, None))
|
||||
val send = Send(message, senderOption, recipient, None)
|
||||
if (largeMessageChannelEnabled && isLargeMessageDestination(recipient))
|
||||
largeQueue.offer(send)
|
||||
else
|
||||
queue.offer(send)
|
||||
}
|
||||
} else if (log.isDebugEnabled)
|
||||
log.debug("Dropping message to quarantined system {}", remoteAddress)
|
||||
}
|
||||
|
||||
private def isLargeMessageDestination(recipient: ActorRef): Boolean = {
|
||||
recipient match {
|
||||
case r: RemoteActorRef if r.cachedLargeMessageDestinationFlag ne null ⇒ r.cachedLargeMessageDestinationFlag == LargeDestination
|
||||
case r: RemoteActorRef ⇒
|
||||
if (largeMessageDestinations.find(r.path.elements.iterator).data.isEmpty) {
|
||||
r.cachedLargeMessageDestinationFlag = RegularDestination
|
||||
false
|
||||
} else {
|
||||
log.debug("Using large message stream for {}", r.path)
|
||||
r.cachedLargeMessageDestinationFlag = LargeDestination
|
||||
true
|
||||
}
|
||||
case _ ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
// FIXME we should be able to Send without a recipient ActorRef
|
||||
override val dummyRecipient: RemoteActorRef =
|
||||
transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef]
|
||||
|
|
@ -195,6 +216,10 @@ private[akka] class Association(
|
|||
// so that outboundControlIngress is ready when stages for all streams start
|
||||
runOutboundControlStream()
|
||||
runOutboundOrdinaryMessagesStream()
|
||||
|
||||
if (largeMessageChannelEnabled) {
|
||||
runOutboundLargeMessagesStream()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -225,6 +250,14 @@ private[akka] class Association(
|
|||
attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream())
|
||||
}
|
||||
|
||||
private def runOutboundLargeMessagesStream(): Unit = {
|
||||
val (q, completed) = Source.queue(256, OverflowStrategy.dropBuffer)
|
||||
.toMat(transport.outboundLarge(this))(Keep.both)
|
||||
.run()(materializer)
|
||||
largeQueue = q
|
||||
attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream())
|
||||
}
|
||||
|
||||
private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: Throwable ⇒ Unit): Unit = {
|
||||
implicit val ec = materializer.executionContext
|
||||
streamCompleted.onFailure {
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue