2016-05-09 07:31:41 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2016 Lightbend Inc. <http://www.lightbend.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.remote.artery
|
|
|
|
|
|
|
|
|
|
import scala.concurrent.Promise
|
2016-05-13 08:06:13 +02:00
|
|
|
import scala.util.Success
|
2016-05-09 07:31:41 +02:00
|
|
|
import akka.actor.ActorRef
|
|
|
|
|
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-09 07:31:41 +02:00
|
|
|
import akka.remote.EndpointManager.Send
|
|
|
|
|
import akka.remote.RemoteActorRef
|
|
|
|
|
import akka.remote.UniqueAddress
|
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-09 07:31:41 +02:00
|
|
|
import akka.stream.Materializer
|
|
|
|
|
import akka.stream.OverflowStrategy
|
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
|
|
|
|
|
import akka.stream.scaladsl.SourceQueueWithComplete
|
2016-05-13 08:06:13 +02:00
|
|
|
import akka.util.Unsafe
|
|
|
|
|
import java.util.concurrent.locks.ReentrantLock
|
|
|
|
|
import java.util.concurrent.CountDownLatch
|
|
|
|
|
import java.util.concurrent.TimeUnit
|
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.
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class Association(
|
|
|
|
|
val transport: ArteryTransport,
|
|
|
|
|
val materializer: Materializer,
|
|
|
|
|
override val remoteAddress: Address,
|
2016-05-13 08:06:13 +02:00
|
|
|
override val controlSubject: ControlMessageSubject)
|
|
|
|
|
extends AbstractAssociation with OutboundContext {
|
|
|
|
|
|
|
|
|
|
private val log = Logging(transport.system, getClass.getName)
|
2016-05-09 07:31:41 +02:00
|
|
|
|
|
|
|
|
@volatile private[this] var queue: SourceQueueWithComplete[Send] = _
|
2016-05-13 08:06:13 +02:00
|
|
|
@volatile private[this] var controlQueue: SourceQueueWithComplete[Send] = _
|
2016-05-12 08:56:28 +02:00
|
|
|
@volatile private[this] var _outboundControlIngress: OutboundControlIngress = _
|
2016-05-13 08:06:13 +02:00
|
|
|
private val materializing = new CountDownLatch(1)
|
2016-05-11 15:55:06 +02:00
|
|
|
|
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
|
|
|
|
|
private[this] var _sharedStateDoNotCallMeDirectly: AssociationState =
|
|
|
|
|
new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise())
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* 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]
|
|
|
|
|
|
|
|
|
|
override def completeHandshake(peer: UniqueAddress): Unit = {
|
|
|
|
|
require(remoteAddress == peer.address,
|
|
|
|
|
s"wrong remote address in completeHandshake, got ${peer.address}, expected ${remoteAddress}")
|
|
|
|
|
val current = associationState
|
|
|
|
|
current.uniqueRemoteAddressPromise.trySuccess(peer)
|
|
|
|
|
current.uniqueRemoteAddress.value match {
|
|
|
|
|
case Some(Success(`peer`)) ⇒ // our value
|
|
|
|
|
case _ ⇒
|
|
|
|
|
val newState = new AssociationState(incarnation = current.incarnation + 1, Promise.successful(peer))
|
|
|
|
|
if (swapState(current, newState)) {
|
|
|
|
|
current.uniqueRemoteAddress.value match {
|
|
|
|
|
case Some(Success(old)) ⇒
|
|
|
|
|
log.debug("Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])",
|
|
|
|
|
newState.incarnation, peer.address, peer.uid, old.uid)
|
|
|
|
|
quarantine(Some(old.uid))
|
|
|
|
|
case _ ⇒ // Failed, nothing to do
|
|
|
|
|
}
|
|
|
|
|
// 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 = {
|
|
|
|
|
// TODO: lookup subchannel
|
|
|
|
|
// FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly
|
|
|
|
|
message match {
|
2016-05-11 15:55:06 +02:00
|
|
|
case _: SystemMessage ⇒
|
2016-05-09 07:31:41 +02:00
|
|
|
implicit val ec = materializer.executionContext
|
2016-05-13 08:06:13 +02:00
|
|
|
controlQueue.offer(Send(message, senderOption, recipient, None)).onFailure {
|
2016-05-09 07:31:41 +02:00
|
|
|
case e ⇒
|
|
|
|
|
// FIXME proper error handling, and quarantining
|
|
|
|
|
println(s"# System message dropped, due to $e") // FIXME
|
|
|
|
|
}
|
|
|
|
|
case _ ⇒
|
|
|
|
|
queue.offer(Send(message, senderOption, recipient, None))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// 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 08:06:13 +02:00
|
|
|
def quarantine(uid: Option[Int]): Unit = {
|
|
|
|
|
// FIXME implement
|
|
|
|
|
log.error("Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address.",
|
|
|
|
|
remoteAddress, uid.getOrElse("unknown"))
|
|
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
|
|
|
|
|
// Idempotent
|
|
|
|
|
def associate(): Unit = {
|
|
|
|
|
// FIXME detect and handle stream failure, e.g. handshake timeout
|
2016-05-13 08:06:13 +02:00
|
|
|
|
|
|
|
|
// it's important to materialize the outboundControl stream first,
|
|
|
|
|
// so that outboundControlIngress is ready when stages for all streams start
|
|
|
|
|
if (controlQueue eq null) {
|
2016-05-11 15:55:06 +02:00
|
|
|
val (q, control) = Source.queue(256, OverflowStrategy.dropBuffer)
|
2016-05-12 08:56:28 +02:00
|
|
|
.toMat(transport.outboundControl(this))(Keep.both)
|
2016-05-11 15:55:06 +02:00
|
|
|
.run()(materializer)
|
2016-05-13 08:06:13 +02:00
|
|
|
controlQueue = q
|
2016-05-12 08:56:28 +02:00
|
|
|
_outboundControlIngress = control
|
2016-05-13 08:06:13 +02:00
|
|
|
// 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.countDown()
|
|
|
|
|
|
|
|
|
|
queue = Source.queue(256, OverflowStrategy.dropBuffer)
|
|
|
|
|
.to(transport.outbound(this)).run()(materializer)
|
2016-05-11 15:55:06 +02:00
|
|
|
}
|
2016-05-09 07:31:41 +02:00
|
|
|
}
|
|
|
|
|
}
|