Merge pull request #24023 from akka/wip-23967-stop-idle-patriknw
Stop unused Artery outbound streams, #23967
This commit is contained in:
commit
f0709774dc
23 changed files with 909 additions and 320 deletions
|
|
@ -115,7 +115,7 @@ class SendQueueBenchmark {
|
||||||
val burstSize = 1000
|
val burstSize = 1000
|
||||||
|
|
||||||
val queue = new ManyToOneConcurrentArrayQueue[Int](1024)
|
val queue = new ManyToOneConcurrentArrayQueue[Int](1024)
|
||||||
val source = Source.fromGraph(new SendQueue[Int](system.deadLetters))
|
val source = Source.fromGraph(new SendQueue[Int](_ ⇒ ()))
|
||||||
|
|
||||||
val (sendQueue, killSwitch) = source.viaMat(KillSwitches.single)(Keep.both)
|
val (sendQueue, killSwitch) = source.viaMat(KillSwitches.single)(Keep.both)
|
||||||
.toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left).run()(materializer)
|
.toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left).run()(materializer)
|
||||||
|
|
|
||||||
|
|
@ -7,9 +7,12 @@ import scala.concurrent.duration._
|
||||||
|
|
||||||
import akka.actor.ActorIdentity
|
import akka.actor.ActorIdentity
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.actor.Identify
|
import akka.actor.Identify
|
||||||
import akka.actor.PoisonPill
|
import akka.actor.PoisonPill
|
||||||
import akka.cluster.ClusterEvent.UnreachableMember
|
import akka.cluster.ClusterEvent.UnreachableMember
|
||||||
|
import akka.remote.RARP
|
||||||
|
import akka.remote.artery.ArterySettings
|
||||||
import akka.remote.testconductor.RoleName
|
import akka.remote.testconductor.RoleName
|
||||||
import akka.remote.testkit.MultiNodeConfig
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
|
|
@ -26,14 +29,14 @@ object LargeMessageClusterMultiJvmSpec extends MultiNodeConfig {
|
||||||
commonConfig(ConfigFactory.parseString(
|
commonConfig(ConfigFactory.parseString(
|
||||||
"""
|
"""
|
||||||
akka {
|
akka {
|
||||||
#loglevel = DEBUG
|
loglevel = DEBUG
|
||||||
cluster.debug.verbose-heartbeat-logging = on
|
cluster.debug.verbose-heartbeat-logging = on
|
||||||
loggers = ["akka.testkit.TestEventListener"]
|
loggers = ["akka.testkit.TestEventListener"]
|
||||||
actor.provider = cluster
|
actor.provider = cluster
|
||||||
|
|
||||||
testconductor.barrier-timeout = 3 minutes
|
testconductor.barrier-timeout = 3 minutes
|
||||||
|
|
||||||
cluster.failure-detector.acceptable-heartbeat-pause = 3 s
|
cluster.failure-detector.acceptable-heartbeat-pause = 5 s
|
||||||
|
|
||||||
remote.artery {
|
remote.artery {
|
||||||
enabled = on
|
enabled = on
|
||||||
|
|
@ -128,21 +131,19 @@ abstract class LargeMessageClusterSpec extends MultiNodeSpec(LargeMessageCluster
|
||||||
|
|
||||||
"not disturb cluster heartbeat messages when saturated" taggedAs LongRunningTest in {
|
"not disturb cluster heartbeat messages when saturated" taggedAs LongRunningTest in {
|
||||||
|
|
||||||
|
// FIXME only enabled for Aeron transport until #24576 is fixed
|
||||||
|
val arterySettings = ArterySettings(system.settings.config.getConfig("akka.remote.artery"))
|
||||||
|
if (!arterySettings.Enabled || arterySettings.Transport != ArterySettings.AeronUpd)
|
||||||
|
pending
|
||||||
|
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
val echo2 = identify(second, "echo")
|
|
||||||
val echo3 = identify(third, "echo")
|
|
||||||
val largeEcho2 = identify(second, "largeEcho")
|
val largeEcho2 = identify(second, "largeEcho")
|
||||||
val largeEcho3 = identify(third, "largeEcho")
|
val largeEcho3 = identify(third, "largeEcho")
|
||||||
|
|
||||||
val ordinaryMsgSize = 10 * 1024
|
val largeMsgSize = 1 * 1000 * 1000
|
||||||
val ordinaryMsg = ("0" * ordinaryMsgSize).getBytes("utf-8")
|
|
||||||
(1 to 5).foreach { _ ⇒
|
|
||||||
echo2.tell(ordinaryMsg, echo3)
|
|
||||||
}
|
|
||||||
|
|
||||||
val largeMsgSize = 2 * 1000 * 1000
|
|
||||||
val largeMsg = ("0" * largeMsgSize).getBytes("utf-8")
|
val largeMsg = ("0" * largeMsgSize).getBytes("utf-8")
|
||||||
(1 to 5).foreach { _ ⇒
|
(1 to 3).foreach { _ ⇒
|
||||||
|
// this will ping-pong between second and third
|
||||||
largeEcho2.tell(largeMsg, largeEcho3)
|
largeEcho2.tell(largeMsg, largeEcho3)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -168,6 +168,9 @@ abstract class AeronStreamLatencySpec
|
||||||
stats.print(System.out)
|
stats.print(System.out)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def sendToDeadLetters[T](pending: Vector[T]): Unit =
|
||||||
|
pending.foreach(system.deadLetters ! _)
|
||||||
|
|
||||||
val scenarios = List(
|
val scenarios = List(
|
||||||
TestSettings(
|
TestSettings(
|
||||||
testName = "rate-100-size-100",
|
testName = "rate-100-size-100",
|
||||||
|
|
@ -259,7 +262,7 @@ abstract class AeronStreamLatencySpec
|
||||||
envelope
|
envelope
|
||||||
}
|
}
|
||||||
|
|
||||||
val queueValue = Source.fromGraph(new SendQueue[Unit](system.deadLetters))
|
val queueValue = Source.fromGraph(new SendQueue[Unit](sendToDeadLetters))
|
||||||
.via(sendFlow)
|
.via(sendFlow)
|
||||||
.to(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink))
|
.to(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink))
|
||||||
.run()
|
.run()
|
||||||
|
|
|
||||||
|
|
@ -26,6 +26,20 @@ ProblemFilters.exclude[Problem]("akka.remote.artery.AeronSource*")
|
||||||
ProblemFilters.exclude[Problem]("akka.remote.artery.TaskRunner*")
|
ProblemFilters.exclude[Problem]("akka.remote.artery.TaskRunner*")
|
||||||
ProblemFilters.exclude[Problem]("akka.remote.artery.AeronErrorLog*")
|
ProblemFilters.exclude[Problem]("akka.remote.artery.AeronErrorLog*")
|
||||||
|
|
||||||
|
# #23967 Stop unused Artery outbound streams
|
||||||
|
ProblemFilters.exclude[Problem]("akka.remote.artery.InboundControlJunction*")
|
||||||
|
ProblemFilters.exclude[Problem]("akka.remote.artery.Association*")
|
||||||
|
ProblemFilters.exclude[Problem]("akka.remote.artery.OutboundContext*")
|
||||||
|
ProblemFilters.exclude[Problem]("akka.remote.artery.Decoder*")
|
||||||
|
ProblemFilters.exclude[Problem]("akka.remote.artery.AssociationState*")
|
||||||
|
ProblemFilters.exclude[Problem]("akka.remote.artery.compress.InboundCompressions*")
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -972,6 +972,21 @@ akka {
|
||||||
# of a network partition that you need to survive.
|
# of a network partition that you need to survive.
|
||||||
give-up-system-message-after = 6 hours
|
give-up-system-message-after = 6 hours
|
||||||
|
|
||||||
|
# After catastrophic communication failures that could result in the loss of system
|
||||||
|
# messages or after the remote DeathWatch triggers the remote system gets
|
||||||
|
# quarantined to prevent inconsistent behavior.
|
||||||
|
# This setting controls how long the quarantined association will be kept around
|
||||||
|
# before being removed to avoid long-term memory leaks. It must be quarantined
|
||||||
|
# and also unused for this duration before it's removed. When removed the historical
|
||||||
|
# information about which UIDs that were quarantined for that hostname:port is
|
||||||
|
# gone which could result in communication with a previously quarantined node
|
||||||
|
# if it wakes up again. Therfore this shouldn't be set too low.
|
||||||
|
remove-quarantined-association-after = 1 h
|
||||||
|
|
||||||
|
# Outbound streams are stopped when they haven't been used for this duration.
|
||||||
|
# They are started again when new messages are sent.
|
||||||
|
stop-idle-outbound-after = 5.minutes
|
||||||
|
|
||||||
# during ActorSystem termination the remoting will wait this long for
|
# during ActorSystem termination the remoting will wait this long for
|
||||||
# an acknowledgment by the destination system that flushing of outstanding
|
# an acknowledgment by the destination system that flushing of outstanding
|
||||||
# remote messages has been completed
|
# remote messages has been completed
|
||||||
|
|
|
||||||
|
|
@ -143,6 +143,11 @@ private[akka] final class ArterySettings private (config: Config) {
|
||||||
val GiveUpSystemMessageAfter: FiniteDuration =
|
val GiveUpSystemMessageAfter: FiniteDuration =
|
||||||
config.getMillisDuration("give-up-system-message-after").requiring(interval ⇒
|
config.getMillisDuration("give-up-system-message-after").requiring(interval ⇒
|
||||||
interval > Duration.Zero, "give-up-system-message-after must be more than zero")
|
interval > Duration.Zero, "give-up-system-message-after must be more than zero")
|
||||||
|
val RemoveQuarantinedAssociationAfter: FiniteDuration =
|
||||||
|
config.getMillisDuration("remove-quarantined-association-after").requiring(interval ⇒
|
||||||
|
interval > Duration.Zero, "remove-quarantined-association-after must be more than zero")
|
||||||
|
val StopIdleOutboundAfter: FiniteDuration = config.getMillisDuration("stop-idle-outbound-after").requiring(interval ⇒
|
||||||
|
interval > Duration.Zero, "stop-idle-outbound-after must be more than zero")
|
||||||
val ShutdownFlushTimeout: FiniteDuration =
|
val ShutdownFlushTimeout: FiniteDuration =
|
||||||
config.getMillisDuration("shutdown-flush-timeout").requiring(interval ⇒
|
config.getMillisDuration("shutdown-flush-timeout").requiring(interval ⇒
|
||||||
interval > Duration.Zero, "shutdown-flush-timeout must be more than zero")
|
interval > Duration.Zero, "shutdown-flush-timeout must be more than zero")
|
||||||
|
|
|
||||||
|
|
@ -10,6 +10,7 @@ import java.nio.channels.ServerSocketChannel
|
||||||
import java.nio.file.Path
|
import java.nio.file.Path
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
|
import java.util.concurrent.atomic.AtomicLong
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
|
@ -36,7 +37,6 @@ import akka.remote.RemoteActorRefProvider
|
||||||
import akka.remote.RemoteTransport
|
import akka.remote.RemoteTransport
|
||||||
import akka.remote.ThisActorSystemQuarantinedEvent
|
import akka.remote.ThisActorSystemQuarantinedEvent
|
||||||
import akka.remote.UniqueAddress
|
import akka.remote.UniqueAddress
|
||||||
import akka.remote.artery.ArteryTransport.ShuttingDown
|
|
||||||
import akka.remote.artery.Decoder.InboundCompressionAccess
|
import akka.remote.artery.Decoder.InboundCompressionAccess
|
||||||
import akka.remote.artery.Encoder.OutboundCompressionAccess
|
import akka.remote.artery.Encoder.OutboundCompressionAccess
|
||||||
import akka.remote.artery.InboundControlJunction.ControlMessageObserver
|
import akka.remote.artery.InboundControlJunction.ControlMessageObserver
|
||||||
|
|
@ -102,6 +102,8 @@ private[remote] object AssociationState {
|
||||||
new AssociationState(
|
new AssociationState(
|
||||||
incarnation = 1,
|
incarnation = 1,
|
||||||
uniqueRemoteAddressPromise = Promise(),
|
uniqueRemoteAddressPromise = Promise(),
|
||||||
|
lastUsedTimestamp = new AtomicLong(System.nanoTime()),
|
||||||
|
controlIdleKillSwitch = OptionVal.None,
|
||||||
quarantined = ImmutableLongMap.empty[QuarantinedTimestamp])
|
quarantined = ImmutableLongMap.empty[QuarantinedTimestamp])
|
||||||
|
|
||||||
final case class QuarantinedTimestamp(nanoTime: Long) {
|
final case class QuarantinedTimestamp(nanoTime: Long) {
|
||||||
|
|
@ -116,6 +118,8 @@ private[remote] object AssociationState {
|
||||||
private[remote] final class AssociationState(
|
private[remote] final class AssociationState(
|
||||||
val incarnation: Int,
|
val incarnation: Int,
|
||||||
val uniqueRemoteAddressPromise: Promise[UniqueAddress],
|
val uniqueRemoteAddressPromise: Promise[UniqueAddress],
|
||||||
|
val lastUsedTimestamp: AtomicLong, // System.nanoTime timestamp
|
||||||
|
val controlIdleKillSwitch: OptionVal[SharedKillSwitch],
|
||||||
val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) {
|
val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) {
|
||||||
|
|
||||||
import AssociationState.QuarantinedTimestamp
|
import AssociationState.QuarantinedTimestamp
|
||||||
|
|
@ -143,7 +147,8 @@ private[remote] final class AssociationState(
|
||||||
}
|
}
|
||||||
|
|
||||||
def newIncarnation(remoteAddressPromise: Promise[UniqueAddress]): AssociationState =
|
def newIncarnation(remoteAddressPromise: Promise[UniqueAddress]): AssociationState =
|
||||||
new AssociationState(incarnation + 1, remoteAddressPromise, quarantined)
|
new AssociationState(incarnation + 1, remoteAddressPromise,
|
||||||
|
lastUsedTimestamp = new AtomicLong(System.nanoTime()), controlIdleKillSwitch, quarantined)
|
||||||
|
|
||||||
def newQuarantined(): AssociationState =
|
def newQuarantined(): AssociationState =
|
||||||
uniqueRemoteAddressPromise.future.value match {
|
uniqueRemoteAddressPromise.future.value match {
|
||||||
|
|
@ -151,6 +156,8 @@ private[remote] final class AssociationState(
|
||||||
new AssociationState(
|
new AssociationState(
|
||||||
incarnation,
|
incarnation,
|
||||||
uniqueRemoteAddressPromise,
|
uniqueRemoteAddressPromise,
|
||||||
|
lastUsedTimestamp = new AtomicLong(System.nanoTime()),
|
||||||
|
controlIdleKillSwitch,
|
||||||
quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime())))
|
quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime())))
|
||||||
case _ ⇒ this
|
case _ ⇒ this
|
||||||
}
|
}
|
||||||
|
|
@ -164,6 +171,10 @@ private[remote] final class AssociationState(
|
||||||
|
|
||||||
def isQuarantined(uid: Long): Boolean = quarantined.contains(uid)
|
def isQuarantined(uid: Long): Boolean = quarantined.contains(uid)
|
||||||
|
|
||||||
|
def withControlIdleKillSwitch(killSwitch: OptionVal[SharedKillSwitch]): AssociationState =
|
||||||
|
new AssociationState(incarnation, uniqueRemoteAddressPromise, lastUsedTimestamp,
|
||||||
|
controlIdleKillSwitch = killSwitch, quarantined)
|
||||||
|
|
||||||
override def toString(): String = {
|
override def toString(): String = {
|
||||||
val a = uniqueRemoteAddressPromise.future.value match {
|
val a = uniqueRemoteAddressPromise.future.value match {
|
||||||
case Some(Success(a)) ⇒ a
|
case Some(Success(a)) ⇒ a
|
||||||
|
|
@ -201,6 +212,11 @@ private[remote] trait OutboundContext {
|
||||||
*/
|
*/
|
||||||
def sendControl(message: ControlMessage): Unit
|
def sendControl(message: ControlMessage): Unit
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return `true` if any of the streams are active (not stopped due to idle)
|
||||||
|
*/
|
||||||
|
def isOrdinaryMessageStreamActive(): Boolean
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An outbound stage can listen to control messages
|
* An outbound stage can listen to control messages
|
||||||
* via this observer subject.
|
* via this observer subject.
|
||||||
|
|
@ -364,7 +380,10 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity =
|
private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity =
|
||||||
settings.Advanced.OutboundMessageQueueSize * settings.Advanced.OutboundLanes * 3)
|
settings.Advanced.OutboundMessageQueueSize * settings.Advanced.OutboundLanes * 3)
|
||||||
|
|
||||||
protected val topLevelFREvents =
|
/**
|
||||||
|
* Thread-safe flight recorder for top level events.
|
||||||
|
*/
|
||||||
|
val topLevelFlightRecorder: EventSink =
|
||||||
createFlightRecorderEventSink(synchr = true)
|
createFlightRecorderEventSink(synchr = true)
|
||||||
|
|
||||||
def createFlightRecorderEventSink(synchr: Boolean = false): EventSink = {
|
def createFlightRecorderEventSink(synchr: Boolean = false): EventSink = {
|
||||||
|
|
@ -389,6 +408,8 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
priorityMessageDestinations,
|
priorityMessageDestinations,
|
||||||
outboundEnvelopePool))
|
outboundEnvelopePool))
|
||||||
|
|
||||||
|
def remoteAddresses: Set[Address] = associationRegistry.allAssociations.map(_.remoteAddress)
|
||||||
|
|
||||||
override def settings: ArterySettings = provider.remoteSettings.Artery
|
override def settings: ArterySettings = provider.remoteSettings.Artery
|
||||||
|
|
||||||
override def start(): Unit = {
|
override def start(): Unit = {
|
||||||
|
|
@ -396,7 +417,7 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
Runtime.getRuntime.addShutdownHook(shutdownHook)
|
Runtime.getRuntime.addShutdownHook(shutdownHook)
|
||||||
|
|
||||||
startTransport()
|
startTransport()
|
||||||
topLevelFREvents.loFreq(Transport_Started, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_Started, NoMetaData)
|
||||||
|
|
||||||
val udp = settings.Transport == ArterySettings.AeronUpd
|
val udp = settings.Transport == ArterySettings.AeronUpd
|
||||||
val port =
|
val port =
|
||||||
|
|
@ -420,7 +441,7 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
AddressUidExtension(system).longAddressUid)
|
AddressUidExtension(system).longAddressUid)
|
||||||
|
|
||||||
// TODO: This probably needs to be a global value instead of an event as events might rotate out of the log
|
// TODO: This probably needs to be a global value instead of an event as events might rotate out of the log
|
||||||
topLevelFREvents.loFreq(Transport_UniqueAddressSet, _localAddress.toString())
|
topLevelFlightRecorder.loFreq(Transport_UniqueAddressSet, _localAddress.toString())
|
||||||
|
|
||||||
materializer = ActorMaterializer.systemMaterializer(settings.Advanced.MaterializerSettings, "remote", system)
|
materializer = ActorMaterializer.systemMaterializer(settings.Advanced.MaterializerSettings, "remote", system)
|
||||||
controlMaterializer = ActorMaterializer.systemMaterializer(
|
controlMaterializer = ActorMaterializer.systemMaterializer(
|
||||||
|
|
@ -428,10 +449,12 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
"remoteControl", system)
|
"remoteControl", system)
|
||||||
|
|
||||||
messageDispatcher = new MessageDispatcher(system, provider)
|
messageDispatcher = new MessageDispatcher(system, provider)
|
||||||
topLevelFREvents.loFreq(Transport_MaterializerStarted, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_MaterializerStarted, NoMetaData)
|
||||||
|
|
||||||
runInboundStreams()
|
runInboundStreams()
|
||||||
topLevelFREvents.loFreq(Transport_StartupFinished, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_StartupFinished, NoMetaData)
|
||||||
|
|
||||||
|
startRemoveQuarantinedAssociationTask()
|
||||||
|
|
||||||
log.info(
|
log.info(
|
||||||
"Remoting started with transport [Artery {}]; listening on address [{}] with UID [{}]",
|
"Remoting started with transport [Artery {}]; listening on address [{}] with UID [{}]",
|
||||||
|
|
@ -442,6 +465,15 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
|
|
||||||
protected def runInboundStreams(): Unit
|
protected def runInboundStreams(): Unit
|
||||||
|
|
||||||
|
private def startRemoveQuarantinedAssociationTask(): Unit = {
|
||||||
|
val removeAfter = settings.Advanced.RemoveQuarantinedAssociationAfter
|
||||||
|
val interval = removeAfter / 2
|
||||||
|
system.scheduler.schedule(removeAfter, interval) {
|
||||||
|
if (!isShutdown)
|
||||||
|
associationRegistry.removeUnusedQuarantined(removeAfter)
|
||||||
|
}(system.dispatcher)
|
||||||
|
}
|
||||||
|
|
||||||
// Select inbound lane based on destination to preserve message order,
|
// Select inbound lane based on destination to preserve message order,
|
||||||
// Also include the uid of the sending system in the hash to spread
|
// Also include the uid of the sending system in the hash to spread
|
||||||
// "hot" destinations, e.g. ActorSelection anchor.
|
// "hot" destinations, e.g. ActorSelection anchor.
|
||||||
|
|
@ -552,6 +584,8 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
case ShuttingDown ⇒ // silence it
|
case ShuttingDown ⇒ // silence it
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def controlSubjectCompleted(signal: Try[Done]): Unit = ()
|
||||||
})
|
})
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -568,6 +602,7 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
case cause ⇒
|
case cause ⇒
|
||||||
if (restartCounter.restart()) {
|
if (restartCounter.restart()) {
|
||||||
log.error(cause, "{} failed. Restarting it. {}", streamName, cause.getMessage)
|
log.error(cause, "{} failed. Restarting it. {}", streamName, cause.getMessage)
|
||||||
|
topLevelFlightRecorder.loFreq(Transport_RestartInbound, s"$localAddress - $streamName")
|
||||||
restart()
|
restart()
|
||||||
} else {
|
} else {
|
||||||
log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system. {}",
|
log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system. {}",
|
||||||
|
|
@ -602,7 +637,7 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
killSwitch.abort(ShutdownSignal)
|
killSwitch.abort(ShutdownSignal)
|
||||||
topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_KillSwitchPulled, NoMetaData)
|
||||||
for {
|
for {
|
||||||
_ ← streamsCompleted.recover { case _ ⇒ Done }
|
_ ← streamsCompleted.recover { case _ ⇒ Done }
|
||||||
_ ← shutdownTransport().recover { case _ ⇒ Done }
|
_ ← shutdownTransport().recover { case _ ⇒ Done }
|
||||||
|
|
@ -610,7 +645,7 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
// no need to explicitly shut down the contained access since it's lifecycle is bound to the Decoder
|
// no need to explicitly shut down the contained access since it's lifecycle is bound to the Decoder
|
||||||
_inboundCompressionAccess = OptionVal.None
|
_inboundCompressionAccess = OptionVal.None
|
||||||
|
|
||||||
topLevelFREvents.loFreq(Transport_FlightRecorderClose, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_FlightRecorderClose, NoMetaData)
|
||||||
flightRecorder.foreach(_.close())
|
flightRecorder.foreach(_.close())
|
||||||
afrFileChannel.foreach(_.force(true))
|
afrFileChannel.foreach(_.force(true))
|
||||||
afrFileChannel.foreach(_.close())
|
afrFileChannel.foreach(_.close())
|
||||||
|
|
@ -692,8 +727,7 @@ private[remote] abstract class ArteryTransport(_system: ExtendedActorSystem, _pr
|
||||||
|
|
||||||
override def completeHandshake(peer: UniqueAddress): Future[Done] = {
|
override def completeHandshake(peer: UniqueAddress): Future[Done] = {
|
||||||
try {
|
try {
|
||||||
val a = associationRegistry.setUID(peer)
|
associationRegistry.setUID(peer).completeHandshake(peer)
|
||||||
a.completeHandshake(peer)
|
|
||||||
} catch {
|
} catch {
|
||||||
case ShuttingDown ⇒ Future.successful(Done) // silence it
|
case ShuttingDown ⇒ Future.successful(Done) // silence it
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,7 @@ package akka.remote.artery
|
||||||
|
|
||||||
import java.util.Queue
|
import java.util.Queue
|
||||||
import java.util.concurrent.CountDownLatch
|
import java.util.concurrent.CountDownLatch
|
||||||
|
import java.util.concurrent.LinkedBlockingQueue
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
|
|
@ -13,26 +14,23 @@ import scala.annotation.tailrec
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
import scala.concurrent.Promise
|
import scala.concurrent.Promise
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.duration.FiniteDuration
|
|
||||||
import akka.{ Done, NotUsed }
|
import akka.{ Done, NotUsed }
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.actor.ActorSelectionMessage
|
import akka.actor.ActorSelectionMessage
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.dispatch.sysmsg.SystemMessage
|
import akka.dispatch.sysmsg.SystemMessage
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
import akka.pattern.after
|
|
||||||
import akka.remote._
|
import akka.remote._
|
||||||
import akka.remote.DaemonMsgCreate
|
import akka.remote.DaemonMsgCreate
|
||||||
import akka.remote.QuarantinedEvent
|
import akka.remote.QuarantinedEvent
|
||||||
import akka.remote.artery.aeron.AeronSink.GaveUpMessageException
|
import akka.remote.artery.aeron.AeronSink.GaveUpMessageException
|
||||||
import akka.remote.artery.ArteryTransport.{ AeronTerminated, ShuttingDown }
|
import akka.remote.artery.ArteryTransport.{ AeronTerminated, ShuttingDown }
|
||||||
import akka.remote.artery.Encoder.OutboundCompressionAccess
|
import akka.remote.artery.Encoder.OutboundCompressionAccess
|
||||||
import akka.remote.artery.Encoder.AccessOutboundCompressionFailed
|
|
||||||
import akka.remote.artery.InboundControlJunction.ControlMessageSubject
|
import akka.remote.artery.InboundControlJunction.ControlMessageSubject
|
||||||
import akka.remote.artery.OutboundControlJunction.OutboundControlIngress
|
import akka.remote.artery.OutboundControlJunction.OutboundControlIngress
|
||||||
import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException
|
import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException
|
||||||
import akka.remote.artery.SystemMessageDelivery.ClearSystemMessageDelivery
|
import akka.remote.artery.SystemMessageDelivery.ClearSystemMessageDelivery
|
||||||
import akka.remote.artery.compress.CompressionProtocol._
|
|
||||||
import akka.remote.artery.compress.CompressionTable
|
import akka.remote.artery.compress.CompressionTable
|
||||||
import akka.stream.AbruptTerminationException
|
import akka.stream.AbruptTerminationException
|
||||||
import akka.stream.KillSwitches
|
import akka.stream.KillSwitches
|
||||||
|
|
@ -44,9 +42,10 @@ import akka.util.{ Unsafe, WildcardIndex }
|
||||||
import akka.util.OptionVal
|
import akka.util.OptionVal
|
||||||
import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
|
import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
|
||||||
import akka.stream.SharedKillSwitch
|
import akka.stream.SharedKillSwitch
|
||||||
|
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
import akka.actor.Cancellable
|
import akka.actor.Cancellable
|
||||||
|
import akka.stream.StreamTcpException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -72,6 +71,15 @@ private[remote] object Association {
|
||||||
override def isEnabled: Boolean = false
|
override def isEnabled: Boolean = false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
object RemovedQueueWrapper extends QueueWrapper {
|
||||||
|
override def queue: java.util.Queue[OutboundEnvelope] =
|
||||||
|
throw new UnsupportedOperationException("The Queue is removed")
|
||||||
|
|
||||||
|
override def offer(message: OutboundEnvelope): Boolean = false
|
||||||
|
|
||||||
|
override def isEnabled: Boolean = false
|
||||||
|
}
|
||||||
|
|
||||||
final case class LazyQueueWrapper(queue: Queue[OutboundEnvelope], materialize: () ⇒ Unit) extends QueueWrapper {
|
final case class LazyQueueWrapper(queue: Queue[OutboundEnvelope], materialize: () ⇒ Unit) extends QueueWrapper {
|
||||||
private val onlyOnce = new AtomicBoolean
|
private val onlyOnce = new AtomicBoolean
|
||||||
|
|
||||||
|
|
@ -92,9 +100,14 @@ private[remote] object Association {
|
||||||
final val LargeQueueIndex = 1
|
final val LargeQueueIndex = 1
|
||||||
final val OrdinaryQueueIndex = 2
|
final val OrdinaryQueueIndex = 2
|
||||||
|
|
||||||
private object OutboundStreamStopSignal extends RuntimeException with NoStackTrace
|
sealed trait StopSignal
|
||||||
|
case object OutboundStreamStopIdleSignal extends RuntimeException("") with StopSignal with NoStackTrace
|
||||||
|
case object OutboundStreamStopQuarantinedSignal extends RuntimeException("") with StopSignal with NoStackTrace
|
||||||
|
|
||||||
final case class OutboundStreamMatValues(streamKillSwitch: SharedKillSwitch, completed: Future[Done])
|
final case class OutboundStreamMatValues(
|
||||||
|
streamKillSwitch: OptionVal[SharedKillSwitch],
|
||||||
|
completed: Future[Done],
|
||||||
|
stopping: OptionVal[StopSignal])
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -117,7 +130,7 @@ private[remote] class Association(
|
||||||
import FlightRecorderEvents._
|
import FlightRecorderEvents._
|
||||||
|
|
||||||
private val log = Logging(transport.system, getClass.getName)
|
private val log = Logging(transport.system, getClass.getName)
|
||||||
private val flightRecorder = transport.createFlightRecorderEventSink(synchr = true)
|
private def flightRecorder = transport.topLevelFlightRecorder
|
||||||
|
|
||||||
override def settings = transport.settings
|
override def settings = transport.settings
|
||||||
private def advancedSettings = transport.settings.Advanced
|
private def advancedSettings = transport.settings.Advanced
|
||||||
|
|
@ -128,8 +141,13 @@ private[remote] class Association(
|
||||||
// the `SendQueue` after materialization. Using same underlying queue. This makes it possible to
|
// the `SendQueue` after materialization. Using same underlying queue. This makes it possible to
|
||||||
// start sending (enqueuing) to the Association immediate after construction.
|
// start sending (enqueuing) to the Association immediate after construction.
|
||||||
|
|
||||||
def createQueue(capacity: Int): Queue[OutboundEnvelope] =
|
def createQueue(capacity: Int, queueIndex: Int): Queue[OutboundEnvelope] = {
|
||||||
new ManyToOneConcurrentArrayQueue[OutboundEnvelope](capacity)
|
val linked = queueIndex == ControlQueueIndex || queueIndex == LargeQueueIndex
|
||||||
|
if (linked)
|
||||||
|
new LinkedBlockingQueue[OutboundEnvelope](capacity) // less memory than ManyToOneConcurrentArrayQueue
|
||||||
|
else
|
||||||
|
new ManyToOneConcurrentArrayQueue[OutboundEnvelope](capacity)
|
||||||
|
}
|
||||||
|
|
||||||
private val outboundLanes = advancedSettings.OutboundLanes
|
private val outboundLanes = advancedSettings.OutboundLanes
|
||||||
private val controlQueueSize = advancedSettings.OutboundControlQueueSize
|
private val controlQueueSize = advancedSettings.OutboundControlQueueSize
|
||||||
|
|
@ -137,15 +155,15 @@ private[remote] class Association(
|
||||||
private val largeQueueSize = advancedSettings.OutboundLargeMessageQueueSize
|
private val largeQueueSize = advancedSettings.OutboundLargeMessageQueueSize
|
||||||
|
|
||||||
private[this] val queues: Array[SendQueue.ProducerApi[OutboundEnvelope]] = new Array(2 + outboundLanes)
|
private[this] val queues: Array[SendQueue.ProducerApi[OutboundEnvelope]] = new Array(2 + outboundLanes)
|
||||||
queues(ControlQueueIndex) = QueueWrapperImpl(createQueue(controlQueueSize)) // control stream
|
queues(ControlQueueIndex) = QueueWrapperImpl(createQueue(controlQueueSize, ControlQueueIndex)) // control stream
|
||||||
queues(LargeQueueIndex) =
|
queues(LargeQueueIndex) =
|
||||||
if (transport.largeMessageChannelEnabled) // large messages stream
|
if (transport.largeMessageChannelEnabled) // large messages stream
|
||||||
QueueWrapperImpl(createQueue(largeQueueSize))
|
QueueWrapperImpl(createQueue(largeQueueSize, LargeQueueIndex))
|
||||||
else
|
else
|
||||||
DisabledQueueWrapper
|
DisabledQueueWrapper
|
||||||
|
|
||||||
(0 until outboundLanes).foreach { i ⇒
|
(0 until outboundLanes).foreach { i ⇒
|
||||||
queues(OrdinaryQueueIndex + i) = QueueWrapperImpl(createQueue(queueSize)) // ordinary messages stream
|
queues(OrdinaryQueueIndex + i) = QueueWrapperImpl(createQueue(queueSize, OrdinaryQueueIndex + i)) // ordinary messages stream
|
||||||
}
|
}
|
||||||
@volatile private[this] var queuesVisibility = false
|
@volatile private[this] var queuesVisibility = false
|
||||||
|
|
||||||
|
|
@ -158,37 +176,27 @@ private[remote] class Association(
|
||||||
// in case there is a restart at the same time as a compression table update
|
// in case there is a restart at the same time as a compression table update
|
||||||
private val changeCompressionTimeout = 5.seconds
|
private val changeCompressionTimeout = 5.seconds
|
||||||
|
|
||||||
private[remote] def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = {
|
|
||||||
import transport.system.dispatcher
|
|
||||||
val c = outboundCompressionAccess
|
|
||||||
val result =
|
|
||||||
if (c.isEmpty) Future.successful(Done)
|
|
||||||
else if (c.size == 1) c.head.changeActorRefCompression(table)
|
|
||||||
else Future.sequence(c.map(_.changeActorRefCompression(table))).map(_ ⇒ Done)
|
|
||||||
timeoutAfter(result, changeCompressionTimeout, new AccessOutboundCompressionFailed)
|
|
||||||
}
|
|
||||||
// keyed by stream queue index
|
// keyed by stream queue index
|
||||||
private[this] val streamMatValues = new AtomicReference(Map.empty[Int, OutboundStreamMatValues])
|
private[this] val streamMatValues = new AtomicReference(Map.empty[Int, OutboundStreamMatValues])
|
||||||
private[this] val idle = new AtomicReference[Option[Cancellable]](None)
|
|
||||||
|
|
||||||
private[remote] def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = {
|
private[this] val idleTask = new AtomicReference[Option[Cancellable]](None)
|
||||||
|
private[this] val quarantinedIdleTask = new AtomicReference[Option[Cancellable]](None)
|
||||||
|
|
||||||
|
private[remote] def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] =
|
||||||
|
updateOutboundCompression(c ⇒ c.changeActorRefCompression(table))
|
||||||
|
|
||||||
|
private[remote] def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] =
|
||||||
|
updateOutboundCompression(c ⇒ c.changeClassManifestCompression(table))
|
||||||
|
|
||||||
|
private def clearOutboundCompression(): Future[Done] =
|
||||||
|
updateOutboundCompression(c ⇒ c.clearCompression())
|
||||||
|
|
||||||
|
private def updateOutboundCompression(action: OutboundCompressionAccess ⇒ Future[Done]): Future[Done] = {
|
||||||
import transport.system.dispatcher
|
import transport.system.dispatcher
|
||||||
val c = outboundCompressionAccess
|
val c = outboundCompressionAccess
|
||||||
val result =
|
if (c.isEmpty) Future.successful(Done)
|
||||||
if (c.isEmpty) Future.successful(Done)
|
else if (c.size == 1) action(c.head)
|
||||||
else if (c.size == 1) c.head.changeClassManifestCompression(table)
|
else Future.sequence(c.map(action(_))).map(_ ⇒ Done)
|
||||||
else Future.sequence(c.map(_.changeClassManifestCompression(table))).map(_ ⇒ Done)
|
|
||||||
timeoutAfter(result, changeCompressionTimeout, new AccessOutboundCompressionFailed)
|
|
||||||
}
|
|
||||||
|
|
||||||
private def clearOutboundCompression(): Future[Done] = {
|
|
||||||
import transport.system.dispatcher
|
|
||||||
val c = outboundCompressionAccess
|
|
||||||
val result =
|
|
||||||
if (c.isEmpty) Future.successful(Done)
|
|
||||||
else if (c.size == 1) c.head.clearCompression()
|
|
||||||
else Future.sequence(c.map(_.clearCompression())).map(_ ⇒ Done)
|
|
||||||
timeoutAfter(result, changeCompressionTimeout, new AccessOutboundCompressionFailed)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def clearInboundCompression(originUid: Long): Unit =
|
private def clearInboundCompression(originUid: Long): Unit =
|
||||||
|
|
@ -197,12 +205,6 @@ private[remote] class Association(
|
||||||
case _ ⇒ // do nothing
|
case _ ⇒ // do nothing
|
||||||
}
|
}
|
||||||
|
|
||||||
private def timeoutAfter[T](f: Future[T], timeout: FiniteDuration, e: ⇒ Throwable): Future[T] = {
|
|
||||||
import transport.system.dispatcher
|
|
||||||
val f2 = after(timeout, transport.system.scheduler)(Future.failed(e))
|
|
||||||
Future.firstCompletedOf(List(f, f2))
|
|
||||||
}
|
|
||||||
|
|
||||||
private def deadletters = transport.system.deadLetters
|
private def deadletters = transport.system.deadLetters
|
||||||
|
|
||||||
def outboundControlIngress: OutboundControlIngress = {
|
def outboundControlIngress: OutboundControlIngress = {
|
||||||
|
|
@ -241,7 +243,7 @@ private[remote] class Association(
|
||||||
* @return Whether the previous state matched correctly
|
* @return Whether the previous state matched correctly
|
||||||
*/
|
*/
|
||||||
@inline
|
@inline
|
||||||
private[this] def swapState(oldState: AssociationState, newState: AssociationState): Boolean =
|
private[artery] def swapState(oldState: AssociationState, newState: AssociationState): Boolean =
|
||||||
Unsafe.instance.compareAndSwapObject(this, AbstractAssociation.sharedStateOffset, oldState, newState)
|
Unsafe.instance.compareAndSwapObject(this, AbstractAssociation.sharedStateOffset, oldState, newState)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -250,6 +252,13 @@ private[remote] class Association(
|
||||||
def associationState: AssociationState =
|
def associationState: AssociationState =
|
||||||
Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[AssociationState]
|
Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[AssociationState]
|
||||||
|
|
||||||
|
def setControlIdleKillSwitch(killSwitch: OptionVal[SharedKillSwitch]): Unit = {
|
||||||
|
val current = associationState
|
||||||
|
swapState(current, current.withControlIdleKillSwitch(killSwitch))
|
||||||
|
if (killSwitch.isDefined)
|
||||||
|
startIdleTimer()
|
||||||
|
}
|
||||||
|
|
||||||
def completeHandshake(peer: UniqueAddress): Future[Done] = {
|
def completeHandshake(peer: UniqueAddress): Future[Done] = {
|
||||||
require(
|
require(
|
||||||
remoteAddress == peer.address,
|
remoteAddress == peer.address,
|
||||||
|
|
@ -275,7 +284,7 @@ private[remote] class Association(
|
||||||
if (swapState(current, newState)) {
|
if (swapState(current, newState)) {
|
||||||
current.uniqueRemoteAddressValue() match {
|
current.uniqueRemoteAddressValue() match {
|
||||||
case Some(old) ⇒
|
case Some(old) ⇒
|
||||||
cancelIdleTimer()
|
cancelQuarantinedIdleTimer()
|
||||||
log.debug(
|
log.debug(
|
||||||
"Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])",
|
"Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])",
|
||||||
newState.incarnation, peer.address, peer.uid, old.uid)
|
newState.incarnation, peer.address, peer.uid, old.uid)
|
||||||
|
|
@ -294,11 +303,11 @@ private[remote] class Association(
|
||||||
// OutboundContext
|
// OutboundContext
|
||||||
override def sendControl(message: ControlMessage): Unit = {
|
override def sendControl(message: ControlMessage): Unit = {
|
||||||
try {
|
try {
|
||||||
if (!transport.isShutdown) {
|
if (!transport.isShutdown && !isRemovedAfterQuarantined()) {
|
||||||
if (associationState.isQuarantined()) {
|
if (associationState.isQuarantined()) {
|
||||||
log.debug("Send control message [{}] to quarantined [{}]", Logging.messageClassName(message),
|
log.debug("Send control message [{}] to quarantined [{}]", Logging.messageClassName(message),
|
||||||
remoteAddress)
|
remoteAddress)
|
||||||
startIdleTimer()
|
startQuarantinedIdleTimer()
|
||||||
}
|
}
|
||||||
outboundControlIngress.sendControlMessage(message)
|
outboundControlIngress.sendControlMessage(message)
|
||||||
}
|
}
|
||||||
|
|
@ -316,20 +325,31 @@ private[remote] class Association(
|
||||||
val unused = queuesVisibility
|
val unused = queuesVisibility
|
||||||
|
|
||||||
def dropped(queueIndex: Int, qSize: Int, env: OutboundEnvelope): Unit = {
|
def dropped(queueIndex: Int, qSize: Int, env: OutboundEnvelope): Unit = {
|
||||||
log.debug(
|
val removed = isRemovedAfterQuarantined()
|
||||||
"Dropping message [{}] from [{}] to [{}] due to overflow of send queue, size [{}]",
|
if (removed) recipient match {
|
||||||
Logging.messageClassName(message), sender.getOrElse(deadletters), recipient.getOrElse(recipient), qSize)
|
case OptionVal.Some(ref) ⇒ ref.cachedAssociation = null // don't use this Association instance any more
|
||||||
|
case OptionVal.None ⇒
|
||||||
|
}
|
||||||
|
if (log.isDebugEnabled) {
|
||||||
|
val reason =
|
||||||
|
if (removed) "removed unused quarantined association"
|
||||||
|
else s"overflow of send queue, size [$queueSize]"
|
||||||
|
log.debug(
|
||||||
|
"Dropping message [{}] from [{}] to [{}] due to {}",
|
||||||
|
Logging.messageClassName(message), sender.getOrElse(deadletters), recipient.getOrElse(recipient), reason)
|
||||||
|
}
|
||||||
flightRecorder.hiFreq(Transport_SendQueueOverflow, queueIndex)
|
flightRecorder.hiFreq(Transport_SendQueueOverflow, queueIndex)
|
||||||
deadletters ! env
|
deadletters ! env
|
||||||
}
|
}
|
||||||
|
|
||||||
val quarantined = associationState.isQuarantined()
|
val state = associationState
|
||||||
|
val quarantined = state.isQuarantined()
|
||||||
|
|
||||||
// allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system
|
// allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system
|
||||||
if (message.isInstanceOf[ActorSelectionMessage] || !quarantined || message == ClearSystemMessageDelivery) {
|
if (message.isInstanceOf[ActorSelectionMessage] || !quarantined || message == ClearSystemMessageDelivery) {
|
||||||
if (quarantined && message != ClearSystemMessageDelivery) {
|
if (quarantined && message != ClearSystemMessageDelivery) {
|
||||||
log.debug("Quarantine piercing attempt with message [{}] to [{}]", Logging.messageClassName(message), recipient.getOrElse(""))
|
log.debug("Quarantine piercing attempt with message [{}] to [{}]", Logging.messageClassName(message), recipient.getOrElse(""))
|
||||||
startIdleTimer()
|
startQuarantinedIdleTimer()
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
val outboundEnvelope = createOutboundEnvelope()
|
val outboundEnvelope = createOutboundEnvelope()
|
||||||
|
|
@ -398,11 +418,23 @@ private[remote] class Association(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def isOrdinaryMessageStreamActive(): Boolean =
|
||||||
|
isStreamActive(OrdinaryQueueIndex)
|
||||||
|
|
||||||
|
def isStreamActive(queueIndex: Int): Boolean = {
|
||||||
|
queues(queueIndex) match {
|
||||||
|
case _: LazyQueueWrapper ⇒ false
|
||||||
|
case DisabledQueueWrapper ⇒ false
|
||||||
|
case RemovedQueueWrapper ⇒ false
|
||||||
|
case _ ⇒ true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
def sendTerminationHint(replyTo: ActorRef): Int = {
|
def sendTerminationHint(replyTo: ActorRef): Int = {
|
||||||
if (!associationState.isQuarantined()) {
|
if (!associationState.isQuarantined()) {
|
||||||
val msg = ActorSystemTerminating(localAddress)
|
val msg = ActorSystemTerminating(localAddress)
|
||||||
var sent = 0
|
var sent = 0
|
||||||
queues.iterator.filter(_.isEnabled).foreach { queue ⇒
|
queues.iterator.filter(q ⇒ q.isEnabled && !q.isInstanceOf[LazyQueueWrapper]).foreach { queue ⇒
|
||||||
try {
|
try {
|
||||||
val envelope = outboundEnvelopePool.acquire()
|
val envelope = outboundEnvelopePool.acquire()
|
||||||
.init(OptionVal.None, msg, OptionVal.Some(replyTo))
|
.init(OptionVal.None, msg, OptionVal.Some(replyTo))
|
||||||
|
|
@ -439,13 +471,14 @@ private[remote] class Association(
|
||||||
"Remote actorsystem must be restarted to recover from this situation. {}",
|
"Remote actorsystem must be restarted to recover from this situation. {}",
|
||||||
remoteAddress, u, reason)
|
remoteAddress, u, reason)
|
||||||
transport.system.eventStream.publish(QuarantinedEvent(remoteAddress, u))
|
transport.system.eventStream.publish(QuarantinedEvent(remoteAddress, u))
|
||||||
|
flightRecorder.loFreq(Transport_Quarantined, s"$remoteAddress - $u")
|
||||||
clearOutboundCompression()
|
clearOutboundCompression()
|
||||||
clearInboundCompression(u)
|
clearInboundCompression(u)
|
||||||
// end delivery of system messages to that incarnation after this point
|
// end delivery of system messages to that incarnation after this point
|
||||||
send(ClearSystemMessageDelivery, OptionVal.None, OptionVal.None)
|
send(ClearSystemMessageDelivery, OptionVal.None, OptionVal.None)
|
||||||
// try to tell the other system that we have quarantined it
|
// try to tell the other system that we have quarantined it
|
||||||
sendControl(Quarantined(localAddress, peer))
|
sendControl(Quarantined(localAddress, peer))
|
||||||
startIdleTimer()
|
startQuarantinedIdleTimer()
|
||||||
} else
|
} else
|
||||||
quarantine(reason, uid) // recursive
|
quarantine(reason, uid) // recursive
|
||||||
}
|
}
|
||||||
|
|
@ -464,20 +497,123 @@ private[remote] class Association(
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def cancelIdleTimer(): Unit = {
|
/**
|
||||||
val current = idle.get
|
* After calling this no messages can be sent with this Association instance
|
||||||
|
*/
|
||||||
|
def removedAfterQuarantined(): Unit = {
|
||||||
|
if (!isRemovedAfterQuarantined()) {
|
||||||
|
flightRecorder.loFreq(Transport_RemovedQuarantined, remoteAddress.toString)
|
||||||
|
queues(ControlQueueIndex) = RemovedQueueWrapper
|
||||||
|
|
||||||
|
if (transport.largeMessageChannelEnabled)
|
||||||
|
queues(LargeQueueIndex) = RemovedQueueWrapper
|
||||||
|
|
||||||
|
(0 until outboundLanes).foreach { i ⇒
|
||||||
|
queues(OrdinaryQueueIndex + i) = RemovedQueueWrapper
|
||||||
|
}
|
||||||
|
queuesVisibility = true // volatile write for visibility of the queues array
|
||||||
|
|
||||||
|
// cleanup
|
||||||
|
_outboundControlIngress = OptionVal.None
|
||||||
|
outboundCompressionAccess = Vector.empty
|
||||||
|
cancelIdleTimer()
|
||||||
|
cancelQuarantinedIdleTimer()
|
||||||
|
abortQuarantined()
|
||||||
|
|
||||||
|
log.info("Unused association to [{}] removed after quarantine", remoteAddress)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def isRemovedAfterQuarantined(): Boolean =
|
||||||
|
queues(ControlQueueIndex) == RemovedQueueWrapper
|
||||||
|
|
||||||
|
private def cancelQuarantinedIdleTimer(): Unit = {
|
||||||
|
val current = quarantinedIdleTask.get
|
||||||
current.foreach(_.cancel())
|
current.foreach(_.cancel())
|
||||||
idle.compareAndSet(current, None)
|
quarantinedIdleTask.compareAndSet(current, None)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def startQuarantinedIdleTimer(): Unit = {
|
||||||
|
cancelQuarantinedIdleTimer()
|
||||||
|
quarantinedIdleTask.set(Some(transport.system.scheduler.scheduleOnce(advancedSettings.StopQuarantinedAfterIdle) {
|
||||||
|
if (associationState.isQuarantined())
|
||||||
|
abortQuarantined()
|
||||||
|
}(transport.system.dispatcher)))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def abortQuarantined(): Unit = {
|
||||||
|
streamMatValues.get.foreach {
|
||||||
|
case (queueIndex, OutboundStreamMatValues(killSwitch, _, _)) ⇒
|
||||||
|
killSwitch match {
|
||||||
|
case OptionVal.Some(k) ⇒
|
||||||
|
setStopReason(queueIndex, OutboundStreamStopQuarantinedSignal)
|
||||||
|
clearStreamKillSwitch(queueIndex, k)
|
||||||
|
k.abort(OutboundStreamStopQuarantinedSignal)
|
||||||
|
case OptionVal.None ⇒ // already aborted
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def cancelIdleTimer(): Unit = {
|
||||||
|
val current = idleTask.get
|
||||||
|
current.foreach(_.cancel())
|
||||||
|
idleTask.compareAndSet(current, None)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def startIdleTimer(): Unit = {
|
private def startIdleTimer(): Unit = {
|
||||||
cancelIdleTimer()
|
cancelIdleTimer()
|
||||||
idle.set(Some(transport.system.scheduler.scheduleOnce(advancedSettings.StopQuarantinedAfterIdle) {
|
val StopIdleOutboundAfter = settings.Advanced.StopIdleOutboundAfter
|
||||||
if (associationState.isQuarantined())
|
val interval = StopIdleOutboundAfter / 2
|
||||||
streamMatValues.get.valuesIterator.foreach {
|
val stopIdleOutboundAfterNanos = StopIdleOutboundAfter.toNanos
|
||||||
case OutboundStreamMatValues(killSwitch, _) ⇒ killSwitch.abort(OutboundStreamStopSignal)
|
val initialDelay = settings.Advanced.ConnectionTimeout.max(StopIdleOutboundAfter) + 1.second
|
||||||
|
val task: Cancellable = transport.system.scheduler.schedule(initialDelay, interval) {
|
||||||
|
if (System.nanoTime() - associationState.lastUsedTimestamp.get >= stopIdleOutboundAfterNanos) {
|
||||||
|
streamMatValues.get.foreach {
|
||||||
|
case (queueIndex, OutboundStreamMatValues(streamKillSwitch, _, stopping)) ⇒
|
||||||
|
if (isStreamActive(queueIndex) && stopping.isEmpty) {
|
||||||
|
if (queueIndex != ControlQueueIndex) {
|
||||||
|
streamKillSwitch match {
|
||||||
|
case OptionVal.Some(k) ⇒
|
||||||
|
// for non-control streams we can stop the entire stream
|
||||||
|
log.info("Stopping idle outbound stream [{}] to [{}]", queueIndex, remoteAddress)
|
||||||
|
flightRecorder.loFreq(Transport_StopIdleOutbound, s"$remoteAddress - $queueIndex")
|
||||||
|
setStopReason(queueIndex, OutboundStreamStopIdleSignal)
|
||||||
|
clearStreamKillSwitch(queueIndex, k)
|
||||||
|
k.abort(OutboundStreamStopIdleSignal)
|
||||||
|
case OptionVal.None ⇒ // already aborted
|
||||||
|
}
|
||||||
|
|
||||||
|
} else {
|
||||||
|
// only stop the transport parts of the stream because SystemMessageDelivery stage has
|
||||||
|
// state (seqno) and system messages might be sent at the same time
|
||||||
|
associationState.controlIdleKillSwitch match {
|
||||||
|
case OptionVal.Some(killSwitch) ⇒
|
||||||
|
log.info("Stopping idle outbound control stream to [{}]", remoteAddress)
|
||||||
|
flightRecorder.loFreq(Transport_StopIdleOutbound, s"$remoteAddress - $queueIndex")
|
||||||
|
setControlIdleKillSwitch(OptionVal.None)
|
||||||
|
killSwitch.abort(OutboundStreamStopIdleSignal)
|
||||||
|
case OptionVal.None ⇒
|
||||||
|
log.debug(
|
||||||
|
"Couldn't stop idle outbound control stream to [{}] due to missing KillSwitch.",
|
||||||
|
remoteAddress)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}(transport.system.dispatcher)))
|
|
||||||
|
cancelIdleTimer()
|
||||||
|
}
|
||||||
|
}(transport.system.dispatcher)
|
||||||
|
|
||||||
|
if (!idleTask.compareAndSet(None, Some(task))) {
|
||||||
|
// another thread did same thing and won
|
||||||
|
task.cancel()
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private def sendToDeadLetters[T](pending: Vector[OutboundEnvelope]): Unit = {
|
||||||
|
pending.foreach(transport.system.deadLetters ! _)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -492,6 +628,7 @@ private[remote] class Association(
|
||||||
if (!controlQueue.isInstanceOf[QueueWrapper])
|
if (!controlQueue.isInstanceOf[QueueWrapper])
|
||||||
throw new IllegalStateException("associate() must only be called once")
|
throw new IllegalStateException("associate() must only be called once")
|
||||||
runOutboundStreams()
|
runOutboundStreams()
|
||||||
|
startIdleTimer()
|
||||||
}
|
}
|
||||||
|
|
||||||
private def runOutboundStreams(): Unit = {
|
private def runOutboundStreams(): Unit = {
|
||||||
|
|
@ -515,8 +652,15 @@ private[remote] class Association(
|
||||||
|
|
||||||
val streamKillSwitch = KillSwitches.shared("outboundControlStreamKillSwitch")
|
val streamKillSwitch = KillSwitches.shared("outboundControlStreamKillSwitch")
|
||||||
|
|
||||||
|
def sendQueuePostStop[T](pending: Vector[OutboundEnvelope]): Unit = {
|
||||||
|
sendToDeadLetters(pending)
|
||||||
|
val systemMessagesCount = pending.count(env ⇒ env.message.isInstanceOf[SystemMessage])
|
||||||
|
if (systemMessagesCount > 0)
|
||||||
|
quarantine(s"SendQueue stopped with [$systemMessagesCount] pending system messages.")
|
||||||
|
}
|
||||||
|
|
||||||
val (queueValue, (control, completed)) =
|
val (queueValue, (control, completed)) =
|
||||||
Source.fromGraph(new SendQueue[OutboundEnvelope](transport.system.deadLetters))
|
Source.fromGraph(new SendQueue[OutboundEnvelope](sendQueuePostStop))
|
||||||
.via(streamKillSwitch.flow)
|
.via(streamKillSwitch.flow)
|
||||||
.toMat(transport.outboundControl(this))(Keep.both)
|
.toMat(transport.outboundControl(this))(Keep.both)
|
||||||
.run()(materializer)
|
.run()(materializer)
|
||||||
|
|
@ -539,12 +683,15 @@ private[remote] class Association(
|
||||||
case existing: QueueWrapper ⇒ existing
|
case existing: QueueWrapper ⇒ existing
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
// use new queue for restarts
|
// use new queue for restarts
|
||||||
QueueWrapperImpl(createQueue(capacity))
|
QueueWrapperImpl(createQueue(capacity, queueIndex))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def runOutboundOrdinaryMessagesStream(): Unit = {
|
private def runOutboundOrdinaryMessagesStream(): Unit = {
|
||||||
if (transport.isShutdown) throw ShuttingDown
|
if (transport.isShutdown) throw ShuttingDown
|
||||||
|
|
||||||
|
val streamKillSwitch = KillSwitches.shared("outboundMessagesKillSwitch")
|
||||||
|
|
||||||
if (outboundLanes == 1) {
|
if (outboundLanes == 1) {
|
||||||
log.debug("Starting outbound message stream to [{}]", remoteAddress)
|
log.debug("Starting outbound message stream to [{}]", remoteAddress)
|
||||||
val queueIndex = OrdinaryQueueIndex
|
val queueIndex = OrdinaryQueueIndex
|
||||||
|
|
@ -552,10 +699,8 @@ private[remote] class Association(
|
||||||
queues(queueIndex) = wrapper // use new underlying queue immediately for restarts
|
queues(queueIndex) = wrapper // use new underlying queue immediately for restarts
|
||||||
queuesVisibility = true // volatile write for visibility of the queues array
|
queuesVisibility = true // volatile write for visibility of the queues array
|
||||||
|
|
||||||
val streamKillSwitch = KillSwitches.shared("outboundMessagesKillSwitch")
|
|
||||||
|
|
||||||
val (queueValue, testMgmt, changeCompression, completed) =
|
val (queueValue, testMgmt, changeCompression, completed) =
|
||||||
Source.fromGraph(new SendQueue[OutboundEnvelope](transport.system.deadLetters))
|
Source.fromGraph(new SendQueue[OutboundEnvelope](sendToDeadLetters))
|
||||||
.via(streamKillSwitch.flow)
|
.via(streamKillSwitch.flow)
|
||||||
.viaMat(transport.outboundTestFlow(this))(Keep.both)
|
.viaMat(transport.outboundTestFlow(this))(Keep.both)
|
||||||
.toMat(transport.outbound(this))({ case ((a, b), (c, d)) ⇒ (a, b, c, d) }) // "keep all, exploded"
|
.toMat(transport.outbound(this))({ case ((a, b), (c, d)) ⇒ (a, b, c, d) }) // "keep all, exploded"
|
||||||
|
|
@ -580,9 +725,7 @@ private[remote] class Association(
|
||||||
wrapper
|
wrapper
|
||||||
}.toVector
|
}.toVector
|
||||||
|
|
||||||
val streamKillSwitch = KillSwitches.shared("outboundMessagesKillSwitch")
|
val lane = Source.fromGraph(new SendQueue[OutboundEnvelope](sendToDeadLetters))
|
||||||
|
|
||||||
val lane = Source.fromGraph(new SendQueue[OutboundEnvelope](transport.system.deadLetters))
|
|
||||||
.via(streamKillSwitch.flow)
|
.via(streamKillSwitch.flow)
|
||||||
.via(transport.outboundTestFlow(this))
|
.via(transport.outboundTestFlow(this))
|
||||||
.viaMat(transport.outboundLane(this))(Keep.both)
|
.viaMat(transport.outboundLane(this))(Keep.both)
|
||||||
|
|
@ -637,7 +780,7 @@ private[remote] class Association(
|
||||||
|
|
||||||
val streamKillSwitch = KillSwitches.shared("outboundLargeMessagesKillSwitch")
|
val streamKillSwitch = KillSwitches.shared("outboundLargeMessagesKillSwitch")
|
||||||
|
|
||||||
val (queueValue, completed) = Source.fromGraph(new SendQueue[OutboundEnvelope](transport.system.deadLetters))
|
val (queueValue, completed) = Source.fromGraph(new SendQueue[OutboundEnvelope](sendToDeadLetters))
|
||||||
.via(streamKillSwitch.flow)
|
.via(streamKillSwitch.flow)
|
||||||
.via(transport.outboundTestFlow(this))
|
.via(transport.outboundTestFlow(this))
|
||||||
.toMat(transport.outboundLarge(this))(Keep.both)
|
.toMat(transport.outboundLarge(this))(Keep.both)
|
||||||
|
|
@ -657,13 +800,21 @@ private[remote] class Association(
|
||||||
streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = {
|
streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = {
|
||||||
|
|
||||||
def lazyRestart(): Unit = {
|
def lazyRestart(): Unit = {
|
||||||
|
flightRecorder.loFreq(Transport_RestartOutbound, s"$remoteAddress - $streamName")
|
||||||
outboundCompressionAccess = Vector.empty
|
outboundCompressionAccess = Vector.empty
|
||||||
if (queueIndex == ControlQueueIndex) {
|
if (queueIndex == ControlQueueIndex) {
|
||||||
materializing = new CountDownLatch(1)
|
materializing = new CountDownLatch(1)
|
||||||
_outboundControlIngress = OptionVal.None
|
_outboundControlIngress = OptionVal.None
|
||||||
}
|
}
|
||||||
// LazyQueueWrapper will invoke the `restart` function when first message is offered
|
// LazyQueueWrapper will invoke the `restart` function when first message is offered
|
||||||
queues(queueIndex) = LazyQueueWrapper(createQueue(queueCapacity), restart)
|
val restartAndStartIdleTimer: () ⇒ Unit = () ⇒ {
|
||||||
|
restart()
|
||||||
|
startIdleTimer()
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!isRemovedAfterQuarantined())
|
||||||
|
queues(queueIndex) = LazyQueueWrapper(createQueue(queueCapacity, queueIndex), restartAndStartIdleTimer)
|
||||||
|
|
||||||
queuesVisibility = true // volatile write for visibility of the queues array
|
queuesVisibility = true // volatile write for visibility of the queues array
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -676,47 +827,73 @@ private[remote] class Association(
|
||||||
streamCompleted.failed.foreach {
|
streamCompleted.failed.foreach {
|
||||||
case ArteryTransport.ShutdownSignal ⇒
|
case ArteryTransport.ShutdownSignal ⇒
|
||||||
// shutdown as expected
|
// shutdown as expected
|
||||||
|
cancelIdleTimer()
|
||||||
// countDown the latch in case threads are waiting on the latch in outboundControlIngress method
|
// countDown the latch in case threads are waiting on the latch in outboundControlIngress method
|
||||||
materializing.countDown()
|
materializing.countDown()
|
||||||
case _: AeronTerminated ⇒ // shutdown already in progress
|
case cause if transport.isShutdown || isRemovedAfterQuarantined() ⇒
|
||||||
case cause if transport.isShutdown ⇒
|
|
||||||
// don't restart after shutdown, but log some details so we notice
|
// don't restart after shutdown, but log some details so we notice
|
||||||
log.error(cause, s"{} to [{}] failed after shutdown. {}", streamName, remoteAddress, cause.getMessage)
|
// for the TCP transport the ShutdownSignal is "converted" to StreamTcpException
|
||||||
|
if (!cause.isInstanceOf[StreamTcpException])
|
||||||
|
log.error(cause, s"{} to [{}] failed after shutdown. {}", streamName, remoteAddress, cause.getMessage)
|
||||||
|
cancelIdleTimer()
|
||||||
// countDown the latch in case threads are waiting on the latch in outboundControlIngress method
|
// countDown the latch in case threads are waiting on the latch in outboundControlIngress method
|
||||||
materializing.countDown()
|
materializing.countDown()
|
||||||
case _: AbruptTerminationException ⇒ // ActorSystem shutdown
|
case _: AeronTerminated ⇒
|
||||||
case OutboundStreamStopSignal ⇒
|
// shutdown already in progress
|
||||||
// stop as expected due to quarantine
|
cancelIdleTimer()
|
||||||
log.debug("{} to [{}] stopped. It will be restarted if used again.", streamName, remoteAddress)
|
case _: AbruptTerminationException ⇒
|
||||||
lazyRestart()
|
// ActorSystem shutdown
|
||||||
case cause: GaveUpMessageException ⇒
|
cancelIdleTimer()
|
||||||
log.debug("{} to [{}] failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage)
|
|
||||||
// restart unconditionally, without counting restarts
|
|
||||||
lazyRestart()
|
|
||||||
case cause ⇒
|
case cause ⇒
|
||||||
if (queueIndex == ControlQueueIndex) {
|
|
||||||
|
// it might have been stopped as expected due to idle or quarantine
|
||||||
|
// for the TCP transport the exception is "converted" to StreamTcpException
|
||||||
|
val stoppedIdle = cause == OutboundStreamStopIdleSignal ||
|
||||||
|
getStopReason(queueIndex).contains(OutboundStreamStopIdleSignal)
|
||||||
|
val stoppedQuarantined = cause == OutboundStreamStopQuarantinedSignal ||
|
||||||
|
getStopReason(queueIndex).contains(OutboundStreamStopQuarantinedSignal)
|
||||||
|
|
||||||
|
// for some cases restart unconditionally, without counting restarts
|
||||||
|
val bypassRestartCounter = cause match {
|
||||||
|
case _: GaveUpMessageException ⇒ true
|
||||||
|
case _ ⇒ stoppedIdle || stoppedQuarantined
|
||||||
|
}
|
||||||
|
|
||||||
|
if (queueIndex == ControlQueueIndex && !stoppedQuarantined) {
|
||||||
cause match {
|
cause match {
|
||||||
case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID
|
case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
// FIXME can we avoid quarantine if all system messages have been delivered?
|
// Must quarantine in case all system messages haven't been delivered.
|
||||||
quarantine("Outbound control stream restarted")
|
// See also comment in the stoppedIdle case below
|
||||||
|
quarantine(s"Outbound control stream restarted. $cause")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (restartCounter.restart()) {
|
if (stoppedIdle) {
|
||||||
|
log.debug("{} to [{}] was idle and stopped. It will be restarted if used again.", streamName, remoteAddress)
|
||||||
|
lazyRestart()
|
||||||
|
} else if (stoppedQuarantined) {
|
||||||
|
log.debug("{} to [{}] was quarantined and stopped. It will be restarted if used again.", streamName, remoteAddress)
|
||||||
|
lazyRestart()
|
||||||
|
} else if (bypassRestartCounter || restartCounter.restart()) {
|
||||||
log.error(cause, "{} to [{}] failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage)
|
log.error(cause, "{} to [{}] failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage)
|
||||||
lazyRestart()
|
lazyRestart()
|
||||||
} else {
|
} else {
|
||||||
log.error(cause, s"{} to [{}] failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}",
|
log.error(cause, s"{} to [{}] failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}",
|
||||||
streamName, remoteAddress, advancedSettings.OutboundMaxRestarts, advancedSettings.OutboundRestartTimeout.toSeconds)
|
streamName, remoteAddress, advancedSettings.OutboundMaxRestarts, advancedSettings.OutboundRestartTimeout.toSeconds)
|
||||||
|
cancelIdleTimer()
|
||||||
transport.system.terminate()
|
transport.system.terminate()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def updateStreamMatValues(streamId: Int, streamKillSwitch: SharedKillSwitch, completed: Future[Done]): Unit = {
|
private def updateStreamMatValues(streamId: Int, streamKillSwitch: SharedKillSwitch,
|
||||||
|
completed: Future[Done]): Unit = {
|
||||||
implicit val ec = materializer.executionContext
|
implicit val ec = materializer.executionContext
|
||||||
updateStreamMatValues(streamId, OutboundStreamMatValues(streamKillSwitch, completed.recover { case _ ⇒ Done }))
|
updateStreamMatValues(
|
||||||
|
streamId,
|
||||||
|
OutboundStreamMatValues(OptionVal.Some(streamKillSwitch), completed.recover { case _ ⇒ Done },
|
||||||
|
stopping = OptionVal.None))
|
||||||
}
|
}
|
||||||
|
|
||||||
@tailrec private def updateStreamMatValues(streamId: Int, values: OutboundStreamMatValues): Unit = {
|
@tailrec private def updateStreamMatValues(streamId: Int, values: OutboundStreamMatValues): Unit = {
|
||||||
|
|
@ -726,6 +903,37 @@ private[remote] class Association(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@tailrec private def setStopReason(streamId: Int, stopSignal: StopSignal): Unit = {
|
||||||
|
val prev = streamMatValues.get()
|
||||||
|
prev.get(streamId) match {
|
||||||
|
case Some(v) ⇒
|
||||||
|
if (!streamMatValues.compareAndSet(prev, prev.updated(streamId, v.copy(stopping = OptionVal.Some(stopSignal)))))
|
||||||
|
setStopReason(streamId, stopSignal)
|
||||||
|
case None ⇒ throw new IllegalStateException(s"Expected streamMatValues for [$streamId]")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def getStopReason(streamId: Int): OptionVal[StopSignal] = {
|
||||||
|
streamMatValues.get().get(streamId) match {
|
||||||
|
case Some(OutboundStreamMatValues(_, _, stopping)) ⇒ stopping
|
||||||
|
case None ⇒ OptionVal.None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// after it has been used we remove the kill switch to cleanup some memory,
|
||||||
|
// not a "leak" but a KillSwitch is rather heavy
|
||||||
|
@tailrec private def clearStreamKillSwitch(streamId: Int, old: SharedKillSwitch): Unit = {
|
||||||
|
val prev = streamMatValues.get()
|
||||||
|
prev.get(streamId) match {
|
||||||
|
case Some(v) ⇒
|
||||||
|
if (v.streamKillSwitch.isDefined && (v.streamKillSwitch.get eq old)) {
|
||||||
|
if (!streamMatValues.compareAndSet(prev, prev.updated(streamId, v.copy(streamKillSwitch = OptionVal.None))))
|
||||||
|
clearStreamKillSwitch(streamId, old)
|
||||||
|
}
|
||||||
|
case None ⇒ throw new IllegalStateException(s"Expected streamMatValues for [$streamId]")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Exposed for orderly shutdown purposes, can not be trusted except for during shutdown as streams may restart.
|
* Exposed for orderly shutdown purposes, can not be trusted except for during shutdown as streams may restart.
|
||||||
* Will complete successfully even if one of the stream completion futures failed
|
* Will complete successfully even if one of the stream completion futures failed
|
||||||
|
|
@ -733,7 +941,7 @@ private[remote] class Association(
|
||||||
def streamsCompleted: Future[Done] = {
|
def streamsCompleted: Future[Done] = {
|
||||||
implicit val ec = materializer.executionContext
|
implicit val ec = materializer.executionContext
|
||||||
Future.sequence(streamMatValues.get().values.map {
|
Future.sequence(streamMatValues.get().values.map {
|
||||||
case OutboundStreamMatValues(_, done) ⇒ done
|
case OutboundStreamMatValues(_, done, _) ⇒ done
|
||||||
}).map(_ ⇒ Done)
|
}).map(_ ⇒ Done)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -774,9 +982,10 @@ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Associa
|
||||||
* @throws ShuttingDown if called while the transport is shutting down
|
* @throws ShuttingDown if called while the transport is shutting down
|
||||||
*/
|
*/
|
||||||
@tailrec final def setUID(peer: UniqueAddress): Association = {
|
@tailrec final def setUID(peer: UniqueAddress): Association = {
|
||||||
val currentMap = associationsByUid.get
|
// Don't create a new association via this method. It's supposed to exist unless it was removed after quarantined.
|
||||||
val a = association(peer.address)
|
val a = association(peer.address)
|
||||||
|
|
||||||
|
val currentMap = associationsByUid.get
|
||||||
currentMap.get(peer.uid) match {
|
currentMap.get(peer.uid) match {
|
||||||
case OptionVal.Some(previous) ⇒
|
case OptionVal.Some(previous) ⇒
|
||||||
if (previous eq a)
|
if (previous eq a)
|
||||||
|
|
@ -797,4 +1006,50 @@ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Associa
|
||||||
|
|
||||||
def allAssociations: Set[Association] =
|
def allAssociations: Set[Association] =
|
||||||
associationsByAddress.get.values.toSet
|
associationsByAddress.get.values.toSet
|
||||||
|
|
||||||
|
def removeUnusedQuarantined(after: FiniteDuration): Unit = {
|
||||||
|
removeUnusedQuarantinedByAddress(after)
|
||||||
|
removeUnusedQuarantinedByUid(after)
|
||||||
|
}
|
||||||
|
|
||||||
|
@tailrec private def removeUnusedQuarantinedByAddress(after: FiniteDuration): Unit = {
|
||||||
|
val now = System.nanoTime()
|
||||||
|
val afterNanos = after.toNanos
|
||||||
|
val currentMap = associationsByAddress.get
|
||||||
|
val remove = currentMap.foldLeft(Map.empty[Address, Association]) {
|
||||||
|
case (acc, (address, association)) ⇒
|
||||||
|
val state = association.associationState
|
||||||
|
if (state.isQuarantined() && ((now - state.lastUsedTimestamp.get) >= afterNanos))
|
||||||
|
acc.updated(address, association)
|
||||||
|
else
|
||||||
|
acc
|
||||||
|
}
|
||||||
|
if (remove.nonEmpty) {
|
||||||
|
val newMap = currentMap -- remove.keysIterator
|
||||||
|
if (associationsByAddress.compareAndSet(currentMap, newMap))
|
||||||
|
remove.valuesIterator.foreach(_.removedAfterQuarantined())
|
||||||
|
else
|
||||||
|
removeUnusedQuarantinedByAddress(after) // CAS fail, recursive
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@tailrec private def removeUnusedQuarantinedByUid(after: FiniteDuration): Unit = {
|
||||||
|
val now = System.nanoTime()
|
||||||
|
val afterNanos = after.toNanos
|
||||||
|
val currentMap = associationsByUid.get
|
||||||
|
var remove = Map.empty[Long, Association]
|
||||||
|
currentMap.keysIterator.foreach { uid ⇒
|
||||||
|
val association = currentMap.get(uid).get
|
||||||
|
val state = association.associationState
|
||||||
|
if (state.isQuarantined() && ((now - state.lastUsedTimestamp.get) >= afterNanos))
|
||||||
|
remove = remove.updated(uid, association)
|
||||||
|
}
|
||||||
|
if (remove.nonEmpty) {
|
||||||
|
val newMap = remove.keysIterator.foldLeft(currentMap)((acc, uid) ⇒ acc.remove(uid))
|
||||||
|
if (associationsByUid.compareAndSet(currentMap, newMap))
|
||||||
|
remove.valuesIterator.foreach(_.removedAfterQuarantined())
|
||||||
|
else
|
||||||
|
removeUnusedQuarantinedByUid(after) // CAS fail, recursive
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -35,10 +35,6 @@ private[remote] object Encoder {
|
||||||
def changeClassManifestCompression(table: CompressionTable[String]): Future[Done]
|
def changeClassManifestCompression(table: CompressionTable[String]): Future[Done]
|
||||||
def clearCompression(): Future[Done]
|
def clearCompression(): Future[Done]
|
||||||
}
|
}
|
||||||
|
|
||||||
private[remote] class AccessOutboundCompressionFailed
|
|
||||||
extends RuntimeException("Change of outbound compression table failed (will be retried), because materialization did not complete yet")
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -71,22 +67,17 @@ private[remote] class Encoder(
|
||||||
|
|
||||||
private val instruments: RemoteInstruments = RemoteInstruments(system)
|
private val instruments: RemoteInstruments = RemoteInstruments(system)
|
||||||
|
|
||||||
private val changeActorRefCompressionCb = getAsyncCallback[(CompressionTable[ActorRef], Promise[Done])] {
|
private val changeActorRefCompressionCb = getAsyncCallback[CompressionTable[ActorRef]] { table ⇒
|
||||||
case (table, done) ⇒
|
headerBuilder.setOutboundActorRefCompression(table)
|
||||||
headerBuilder.setOutboundActorRefCompression(table)
|
|
||||||
done.success(Done)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private val changeClassManifsetCompressionCb = getAsyncCallback[(CompressionTable[String], Promise[Done])] {
|
private val changeClassManifsetCompressionCb = getAsyncCallback[CompressionTable[String]] { table ⇒
|
||||||
case (table, done) ⇒
|
headerBuilder.setOutboundClassManifestCompression(table)
|
||||||
headerBuilder.setOutboundClassManifestCompression(table)
|
|
||||||
done.success(Done)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private val clearCompressionCb = getAsyncCallback[Promise[Done]] { done ⇒
|
private val clearCompressionCb = getAsyncCallback[Unit] { _ ⇒
|
||||||
headerBuilder.setOutboundActorRefCompression(CompressionTable.empty[ActorRef])
|
headerBuilder.setOutboundActorRefCompression(CompressionTable.empty[ActorRef])
|
||||||
headerBuilder.setOutboundClassManifestCompression(CompressionTable.empty[String])
|
headerBuilder.setOutboundClassManifestCompression(CompressionTable.empty[String])
|
||||||
done.success(Done)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override protected def logSource = classOf[Encoder]
|
override protected def logSource = classOf[Encoder]
|
||||||
|
|
@ -177,40 +168,20 @@ private[remote] class Encoder(
|
||||||
/**
|
/**
|
||||||
* External call from ChangeOutboundCompression materialized value
|
* External call from ChangeOutboundCompression materialized value
|
||||||
*/
|
*/
|
||||||
override def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = {
|
override def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] =
|
||||||
val done = Promise[Done]()
|
changeActorRefCompressionCb.invokeWithFeedback(table)
|
||||||
try changeActorRefCompressionCb.invoke((table, done)) catch {
|
|
||||||
// This is a harmless failure, it will be retried on next advertisement or handshake attempt.
|
|
||||||
// It will only occur when callback is invoked before preStart. That is highly unlikely to
|
|
||||||
// happen since advertisement is not done immediately and handshake involves network roundtrip.
|
|
||||||
case NonFatal(_) ⇒ done.tryFailure(new AccessOutboundCompressionFailed)
|
|
||||||
}
|
|
||||||
done.future
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* External call from ChangeOutboundCompression materialized value
|
* External call from ChangeOutboundCompression materialized value
|
||||||
*/
|
*/
|
||||||
override def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = {
|
override def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] =
|
||||||
val done = Promise[Done]()
|
changeClassManifsetCompressionCb.invokeWithFeedback(table)
|
||||||
try changeClassManifsetCompressionCb.invoke((table, done)) catch {
|
|
||||||
// in case materialization not completed yet
|
|
||||||
case NonFatal(_) ⇒ done.tryFailure(new AccessOutboundCompressionFailed)
|
|
||||||
}
|
|
||||||
done.future
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* External call from ChangeOutboundCompression materialized value
|
* External call from ChangeOutboundCompression materialized value
|
||||||
*/
|
*/
|
||||||
override def clearCompression(): Future[Done] = {
|
override def clearCompression(): Future[Done] =
|
||||||
val done = Promise[Done]()
|
clearCompressionCb.invokeWithFeedback(())
|
||||||
try clearCompressionCb.invoke(done) catch {
|
|
||||||
// in case materialization not completed yet
|
|
||||||
case NonFatal(_) ⇒ done.tryFailure(new AccessOutboundCompressionFailed)
|
|
||||||
}
|
|
||||||
done.future
|
|
||||||
}
|
|
||||||
|
|
||||||
setHandlers(in, out, this)
|
setHandlers(in, out, this)
|
||||||
}
|
}
|
||||||
|
|
@ -240,6 +211,9 @@ private[remote] object Decoder {
|
||||||
def runNextActorRefAdvertisement(): Unit
|
def runNextActorRefAdvertisement(): Unit
|
||||||
/** For testing purposes, usually triggered by timer from within Decoder stage. */
|
/** For testing purposes, usually triggered by timer from within Decoder stage. */
|
||||||
def runNextClassManifestAdvertisement(): Unit
|
def runNextClassManifestAdvertisement(): Unit
|
||||||
|
/** For testing purposes */
|
||||||
|
def currentCompressionOriginUids: Future[Set[Long]]
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private[remote] trait InboundCompressionAccessImpl extends InboundCompressionAccess {
|
private[remote] trait InboundCompressionAccessImpl extends InboundCompressionAccess {
|
||||||
|
|
@ -247,20 +221,16 @@ private[remote] object Decoder {
|
||||||
|
|
||||||
def compressions: InboundCompressions
|
def compressions: InboundCompressions
|
||||||
|
|
||||||
private val closeCompressionForCb = getAsyncCallback[(Long, Promise[Done])] {
|
private val closeCompressionForCb = getAsyncCallback[Long] { uid ⇒
|
||||||
case (uid, done) ⇒
|
compressions.close(uid)
|
||||||
compressions.close(uid)
|
|
||||||
done.success(Done)
|
|
||||||
}
|
}
|
||||||
private val confirmActorRefCompressionAdvertisementCb = getAsyncCallback[(ActorRefCompressionAdvertisementAck, Promise[Done])] {
|
private val confirmActorRefCompressionAdvertisementCb = getAsyncCallback[ActorRefCompressionAdvertisementAck] {
|
||||||
case (ActorRefCompressionAdvertisementAck(from, tableVersion), done) ⇒
|
case ActorRefCompressionAdvertisementAck(from, tableVersion) ⇒
|
||||||
compressions.confirmActorRefCompressionAdvertisement(from.uid, tableVersion)
|
compressions.confirmActorRefCompressionAdvertisement(from.uid, tableVersion)
|
||||||
done.success(Done)
|
|
||||||
}
|
}
|
||||||
private val confirmClassManifestCompressionAdvertisementCb = getAsyncCallback[(ClassManifestCompressionAdvertisementAck, Promise[Done])] {
|
private val confirmClassManifestCompressionAdvertisementCb = getAsyncCallback[ClassManifestCompressionAdvertisementAck] {
|
||||||
case (ClassManifestCompressionAdvertisementAck(from, tableVersion), done) ⇒
|
case ClassManifestCompressionAdvertisementAck(from, tableVersion) ⇒
|
||||||
compressions.confirmClassManifestCompressionAdvertisement(from.uid, tableVersion)
|
compressions.confirmClassManifestCompressionAdvertisement(from.uid, tableVersion)
|
||||||
done.success(Done)
|
|
||||||
}
|
}
|
||||||
private val runNextActorRefAdvertisementCb = getAsyncCallback[Unit] {
|
private val runNextActorRefAdvertisementCb = getAsyncCallback[Unit] {
|
||||||
_ ⇒ compressions.runNextActorRefAdvertisement()
|
_ ⇒ compressions.runNextActorRefAdvertisement()
|
||||||
|
|
@ -268,55 +238,49 @@ private[remote] object Decoder {
|
||||||
private val runNextClassManifestAdvertisementCb = getAsyncCallback[Unit] {
|
private val runNextClassManifestAdvertisementCb = getAsyncCallback[Unit] {
|
||||||
_ ⇒ compressions.runNextClassManifestAdvertisement()
|
_ ⇒ compressions.runNextClassManifestAdvertisement()
|
||||||
}
|
}
|
||||||
|
private val currentCompressionOriginUidsCb = getAsyncCallback[Promise[Set[Long]]] { p ⇒
|
||||||
// TODO in practice though all those CB's will always succeed, no need for the futures etc IMO
|
p.success(compressions.currentOriginUids)
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* External call from ChangeInboundCompression materialized value
|
* External call from ChangeInboundCompression materialized value
|
||||||
*/
|
*/
|
||||||
override def closeCompressionFor(originUid: Long): Future[Done] = {
|
override def closeCompressionFor(originUid: Long): Future[Done] =
|
||||||
val done = Promise[Done]()
|
closeCompressionForCb.invokeWithFeedback(originUid)
|
||||||
try closeCompressionForCb.invoke((originUid, done)) catch {
|
|
||||||
// in case materialization not completed yet
|
|
||||||
case NonFatal(_) ⇒ done.tryFailure(new AccessInboundCompressionFailed)
|
|
||||||
}
|
|
||||||
done.future
|
|
||||||
}
|
|
||||||
/**
|
/**
|
||||||
* External call from ChangeInboundCompression materialized value
|
* External call from ChangeInboundCompression materialized value
|
||||||
*/
|
*/
|
||||||
override def confirmActorRefCompressionAdvertisementAck(ack: ActorRefCompressionAdvertisementAck): Future[Done] = {
|
override def confirmActorRefCompressionAdvertisementAck(ack: ActorRefCompressionAdvertisementAck): Future[Done] =
|
||||||
val done = Promise[Done]()
|
confirmActorRefCompressionAdvertisementCb.invokeWithFeedback(ack)
|
||||||
try confirmActorRefCompressionAdvertisementCb.invoke((ack, done)) catch {
|
|
||||||
// in case materialization not completed yet
|
|
||||||
case NonFatal(_) ⇒ done.tryFailure(new AccessInboundCompressionFailed)
|
|
||||||
}
|
|
||||||
done.future
|
|
||||||
}
|
|
||||||
/**
|
/**
|
||||||
* External call from ChangeInboundCompression materialized value
|
* External call from ChangeInboundCompression materialized value
|
||||||
*/
|
*/
|
||||||
override def confirmClassManifestCompressionAdvertisementAck(ack: ClassManifestCompressionAdvertisementAck): Future[Done] = {
|
override def confirmClassManifestCompressionAdvertisementAck(ack: ClassManifestCompressionAdvertisementAck): Future[Done] =
|
||||||
val done = Promise[Done]()
|
confirmClassManifestCompressionAdvertisementCb.invokeWithFeedback(ack)
|
||||||
try confirmClassManifestCompressionAdvertisementCb.invoke((ack, done)) catch {
|
|
||||||
case NonFatal(_) ⇒ done.tryFailure(new AccessInboundCompressionFailed)
|
|
||||||
}
|
|
||||||
done.future
|
|
||||||
}
|
|
||||||
/**
|
/**
|
||||||
* External call from ChangeInboundCompression materialized value
|
* External call from ChangeInboundCompression materialized value
|
||||||
*/
|
*/
|
||||||
override def runNextActorRefAdvertisement(): Unit =
|
override def runNextActorRefAdvertisement(): Unit =
|
||||||
runNextActorRefAdvertisementCb.invoke(())
|
runNextActorRefAdvertisementCb.invoke(())
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* External call from ChangeInboundCompression materialized value
|
* External call from ChangeInboundCompression materialized value
|
||||||
*/
|
*/
|
||||||
override def runNextClassManifestAdvertisement(): Unit =
|
override def runNextClassManifestAdvertisement(): Unit =
|
||||||
runNextClassManifestAdvertisementCb.invoke(())
|
runNextClassManifestAdvertisementCb.invoke(())
|
||||||
}
|
|
||||||
|
|
||||||
private[remote] class AccessInboundCompressionFailed
|
/**
|
||||||
extends RuntimeException("Change of inbound compression table failed (will be retried), because materialization did not complete yet")
|
* External call from ChangeInboundCompression materialized value
|
||||||
|
*/
|
||||||
|
override def currentCompressionOriginUids: Future[Set[Long]] = {
|
||||||
|
val p = Promise[Set[Long]]
|
||||||
|
currentCompressionOriginUidsCb.invoke(p)
|
||||||
|
p.future
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// timer keys
|
// timer keys
|
||||||
private case object AdvertiseActorRefsCompressionTable
|
private case object AdvertiseActorRefsCompressionTable
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,8 @@ import java.util.ArrayDeque
|
||||||
|
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
import scala.concurrent.Promise
|
import scala.concurrent.Promise
|
||||||
|
import scala.util.Try
|
||||||
|
|
||||||
import akka.Done
|
import akka.Done
|
||||||
import akka.stream.Attributes
|
import akka.stream.Attributes
|
||||||
import akka.stream.FlowShape
|
import akka.stream.FlowShape
|
||||||
|
|
@ -61,7 +63,6 @@ private[remote] object InboundControlJunction {
|
||||||
private[remote] trait ControlMessageSubject {
|
private[remote] trait ControlMessageSubject {
|
||||||
def attach(observer: ControlMessageObserver): Future[Done]
|
def attach(observer: ControlMessageObserver): Future[Done]
|
||||||
def detach(observer: ControlMessageObserver): Unit
|
def detach(observer: ControlMessageObserver): Unit
|
||||||
def stopped: Future[Done]
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private[remote] trait ControlMessageObserver {
|
private[remote] trait ControlMessageObserver {
|
||||||
|
|
@ -71,6 +72,8 @@ private[remote] object InboundControlJunction {
|
||||||
* of the envelope is always a `ControlMessage`.
|
* of the envelope is always a `ControlMessage`.
|
||||||
*/
|
*/
|
||||||
def notify(inboundEnvelope: InboundEnvelope): Unit
|
def notify(inboundEnvelope: InboundEnvelope): Unit
|
||||||
|
|
||||||
|
def controlSubjectCompleted(signal: Try[Done]): Unit
|
||||||
}
|
}
|
||||||
|
|
||||||
// messages for the stream callback
|
// messages for the stream callback
|
||||||
|
|
@ -92,7 +95,6 @@ private[remote] class InboundControlJunction
|
||||||
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) = {
|
||||||
val stoppedPromise = Promise[Done]()
|
|
||||||
val logic = new GraphStageLogic(shape) with InHandler with OutHandler with ControlMessageSubject {
|
val logic = new GraphStageLogic(shape) with InHandler with OutHandler with ControlMessageSubject {
|
||||||
|
|
||||||
private var observers: Vector[ControlMessageObserver] = Vector.empty
|
private var observers: Vector[ControlMessageObserver] = Vector.empty
|
||||||
|
|
@ -105,7 +107,10 @@ private[remote] class InboundControlJunction
|
||||||
observers = observers.filterNot(_ == observer)
|
observers = observers.filterNot(_ == observer)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def postStop(): Unit = stoppedPromise.success(Done)
|
override def postStop(): Unit = {
|
||||||
|
observers.foreach(_.controlSubjectCompleted(Try(Done)))
|
||||||
|
observers = Vector.empty
|
||||||
|
}
|
||||||
|
|
||||||
// InHandler
|
// InHandler
|
||||||
override def onPush(): Unit = {
|
override def onPush(): Unit = {
|
||||||
|
|
@ -133,8 +138,6 @@ private[remote] class InboundControlJunction
|
||||||
override def detach(observer: ControlMessageObserver): Unit =
|
override def detach(observer: ControlMessageObserver): Unit =
|
||||||
callback.invoke(Dettach(observer))
|
callback.invoke(Dettach(observer))
|
||||||
|
|
||||||
override def stopped: Future[Done] =
|
|
||||||
stoppedPromise.future
|
|
||||||
}
|
}
|
||||||
|
|
||||||
(logic, logic)
|
(logic, logic)
|
||||||
|
|
|
||||||
|
|
@ -24,6 +24,11 @@ private[remote] object FlightRecorderEvents {
|
||||||
val Transport_MediaFileDeleted = 11
|
val Transport_MediaFileDeleted = 11
|
||||||
val Transport_FlightRecorderClose = 12
|
val Transport_FlightRecorderClose = 12
|
||||||
val Transport_SendQueueOverflow = 13
|
val Transport_SendQueueOverflow = 13
|
||||||
|
val Transport_StopIdleOutbound = 14
|
||||||
|
val Transport_Quarantined = 15
|
||||||
|
val Transport_RemovedQuarantined = 16
|
||||||
|
val Transport_RestartOutbound = 17
|
||||||
|
val Transport_RestartInbound = 18
|
||||||
|
|
||||||
// Aeron Sink events
|
// Aeron Sink events
|
||||||
val AeronSink_Started = 50
|
val AeronSink_Started = 50
|
||||||
|
|
@ -75,6 +80,11 @@ private[remote] object FlightRecorderEvents {
|
||||||
Transport_MediaFileDeleted → "Transport: Media file deleted",
|
Transport_MediaFileDeleted → "Transport: Media file deleted",
|
||||||
Transport_FlightRecorderClose → "Transport: Flight recorder closed",
|
Transport_FlightRecorderClose → "Transport: Flight recorder closed",
|
||||||
Transport_SendQueueOverflow → "Transport: Send queue overflow",
|
Transport_SendQueueOverflow → "Transport: Send queue overflow",
|
||||||
|
Transport_StopIdleOutbound -> "Transport: Remove idle outbound",
|
||||||
|
Transport_Quarantined -> "Transport: Quarantined association",
|
||||||
|
Transport_RemovedQuarantined -> "Transport: Removed idle quarantined association",
|
||||||
|
Transport_RestartOutbound -> "Transport: Restart outbound",
|
||||||
|
Transport_RestartInbound -> "Transport: Restart outbound",
|
||||||
|
|
||||||
// Aeron Sink events
|
// Aeron Sink events
|
||||||
AeronSink_Started → "AeronSink: Started",
|
AeronSink_Started → "AeronSink: Started",
|
||||||
|
|
|
||||||
|
|
@ -132,6 +132,7 @@ private[remote] class OutboundHandshake(
|
||||||
scheduleOnce(InjectHandshakeTick, injectHandshakeInterval)
|
scheduleOnce(InjectHandshakeTick, injectHandshakeInterval)
|
||||||
val env: OutboundEnvelope = outboundEnvelopePool.acquire().init(
|
val env: OutboundEnvelope = outboundEnvelopePool.acquire().init(
|
||||||
recipient = OptionVal.None, message = HandshakeReq(outboundContext.localAddress, outboundContext.remoteAddress), sender = OptionVal.None)
|
recipient = OptionVal.None, message = HandshakeReq(outboundContext.localAddress, outboundContext.remoteAddress), sender = OptionVal.None)
|
||||||
|
outboundContext.associationState.lastUsedTimestamp.set(System.nanoTime())
|
||||||
push(out, env)
|
push(out, env)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -47,7 +47,8 @@ private[remote] object SendQueue {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[remote] final class SendQueue[T](deadLetters: ActorRef) extends GraphStageWithMaterializedValue[SourceShape[T], SendQueue.QueueValue[T]] {
|
private[remote] final class SendQueue[T](postStopAction: Vector[T] ⇒ Unit)
|
||||||
|
extends GraphStageWithMaterializedValue[SourceShape[T], SendQueue.QueueValue[T]] {
|
||||||
import SendQueue._
|
import SendQueue._
|
||||||
|
|
||||||
val out: Outlet[T] = Outlet("SendQueue.out")
|
val out: Outlet[T] = Outlet("SendQueue.out")
|
||||||
|
|
@ -105,15 +106,17 @@ private[remote] final class SendQueue[T](deadLetters: ActorRef) extends GraphSta
|
||||||
}
|
}
|
||||||
|
|
||||||
override def postStop(): Unit = {
|
override def postStop(): Unit = {
|
||||||
// TODO quarantine will currently always be done when control stream is terminated, see issue #21359
|
var pending = Vector.newBuilder[T]
|
||||||
if (consumerQueue ne null) {
|
if (consumerQueue ne null) {
|
||||||
var msg = consumerQueue.poll()
|
var msg = consumerQueue.poll()
|
||||||
while (msg != null) {
|
while (msg != null) {
|
||||||
deadLetters ! msg
|
pending += msg
|
||||||
msg = consumerQueue.poll()
|
msg = consumerQueue.poll()
|
||||||
}
|
}
|
||||||
consumerQueue.clear()
|
consumerQueue.clear()
|
||||||
}
|
}
|
||||||
|
postStopAction(pending.result())
|
||||||
|
|
||||||
super.postStop()
|
super.postStop()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -31,7 +31,6 @@ import scala.util.control.NoStackTrace
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[remote] object SystemMessageDelivery {
|
private[remote] object SystemMessageDelivery {
|
||||||
// FIXME serialization of these messages
|
|
||||||
final case class SystemMessageEnvelope(message: AnyRef, seqNo: Long, ackReplyTo: UniqueAddress) extends ArteryMessage
|
final case class SystemMessageEnvelope(message: AnyRef, seqNo: Long, ackReplyTo: UniqueAddress) extends ArteryMessage
|
||||||
final case class Ack(seqNo: Long, from: UniqueAddress) extends Reply
|
final case class Ack(seqNo: Long, from: UniqueAddress) extends Reply
|
||||||
final case class Nack(seqNo: Long, from: UniqueAddress) extends Reply
|
final case class Nack(seqNo: Long, from: UniqueAddress) extends Reply
|
||||||
|
|
@ -89,19 +88,14 @@ private[remote] class SystemMessageDelivery(
|
||||||
pull(in) // onPull from downstream already called
|
pull(in) // onPull from downstream already called
|
||||||
}.invoke
|
}.invoke
|
||||||
}
|
}
|
||||||
|
|
||||||
outboundContext.controlSubject.stopped.onComplete {
|
|
||||||
getAsyncCallback[Try[Done]] {
|
|
||||||
case Success(_) ⇒ completeStage()
|
|
||||||
case Failure(cause) ⇒ failStage(cause)
|
|
||||||
}.invoke
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override def postStop(): Unit = {
|
override def postStop(): Unit = {
|
||||||
// TODO quarantine will currently always be done when control stream is terminated, see issue #21359
|
val pendingCount = unacknowledged.size
|
||||||
sendUnacknowledgedToDeadLetters()
|
sendUnacknowledgedToDeadLetters()
|
||||||
unacknowledged.clear()
|
unacknowledged.clear()
|
||||||
|
if (pendingCount > 0)
|
||||||
|
outboundContext.quarantine(s"SystemMessageDelivery stopped with [$pendingCount] pending system messages.")
|
||||||
outboundContext.controlSubject.detach(this)
|
outboundContext.controlSubject.detach(this)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -133,6 +127,14 @@ private[remote] class SystemMessageDelivery(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ControlMessageObserver, external call
|
||||||
|
override def controlSubjectCompleted(signal: Try[Done]): Unit = {
|
||||||
|
getAsyncCallback[Try[Done]] {
|
||||||
|
case Success(_) ⇒ completeStage()
|
||||||
|
case Failure(cause) ⇒ failStage(cause)
|
||||||
|
}.invoke(signal)
|
||||||
|
}
|
||||||
|
|
||||||
private val ackCallback = getAsyncCallback[Ack] { reply ⇒
|
private val ackCallback = getAsyncCallback[Ack] { reply ⇒
|
||||||
ack(reply.seqNo)
|
ack(reply.seqNo)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -24,6 +24,7 @@ import akka.remote.RemoteActorRefProvider
|
||||||
import akka.remote.RemoteTransportException
|
import akka.remote.RemoteTransportException
|
||||||
import akka.remote.artery.compress._
|
import akka.remote.artery.compress._
|
||||||
import akka.stream.KillSwitches
|
import akka.stream.KillSwitches
|
||||||
|
import akka.stream.SharedKillSwitch
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.scaladsl.Keep
|
import akka.stream.scaladsl.Keep
|
||||||
import akka.stream.scaladsl.Sink
|
import akka.stream.scaladsl.Sink
|
||||||
|
|
@ -72,12 +73,12 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro
|
||||||
startMediaDriver()
|
startMediaDriver()
|
||||||
startAeron()
|
startAeron()
|
||||||
startAeronErrorLog()
|
startAeronErrorLog()
|
||||||
topLevelFREvents.loFreq(Transport_AeronErrorLogStarted, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_AeronErrorLogStarted, NoMetaData)
|
||||||
if (settings.LogAeronCounters) {
|
if (settings.LogAeronCounters) {
|
||||||
startAeronCounterLog()
|
startAeronCounterLog()
|
||||||
}
|
}
|
||||||
taskRunner.start()
|
taskRunner.start()
|
||||||
topLevelFREvents.loFreq(Transport_TaskRunnerStarted, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_TaskRunnerStarted, NoMetaData)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def startMediaDriver(): Unit = {
|
private def startMediaDriver(): Unit = {
|
||||||
|
|
@ -119,7 +120,7 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro
|
||||||
|
|
||||||
val driver = MediaDriver.launchEmbedded(driverContext)
|
val driver = MediaDriver.launchEmbedded(driverContext)
|
||||||
log.info("Started embedded media driver in directory [{}]", driver.aeronDirectoryName)
|
log.info("Started embedded media driver in directory [{}]", driver.aeronDirectoryName)
|
||||||
topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName())
|
topLevelFlightRecorder.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName())
|
||||||
if (!mediaDriver.compareAndSet(None, Some(driver))) {
|
if (!mediaDriver.compareAndSet(None, Some(driver))) {
|
||||||
throw new IllegalStateException("media driver started more than once")
|
throw new IllegalStateException("media driver started more than once")
|
||||||
}
|
}
|
||||||
|
|
@ -145,7 +146,7 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro
|
||||||
try {
|
try {
|
||||||
if (settings.Advanced.DeleteAeronDirectory) {
|
if (settings.Advanced.DeleteAeronDirectory) {
|
||||||
IoUtil.delete(new File(driver.aeronDirectoryName), false)
|
IoUtil.delete(new File(driver.aeronDirectoryName), false)
|
||||||
topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_MediaFileDeleted, NoMetaData)
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case NonFatal(e) ⇒
|
case NonFatal(e) ⇒
|
||||||
|
|
@ -285,11 +286,17 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override protected def outboundTransportSink(outboundContext: OutboundContext, streamId: Int,
|
override protected def outboundTransportSink(
|
||||||
bufferPool: EnvelopeBufferPool): Sink[EnvelopeBuffer, Future[Done]] = {
|
outboundContext: OutboundContext,
|
||||||
|
streamId: Int,
|
||||||
|
bufferPool: EnvelopeBufferPool): Sink[EnvelopeBuffer, Future[Done]] = {
|
||||||
val giveUpAfter =
|
val giveUpAfter =
|
||||||
if (streamId == ControlStreamId) settings.Advanced.GiveUpSystemMessageAfter
|
if (streamId == ControlStreamId) settings.Advanced.GiveUpSystemMessageAfter
|
||||||
else settings.Advanced.GiveUpMessageAfter
|
else settings.Advanced.GiveUpMessageAfter
|
||||||
|
// TODO: Note that the AssociationState.controlStreamIdleKillSwitch in control stream is not used for the
|
||||||
|
// Aeron transport. Would be difficult to handle the Future[Done] materialized value.
|
||||||
|
// If we want to stop for Aeron also it is probably easier to stop the publication inside the
|
||||||
|
// AeronSink, i.e. not using a KillSwitch.
|
||||||
Sink.fromGraph(new AeronSink(outboundChannel(outboundContext.remoteAddress), streamId, aeron, taskRunner,
|
Sink.fromGraph(new AeronSink(outboundChannel(outboundContext.remoteAddress), streamId, aeron, taskRunner,
|
||||||
bufferPool, giveUpAfter, createFlightRecorderEventSink()))
|
bufferPool, giveUpAfter, createFlightRecorderEventSink()))
|
||||||
}
|
}
|
||||||
|
|
@ -395,10 +402,10 @@ private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _pro
|
||||||
override protected def shutdownTransport(): Future[Done] = {
|
override protected def shutdownTransport(): Future[Done] = {
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
taskRunner.stop().map { _ ⇒
|
taskRunner.stop().map { _ ⇒
|
||||||
topLevelFREvents.loFreq(Transport_Stopped, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_Stopped, NoMetaData)
|
||||||
if (aeronErrorLogTask != null) {
|
if (aeronErrorLogTask != null) {
|
||||||
aeronErrorLogTask.cancel()
|
aeronErrorLogTask.cancel()
|
||||||
topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData)
|
||||||
}
|
}
|
||||||
if (aeron != null) aeron.close()
|
if (aeron != null) aeron.close()
|
||||||
if (aeronErrorLog != null) aeronErrorLog.close()
|
if (aeronErrorLog != null) aeronErrorLog.close()
|
||||||
|
|
|
||||||
|
|
@ -4,19 +4,19 @@
|
||||||
|
|
||||||
package akka.remote.artery.compress
|
package akka.remote.artery.compress
|
||||||
|
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
import java.util.function.LongFunction
|
||||||
import java.util.function.{ Consumer, LongFunction }
|
|
||||||
|
|
||||||
import akka.actor.{ ActorRef, ActorSystem, Address }
|
import scala.annotation.tailrec
|
||||||
import akka.event.{ Logging, LoggingAdapter }
|
|
||||||
|
import akka.actor.ActorRef
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
import akka.actor.Address
|
||||||
|
import akka.event.Logging
|
||||||
|
import akka.event.LoggingAdapter
|
||||||
import akka.remote.artery._
|
import akka.remote.artery._
|
||||||
import akka.util.OptionVal
|
import akka.util.OptionVal
|
||||||
import org.agrona.collections.Long2ObjectHashMap
|
import org.agrona.collections.Long2ObjectHashMap
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
* Decompress and cause compression advertisements.
|
* Decompress and cause compression advertisements.
|
||||||
|
|
@ -36,6 +36,8 @@ private[remote] trait InboundCompressions {
|
||||||
/** Triggers compression advertisement via control message. */
|
/** Triggers compression advertisement via control message. */
|
||||||
def runNextClassManifestAdvertisement(): Unit
|
def runNextClassManifestAdvertisement(): Unit
|
||||||
|
|
||||||
|
def currentOriginUids: Set[Long]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Remove compression and cancel advertisement scheduling for a specific origin
|
* Remove compression and cancel advertisement scheduling for a specific origin
|
||||||
*/
|
*/
|
||||||
|
|
@ -47,6 +49,7 @@ private[remote] trait InboundCompressions {
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*
|
*
|
||||||
* One per incoming Aeron stream, actual compression tables are kept per-originUid and created on demand.
|
* One per incoming Aeron stream, actual compression tables are kept per-originUid and created on demand.
|
||||||
|
* All access is via the Decoder stage.
|
||||||
*/
|
*/
|
||||||
private[remote] final class InboundCompressionsImpl(
|
private[remote] final class InboundCompressionsImpl(
|
||||||
system: ActorSystem,
|
system: ActorSystem,
|
||||||
|
|
@ -54,101 +57,104 @@ private[remote] final class InboundCompressionsImpl(
|
||||||
settings: ArterySettings.Compression,
|
settings: ArterySettings.Compression,
|
||||||
eventSink: EventSink = IgnoreEventSink) extends InboundCompressions {
|
eventSink: EventSink = IgnoreEventSink) extends InboundCompressions {
|
||||||
|
|
||||||
// None is used as tombstone value after closed
|
private[this] val _actorRefsIns = new Long2ObjectHashMap[InboundActorRefCompression]()
|
||||||
// TODO would be nice if we can cleanup the tombstones
|
|
||||||
// FIXME we should be able to remove the tombstones easily now
|
|
||||||
private[this] val _actorRefsIns = new Long2ObjectHashMap[Option[InboundActorRefCompression]]()
|
|
||||||
private[this] val _inboundActorRefsLog = Logging(system, classOf[InboundActorRefCompression])
|
private[this] val _inboundActorRefsLog = Logging(system, classOf[InboundActorRefCompression])
|
||||||
private val createInboundActorRefsForOrigin = new LongFunction[Option[InboundActorRefCompression]] {
|
private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] {
|
||||||
override def apply(originUid: Long): Option[InboundActorRefCompression] = {
|
override def apply(originUid: Long): InboundActorRefCompression = {
|
||||||
val actorRefHitters = new TopHeavyHitters[ActorRef](settings.ActorRefs.Max)
|
val actorRefHitters = new TopHeavyHitters[ActorRef](settings.ActorRefs.Max)
|
||||||
Some(new InboundActorRefCompression(_inboundActorRefsLog, settings, originUid, inboundContext, actorRefHitters))
|
new InboundActorRefCompression(_inboundActorRefsLog, settings, originUid, inboundContext, actorRefHitters)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
private def actorRefsIn(originUid: Long): Option[InboundActorRefCompression] =
|
private def actorRefsIn(originUid: Long): InboundActorRefCompression =
|
||||||
_actorRefsIns.computeIfAbsent(originUid, createInboundActorRefsForOrigin)
|
_actorRefsIns.computeIfAbsent(originUid, createInboundActorRefsForOrigin)
|
||||||
|
|
||||||
// None is used as tombstone value after closed
|
private[this] val _classManifestsIns = new Long2ObjectHashMap[InboundManifestCompression]()
|
||||||
private[this] val _classManifestsIns = new Long2ObjectHashMap[Option[InboundManifestCompression]]()
|
|
||||||
|
|
||||||
private[this] val _inboundManifestLog = Logging(system, classOf[InboundManifestCompression])
|
private[this] val _inboundManifestLog = Logging(system, classOf[InboundManifestCompression])
|
||||||
private val createInboundManifestsForOrigin = new LongFunction[Option[InboundManifestCompression]] {
|
private val createInboundManifestsForOrigin = new LongFunction[InboundManifestCompression] {
|
||||||
override def apply(originUid: Long): Option[InboundManifestCompression] = {
|
override def apply(originUid: Long): InboundManifestCompression = {
|
||||||
val manifestHitters = new TopHeavyHitters[String](settings.Manifests.Max)
|
val manifestHitters = new TopHeavyHitters[String](settings.Manifests.Max)
|
||||||
Some(new InboundManifestCompression(_inboundManifestLog, settings, originUid, inboundContext, manifestHitters))
|
new InboundManifestCompression(_inboundManifestLog, settings, originUid, inboundContext, manifestHitters)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
private def classManifestsIn(originUid: Long): Option[InboundManifestCompression] =
|
private def classManifestsIn(originUid: Long): InboundManifestCompression =
|
||||||
_classManifestsIns.computeIfAbsent(originUid, createInboundManifestsForOrigin)
|
_classManifestsIns.computeIfAbsent(originUid, createInboundManifestsForOrigin)
|
||||||
|
|
||||||
// actor ref compression ---
|
// actor ref compression ---
|
||||||
|
|
||||||
override def decompressActorRef(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[ActorRef] =
|
override def decompressActorRef(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[ActorRef] =
|
||||||
actorRefsIn(originUid) match {
|
actorRefsIn(originUid).decompress(tableVersion, idx)
|
||||||
case Some(a) ⇒ a.decompress(tableVersion, idx)
|
|
||||||
case None ⇒ OptionVal.None
|
|
||||||
}
|
|
||||||
override def hitActorRef(originUid: Long, address: Address, ref: ActorRef, n: Int): Unit = {
|
override def hitActorRef(originUid: Long, address: Address, ref: ActorRef, n: Int): Unit = {
|
||||||
if (ArterySettings.Compression.Debug) println(s"[compress] hitActorRef($originUid, $address, $ref, $n)")
|
if (ArterySettings.Compression.Debug) println(s"[compress] hitActorRef($originUid, $address, $ref, $n)")
|
||||||
actorRefsIn(originUid) match {
|
actorRefsIn(originUid).increment(address, ref, n)
|
||||||
case Some(a) ⇒ a.increment(address, ref, n)
|
|
||||||
case None ⇒ // closed
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = {
|
override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = {
|
||||||
_actorRefsIns.get(originUid) match {
|
_actorRefsIns.get(originUid) match {
|
||||||
case null ⇒ // ignore
|
case null ⇒ // ignore
|
||||||
case Some(a) ⇒ a.confirmAdvertisement(tableVersion)
|
case a ⇒ a.confirmAdvertisement(tableVersion)
|
||||||
case None ⇒ // closed
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
/** Send compression table advertisement over control stream. Should be called from Decoder. */
|
/** Send compression table advertisement over control stream. Should be called from Decoder. */
|
||||||
override def runNextActorRefAdvertisement(): Unit = {
|
override def runNextActorRefAdvertisement(): Unit = {
|
||||||
val vs = _actorRefsIns.values.iterator()
|
val vs = _actorRefsIns.values.iterator()
|
||||||
while (vs.hasNext) vs.next() match {
|
var remove = Vector.empty[Long]
|
||||||
case Some(inbound) ⇒
|
while (vs.hasNext) {
|
||||||
eventSink.hiFreq(FlightRecorderEvents.Compression_Inbound_RunActorRefAdvertisement, 1)
|
val inbound = vs.next()
|
||||||
inbound.runNextTableAdvertisement()
|
inboundContext.association(inbound.originUid) match {
|
||||||
case None ⇒ // do nothing...
|
case OptionVal.Some(a) if !a.associationState.isQuarantined(inbound.originUid) ⇒
|
||||||
|
eventSink.hiFreq(FlightRecorderEvents.Compression_Inbound_RunActorRefAdvertisement, inbound.originUid)
|
||||||
|
inbound.runNextTableAdvertisement()
|
||||||
|
case _ ⇒ remove :+= inbound.originUid
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
if (remove.nonEmpty) remove.foreach(close)
|
||||||
}
|
}
|
||||||
|
|
||||||
// class manifest compression ---
|
// class manifest compression ---
|
||||||
|
|
||||||
override def decompressClassManifest(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[String] =
|
override def decompressClassManifest(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[String] =
|
||||||
classManifestsIn(originUid) match {
|
classManifestsIn(originUid).decompress(tableVersion, idx)
|
||||||
case Some(a) ⇒ a.decompress(tableVersion, idx)
|
|
||||||
case None ⇒ OptionVal.None
|
|
||||||
}
|
|
||||||
|
|
||||||
override def hitClassManifest(originUid: Long, address: Address, manifest: String, n: Int): Unit = {
|
override def hitClassManifest(originUid: Long, address: Address, manifest: String, n: Int): Unit = {
|
||||||
if (ArterySettings.Compression.Debug) println(s"[compress] hitClassManifest($originUid, $address, $manifest, $n)")
|
if (ArterySettings.Compression.Debug) println(s"[compress] hitClassManifest($originUid, $address, $manifest, $n)")
|
||||||
classManifestsIn(originUid) match {
|
classManifestsIn(originUid).increment(address, manifest, n)
|
||||||
case Some(a) ⇒ a.increment(address, manifest, n)
|
|
||||||
case None ⇒ // closed
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = {
|
override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = {
|
||||||
_classManifestsIns.get(originUid) match {
|
_classManifestsIns.get(originUid) match {
|
||||||
case null ⇒ // ignore
|
case null ⇒ // ignore
|
||||||
case Some(a) ⇒ a.confirmAdvertisement(tableVersion)
|
case a ⇒ a.confirmAdvertisement(tableVersion)
|
||||||
case None ⇒ // closed
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
/** Send compression table advertisement over control stream. Should be called from Decoder. */
|
/** Send compression table advertisement over control stream. Should be called from Decoder. */
|
||||||
override def runNextClassManifestAdvertisement(): Unit = {
|
override def runNextClassManifestAdvertisement(): Unit = {
|
||||||
val vs = _classManifestsIns.values.iterator()
|
val vs = _classManifestsIns.values.iterator()
|
||||||
while (vs.hasNext) vs.next() match {
|
var remove = Vector.empty[Long]
|
||||||
case Some(inbound) ⇒
|
while (vs.hasNext) {
|
||||||
eventSink.hiFreq(FlightRecorderEvents.Compression_Inbound_RunClassManifestAdvertisement, 1)
|
val inbound = vs.next()
|
||||||
inbound.runNextTableAdvertisement()
|
inboundContext.association(inbound.originUid) match {
|
||||||
case None ⇒ // do nothing...
|
case OptionVal.Some(a) if !a.associationState.isQuarantined(inbound.originUid) ⇒
|
||||||
|
eventSink.hiFreq(FlightRecorderEvents.Compression_Inbound_RunClassManifestAdvertisement, inbound.originUid)
|
||||||
|
inbound.runNextTableAdvertisement()
|
||||||
|
case _ ⇒ remove :+= inbound.originUid
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
if (remove.nonEmpty) remove.foreach(close)
|
||||||
|
}
|
||||||
|
|
||||||
|
override def currentOriginUids: Set[Long] = {
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
// can't use union because of java.lang.Long and Scala Long mismatch,
|
||||||
|
// only used for testing so doesn't matter
|
||||||
|
val result = Set.empty[java.lang.Long] ++ _actorRefsIns.keySet.asScala.iterator ++
|
||||||
|
_classManifestsIns.keySet.asScala.iterator
|
||||||
|
result.map(_.longValue)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def close(originUid: Long): Unit = {
|
override def close(originUid: Long): Unit = {
|
||||||
_actorRefsIns.putIfAbsent(originUid, None)
|
_actorRefsIns.remove(originUid)
|
||||||
_classManifestsIns.putIfAbsent(originUid, None)
|
_classManifestsIns.remove(originUid)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -281,7 +287,7 @@ private[remote] object InboundCompression {
|
||||||
private[remote] abstract class InboundCompression[T >: Null](
|
private[remote] abstract class InboundCompression[T >: Null](
|
||||||
val log: LoggingAdapter,
|
val log: LoggingAdapter,
|
||||||
val settings: ArterySettings.Compression,
|
val settings: ArterySettings.Compression,
|
||||||
originUid: Long,
|
val originUid: Long,
|
||||||
inboundContext: InboundContext,
|
inboundContext: InboundContext,
|
||||||
val heavyHitters: TopHeavyHitters[T]) {
|
val heavyHitters: TopHeavyHitters[T]) {
|
||||||
|
|
||||||
|
|
@ -396,10 +402,7 @@ private[remote] abstract class InboundCompression[T >: Null](
|
||||||
case None ⇒
|
case None ⇒
|
||||||
inboundContext.association(originUid) match {
|
inboundContext.association(originUid) match {
|
||||||
case OptionVal.Some(association) ⇒
|
case OptionVal.Some(association) ⇒
|
||||||
if (association.associationState.isQuarantined(originUid)) {
|
if (alive && association.isOrdinaryMessageStreamActive()) {
|
||||||
// FIXME cleanup compresssion for quarantined associations, see #23967
|
|
||||||
log.debug("Ignoring {} for quarantined originUid [{}].", Logging.simpleName(tables.activeTable), originUid)
|
|
||||||
} else if (alive) {
|
|
||||||
val table = prepareCompressionAdvertisement(tables.nextTable.version)
|
val table = prepareCompressionAdvertisement(tables.nextTable.version)
|
||||||
// TODO expensive, check if building the other way wouldn't be faster?
|
// TODO expensive, check if building the other way wouldn't be faster?
|
||||||
val nextState = tables.copy(nextTable = table.invert, advertisementInProgress = Some(table))
|
val nextState = tables.copy(nextTable = table.invert, advertisementInProgress = Some(table))
|
||||||
|
|
@ -424,16 +427,10 @@ private[remote] abstract class InboundCompression[T >: Null](
|
||||||
|
|
||||||
inboundContext.association(originUid) match {
|
inboundContext.association(originUid) match {
|
||||||
case OptionVal.Some(association) ⇒
|
case OptionVal.Some(association) ⇒
|
||||||
if (association.associationState.isQuarantined(originUid)) {
|
log.debug(
|
||||||
// give up
|
"Advertisement in progress for originUid [{}] version {}, resending",
|
||||||
log.debug("Skipping advertisement in progress for quarantined originUid [{}].", originUid)
|
originUid, inProgress.version)
|
||||||
confirmAdvertisement(inProgress.version)
|
advertiseCompressionTable(association, inProgress) // resend
|
||||||
} else {
|
|
||||||
log.debug(
|
|
||||||
"Advertisement in progress for originUid [{}] version {}, resending",
|
|
||||||
originUid, inProgress.version)
|
|
||||||
advertiseCompressionTable(association, inProgress) // resend
|
|
||||||
}
|
|
||||||
case OptionVal.None ⇒
|
case OptionVal.None ⇒
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
|
@ -497,5 +494,7 @@ private[remote] case object NoInboundCompressions extends InboundCompressions {
|
||||||
override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = ()
|
override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = ()
|
||||||
override def runNextClassManifestAdvertisement(): Unit = ()
|
override def runNextClassManifestAdvertisement(): Unit = ()
|
||||||
|
|
||||||
|
override def currentOriginUids: Set[Long] = Set.empty
|
||||||
|
|
||||||
override def close(originUid: Long): Unit = ()
|
override def close(originUid: Long): Unit = ()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -128,13 +128,30 @@ private[remote] class ArteryTcpTransport(_system: ExtendedActorSystem, _provider
|
||||||
TcpOutbound_Connected,
|
TcpOutbound_Connected,
|
||||||
s"${outboundContext.remoteAddress.host.get}:${outboundContext.remoteAddress.port.get} " +
|
s"${outboundContext.remoteAddress.host.get}:${outboundContext.remoteAddress.port.get} " +
|
||||||
s"/ ${streamName(streamId)}")
|
s"/ ${streamName(streamId)}")
|
||||||
Flow[ByteString]
|
|
||||||
.prepend(Source.single(TcpFraming.encodeConnectionHeader(streamId)))
|
// FIXME use the Flow.lazyInit from https://github.com/akka/akka/pull/24527
|
||||||
.via(connectionFlow)
|
|
||||||
.mapMaterializedValue(_ ⇒ NotUsed)
|
val flow =
|
||||||
.recoverWithRetries(1, { case ArteryTransport.ShutdownSignal ⇒ Source.empty })
|
Flow[ByteString]
|
||||||
.log(name = s"outbound connection to [${outboundContext.remoteAddress}], ${streamName(streamId)} stream")
|
.prepend(Source.single(TcpFraming.encodeConnectionHeader(streamId)))
|
||||||
.addAttributes(Attributes.logLevels(onElement = LogLevels.Off, onFailure = Logging.WarningLevel))
|
.via(connectionFlow)
|
||||||
|
.recoverWithRetries(1, { case ArteryTransport.ShutdownSignal ⇒ Source.empty })
|
||||||
|
.log(name = s"outbound connection to [${outboundContext.remoteAddress}], ${streamName(streamId)} stream")
|
||||||
|
.addAttributes(Attributes.logLevels(onElement = LogLevels.Off, onFailure = Logging.WarningLevel))
|
||||||
|
|
||||||
|
if (streamId == ControlStreamId) {
|
||||||
|
// must replace the KillSwitch when restarted
|
||||||
|
val controlIdleKillSwitch = KillSwitches.shared("outboundControlStreamIdleKillSwitch")
|
||||||
|
Flow[ByteString]
|
||||||
|
.via(controlIdleKillSwitch.flow)
|
||||||
|
.via(flow)
|
||||||
|
.mapMaterializedValue { _ ⇒
|
||||||
|
outboundContext.asInstanceOf[Association].setControlIdleKillSwitch(OptionVal.Some(controlIdleKillSwitch))
|
||||||
|
NotUsed
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
flow
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (streamId == ControlStreamId) {
|
if (streamId == ControlStreamId) {
|
||||||
|
|
@ -145,7 +162,6 @@ private[remote] class ArteryTcpTransport(_system: ExtendedActorSystem, _provider
|
||||||
settings.Advanced.GiveUpSystemMessageAfter, 0.1)(flowFactory)
|
settings.Advanced.GiveUpSystemMessageAfter, 0.1)(flowFactory)
|
||||||
} else {
|
} else {
|
||||||
// Best effort retry a few times
|
// Best effort retry a few times
|
||||||
// FIXME only restart on failures?, but missing in RestartFlow, see https://github.com/akka/akka/pull/23911
|
|
||||||
RestartFlow.withBackoff[ByteString, ByteString](
|
RestartFlow.withBackoff[ByteString, ByteString](
|
||||||
settings.Advanced.OutboundRestartBackoff,
|
settings.Advanced.OutboundRestartBackoff,
|
||||||
settings.Advanced.OutboundRestartBackoff * 5, 0.1, maxRestarts = 3)(flowFactory)
|
settings.Advanced.OutboundRestartBackoff * 5, 0.1, maxRestarts = 3)(flowFactory)
|
||||||
|
|
@ -397,7 +413,7 @@ private[remote] class ArteryTcpTransport(_system: ExtendedActorSystem, _provider
|
||||||
implicit val ec: ExecutionContext = materializer.executionContext
|
implicit val ec: ExecutionContext = materializer.executionContext
|
||||||
inboundKillSwitch.shutdown()
|
inboundKillSwitch.shutdown()
|
||||||
unbind().map { _ ⇒
|
unbind().map { _ ⇒
|
||||||
topLevelFREvents.loFreq(Transport_Stopped, NoMetaData)
|
topLevelFlightRecorder.loFreq(Transport_Stopped, NoMetaData)
|
||||||
Done
|
Done
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -410,7 +426,7 @@ private[remote] class ArteryTcpTransport(_system: ExtendedActorSystem, _provider
|
||||||
b ← binding
|
b ← binding
|
||||||
_ ← b.unbind()
|
_ ← b.unbind()
|
||||||
} yield {
|
} yield {
|
||||||
topLevelFREvents.loFreq(TcpInbound_Bound, s"${localAddress.address.host.get}:${localAddress.address.port}")
|
topLevelFlightRecorder.loFreq(TcpInbound_Bound, s"${localAddress.address.host.get}:${localAddress.address.port}")
|
||||||
Done
|
Done
|
||||||
}
|
}
|
||||||
case None ⇒
|
case None ⇒
|
||||||
|
|
|
||||||
|
|
@ -49,6 +49,7 @@ class EnvelopeBufferSpec extends AkkaSpec {
|
||||||
|
|
||||||
override def runNextActorRefAdvertisement(): Unit = ???
|
override def runNextActorRefAdvertisement(): Unit = ???
|
||||||
override def runNextClassManifestAdvertisement(): Unit = ???
|
override def runNextClassManifestAdvertisement(): Unit = ???
|
||||||
|
override def currentOriginUids: Set[Long] = ???
|
||||||
}
|
}
|
||||||
|
|
||||||
val version = ArteryTransport.HighestVersion
|
val version = ArteryTransport.HighestVersion
|
||||||
|
|
|
||||||
|
|
@ -3,6 +3,9 @@
|
||||||
*/
|
*/
|
||||||
package akka.remote.artery
|
package akka.remote.artery
|
||||||
|
|
||||||
|
import scala.util.Try
|
||||||
|
|
||||||
|
import akka.Done
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.remote.UniqueAddress
|
import akka.remote.UniqueAddress
|
||||||
import akka.remote.artery.InboundControlJunction.ControlMessageObserver
|
import akka.remote.artery.InboundControlJunction.ControlMessageObserver
|
||||||
|
|
@ -56,6 +59,7 @@ class InboundControlJunctionSpec
|
||||||
override def notify(env: InboundEnvelope) = {
|
override def notify(env: InboundEnvelope) = {
|
||||||
observerProbe.ref ! env.message
|
observerProbe.ref ! env.message
|
||||||
}
|
}
|
||||||
|
override def controlSubjectCompleted(signal: Try[Done]): Unit = ()
|
||||||
})
|
})
|
||||||
|
|
||||||
downstream.request(10)
|
downstream.request(10)
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,209 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
package akka.remote.artery
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
import akka.actor.ActorRef
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
import akka.actor.Address
|
||||||
|
import akka.actor.RootActorPath
|
||||||
|
import akka.remote.RARP
|
||||||
|
import akka.testkit.ImplicitSender
|
||||||
|
import akka.testkit.TestActors
|
||||||
|
import akka.testkit.TestProbe
|
||||||
|
import org.scalatest.concurrent.Eventually
|
||||||
|
import org.scalatest.time.Span
|
||||||
|
|
||||||
|
class OutboundIdleShutdownSpec extends ArteryMultiNodeSpec(s"""
|
||||||
|
akka.loglevel=INFO
|
||||||
|
akka.remote.artery.advanced.stop-idle-outbound-after = 1 s
|
||||||
|
akka.remote.artery.advanced.connection-timeout = 2 s
|
||||||
|
akka.remote.artery.advanced.remove-quarantined-association-after = 1 s
|
||||||
|
akka.remote.artery.advanced.compression {
|
||||||
|
actor-refs.advertisement-interval = 5 seconds
|
||||||
|
}
|
||||||
|
""") with ImplicitSender with Eventually {
|
||||||
|
|
||||||
|
override implicit val patience: PatienceConfig = PatienceConfig(
|
||||||
|
testKitSettings.DefaultTimeout.duration * 2,
|
||||||
|
Span(200, org.scalatest.time.Millis))
|
||||||
|
|
||||||
|
private def isArteryTcp: Boolean =
|
||||||
|
RARP(system).provider.transport.asInstanceOf[ArteryTransport].settings.Transport == ArterySettings.Tcp
|
||||||
|
|
||||||
|
private def assertStreamActive(association: Association, queueIndex: Int, expected: Boolean): Unit = {
|
||||||
|
if (queueIndex == Association.ControlQueueIndex) {
|
||||||
|
// the control stream is not stopped, but for TCP the connection is closed
|
||||||
|
if (expected)
|
||||||
|
association.isStreamActive(queueIndex) shouldBe expected
|
||||||
|
else if (isArteryTcp && !association.isRemovedAfterQuarantined()) {
|
||||||
|
association.associationState.controlIdleKillSwitch.isDefined shouldBe expected
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
association.isStreamActive(queueIndex) shouldBe expected
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
"Outbound streams" should {
|
||||||
|
|
||||||
|
"be stopped when they are idle" in withAssociation {
|
||||||
|
(_, remoteAddress, remoteEcho, localArtery, localProbe) ⇒
|
||||||
|
|
||||||
|
val association = localArtery.association(remoteAddress)
|
||||||
|
withClue("When initiating a connection, both the control and ordinary streams are opened") {
|
||||||
|
assertStreamActive(association, Association.ControlQueueIndex, expected = true)
|
||||||
|
assertStreamActive(association, Association.OrdinaryQueueIndex, expected = true)
|
||||||
|
}
|
||||||
|
|
||||||
|
eventually {
|
||||||
|
assertStreamActive(association, Association.ControlQueueIndex, expected = false)
|
||||||
|
assertStreamActive(association, Association.OrdinaryQueueIndex, expected = false)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"still be resumable after they have been stopped" in withAssociation {
|
||||||
|
(_, remoteAddress, remoteEcho, localArtery, localProbe) ⇒
|
||||||
|
val firstAssociation = localArtery.association(remoteAddress)
|
||||||
|
|
||||||
|
eventually {
|
||||||
|
assertStreamActive(firstAssociation, Association.ControlQueueIndex, expected = false)
|
||||||
|
assertStreamActive(firstAssociation, Association.OrdinaryQueueIndex, expected = false)
|
||||||
|
}
|
||||||
|
|
||||||
|
withClue("re-initiating the connection should be the same as starting it the first time") {
|
||||||
|
|
||||||
|
eventually {
|
||||||
|
remoteEcho.tell("ping", localProbe.ref)
|
||||||
|
localProbe.expectMsg("ping")
|
||||||
|
val secondAssociation = localArtery.association(remoteAddress)
|
||||||
|
assertStreamActive(secondAssociation, Association.ControlQueueIndex, expected = true)
|
||||||
|
assertStreamActive(secondAssociation, Association.OrdinaryQueueIndex, expected = true)
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"eliminate quarantined association when not used" in withAssociation {
|
||||||
|
(_, remoteAddress, remoteEcho, localArtery, localProbe) ⇒
|
||||||
|
|
||||||
|
val association = localArtery.association(remoteAddress)
|
||||||
|
withClue("When initiating a connection, both the control and ordinary streams are opened") {
|
||||||
|
assertStreamActive(association, Association.ControlQueueIndex, expected = true)
|
||||||
|
assertStreamActive(association, Association.OrdinaryQueueIndex, expected = true)
|
||||||
|
}
|
||||||
|
|
||||||
|
val remoteUid = association.associationState.uniqueRemoteAddress.futureValue.uid
|
||||||
|
|
||||||
|
localArtery.quarantine(remoteAddress, Some(remoteUid), "Test")
|
||||||
|
|
||||||
|
eventually {
|
||||||
|
assertStreamActive(association, Association.ControlQueueIndex, expected = false)
|
||||||
|
assertStreamActive(association, Association.OrdinaryQueueIndex, expected = false)
|
||||||
|
}
|
||||||
|
|
||||||
|
Thread.sleep(2000)
|
||||||
|
// localArtery.quarantine(remoteAddress, Some(remoteUid), "Test")
|
||||||
|
|
||||||
|
// the outbound streams are inactive and association quarantined, then it's completely removed
|
||||||
|
eventually {
|
||||||
|
localArtery.remoteAddresses should not contain remoteAddress
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"remove inbound compression after quarantine" in withAssociation {
|
||||||
|
(_, remoteAddress, remoteEcho, localArtery, localProbe) ⇒
|
||||||
|
|
||||||
|
val association = localArtery.association(remoteAddress)
|
||||||
|
val remoteUid = association.associationState.uniqueRemoteAddress.futureValue.uid
|
||||||
|
|
||||||
|
localArtery.inboundCompressionAccess.get.currentCompressionOriginUids.futureValue should contain(remoteUid)
|
||||||
|
|
||||||
|
eventually {
|
||||||
|
assertStreamActive(association, Association.OrdinaryQueueIndex, expected = false)
|
||||||
|
}
|
||||||
|
// compression still exists when idle
|
||||||
|
localArtery.inboundCompressionAccess.get.currentCompressionOriginUids.futureValue should contain(remoteUid)
|
||||||
|
|
||||||
|
localArtery.quarantine(remoteAddress, Some(remoteUid), "Test")
|
||||||
|
// after quarantine it should be removed
|
||||||
|
eventually {
|
||||||
|
localArtery.inboundCompressionAccess.get.currentCompressionOriginUids.futureValue should not contain remoteUid
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"remove inbound compression after restart with same host:port" in withAssociation {
|
||||||
|
(remoteSystem, remoteAddress, remoteEcho, localArtery, localProbe) ⇒
|
||||||
|
|
||||||
|
val association = localArtery.association(remoteAddress)
|
||||||
|
val remoteUid = association.associationState.uniqueRemoteAddress.futureValue.uid
|
||||||
|
|
||||||
|
localArtery.inboundCompressionAccess.get.currentCompressionOriginUids.futureValue should contain(remoteUid)
|
||||||
|
|
||||||
|
shutdown(remoteSystem, verifySystemShutdown = true)
|
||||||
|
|
||||||
|
val remoteSystem2 = newRemoteSystem(Some(s"""
|
||||||
|
akka.remote.artery.canonical.hostname = ${remoteAddress.host.get}
|
||||||
|
akka.remote.artery.canonical.port = ${remoteAddress.port.get}
|
||||||
|
"""), name = Some(remoteAddress.system))
|
||||||
|
try {
|
||||||
|
|
||||||
|
remoteSystem2.actorOf(TestActors.echoActorProps, "echo2")
|
||||||
|
|
||||||
|
def remoteEcho = system.actorSelection(RootActorPath(remoteAddress) / "user" / "echo2")
|
||||||
|
|
||||||
|
val echoRef = eventually {
|
||||||
|
remoteEcho.resolveOne(1.seconds).futureValue
|
||||||
|
}
|
||||||
|
|
||||||
|
echoRef.tell("ping2", localProbe.ref)
|
||||||
|
localProbe.expectMsg("ping2")
|
||||||
|
|
||||||
|
val association2 = localArtery.association(remoteAddress)
|
||||||
|
val remoteUid2 = association2.associationState.uniqueRemoteAddress.futureValue.uid
|
||||||
|
|
||||||
|
remoteUid2 should !==(remoteUid)
|
||||||
|
|
||||||
|
eventually {
|
||||||
|
localArtery.inboundCompressionAccess.get.currentCompressionOriginUids.futureValue should contain(remoteUid2)
|
||||||
|
}
|
||||||
|
eventually {
|
||||||
|
localArtery.inboundCompressionAccess.get.currentCompressionOriginUids.futureValue should not contain remoteUid
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
shutdown(remoteSystem2)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test setup fixture:
|
||||||
|
* 1. A 'remote' ActorSystem is created to spawn an Echo actor,
|
||||||
|
* 2. A TestProbe is spawned locally to initiate communication with the Echo actor
|
||||||
|
* 3. Details (remoteAddress, remoteEcho, localArtery, localProbe) are supplied to the test
|
||||||
|
*/
|
||||||
|
def withAssociation(test: (ActorSystem, Address, ActorRef, ArteryTransport, TestProbe) ⇒ Any): Unit = {
|
||||||
|
val remoteSystem = newRemoteSystem()
|
||||||
|
try {
|
||||||
|
remoteSystem.actorOf(TestActors.echoActorProps, "echo")
|
||||||
|
val remoteAddress = RARP(remoteSystem).provider.getDefaultAddress
|
||||||
|
|
||||||
|
def remoteEcho = system.actorSelection(RootActorPath(remoteAddress) / "user" / "echo")
|
||||||
|
|
||||||
|
val echoRef = remoteEcho.resolveOne(3.seconds).futureValue
|
||||||
|
val localProbe = new TestProbe(localSystem)
|
||||||
|
|
||||||
|
echoRef.tell("ping", localProbe.ref)
|
||||||
|
localProbe.expectMsg("ping")
|
||||||
|
|
||||||
|
val artery = RARP(system).provider.transport.asInstanceOf[ArteryTransport]
|
||||||
|
|
||||||
|
test(remoteSystem, remoteAddress, echoRef, artery, localProbe)
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
shutdown(remoteSystem)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -55,11 +55,19 @@ class SendQueueSpec extends AkkaSpec("akka.actor.serialize-messages = off") with
|
||||||
val matSettings = ActorMaterializerSettings(system).withFuzzing(true)
|
val matSettings = ActorMaterializerSettings(system).withFuzzing(true)
|
||||||
implicit val mat = ActorMaterializer(matSettings)(system)
|
implicit val mat = ActorMaterializer(matSettings)(system)
|
||||||
|
|
||||||
|
def sendToDeadLetters[T](pending: Vector[T]): Unit =
|
||||||
|
pending.foreach(system.deadLetters ! _)
|
||||||
|
|
||||||
|
def createQueue[E](capacity: Int): Queue[E] = {
|
||||||
|
// new java.util.concurrent.LinkedBlockingQueue[E](capacity)
|
||||||
|
new ManyToOneConcurrentArrayQueue[E](capacity)
|
||||||
|
}
|
||||||
|
|
||||||
"SendQueue" must {
|
"SendQueue" must {
|
||||||
|
|
||||||
"deliver all messages" in {
|
"deliver all messages" in {
|
||||||
val queue = new ManyToOneConcurrentArrayQueue[String](128)
|
val queue = createQueue[String](128)
|
||||||
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[String](system.deadLetters))
|
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[String](sendToDeadLetters))
|
||||||
.toMat(TestSink.probe)(Keep.both).run()
|
.toMat(TestSink.probe)(Keep.both).run()
|
||||||
|
|
||||||
downstream.request(10)
|
downstream.request(10)
|
||||||
|
|
@ -74,11 +82,11 @@ class SendQueueSpec extends AkkaSpec("akka.actor.serialize-messages = off") with
|
||||||
}
|
}
|
||||||
|
|
||||||
"deliver messages enqueued before materialization" in {
|
"deliver messages enqueued before materialization" in {
|
||||||
val queue = new ManyToOneConcurrentArrayQueue[String](128)
|
val queue = createQueue[String](128)
|
||||||
queue.offer("a")
|
queue.offer("a")
|
||||||
queue.offer("b")
|
queue.offer("b")
|
||||||
|
|
||||||
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[String](system.deadLetters))
|
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[String](sendToDeadLetters))
|
||||||
.toMat(TestSink.probe)(Keep.both).run()
|
.toMat(TestSink.probe)(Keep.both).run()
|
||||||
|
|
||||||
downstream.request(10)
|
downstream.request(10)
|
||||||
|
|
@ -94,9 +102,9 @@ class SendQueueSpec extends AkkaSpec("akka.actor.serialize-messages = off") with
|
||||||
|
|
||||||
"deliver bursts of messages" in {
|
"deliver bursts of messages" in {
|
||||||
// this test verifies that the wakeup signal is triggered correctly
|
// this test verifies that the wakeup signal is triggered correctly
|
||||||
val queue = new ManyToOneConcurrentArrayQueue[Int](128)
|
val queue = createQueue[Int](128)
|
||||||
val burstSize = 100
|
val burstSize = 100
|
||||||
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[Int](system.deadLetters))
|
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[Int](sendToDeadLetters))
|
||||||
.grouped(burstSize)
|
.grouped(burstSize)
|
||||||
.async
|
.async
|
||||||
.toMat(TestSink.probe)(Keep.both).run()
|
.toMat(TestSink.probe)(Keep.both).run()
|
||||||
|
|
@ -118,13 +126,13 @@ class SendQueueSpec extends AkkaSpec("akka.actor.serialize-messages = off") with
|
||||||
|
|
||||||
"support multiple producers" in {
|
"support multiple producers" in {
|
||||||
val numberOfProducers = 5
|
val numberOfProducers = 5
|
||||||
val queue = new ManyToOneConcurrentArrayQueue[Msg](numberOfProducers * 512)
|
val queue = createQueue[Msg](numberOfProducers * 512)
|
||||||
val producers = Vector.tabulate(numberOfProducers)(i ⇒ system.actorOf(producerProps(s"producer-$i")))
|
val producers = Vector.tabulate(numberOfProducers)(i ⇒ system.actorOf(producerProps(s"producer-$i")))
|
||||||
|
|
||||||
// send 100 per producer before materializing
|
// send 100 per producer before materializing
|
||||||
producers.foreach(_ ! ProduceToQueue(0, 100, queue))
|
producers.foreach(_ ! ProduceToQueue(0, 100, queue))
|
||||||
|
|
||||||
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[Msg](system.deadLetters))
|
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[Msg](sendToDeadLetters))
|
||||||
.toMat(TestSink.probe)(Keep.both).run()
|
.toMat(TestSink.probe)(Keep.both).run()
|
||||||
|
|
||||||
sendQueue.inject(queue)
|
sendQueue.inject(queue)
|
||||||
|
|
@ -150,11 +158,11 @@ class SendQueueSpec extends AkkaSpec("akka.actor.serialize-messages = off") with
|
||||||
|
|
||||||
"deliver first message" in {
|
"deliver first message" in {
|
||||||
|
|
||||||
def test(f: (ManyToOneConcurrentArrayQueue[String], SendQueue.QueueValue[String], TestSubscriber.Probe[String]) ⇒ Unit): Unit = {
|
def test(f: (Queue[String], SendQueue.QueueValue[String], TestSubscriber.Probe[String]) ⇒ Unit): Unit = {
|
||||||
|
|
||||||
(1 to 100).foreach { n ⇒
|
(1 to 100).foreach { n ⇒
|
||||||
val queue = new ManyToOneConcurrentArrayQueue[String](16)
|
val queue = createQueue[String](16)
|
||||||
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[String](system.deadLetters))
|
val (sendQueue, downstream) = Source.fromGraph(new SendQueue[String](sendToDeadLetters))
|
||||||
.toMat(TestSink.probe)(Keep.both).run()
|
.toMat(TestSink.probe)(Keep.both).run()
|
||||||
|
|
||||||
f(queue, sendQueue, downstream)
|
f(queue, sendQueue, downstream)
|
||||||
|
|
|
||||||
|
|
@ -25,6 +25,7 @@ import akka.stream.testkit.scaladsl.TestSink
|
||||||
import akka.testkit.ImplicitSender
|
import akka.testkit.ImplicitSender
|
||||||
import akka.testkit.TestActors
|
import akka.testkit.TestActors
|
||||||
import akka.testkit.TestProbe
|
import akka.testkit.TestProbe
|
||||||
|
import com.typesafe.config.Config
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.util.OptionVal
|
import akka.util.OptionVal
|
||||||
|
|
||||||
|
|
@ -33,8 +34,13 @@ object SystemMessageDeliverySpec {
|
||||||
case class TestSysMsg(s: String) extends SystemMessageDelivery.AckedDeliveryMessage
|
case class TestSysMsg(s: String) extends SystemMessageDelivery.AckedDeliveryMessage
|
||||||
|
|
||||||
val config = ConfigFactory.parseString(
|
val config = ConfigFactory.parseString(
|
||||||
"""
|
s"""
|
||||||
akka.loglevel = DEBUG
|
akka.loglevel = INFO
|
||||||
|
akka.remote.artery.advanced.stop-idle-outbound-after = 1000 ms
|
||||||
|
akka.remote.artery.advanced.inject-handshake-interval = 500 ms
|
||||||
|
akka.remote.watch-failure-detector.heartbeat-interval = 2 s
|
||||||
|
akka.remote.artery.log-received-messages = on
|
||||||
|
akka.remote.artery.log-sent-messages = on
|
||||||
""".stripMargin).withFallback(ArterySpecSupport.defaultConfig)
|
""".stripMargin).withFallback(ArterySpecSupport.defaultConfig)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -103,7 +109,7 @@ class SystemMessageDeliverySpec extends ArteryMultiNodeSpec(SystemMessageDeliver
|
||||||
"System messages" must {
|
"System messages" must {
|
||||||
|
|
||||||
"be delivered with real actors" in {
|
"be delivered with real actors" in {
|
||||||
systemB.actorOf(TestActors.echoActorProps, "echo")
|
val systemBRef = systemB.actorOf(TestActors.echoActorProps, "echo")
|
||||||
|
|
||||||
val remoteRef = {
|
val remoteRef = {
|
||||||
system.actorSelection(rootB / "user" / "echo") ! Identify(None)
|
system.actorSelection(rootB / "user" / "echo") ! Identify(None)
|
||||||
|
|
@ -111,10 +117,39 @@ class SystemMessageDeliverySpec extends ArteryMultiNodeSpec(SystemMessageDeliver
|
||||||
}
|
}
|
||||||
|
|
||||||
watch(remoteRef)
|
watch(remoteRef)
|
||||||
remoteRef ! PoisonPill
|
systemB.stop(systemBRef)
|
||||||
expectTerminated(remoteRef)
|
expectTerminated(remoteRef)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
"be delivered when concurrent idle stopping" in {
|
||||||
|
// it's configured with short stop-idle-outbound-after to stress exercise stopping of idle outbound streams
|
||||||
|
// at the same time as system messages are sent
|
||||||
|
|
||||||
|
val systemBRef = systemB.actorOf(TestActors.echoActorProps, "echo2")
|
||||||
|
|
||||||
|
val remoteRef = {
|
||||||
|
system.actorSelection(rootB / "user" / "echo2") ! Identify(None)
|
||||||
|
expectMsgType[ActorIdentity].ref.get
|
||||||
|
}
|
||||||
|
|
||||||
|
val idleTimeout = RARP(system).provider.transport.asInstanceOf[ArteryTransport].settings.Advanced.StopIdleOutboundAfter
|
||||||
|
val rnd = ThreadLocalRandom.current()
|
||||||
|
|
||||||
|
(1 to 5).foreach { _ ⇒
|
||||||
|
(1 to 1).foreach { _ ⇒
|
||||||
|
watch(remoteRef)
|
||||||
|
unwatch(remoteRef)
|
||||||
|
}
|
||||||
|
Thread.sleep((idleTimeout - 10.millis).toMillis + rnd.nextInt(20))
|
||||||
|
}
|
||||||
|
|
||||||
|
watch(remoteRef)
|
||||||
|
remoteRef ! "ping2"
|
||||||
|
expectMsg("ping2")
|
||||||
|
systemB.stop(systemBRef)
|
||||||
|
expectTerminated(remoteRef, 5.seconds)
|
||||||
|
}
|
||||||
|
|
||||||
"be flushed on shutdown" in {
|
"be flushed on shutdown" in {
|
||||||
val systemC = ActorSystem("systemC", system.settings.config)
|
val systemC = ActorSystem("systemC", system.settings.config)
|
||||||
try {
|
try {
|
||||||
|
|
|
||||||
|
|
@ -90,6 +90,8 @@ private[remote] class TestOutboundContext(
|
||||||
_associationState = _associationState.newQuarantined()
|
_associationState = _associationState.newQuarantined()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def isOrdinaryMessageStreamActive(): Boolean = true
|
||||||
|
|
||||||
override def sendControl(message: ControlMessage) = {
|
override def sendControl(message: ControlMessage) = {
|
||||||
controlProbe.foreach(_ ! message)
|
controlProbe.foreach(_ ! message)
|
||||||
controlSubject.sendControl(InboundEnvelope(OptionVal.None, message, OptionVal.None, localAddress.uid,
|
controlSubject.sendControl(InboundEnvelope(OptionVal.None, message, OptionVal.None, localAddress.uid,
|
||||||
|
|
@ -114,8 +116,6 @@ private[remote] class TestControlMessageSubject extends ControlMessageSubject {
|
||||||
observers.remove(observer)
|
observers.remove(observer)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def stopped: Future[Done] = Promise[Done]().future
|
|
||||||
|
|
||||||
def sendControl(env: InboundEnvelope): Unit = {
|
def sendControl(env: InboundEnvelope): Unit = {
|
||||||
val iter = observers.iterator()
|
val iter = observers.iterator()
|
||||||
while (iter.hasNext())
|
while (iter.hasNext())
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue