Merge branch 'master' into wip-sync-artery-patriknw
This commit is contained in:
commit
839ec5f167
757 changed files with 9166 additions and 6642 deletions
|
|
@ -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()))
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue