2016-05-09 07:31:41 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Copyright (C) 2016 Lightbend Inc. <http://www.lightbend.com>
|
|
|
|
|
|
*/
|
|
|
|
|
|
package akka.remote.artery
|
|
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
import java.util.Queue
|
|
|
|
|
|
|
2016-05-25 12:28:44 +02:00
|
|
|
|
import java.util.concurrent.ConcurrentHashMap
|
2016-06-02 07:21:32 +02:00
|
|
|
|
import java.util.concurrent.CopyOnWriteArrayList
|
2016-05-17 17:34:57 +02:00
|
|
|
|
import java.util.concurrent.CountDownLatch
|
|
|
|
|
|
import java.util.concurrent.TimeUnit
|
2016-05-29 22:15:48 +02:00
|
|
|
|
import java.util.concurrent.atomic.AtomicReference
|
2016-05-13 15:34:37 +02:00
|
|
|
|
import scala.annotation.tailrec
|
2016-05-17 17:34:57 +02:00
|
|
|
|
import scala.concurrent.Future
|
2016-05-09 07:31:41 +02:00
|
|
|
|
import scala.concurrent.Promise
|
2016-05-17 17:34:57 +02:00
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
|
import scala.concurrent.duration.FiniteDuration
|
2016-05-13 08:06:13 +02:00
|
|
|
|
import scala.util.Success
|
2016-05-20 12:40:56 +02:00
|
|
|
|
import akka.{ Done, NotUsed }
|
2016-05-09 07:31:41 +02:00
|
|
|
|
import akka.actor.ActorRef
|
2016-05-17 17:34:57 +02:00
|
|
|
|
import akka.actor.ActorSelectionMessage
|
2016-05-09 07:31:41 +02:00
|
|
|
|
import akka.actor.Address
|
|
|
|
|
|
import akka.actor.RootActorPath
|
|
|
|
|
|
import akka.dispatch.sysmsg.SystemMessage
|
2016-05-13 08:06:13 +02:00
|
|
|
|
import akka.event.Logging
|
2016-05-20 12:40:56 +02:00
|
|
|
|
import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef, UniqueAddress }
|
2016-05-25 12:28:44 +02:00
|
|
|
|
import akka.remote.EndpointManager.Send
|
2016-05-19 08:24:27 +02:00
|
|
|
|
import akka.remote.artery.AeronSink.GaveUpSendingException
|
2016-05-12 08:56:28 +02:00
|
|
|
|
import akka.remote.artery.InboundControlJunction.ControlMessageSubject
|
2016-05-13 08:06:13 +02:00
|
|
|
|
import akka.remote.artery.OutboundControlJunction.OutboundControlIngress
|
2016-05-17 17:34:57 +02:00
|
|
|
|
import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException
|
|
|
|
|
|
import akka.remote.artery.SystemMessageDelivery.ClearSystemMessageDelivery
|
|
|
|
|
|
import akka.stream.AbruptTerminationException
|
2016-05-09 07:31:41 +02:00
|
|
|
|
import akka.stream.Materializer
|
2016-05-13 08:06:13 +02:00
|
|
|
|
import akka.stream.scaladsl.Keep
|
2016-05-09 07:31:41 +02:00
|
|
|
|
import akka.stream.scaladsl.Source
|
2016-05-20 12:40:56 +02:00
|
|
|
|
import akka.util.{ Unsafe, WildcardTree }
|
2016-05-29 22:15:48 +02:00
|
|
|
|
import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
|
*/
|
|
|
|
|
|
private[remote] object Association {
|
|
|
|
|
|
final case class QueueWrapper(queue: Queue[Send]) extends SendQueue.ProducerApi[Send] {
|
|
|
|
|
|
override def offer(message: Send): Boolean = queue.offer(message)
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
|
*
|
|
|
|
|
|
* Thread-safe, mutable holder for association state. Main entry point for remote destined message to a specific
|
|
|
|
|
|
* remote address.
|
|
|
|
|
|
*/
|
2016-05-29 22:15:48 +02:00
|
|
|
|
private[remote] class Association(
|
2016-06-03 11:59:00 +02:00
|
|
|
|
val transport: ArteryTransport,
|
|
|
|
|
|
val materializer: Materializer,
|
|
|
|
|
|
override val remoteAddress: Address,
|
2016-05-20 12:40:56 +02:00
|
|
|
|
override val controlSubject: ControlMessageSubject,
|
2016-06-03 11:59:00 +02:00
|
|
|
|
largeMessageDestinations: WildcardTree[NotUsed])
|
2016-05-13 08:06:13 +02:00
|
|
|
|
extends AbstractAssociation with OutboundContext {
|
2016-05-29 22:15:48 +02:00
|
|
|
|
import Association._
|
2016-05-13 08:06:13 +02:00
|
|
|
|
|
|
|
|
|
|
private val log = Logging(transport.system, getClass.getName)
|
2016-06-02 07:21:32 +02:00
|
|
|
|
private val controlQueueSize = transport.remoteSettings.SysMsgBufferSize
|
2016-05-29 22:15:48 +02:00
|
|
|
|
// FIXME config queue size, and it should perhaps also be possible to use some kind of LinkedQueue
|
|
|
|
|
|
// such as agrona.ManyToOneConcurrentLinkedQueue or AbstractNodeQueue for less memory consumption
|
|
|
|
|
|
private val queueSize = 3072
|
|
|
|
|
|
private val largeQueueSize = 256
|
2016-05-09 07:31:41 +02:00
|
|
|
|
|
2016-05-17 17:34:57 +02:00
|
|
|
|
private val restartTimeout: FiniteDuration = 5.seconds // FIXME config
|
|
|
|
|
|
private val maxRestarts = 5 // FIXME config
|
|
|
|
|
|
private val restartCounter = new RestartCounter(maxRestarts, restartTimeout)
|
2016-05-20 12:40:56 +02:00
|
|
|
|
private val largeMessageChannelEnabled = largeMessageDestinations.children.nonEmpty
|
2016-05-17 17:34:57 +02:00
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
// We start with the raw wrapped queue and then it is replaced with the materialized value of
|
|
|
|
|
|
// the `SendQueue` after materialization. Using same underlying queue. This makes it possible to
|
|
|
|
|
|
// start sending (enqueuing) to the Association immediate after construction.
|
|
|
|
|
|
|
|
|
|
|
|
def createQueue(capacity: Int): Queue[Send] =
|
|
|
|
|
|
new ManyToOneConcurrentArrayQueue[Send](capacity)
|
|
|
|
|
|
|
|
|
|
|
|
@volatile private[this] var queue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(queueSize))
|
|
|
|
|
|
@volatile private[this] var largeQueue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(largeQueueSize))
|
|
|
|
|
|
@volatile private[this] var controlQueue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(controlQueueSize))
|
2016-05-12 08:56:28 +02:00
|
|
|
|
@volatile private[this] var _outboundControlIngress: OutboundControlIngress = _
|
2016-05-17 17:34:57 +02:00
|
|
|
|
@volatile private[this] var materializing = new CountDownLatch(1)
|
2016-05-11 15:55:06 +02:00
|
|
|
|
|
2016-06-02 07:21:32 +02:00
|
|
|
|
private val _testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList
|
|
|
|
|
|
|
|
|
|
|
|
def testStages(): List[TestManagementApi] = {
|
|
|
|
|
|
import scala.collection.JavaConverters._
|
|
|
|
|
|
_testStages.asScala.toList
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-12 08:56:28 +02:00
|
|
|
|
def outboundControlIngress: OutboundControlIngress = {
|
2016-05-13 08:06:13 +02:00
|
|
|
|
if (_outboundControlIngress ne null)
|
|
|
|
|
|
_outboundControlIngress
|
|
|
|
|
|
else {
|
|
|
|
|
|
// materialization not completed yet
|
|
|
|
|
|
materializing.await(10, TimeUnit.SECONDS)
|
|
|
|
|
|
if (_outboundControlIngress eq null)
|
|
|
|
|
|
throw new IllegalStateException("outboundControlIngress not initialized yet")
|
|
|
|
|
|
_outboundControlIngress
|
|
|
|
|
|
}
|
2016-05-11 15:55:06 +02:00
|
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
|
|
|
|
|
|
|
override def localAddress: UniqueAddress = transport.localAddress
|
|
|
|
|
|
|
2016-05-13 08:06:13 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Holds reference to shared state of Association - *access only via helper methods*
|
|
|
|
|
|
*/
|
|
|
|
|
|
@volatile
|
2016-05-13 15:34:37 +02:00
|
|
|
|
private[this] var _sharedStateDoNotCallMeDirectly: AssociationState = AssociationState()
|
2016-05-13 08:06:13 +02:00
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Helper method for access to underlying state via Unsafe
|
|
|
|
|
|
*
|
|
|
|
|
|
* @param oldState Previous state
|
|
|
|
|
|
* @param newState Next state on transition
|
|
|
|
|
|
* @return Whether the previous state matched correctly
|
|
|
|
|
|
*/
|
|
|
|
|
|
@inline
|
|
|
|
|
|
private[this] def swapState(oldState: AssociationState, newState: AssociationState): Boolean =
|
|
|
|
|
|
Unsafe.instance.compareAndSwapObject(this, AbstractAssociation.sharedStateOffset, oldState, newState)
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* @return Reference to current shared state
|
|
|
|
|
|
*/
|
|
|
|
|
|
def associationState: AssociationState =
|
|
|
|
|
|
Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[AssociationState]
|
|
|
|
|
|
|
2016-05-25 12:28:44 +02:00
|
|
|
|
def completeHandshake(peer: UniqueAddress): Unit = {
|
2016-06-03 11:59:00 +02:00
|
|
|
|
require(
|
|
|
|
|
|
remoteAddress == peer.address,
|
2016-05-13 08:06:13 +02:00
|
|
|
|
s"wrong remote address in completeHandshake, got ${peer.address}, expected ${remoteAddress}")
|
|
|
|
|
|
val current = associationState
|
|
|
|
|
|
current.uniqueRemoteAddressPromise.trySuccess(peer)
|
2016-05-13 15:34:37 +02:00
|
|
|
|
current.uniqueRemoteAddressValue() match {
|
2016-06-04 22:14:28 +02:00
|
|
|
|
case Some(`peer`) ⇒ // our value
|
2016-05-13 08:06:13 +02:00
|
|
|
|
case _ ⇒
|
2016-05-13 15:34:37 +02:00
|
|
|
|
val newState = current.newIncarnation(Promise.successful(peer))
|
2016-05-13 08:06:13 +02:00
|
|
|
|
if (swapState(current, newState)) {
|
2016-05-13 15:34:37 +02:00
|
|
|
|
current.uniqueRemoteAddressValue() match {
|
2016-06-04 22:14:28 +02:00
|
|
|
|
case Some(old) ⇒
|
2016-06-03 11:59:00 +02:00
|
|
|
|
log.debug(
|
|
|
|
|
|
"Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])",
|
2016-05-13 08:06:13 +02:00
|
|
|
|
newState.incarnation, peer.address, peer.uid, old.uid)
|
2016-06-04 22:14:28 +02:00
|
|
|
|
case None ⇒
|
2016-05-13 15:34:37 +02:00
|
|
|
|
// Failed, nothing to do
|
2016-05-13 08:06:13 +02:00
|
|
|
|
}
|
|
|
|
|
|
// if swap failed someone else completed before us, and that is fine
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-13 08:06:13 +02:00
|
|
|
|
// OutboundContext
|
|
|
|
|
|
override def sendControl(message: ControlMessage): Unit =
|
|
|
|
|
|
outboundControlIngress.sendControlMessage(message)
|
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
|
def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = {
|
2016-05-13 15:34:37 +02:00
|
|
|
|
// allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system
|
|
|
|
|
|
// FIXME where is that ActorSelectionMessage check in old remoting?
|
2016-05-17 17:34:57 +02:00
|
|
|
|
if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) {
|
2016-05-13 15:34:37 +02:00
|
|
|
|
// FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly
|
|
|
|
|
|
message match {
|
|
|
|
|
|
case _: SystemMessage | ClearSystemMessageDelivery ⇒
|
2016-05-29 22:15:48 +02:00
|
|
|
|
val send = Send(message, senderOption, recipient, None)
|
|
|
|
|
|
if (!controlQueue.offer(send)) {
|
|
|
|
|
|
quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]")
|
|
|
|
|
|
transport.system.deadLetters ! send
|
2016-05-13 15:34:37 +02:00
|
|
|
|
}
|
|
|
|
|
|
case _ ⇒
|
2016-05-20 12:40:56 +02:00
|
|
|
|
val send = Send(message, senderOption, recipient, None)
|
2016-05-29 22:15:48 +02:00
|
|
|
|
val offerOk =
|
|
|
|
|
|
if (largeMessageChannelEnabled && isLargeMessageDestination(recipient))
|
|
|
|
|
|
largeQueue.offer(send)
|
|
|
|
|
|
else
|
|
|
|
|
|
queue.offer(send)
|
|
|
|
|
|
if (!offerOk)
|
|
|
|
|
|
transport.system.deadLetters ! send
|
2016-05-13 15:34:37 +02:00
|
|
|
|
}
|
|
|
|
|
|
} else if (log.isDebugEnabled)
|
|
|
|
|
|
log.debug("Dropping message to quarantined system {}", remoteAddress)
|
2016-05-09 07:31:41 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-20 12:40:56 +02:00
|
|
|
|
private def isLargeMessageDestination(recipient: ActorRef): Boolean = {
|
|
|
|
|
|
recipient match {
|
2016-05-26 10:42:08 +02:00
|
|
|
|
case r: RemoteActorRef if r.cachedLargeMessageDestinationFlag ne null ⇒ r.cachedLargeMessageDestinationFlag eq LargeDestination
|
2016-05-20 12:40:56 +02:00
|
|
|
|
case r: RemoteActorRef ⇒
|
|
|
|
|
|
if (largeMessageDestinations.find(r.path.elements.iterator).data.isEmpty) {
|
|
|
|
|
|
r.cachedLargeMessageDestinationFlag = RegularDestination
|
|
|
|
|
|
false
|
|
|
|
|
|
} else {
|
|
|
|
|
|
log.debug("Using large message stream for {}", r.path)
|
|
|
|
|
|
r.cachedLargeMessageDestinationFlag = LargeDestination
|
|
|
|
|
|
true
|
|
|
|
|
|
}
|
|
|
|
|
|
case _ ⇒ false
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-09 07:31:41 +02:00
|
|
|
|
// FIXME we should be able to Send without a recipient ActorRef
|
|
|
|
|
|
override val dummyRecipient: RemoteActorRef =
|
|
|
|
|
|
transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef]
|
|
|
|
|
|
|
2016-05-13 15:34:37 +02:00
|
|
|
|
// OutboundContext
|
|
|
|
|
|
override def quarantine(reason: String): Unit = {
|
2016-06-04 22:14:28 +02:00
|
|
|
|
val uid = associationState.uniqueRemoteAddressValue().map(_.uid)
|
2016-05-13 15:34:37 +02:00
|
|
|
|
quarantine(reason, uid)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-25 12:28:44 +02:00
|
|
|
|
@tailrec final def quarantine(reason: String, uid: Option[Long]): Unit = {
|
2016-05-13 15:34:37 +02:00
|
|
|
|
uid match {
|
|
|
|
|
|
case Some(u) ⇒
|
|
|
|
|
|
val current = associationState
|
|
|
|
|
|
current.uniqueRemoteAddressValue() match {
|
2016-06-04 22:14:28 +02:00
|
|
|
|
case Some(peer) if peer.uid == u ⇒
|
2016-05-13 15:34:37 +02:00
|
|
|
|
if (!current.isQuarantined(u)) {
|
|
|
|
|
|
val newState = current.newQuarantined()
|
|
|
|
|
|
if (swapState(current, newState)) {
|
|
|
|
|
|
// quarantine state change was performed
|
2016-06-03 11:59:00 +02:00
|
|
|
|
log.warning(
|
|
|
|
|
|
"Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address. {}",
|
2016-05-13 15:34:37 +02:00
|
|
|
|
remoteAddress, u, reason)
|
|
|
|
|
|
// end delivery of system messages to that incarnation after this point
|
|
|
|
|
|
send(ClearSystemMessageDelivery, None, dummyRecipient)
|
|
|
|
|
|
// try to tell the other system that we have quarantined it
|
|
|
|
|
|
sendControl(Quarantined(localAddress, peer))
|
|
|
|
|
|
} else
|
|
|
|
|
|
quarantine(reason, uid) // recursive
|
|
|
|
|
|
}
|
2016-06-04 22:14:28 +02:00
|
|
|
|
case Some(peer) ⇒
|
2016-06-03 11:59:00 +02:00
|
|
|
|
log.debug(
|
|
|
|
|
|
"Quarantine of [{}] ignored due to non-matching UID, quarantine requested for [{}] but current is [{}]. {}",
|
2016-05-13 15:34:37 +02:00
|
|
|
|
remoteAddress, u, peer.uid, reason)
|
|
|
|
|
|
case None ⇒
|
2016-06-03 11:59:00 +02:00
|
|
|
|
log.debug(
|
|
|
|
|
|
"Quarantine of [{}] ignored because handshake not completed, quarantine request was for old incarnation. {}",
|
2016-05-13 15:34:37 +02:00
|
|
|
|
remoteAddress, reason)
|
|
|
|
|
|
}
|
|
|
|
|
|
case None ⇒
|
|
|
|
|
|
// FIXME should we do something more, old impl used gating?
|
|
|
|
|
|
log.warning("Quarantine of [{}] ignored because unknown UID", remoteAddress)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-13 08:06:13 +02:00
|
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Called once after construction when the `Association` instance
|
|
|
|
|
|
* wins the CAS in the `AssociationRegistry`. It will materialize
|
|
|
|
|
|
* the streams. It is possible to sending (enqueuing) to the association
|
|
|
|
|
|
* before this method is called.
|
|
|
|
|
|
*/
|
2016-05-09 07:31:41 +02:00
|
|
|
|
def associate(): Unit = {
|
2016-05-29 22:15:48 +02:00
|
|
|
|
if (!controlQueue.isInstanceOf[QueueWrapper])
|
|
|
|
|
|
throw new IllegalStateException("associate() must only be called once")
|
2016-05-20 12:40:56 +02:00
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
// it's important to materialize the outboundControl stream first,
|
|
|
|
|
|
// so that outboundControlIngress is ready when stages for all streams start
|
|
|
|
|
|
runOutboundControlStream()
|
|
|
|
|
|
runOutboundOrdinaryMessagesStream()
|
|
|
|
|
|
|
|
|
|
|
|
if (largeMessageChannelEnabled) {
|
|
|
|
|
|
runOutboundLargeMessagesStream()
|
2016-05-17 17:34:57 +02:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
private def runOutboundControlStream(): Unit = {
|
|
|
|
|
|
// stage in the control stream may access the outboundControlIngress before returned here
|
|
|
|
|
|
// using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress
|
|
|
|
|
|
materializing = new CountDownLatch(1)
|
2016-05-29 22:15:48 +02:00
|
|
|
|
|
|
|
|
|
|
val wrapper = getOrCreateQueueWrapper(controlQueue, queueSize)
|
|
|
|
|
|
controlQueue = wrapper // use new underlying queue immediately for restarts
|
2016-06-02 07:21:32 +02:00
|
|
|
|
|
|
|
|
|
|
val (queueValue, (control, completed)) =
|
|
|
|
|
|
if (transport.remoteSettings.TestMode) {
|
|
|
|
|
|
val ((queueValue, mgmt), (control, completed)) =
|
|
|
|
|
|
Source.fromGraph(new SendQueue[Send])
|
|
|
|
|
|
.viaMat(transport.outboundTestFlow(this))(Keep.both)
|
|
|
|
|
|
.toMat(transport.outboundControl(this))(Keep.both)
|
|
|
|
|
|
.run()(materializer)
|
|
|
|
|
|
_testStages.add(mgmt)
|
|
|
|
|
|
(queueValue, (control, completed))
|
|
|
|
|
|
} else {
|
|
|
|
|
|
Source.fromGraph(new SendQueue[Send])
|
|
|
|
|
|
.toMat(transport.outboundControl(this))(Keep.both)
|
|
|
|
|
|
.run()(materializer)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
queueValue.inject(wrapper.queue)
|
|
|
|
|
|
// replace with the materialized value, still same underlying queue
|
|
|
|
|
|
controlQueue = queueValue
|
2016-05-17 17:34:57 +02:00
|
|
|
|
_outboundControlIngress = control
|
|
|
|
|
|
materializing.countDown()
|
|
|
|
|
|
attachStreamRestart("Outbound control stream", completed, cause ⇒ {
|
|
|
|
|
|
runOutboundControlStream()
|
|
|
|
|
|
cause match {
|
|
|
|
|
|
case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID
|
|
|
|
|
|
case _ ⇒ quarantine("Outbound control stream restarted")
|
|
|
|
|
|
}
|
|
|
|
|
|
})
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
private def getOrCreateQueueWrapper(q: SendQueue.ProducerApi[Send], capacity: Int): QueueWrapper =
|
|
|
|
|
|
q match {
|
|
|
|
|
|
case existing: QueueWrapper ⇒ existing
|
|
|
|
|
|
case _ ⇒
|
|
|
|
|
|
// use new queue for restarts
|
|
|
|
|
|
QueueWrapper(createQueue(capacity))
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-17 17:34:57 +02:00
|
|
|
|
private def runOutboundOrdinaryMessagesStream(): Unit = {
|
2016-05-29 22:15:48 +02:00
|
|
|
|
val wrapper = getOrCreateQueueWrapper(queue, queueSize)
|
|
|
|
|
|
queue = wrapper // use new underlying queue immediately for restarts
|
2016-06-02 07:21:32 +02:00
|
|
|
|
|
|
|
|
|
|
val (queueValue, completed) =
|
|
|
|
|
|
if (transport.remoteSettings.TestMode) {
|
|
|
|
|
|
val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[Send])
|
|
|
|
|
|
.viaMat(transport.outboundTestFlow(this))(Keep.both)
|
|
|
|
|
|
.toMat(transport.outbound(this))(Keep.both)
|
|
|
|
|
|
.run()(materializer)
|
|
|
|
|
|
_testStages.add(mgmt)
|
|
|
|
|
|
(queueValue, completed)
|
|
|
|
|
|
} else {
|
|
|
|
|
|
Source.fromGraph(new SendQueue[Send])
|
|
|
|
|
|
.toMat(transport.outbound(this))(Keep.both)
|
|
|
|
|
|
.run()(materializer)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
queueValue.inject(wrapper.queue)
|
|
|
|
|
|
// replace with the materialized value, still same underlying queue
|
|
|
|
|
|
queue = queueValue
|
2016-06-02 07:21:32 +02:00
|
|
|
|
|
2016-05-17 17:34:57 +02:00
|
|
|
|
attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream())
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-20 12:40:56 +02:00
|
|
|
|
private def runOutboundLargeMessagesStream(): Unit = {
|
2016-05-29 22:15:48 +02:00
|
|
|
|
val wrapper = getOrCreateQueueWrapper(queue, largeQueueSize)
|
|
|
|
|
|
largeQueue = wrapper // use new underlying queue immediately for restarts
|
2016-06-02 07:21:32 +02:00
|
|
|
|
|
|
|
|
|
|
val (queueValue, completed) =
|
|
|
|
|
|
if (transport.remoteSettings.TestMode) {
|
|
|
|
|
|
val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[Send])
|
|
|
|
|
|
.viaMat(transport.outboundTestFlow(this))(Keep.both)
|
|
|
|
|
|
.toMat(transport.outboundLarge(this))(Keep.both)
|
|
|
|
|
|
.run()(materializer)
|
|
|
|
|
|
_testStages.add(mgmt)
|
|
|
|
|
|
(queueValue, completed)
|
|
|
|
|
|
} else {
|
|
|
|
|
|
Source.fromGraph(new SendQueue[Send])
|
|
|
|
|
|
.toMat(transport.outboundLarge(this))(Keep.both)
|
|
|
|
|
|
.run()(materializer)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
queueValue.inject(wrapper.queue)
|
|
|
|
|
|
// replace with the materialized value, still same underlying queue
|
|
|
|
|
|
largeQueue = queueValue
|
2016-05-20 12:40:56 +02:00
|
|
|
|
attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream())
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-05-17 17:34:57 +02:00
|
|
|
|
private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: Throwable ⇒ Unit): Unit = {
|
|
|
|
|
|
implicit val ec = materializer.executionContext
|
|
|
|
|
|
streamCompleted.onFailure {
|
2016-05-19 08:24:27 +02:00
|
|
|
|
case _ if transport.isShutdown ⇒ // don't restart after shutdown
|
2016-05-17 17:34:57 +02:00
|
|
|
|
case _: AbruptTerminationException ⇒ // ActorSystem shutdown
|
2016-05-19 08:24:27 +02:00
|
|
|
|
case cause: GaveUpSendingException ⇒
|
2016-05-19 21:12:47 +02:00
|
|
|
|
log.debug("{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage)
|
2016-05-19 08:24:27 +02:00
|
|
|
|
// restart unconditionally, without counting restarts
|
|
|
|
|
|
restart(cause)
|
2016-05-17 17:34:57 +02:00
|
|
|
|
case cause ⇒
|
2016-05-19 08:24:27 +02:00
|
|
|
|
if (restartCounter.restart()) {
|
2016-05-19 21:12:47 +02:00
|
|
|
|
log.error(cause, "{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage)
|
2016-05-19 08:24:27 +02:00
|
|
|
|
restart(cause)
|
|
|
|
|
|
} else {
|
2016-05-19 21:12:47 +02:00
|
|
|
|
log.error(cause, s"{} to {} failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}",
|
|
|
|
|
|
streamName, remoteAddress, maxRestarts, restartTimeout.toSeconds)
|
2016-05-19 08:24:27 +02:00
|
|
|
|
transport.system.terminate()
|
|
|
|
|
|
}
|
2016-05-11 15:55:06 +02:00
|
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
|
}
|
2016-05-25 12:28:44 +02:00
|
|
|
|
|
|
|
|
|
|
override def toString(): String =
|
|
|
|
|
|
s"Association($localAddress -> $remoteAddress with $associationState)"
|
|
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
|
*/
|
|
|
|
|
|
private[remote] class AssociationRegistry(createAssociation: Address ⇒ Association) {
|
2016-05-29 22:15:48 +02:00
|
|
|
|
private[this] val associationsByAddress = new AtomicReference[Map[Address, Association]](Map.empty)
|
|
|
|
|
|
private[this] val associationsByUid = new ConcurrentHashMap[Long, Association]() // FIXME replace with specialized Long Map
|
2016-05-25 12:28:44 +02:00
|
|
|
|
|
2016-05-29 22:15:48 +02:00
|
|
|
|
@tailrec final def association(remoteAddress: Address): Association = {
|
|
|
|
|
|
val currentMap = associationsByAddress.get
|
|
|
|
|
|
currentMap.get(remoteAddress) match {
|
|
|
|
|
|
case Some(existing) ⇒ existing
|
|
|
|
|
|
case None ⇒
|
|
|
|
|
|
val newAssociation = createAssociation(remoteAddress)
|
|
|
|
|
|
val newMap = currentMap.updated(remoteAddress, newAssociation)
|
|
|
|
|
|
if (associationsByAddress.compareAndSet(currentMap, newMap)) {
|
|
|
|
|
|
newAssociation.associate() // start it, only once
|
2016-05-25 12:28:44 +02:00
|
|
|
|
newAssociation
|
2016-05-29 22:15:48 +02:00
|
|
|
|
} else
|
|
|
|
|
|
association(remoteAddress) // lost CAS, retry
|
2016-05-25 12:28:44 +02:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
def association(uid: Long): Association =
|
|
|
|
|
|
associationsByUid.get(uid)
|
|
|
|
|
|
|
|
|
|
|
|
def setUID(peer: UniqueAddress): Association = {
|
|
|
|
|
|
val a = association(peer.address)
|
|
|
|
|
|
val previous = associationsByUid.put(peer.uid, a)
|
|
|
|
|
|
if ((previous ne null) && (previous ne a))
|
|
|
|
|
|
throw new IllegalArgumentException(s"UID collision old [$previous] new [$a]")
|
|
|
|
|
|
a
|
|
|
|
|
|
}
|
2016-06-02 07:21:32 +02:00
|
|
|
|
|
|
|
|
|
|
def allAssociations: Set[Association] =
|
|
|
|
|
|
associationsByAddress.get.values.toSet
|
2016-05-09 07:31:41 +02:00
|
|
|
|
}
|