Merge branch 'master' into wip-sync-artery-patriknw

This commit is contained in:
Patrik Nordwall 2016-06-03 11:09:17 +02:00
commit 839ec5f167
757 changed files with 9166 additions and 6642 deletions

View file

@ -178,13 +178,14 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
val addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]] = Promise()
manager ! Listen(addressesPromise)
val transports: Seq[(AkkaProtocolTransport, Address)] = Await.result(addressesPromise.future,
val transports: Seq[(AkkaProtocolTransport, Address)] = Await.result(
addressesPromise.future,
StartupTimeout.duration)
if (transports.isEmpty) throw new RemoteTransportException("No transport drivers were loaded.", null)
transportMapping = transports.groupBy {
case (transport, _) transport.schemeIdentifier
} map { case (k, v) k -> v.toSet }
} map { case (k, v) k v.toSet }
defaultAddress = transports.head._2
addresses = transports.map { _._2 }.toSet
@ -230,7 +231,7 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
private[akka] def boundAddresses: Map[String, Set[Address]] = {
transportMapping.map {
case (scheme, transports)
scheme -> transports.flatMap {
scheme transports.flatMap {
// Need to do like this for binary compatibility reasons
case (t, _) Option(t.boundAddress)
}
@ -262,8 +263,9 @@ private[remote] object EndpointManager {
// Messages internal to EndpointManager
case object Prune extends NoSerializationVerificationNeeded
final case class ListensResult(addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]],
results: Seq[(AkkaProtocolTransport, Address, Promise[AssociationEventListener])])
final case class ListensResult(
addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]],
results: Seq[(AkkaProtocolTransport, Address, Promise[AssociationEventListener])])
extends NoSerializationVerificationNeeded
final case class ListensFailure(addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]], cause: Throwable)
extends NoSerializationVerificationNeeded
@ -283,9 +285,12 @@ private[remote] object EndpointManager {
final case class Pass(endpoint: ActorRef, uid: Option[Int], refuseUid: Option[Int]) extends EndpointPolicy {
override def isTombstone: Boolean = false
}
final case class Gated(timeOfRelease: Deadline) extends EndpointPolicy {
final case class Gated(timeOfRelease: Deadline, refuseUid: Option[Int]) extends EndpointPolicy {
override def isTombstone: Boolean = true
}
final case class WasGated(refuseUid: Option[Int]) extends EndpointPolicy {
override def isTombstone: Boolean = false
}
final case class Quarantined(uid: Int, timeOfRelease: Deadline) extends EndpointPolicy {
override def isTombstone: Boolean = true
}
@ -302,21 +307,30 @@ private[remote] object EndpointManager {
case Some(Pass(e, _, _))
throw new IllegalArgumentException(s"Attempting to overwrite existing endpoint [$e] with [$endpoint]")
case _
addressToWritable += address -> Pass(endpoint, uid, refuseUid)
writableToAddress += endpoint -> address
addressToWritable += address Pass(endpoint, uid, refuseUid)
writableToAddress += endpoint address
endpoint
}
def registerWritableEndpointUid(remoteAddress: Address, uid: Int): Unit = {
addressToWritable.get(remoteAddress) match {
case Some(Pass(ep, _, refuseUid)) addressToWritable += remoteAddress -> Pass(ep, Some(uid), refuseUid)
case other // the GotUid might have lost the race with some failure
case Some(Pass(ep, _, refuseUid)) addressToWritable += remoteAddress Pass(ep, Some(uid), refuseUid)
case other
}
}
def registerWritableEndpointRefuseUid(remoteAddress: Address, refuseUid: Int): Unit = {
addressToWritable.get(remoteAddress) match {
case Some(Pass(ep, uid, _)) addressToWritable += remoteAddress Pass(ep, uid, Some(refuseUid))
case Some(g: Gated) addressToWritable += remoteAddress g.copy(refuseUid = Some(refuseUid))
case Some(w: WasGated) addressToWritable += remoteAddress w.copy(refuseUid = Some(refuseUid))
case other
}
}
def registerReadOnlyEndpoint(address: Address, endpoint: ActorRef, uid: Int): ActorRef = {
addressToReadonly += address -> ((endpoint, uid))
readonlyToAddress += endpoint -> address
addressToReadonly += address ((endpoint, uid))
readonlyToAddress += endpoint address
endpoint
}
@ -338,8 +352,8 @@ private[remote] object EndpointManager {
def writableEndpointWithPolicyFor(address: Address): Option[EndpointPolicy] = addressToWritable.get(address)
def hasWritableEndpointFor(address: Address): Boolean = writableEndpointWithPolicyFor(address) match {
case Some(Pass(_, _, _)) true
case _ false
case Some(_: Pass | _: WasGated) true
case _ false
}
def readOnlyEndpointFor(address: Address): Option[(ActorRef, Int)] = addressToReadonly.get(address)
@ -360,7 +374,9 @@ private[remote] object EndpointManager {
// known fact that it is quarantined.
case Some(Quarantined(uid, _)) Some(uid)
case Some(Pass(_, _, refuseUid)) refuseUid
case _ None
case Some(Gated(_, refuseUid)) refuseUid
case Some(WasGated(refuseUid)) refuseUid
case None None
}
/**
@ -369,15 +385,27 @@ private[remote] object EndpointManager {
*/
def markAsFailed(endpoint: ActorRef, timeOfRelease: Deadline): Unit =
if (isWritable(endpoint)) {
addressToWritable += writableToAddress(endpoint) -> Gated(timeOfRelease)
writableToAddress -= endpoint
val address = writableToAddress(endpoint)
addressToWritable.get(address) match {
case Some(Quarantined(_, _)) // don't overwrite Quarantined with Gated
case Some(Pass(_, _, refuseUid))
addressToWritable += address Gated(timeOfRelease, refuseUid)
writableToAddress -= endpoint
case Some(WasGated(refuseUid))
addressToWritable += address Gated(timeOfRelease, refuseUid)
writableToAddress -= endpoint
case Some(Gated(_, _)) // already gated
case None
addressToWritable += address Gated(timeOfRelease, refuseUid = None)
writableToAddress -= endpoint
}
} else if (isReadOnly(endpoint)) {
addressToReadonly -= readonlyToAddress(endpoint)
readonlyToAddress -= endpoint
}
def markAsQuarantined(address: Address, uid: Int, timeOfRelease: Deadline): Unit =
addressToWritable += address -> Quarantined(uid, timeOfRelease)
addressToWritable += address Quarantined(uid, timeOfRelease)
def removePolicy(address: Address): Unit =
addressToWritable -= address
@ -385,10 +413,14 @@ private[remote] object EndpointManager {
def allEndpoints: collection.Iterable[ActorRef] = writableToAddress.keys ++ readonlyToAddress.keys
def prune(): Unit = {
addressToWritable = addressToWritable.filter {
case (_, Gated(timeOfRelease)) timeOfRelease.hasTimeLeft
case (_, Quarantined(_, timeOfRelease)) timeOfRelease.hasTimeLeft
case _ true
addressToWritable = addressToWritable.collect {
case entry @ (key, Gated(timeOfRelease, refuseUid))
if (timeOfRelease.hasTimeLeft) entry
else (key WasGated(refuseUid))
case entry @ (_, Quarantined(_, timeOfRelease)) if timeOfRelease.hasTimeLeft
// Querantined removed when no time left
entry
case entry @ (_, _: Pass | _: WasGated) entry
}
}
}
@ -442,9 +474,10 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
case e @ InvalidAssociation(localAddress, remoteAddress, reason, disassiciationInfo)
keepQuarantinedOr(remoteAddress) {
val causedBy = if (reason.getCause == null) "" else s"Caused by: [${reason.getCause.getMessage}]"
log.warning("Tried to associate with unreachable remote address [{}]. " +
"Address is now gated for {} ms, all messages to this address will be delivered to dead letters. " +
"Reason: [{}] {}",
log.warning(
"Tried to associate with unreachable remote address [{}]. " +
"Address is now gated for {} ms, all messages to this address will be delivered to dead letters. " +
"Reason: [{}] {}",
remoteAddress, settings.RetryGateClosedFor.toMillis, reason.getMessage, causedBy)
endpoints.markAsFailed(sender(), Deadline.now + settings.RetryGateClosedFor)
}
@ -457,8 +490,9 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
case ShutDownAssociation(localAddress, remoteAddress, _)
keepQuarantinedOr(remoteAddress) {
log.debug("Remote system with address [{}] has shut down. " +
"Address is now gated for {} ms, all messages to this address will be delivered to dead letters.",
log.debug(
"Remote system with address [{}] has shut down. " +
"Address is now gated for {} ms, all messages to this address will be delivered to dead letters.",
remoteAddress, settings.RetryGateClosedFor.toMillis)
endpoints.markAsFailed(sender(), Deadline.now + settings.RetryGateClosedFor)
}
@ -477,8 +511,9 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
case HopelessAssociation(localAddress, remoteAddress, None, _)
keepQuarantinedOr(remoteAddress) {
log.warning("Association to [{}] with unknown UID is irrecoverably failed. " +
"Address cannot be quarantined without knowing the UID, gating instead for {} ms.",
log.warning(
"Association to [{}] with unknown UID is irrecoverably failed. " +
"Address cannot be quarantined without knowing the UID, gating instead for {} ms.",
remoteAddress, settings.RetryGateClosedFor.toMillis)
endpoints.markAsFailed(sender(), Deadline.now + settings.RetryGateClosedFor)
}
@ -507,13 +542,13 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
} map {
case (a, t) if t.size > 1
throw new RemoteTransportException(s"There are more than one transports listening on local address [$a]", null)
case (a, t) a -> t.head._1
case (a, t) a t.head._1
}
// Register to each transport as listener and collect mapping to addresses
val transportsAndAddresses = results map {
case (transport, address, promise)
promise.success(ActorAssociationEventListener(self))
transport -> address
transport address
}
addressesPromise.success(transportsAndAddresses)
case ListensFailure(addressesPromise, cause)
@ -541,16 +576,32 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
(endpoints.writableEndpointWithPolicyFor(address), uidToQuarantineOption) match {
case (Some(Pass(endpoint, _, _)), None)
context.stop(endpoint)
log.warning("Association to [{}] with unknown UID is reported as quarantined, but " +
"address cannot be quarantined without knowing the UID, gating instead for {} ms.",
log.warning(
"Association to [{}] with unknown UID is reported as quarantined, but " +
"address cannot be quarantined without knowing the UID, gating instead for {} ms.",
address, settings.RetryGateClosedFor.toMillis)
endpoints.markAsFailed(endpoint, Deadline.now + settings.RetryGateClosedFor)
case (Some(Pass(endpoint, Some(currentUid), _)), Some(quarantineUid)) if currentUid == quarantineUid
context.stop(endpoint)
case _
// Do nothing, because either:
// A: we don't know yet the UID of the writer, it will be checked against current quarantine state later
// B: we know the UID, but it does not match with the UID to be quarantined
case (Some(Pass(endpoint, uidOption, refuseUidOption)), Some(quarantineUid))
uidOption match {
case Some(`quarantineUid`)
endpoints.markAsQuarantined(address, quarantineUid, Deadline.now + settings.QuarantineDuration)
eventPublisher.notifyListeners(QuarantinedEvent(address, quarantineUid))
context.stop(endpoint)
// or it does not match with the UID to be quarantined
case None if !refuseUidOption.contains(quarantineUid)
// the quarantine uid may be got fresh by cluster gossip, so update refuseUid for late handle when the writer got uid
endpoints.registerWritableEndpointRefuseUid(address, quarantineUid)
case _ //the quarantine uid has lost the race with some failure, do nothing
}
case (Some(WasGated(refuseUidOption)), Some(quarantineUid))
if (!refuseUidOption.contains(quarantineUid))
endpoints.registerWritableEndpointRefuseUid(address, quarantineUid)
case (Some(Quarantined(uid, _)), Some(quarantineUid)) if uid == quarantineUid // the UID to be quarantined already exists, do nothing
case (_, Some(quarantineUid))
// the current state is gated or quarantined, and we know the UID, update
endpoints.markAsQuarantined(address, quarantineUid, Deadline.now + settings.QuarantineDuration)
eventPublisher.notifyListeners(QuarantinedEvent(address, quarantineUid))
case _ // the current state is Gated, WasGated or Quarantined, and we don't know the UID, do nothing.
}
// Stop inbound read-only associations
@ -580,7 +631,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
// Stop all matching stashed connections
stashedInbound = stashedInbound.map {
case (writer, associations)
writer -> associations.filter { assoc
writer associations.filter { assoc
val handle = assoc.association.asInstanceOf[AkkaProtocolHandle]
val drop = matchesQuarantine(handle)
if (drop) handle.disassociate()
@ -588,11 +639,6 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
}
}
uidToQuarantineOption foreach { uid
endpoints.markAsQuarantined(address, uid, Deadline.now + settings.QuarantineDuration)
eventPublisher.notifyListeners(QuarantinedEvent(address, uid))
}
case s @ Send(message, senderOption, recipientRef, _)
val recipientAddress = recipientRef.path.address
@ -613,9 +659,11 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
endpoints.writableEndpointWithPolicyFor(recipientAddress) match {
case Some(Pass(endpoint, _, _))
endpoint ! s
case Some(Gated(timeOfRelease))
if (timeOfRelease.isOverdue()) createAndRegisterWritingEndpoint(refuseUid = None) ! s
case Some(Gated(timeOfRelease, refuseUid))
if (timeOfRelease.isOverdue()) createAndRegisterWritingEndpoint(refuseUid) ! s
else extendedSystem.deadLetters ! s
case Some(WasGated(refuseUid))
createAndRegisterWritingEndpoint(refuseUid) ! s
case Some(Quarantined(uid, _))
// timeOfRelease is only used for garbage collection reasons, therefore it is ignored here. We still have
// the Quarantined tombstone and we know what UID we don't want to accept, so use it.
@ -636,8 +684,22 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
case EndpointWriter.TookOver(endpoint, handle)
removePendingReader(takingOverFrom = endpoint, withHandle = handle)
case ReliableDeliverySupervisor.GotUid(uid, remoteAddress)
endpoints.registerWritableEndpointUid(remoteAddress, uid)
handleStashedInbound(sender(), writerIsIdle = false)
endpoints.writableEndpointWithPolicyFor(remoteAddress) match {
case Some(Pass(endpoint, _, refuseUidOption))
if (refuseUidOption.contains(uid)) {
endpoints.markAsQuarantined(remoteAddress, uid, Deadline.now + settings.QuarantineDuration)
eventPublisher.notifyListeners(QuarantinedEvent(remoteAddress, uid))
context.stop(endpoint)
} else endpoints.registerWritableEndpointUid(remoteAddress, uid)
handleStashedInbound(sender(), writerIsIdle = false)
case Some(WasGated(refuseUidOption))
if (refuseUidOption.contains(uid)) {
endpoints.markAsQuarantined(remoteAddress, uid, Deadline.now + settings.QuarantineDuration)
eventPublisher.notifyListeners(QuarantinedEvent(remoteAddress, uid))
} else endpoints.registerWritableEndpointUid(remoteAddress, uid)
handleStashedInbound(sender(), writerIsIdle = false)
case _ // the GotUid might have lost the race with some failure
}
case ReliableDeliverySupervisor.Idle
handleStashedInbound(sender(), writerIsIdle = true)
case Prune
@ -675,7 +737,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
case ia @ InboundAssociation(handle: AkkaProtocolHandle) endpoints.readOnlyEndpointFor(handle.remoteAddress) match {
case Some((endpoint, _))
pendingReadHandoffs.get(endpoint) foreach (_.disassociate())
pendingReadHandoffs += endpoint -> handle
pendingReadHandoffs += endpoint handle
endpoint ! EndpointWriter.TakeOver(handle, self)
endpoints.writableEndpointWithPolicyFor(handle.remoteAddress) match {
case Some(Pass(ep, _, _)) ep ! ReliableDeliverySupervisor.Ungate
@ -690,13 +752,13 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
// to get an unstash event
if (!writerIsIdle) {
ep ! ReliableDeliverySupervisor.IsIdle
stashedInbound += ep -> (stashedInbound.getOrElse(ep, Vector.empty) :+ ia)
stashedInbound += ep (stashedInbound.getOrElse(ep, Vector.empty) :+ ia)
} else
createAndRegisterEndpoint(handle, refuseUid = endpoints.refuseUid(handle.remoteAddress))
case Some(Pass(ep, Some(uid), _))
if (handle.handshakeInfo.uid == uid) {
pendingReadHandoffs.get(ep) foreach (_.disassociate())
pendingReadHandoffs += ep -> handle
pendingReadHandoffs += ep handle
ep ! EndpointWriter.StopReading(ep, self)
ep ! ReliableDeliverySupervisor.Ungate
} else {
@ -741,7 +803,7 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
*/
val transports: Seq[AkkaProtocolTransport] = for ((fqn, adapters, config) settings.Transports) yield {
val args = Seq(classOf[ExtendedActorSystem] -> context.system, classOf[Config] -> config)
val args = Seq(classOf[ExtendedActorSystem] context.system, classOf[Config] config)
// Loads the driver -- the bottom element of the chain.
// The chain at this point:
@ -800,37 +862,40 @@ private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends
pendingReadHandoffs -= takingOverFrom
}
private def createEndpoint(remoteAddress: Address,
localAddress: Address,
transport: AkkaProtocolTransport,
endpointSettings: RemoteSettings,
handleOption: Option[AkkaProtocolHandle],
writing: Boolean,
refuseUid: Option[Int]): ActorRef = {
private def createEndpoint(
remoteAddress: Address,
localAddress: Address,
transport: AkkaProtocolTransport,
endpointSettings: RemoteSettings,
handleOption: Option[AkkaProtocolHandle],
writing: Boolean,
refuseUid: Option[Int]): ActorRef = {
require(transportMapping contains localAddress, "Transport mapping is not defined for the address")
// refuseUid is ignored for read-only endpoints since the UID of the remote system is already known and has passed
// quarantine checks
if (writing) context.watch(context.actorOf(RARP(extendedSystem).configureDispatcher(ReliableDeliverySupervisor.props(
handleOption,
localAddress,
remoteAddress,
refuseUid,
transport,
endpointSettings,
AkkaPduProtobufCodec,
receiveBuffers)).withDeploy(Deploy.local),
if (writing) context.watch(context.actorOf(
RARP(extendedSystem).configureDispatcher(ReliableDeliverySupervisor.props(
handleOption,
localAddress,
remoteAddress,
refuseUid,
transport,
endpointSettings,
AkkaPduProtobufCodec,
receiveBuffers)).withDeploy(Deploy.local),
"reliableEndpointWriter-" + AddressUrlEncoder(remoteAddress) + "-" + endpointId.next()))
else context.watch(context.actorOf(RARP(extendedSystem).configureDispatcher(EndpointWriter.props(
handleOption,
localAddress,
remoteAddress,
refuseUid,
transport,
endpointSettings,
AkkaPduProtobufCodec,
receiveBuffers,
reliableDeliverySupervisor = None)).withDeploy(Deploy.local),
else context.watch(context.actorOf(
RARP(extendedSystem).configureDispatcher(EndpointWriter.props(
handleOption,
localAddress,
remoteAddress,
refuseUid,
transport,
endpointSettings,
AkkaPduProtobufCodec,
receiveBuffers,
reliableDeliverySupervisor = None)).withDeploy(Deploy.local),
"endpointWriter-" + AddressUrlEncoder(remoteAddress) + "-" + endpointId.next()))
}