Disable remote watch and remote deployment outside Cluster (#27126)
This commit is contained in:
parent
8757e35b7e
commit
611e32de91
39 changed files with 1022 additions and 179 deletions
|
|
@ -363,6 +363,8 @@ object ActorSystem {
|
||||||
|
|
||||||
final val ProviderClass: String = ProviderSelectionType.fqcn
|
final val ProviderClass: String = ProviderSelectionType.fqcn
|
||||||
|
|
||||||
|
final val HasCluster: Boolean = ProviderSelectionType.hasCluster
|
||||||
|
|
||||||
final val SupervisorStrategyClass: String = getString("akka.actor.guardian-supervisor-strategy")
|
final val SupervisorStrategyClass: String = getString("akka.actor.guardian-supervisor-strategy")
|
||||||
final val CreationTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.creation-timeout"))
|
final val CreationTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.creation-timeout"))
|
||||||
final val UnstartedPushTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.unstarted-push-timeout"))
|
final val UnstartedPushTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.unstarted-push-timeout"))
|
||||||
|
|
|
||||||
|
|
@ -153,12 +153,11 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
||||||
.toMap
|
.toMap
|
||||||
|
|
||||||
config.root.asScala
|
config.root.asScala
|
||||||
.map {
|
.flatMap {
|
||||||
case ("default", _) => None
|
case ("default", _) => None
|
||||||
case (key, value: ConfigObject) => parseConfig(key, value.toConfig)
|
case (key, value: ConfigObject) => parseConfig(key, value.toConfig)
|
||||||
case _ => None
|
case _ => None
|
||||||
}
|
}
|
||||||
.flatten
|
|
||||||
.foreach(deploy)
|
.foreach(deploy)
|
||||||
|
|
||||||
def lookup(path: ActorPath): Option[Deploy] = lookup(path.elements.drop(1))
|
def lookup(path: ActorPath): Option[Deploy] = lookup(path.elements.drop(1))
|
||||||
|
|
|
||||||
|
|
@ -14,7 +14,6 @@ import akka.actor._
|
||||||
import akka.actor.DeadLetterSuppression
|
import akka.actor.DeadLetterSuppression
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.ClusterEvent._
|
import akka.cluster.ClusterEvent._
|
||||||
import akka.cluster.MemberStatus
|
|
||||||
import akka.cluster.ddata.LWWRegister
|
import akka.cluster.ddata.LWWRegister
|
||||||
import akka.cluster.ddata.LWWRegisterKey
|
import akka.cluster.ddata.LWWRegisterKey
|
||||||
import akka.cluster.ddata.Replicator._
|
import akka.cluster.ddata.Replicator._
|
||||||
|
|
@ -510,8 +509,8 @@ abstract class ShardCoordinator(
|
||||||
|
|
||||||
def isMember(region: ActorRef): Boolean = {
|
def isMember(region: ActorRef): Boolean = {
|
||||||
val regionAddress = region.path.address
|
val regionAddress = region.path.address
|
||||||
(region.path.address == self.path.address ||
|
regionAddress == self.path.address ||
|
||||||
cluster.state.members.exists(m => m.address == regionAddress && m.status == MemberStatus.Up))
|
cluster.state.isMemberUp(regionAddress)
|
||||||
}
|
}
|
||||||
|
|
||||||
def active: Receive =
|
def active: Receive =
|
||||||
|
|
@ -539,17 +538,18 @@ abstract class ShardCoordinator(
|
||||||
}
|
}
|
||||||
|
|
||||||
case RegisterProxy(proxy) =>
|
case RegisterProxy(proxy) =>
|
||||||
log.debug("ShardRegion proxy registered: [{}]", proxy)
|
if (isMember(proxy)) {
|
||||||
if (state.regionProxies.contains(proxy))
|
log.debug("ShardRegion proxy registered: [{}]", proxy)
|
||||||
proxy ! RegisterAck(self)
|
if (state.regionProxies.contains(proxy))
|
||||||
else {
|
|
||||||
update(ShardRegionProxyRegistered(proxy)) { evt =>
|
|
||||||
state = state.updated(evt)
|
|
||||||
context.watch(proxy)
|
|
||||||
proxy ! RegisterAck(self)
|
proxy ! RegisterAck(self)
|
||||||
|
else {
|
||||||
|
update(ShardRegionProxyRegistered(proxy)) { evt =>
|
||||||
|
state = state.updated(evt)
|
||||||
|
context.watch(proxy)
|
||||||
|
proxy ! RegisterAck(self)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
case GetShardHome(shard) =>
|
case GetShardHome(shard) =>
|
||||||
if (!handleGetShardHome(shard)) {
|
if (!handleGetShardHome(shard)) {
|
||||||
// location not know, yet
|
// location not know, yet
|
||||||
|
|
|
||||||
|
|
@ -1,2 +1,5 @@
|
||||||
# #24710 remove internal ClusterReadView.refreshCurrentState
|
# #24710 remove internal ClusterReadView.refreshCurrentState
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ClusterReadView.refreshCurrentState")
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ClusterReadView.refreshCurrentState")
|
||||||
|
|
||||||
|
# Disable remote watch and remote deployment outside Cluster #26176
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ClusterRemoteWatcher.props")
|
||||||
|
|
|
||||||
|
|
@ -5,18 +5,26 @@
|
||||||
package akka.cluster
|
package akka.cluster
|
||||||
|
|
||||||
import akka.ConfigurationException
|
import akka.ConfigurationException
|
||||||
import akka.actor.{ ActorRef, ActorSystem, ActorSystemImpl, Deploy, DynamicAccess, NoScopeGiven, Scope }
|
import akka.actor.ActorPath
|
||||||
|
import akka.actor.Address
|
||||||
|
import akka.actor.ActorRef
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
import akka.actor.ActorSystemImpl
|
||||||
|
import akka.actor.Deploy
|
||||||
|
import akka.actor.DynamicAccess
|
||||||
|
import akka.actor.NoScopeGiven
|
||||||
|
import akka.actor.Scope
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import akka.cluster.routing.{
|
import akka.cluster.routing.ClusterRouterGroup
|
||||||
ClusterRouterGroup,
|
import akka.cluster.routing.ClusterRouterGroupSettings
|
||||||
ClusterRouterGroupSettings,
|
import akka.cluster.routing.ClusterRouterPool
|
||||||
ClusterRouterPool,
|
import akka.cluster.routing.ClusterRouterPoolSettings
|
||||||
ClusterRouterPoolSettings
|
|
||||||
}
|
|
||||||
import akka.event.EventStream
|
import akka.event.EventStream
|
||||||
import akka.remote.{ RemoteActorRefProvider, RemoteDeployer }
|
|
||||||
import akka.remote.routing.RemoteRouterConfig
|
import akka.remote.routing.RemoteRouterConfig
|
||||||
import akka.routing.{ Group, Pool }
|
import akka.remote.RemoteActorRefProvider
|
||||||
|
import akka.remote.RemoteDeployer
|
||||||
|
import akka.routing.Group
|
||||||
|
import akka.routing.Pool
|
||||||
import com.github.ghik.silencer.silent
|
import com.github.ghik.silencer.silent
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
@ -46,15 +54,8 @@ private[akka] class ClusterActorRefProvider(
|
||||||
override protected def createRemoteWatcher(system: ActorSystemImpl): ActorRef = {
|
override protected def createRemoteWatcher(system: ActorSystemImpl): ActorRef = {
|
||||||
// make sure Cluster extension is initialized/loaded from init thread
|
// make sure Cluster extension is initialized/loaded from init thread
|
||||||
Cluster(system)
|
Cluster(system)
|
||||||
|
|
||||||
import remoteSettings._
|
|
||||||
val failureDetector = createRemoteWatcherFailureDetector(system)
|
|
||||||
system.systemActorOf(
|
system.systemActorOf(
|
||||||
ClusterRemoteWatcher.props(
|
ClusterRemoteWatcher.props(createRemoteWatcherFailureDetector(system), remoteSettings),
|
||||||
failureDetector,
|
|
||||||
heartbeatInterval = WatchHeartBeatInterval,
|
|
||||||
unreachableReaperInterval = WatchUnreachableReaperInterval,
|
|
||||||
heartbeatExpectedResponseAfter = WatchHeartbeatExpectedResponseAfter),
|
|
||||||
"remote-watcher")
|
"remote-watcher")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -64,7 +65,11 @@ private[akka] class ClusterActorRefProvider(
|
||||||
*/
|
*/
|
||||||
override protected def createDeployer: ClusterDeployer = new ClusterDeployer(settings, dynamicAccess)
|
override protected def createDeployer: ClusterDeployer = new ClusterDeployer(settings, dynamicAccess)
|
||||||
|
|
||||||
override protected def showDirectUseWarningIfRequired(): Unit = ()
|
override protected def shouldCreateRemoteActorRef(system: ActorSystem, address: Address): Boolean =
|
||||||
|
Cluster(system).state.members.exists(_.address == address) && super.shouldCreateRemoteActorRef(system, address)
|
||||||
|
|
||||||
|
override protected def warnIfNotRemoteActorRef(path: ActorPath): Unit =
|
||||||
|
warnOnUnsafe(s"Remote deploy of [$path] outside this cluster is not allowed, falling back to local.")
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -193,6 +193,14 @@ object ClusterEvent {
|
||||||
def withUnreachableDataCenters(unreachableDataCenters: Set[DataCenter]): CurrentClusterState =
|
def withUnreachableDataCenters(unreachableDataCenters: Set[DataCenter]): CurrentClusterState =
|
||||||
new CurrentClusterState(members, unreachable, seenBy, leader, roleLeaderMap, unreachableDataCenters)
|
new CurrentClusterState(members, unreachable, seenBy, leader, roleLeaderMap, unreachableDataCenters)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
* Returns true if the address is a cluster member and that member is `MemberStatus.Up`.
|
||||||
|
*/
|
||||||
|
@InternalApi
|
||||||
|
private[akka] def isMemberUp(address: Address): Boolean =
|
||||||
|
members.exists(m => m.address == address && m.status == MemberStatus.Up)
|
||||||
|
|
||||||
// for binary compatibility (used to be a case class)
|
// for binary compatibility (used to be a case class)
|
||||||
def copy(
|
def copy(
|
||||||
members: immutable.SortedSet[Member] = this.members,
|
members: immutable.SortedSet[Member] = this.members,
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,7 @@ import akka.cluster.ClusterEvent.MemberRemoved
|
||||||
import akka.cluster.ClusterEvent.MemberWeaklyUp
|
import akka.cluster.ClusterEvent.MemberWeaklyUp
|
||||||
import akka.dispatch.Dispatchers
|
import akka.dispatch.Dispatchers
|
||||||
import akka.remote.FailureDetectorRegistry
|
import akka.remote.FailureDetectorRegistry
|
||||||
|
import akka.remote.RemoteSettings
|
||||||
import akka.remote.RemoteWatcher
|
import akka.remote.RemoteWatcher
|
||||||
import akka.remote.RARP
|
import akka.remote.RARP
|
||||||
|
|
||||||
|
|
@ -26,17 +27,15 @@ private[cluster] object ClusterRemoteWatcher {
|
||||||
/**
|
/**
|
||||||
* Factory method for `ClusterRemoteWatcher` [[akka.actor.Props]].
|
* Factory method for `ClusterRemoteWatcher` [[akka.actor.Props]].
|
||||||
*/
|
*/
|
||||||
def props(
|
def props(failureDetector: FailureDetectorRegistry[Address], settings: RemoteSettings): Props =
|
||||||
failureDetector: FailureDetectorRegistry[Address],
|
|
||||||
heartbeatInterval: FiniteDuration,
|
|
||||||
unreachableReaperInterval: FiniteDuration,
|
|
||||||
heartbeatExpectedResponseAfter: FiniteDuration): Props =
|
|
||||||
Props(
|
Props(
|
||||||
classOf[ClusterRemoteWatcher],
|
new ClusterRemoteWatcher(
|
||||||
failureDetector,
|
failureDetector,
|
||||||
heartbeatInterval,
|
heartbeatInterval = settings.WatchHeartBeatInterval,
|
||||||
unreachableReaperInterval,
|
unreachableReaperInterval = settings.WatchUnreachableReaperInterval,
|
||||||
heartbeatExpectedResponseAfter).withDispatcher(Dispatchers.InternalDispatcherId).withDeploy(Deploy.local)
|
heartbeatExpectedResponseAfter = settings.WatchHeartbeatExpectedResponseAfter))
|
||||||
|
.withDispatcher(Dispatchers.InternalDispatcherId)
|
||||||
|
.withDeploy(Deploy.local)
|
||||||
|
|
||||||
private final case class DelayedQuarantine(m: Member, previousStatus: MemberStatus)
|
private final case class DelayedQuarantine(m: Member, previousStatus: MemberStatus)
|
||||||
extends NoSerializationVerificationNeeded
|
extends NoSerializationVerificationNeeded
|
||||||
|
|
@ -81,7 +80,7 @@ private[cluster] class ClusterRemoteWatcher(
|
||||||
cluster.unsubscribe(self)
|
cluster.unsubscribe(self)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def receive = receiveClusterEvent.orElse(super.receive)
|
override def receive: Receive = receiveClusterEvent.orElse(super.receive)
|
||||||
|
|
||||||
def receiveClusterEvent: Actor.Receive = {
|
def receiveClusterEvent: Actor.Receive = {
|
||||||
case state: CurrentClusterState =>
|
case state: CurrentClusterState =>
|
||||||
|
|
@ -160,6 +159,9 @@ private[cluster] class ClusterRemoteWatcher(
|
||||||
override def watchNode(watchee: InternalActorRef): Unit =
|
override def watchNode(watchee: InternalActorRef): Unit =
|
||||||
if (!clusterNodes(watchee.path.address)) super.watchNode(watchee)
|
if (!clusterNodes(watchee.path.address)) super.watchNode(watchee)
|
||||||
|
|
||||||
|
override protected def shouldWatch(watchee: InternalActorRef): Boolean =
|
||||||
|
clusterNodes(watchee.path.address)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* When a cluster node is added this class takes over the
|
* When a cluster node is added this class takes over the
|
||||||
* responsibility for watchees on that node already handled
|
* responsibility for watchees on that node already handled
|
||||||
|
|
|
||||||
|
|
@ -4,19 +4,20 @@
|
||||||
|
|
||||||
package akka.cluster
|
package akka.cluster
|
||||||
|
|
||||||
import language.postfixOps
|
|
||||||
import scala.concurrent.Await
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.remote.testkit.MultiNodeConfig
|
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
|
||||||
import akka.testkit._
|
|
||||||
import akka.testkit.TestEvent._
|
|
||||||
import akka.actor._
|
|
||||||
import akka.remote.RemoteActorRef
|
|
||||||
import java.util.concurrent.TimeoutException
|
import java.util.concurrent.TimeoutException
|
||||||
|
|
||||||
import akka.remote.RemoteWatcher
|
import scala.concurrent.Await
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
import scala.language.postfixOps
|
||||||
|
|
||||||
|
import akka.actor._
|
||||||
import akka.cluster.MultiNodeClusterSpec.EndActor
|
import akka.cluster.MultiNodeClusterSpec.EndActor
|
||||||
|
import akka.remote.RemoteActorRef
|
||||||
|
import akka.remote.RemoteWatcher
|
||||||
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
|
import akka.testkit.TestEvent._
|
||||||
|
import akka.testkit._
|
||||||
import org.scalatest.concurrent.ScalaFutures
|
import org.scalatest.concurrent.ScalaFutures
|
||||||
|
|
||||||
object ClusterDeathWatchMultiJvmSpec extends MultiNodeConfig {
|
object ClusterDeathWatchMultiJvmSpec extends MultiNodeConfig {
|
||||||
|
|
@ -141,7 +142,7 @@ abstract class ClusterDeathWatchSpec
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to watch actor before node joins cluster, ClusterRemoteWatcher takes over from RemoteWatcher" in within(
|
"not be able to watch an actor before node joins cluster, ClusterRemoteWatcher takes over from RemoteWatcher" in within(
|
||||||
20 seconds) {
|
20 seconds) {
|
||||||
runOn(fifth) {
|
runOn(fifth) {
|
||||||
system.actorOf(
|
system.actorOf(
|
||||||
|
|
@ -155,30 +156,45 @@ abstract class ClusterDeathWatchSpec
|
||||||
val subject5 = expectMsgType[ActorIdentity].ref.get
|
val subject5 = expectMsgType[ActorIdentity].ref.get
|
||||||
watch(subject5)
|
watch(subject5)
|
||||||
|
|
||||||
// fifth is not cluster member, so the watch is handled by the RemoteWatcher
|
// fifth is not cluster member, watch is dropped
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
remoteWatcher ! RemoteWatcher.Stats
|
remoteWatcher ! RemoteWatcher.Stats
|
||||||
val stats = expectMsgType[RemoteWatcher.Stats]
|
expectMsg(RemoteWatcher.Stats.empty)
|
||||||
stats.watchingRefs should contain(subject5 -> testActor)
|
|
||||||
stats.watchingAddresses should contain(address(fifth))
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
enterBarrier("remote-watch")
|
|
||||||
|
|
||||||
// second and third are already removed
|
// second and third are already removed
|
||||||
awaitClusterUp(first, fourth, fifth)
|
awaitClusterUp(first, fourth, fifth)
|
||||||
|
|
||||||
|
runOn(fifth) {
|
||||||
|
// fifth is a member, the watch for subject5 previously deployed would not be in
|
||||||
|
// RemoteWatcher. Therefore we create a new one to test that now, being a member,
|
||||||
|
// will be in RemoteWatcher.
|
||||||
|
system.actorOf(
|
||||||
|
Props(new Actor { def receive = Actor.emptyBehavior }).withDeploy(Deploy.local),
|
||||||
|
name = "subject6")
|
||||||
|
}
|
||||||
|
enterBarrier("subject6-started")
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
// fifth is member, so the node is handled by the ClusterRemoteWatcher,
|
// fifth is member, so the node is handled by the ClusterRemoteWatcher.
|
||||||
// but the watch is still in RemoteWatcher
|
system.actorSelection(RootActorPath(fifth) / "user" / "subject6") ! Identify("subject6")
|
||||||
|
val subject6 = expectMsgType[ActorIdentity].ref.get
|
||||||
|
watch(subject6)
|
||||||
|
|
||||||
|
system.actorSelection(RootActorPath(fifth) / "user" / "subject5") ! Identify("subject5")
|
||||||
|
val subject5 = expectMsgType[ActorIdentity].ref.get
|
||||||
|
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
remoteWatcher ! RemoteWatcher.Stats
|
remoteWatcher ! RemoteWatcher.Stats
|
||||||
val stats = expectMsgType[RemoteWatcher.Stats]
|
val stats = expectMsgType[RemoteWatcher.Stats]
|
||||||
stats.watchingRefs.map(_._1.path.name) should contain("subject5")
|
stats.watchingRefs should contain(subject6 -> testActor)
|
||||||
|
stats.watchingRefs should not contain (subject5 -> testActor)
|
||||||
|
stats.watching shouldEqual 1
|
||||||
stats.watchingAddresses should not contain address(fifth)
|
stats.watchingAddresses should not contain address(fifth)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
enterBarrier("remote-watch")
|
||||||
enterBarrier("cluster-watch")
|
enterBarrier("cluster-watch")
|
||||||
|
|
||||||
runOn(fourth) {
|
runOn(fourth) {
|
||||||
|
|
@ -192,9 +208,19 @@ abstract class ClusterDeathWatchSpec
|
||||||
|
|
||||||
enterBarrier("fifth-terminated")
|
enterBarrier("fifth-terminated")
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
expectMsgType[Terminated].actor.path.name should ===("subject5")
|
// subject5 is not in RemoteWatcher.watching, the terminated for subject5 is from testActor.watch.
|
||||||
}
|
// You can not verify that it is the testActor receiving it, though the remoteWatcher stats proves
|
||||||
|
// it above
|
||||||
|
receiveWhile(messages = 2) {
|
||||||
|
case Terminated(ref) => ref.path.name
|
||||||
|
}.toSet shouldEqual Set("subject5", "subject6")
|
||||||
|
|
||||||
|
awaitAssert {
|
||||||
|
remoteWatcher ! RemoteWatcher.Stats
|
||||||
|
expectMsg(RemoteWatcher.Stats.empty)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
enterBarrier("terminated-subject6")
|
||||||
enterBarrier("after-3")
|
enterBarrier("after-3")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,15 +4,16 @@
|
||||||
|
|
||||||
package akka.cluster
|
package akka.cluster
|
||||||
|
|
||||||
import scala.collection.immutable
|
|
||||||
import com.typesafe.config.ConfigFactory
|
|
||||||
import akka.remote.testkit.MultiNodeConfig
|
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
|
||||||
import akka.testkit._
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.cluster.ClusterEvent.MemberUp
|
import akka.cluster.ClusterEvent.MemberUp
|
||||||
import akka.cluster.ClusterEvent.CurrentClusterState
|
import akka.cluster.ClusterEvent.CurrentClusterState
|
||||||
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
|
import akka.testkit._
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
object DeterministicOldestWhenJoiningMultiJvmSpec extends MultiNodeConfig {
|
object DeterministicOldestWhenJoiningMultiJvmSpec extends MultiNodeConfig {
|
||||||
val seed1 = role("seed1")
|
val seed1 = role("seed1")
|
||||||
|
|
@ -58,7 +59,7 @@ abstract class DeterministicOldestWhenJoiningSpec
|
||||||
cluster.joinSeedNodes(seedNodes)
|
cluster.joinSeedNodes(seedNodes)
|
||||||
}
|
}
|
||||||
|
|
||||||
within(10.seconds) {
|
within(15.seconds) {
|
||||||
val ups = List(expectMsgType[MemberUp], expectMsgType[MemberUp], expectMsgType[MemberUp])
|
val ups = List(expectMsgType[MemberUp], expectMsgType[MemberUp], expectMsgType[MemberUp])
|
||||||
ups.map(_.member).sorted(Member.ageOrdering).head.address should ===(seedNodes.head)
|
ups.map(_.member).sorted(Member.ageOrdering).head.address should ===(seedNodes.head)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,136 @@
|
||||||
|
/*
|
||||||
|
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.cluster
|
||||||
|
|
||||||
|
import akka.actor.Actor
|
||||||
|
import akka.actor.ActorRef
|
||||||
|
import akka.actor.AddressFromURIString
|
||||||
|
import akka.actor.Props
|
||||||
|
import akka.actor.RepointableActorRef
|
||||||
|
import akka.cluster.ClusterRemoteFeatures.AddressPing
|
||||||
|
import akka.remote.RARP
|
||||||
|
import akka.remote.RemoteActorRef
|
||||||
|
import akka.remote.RemoteActorRefProvider
|
||||||
|
import akka.remote.RemoteWatcher.Heartbeat
|
||||||
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
|
import akka.testkit.ImplicitSender
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
import org.scalatest.concurrent.ScalaFutures
|
||||||
|
|
||||||
|
class ClusterRemoteFeaturesConfig(artery: Boolean) extends MultiNodeConfig {
|
||||||
|
val first = role("first")
|
||||||
|
val second = role("second")
|
||||||
|
val third = role("third")
|
||||||
|
|
||||||
|
private val baseConfig = ConfigFactory.parseString(s"""
|
||||||
|
akka.remote.log-remote-lifecycle-events = off
|
||||||
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.artery.canonical.port = 0
|
||||||
|
akka.remote.artery.advanced.flight-recorder.enabled = off
|
||||||
|
akka.log-dead-letters-during-shutdown = off
|
||||||
|
""").withFallback(MultiNodeClusterSpec.clusterConfig)
|
||||||
|
|
||||||
|
commonConfig(debugConfig(on = false).withFallback(baseConfig))
|
||||||
|
|
||||||
|
deployOn(first, """/kattdjur.remote = "@second@" """)
|
||||||
|
deployOn(third, """/kattdjur.remote = "@second@" """)
|
||||||
|
deployOn(second, """/kattdjur.remote = "@third@" """)
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
object ClusterRemoteFeatures {
|
||||||
|
class AddressPing extends Actor {
|
||||||
|
def receive: Receive = {
|
||||||
|
case "ping" => sender() ! self
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class ArteryClusterRemoteFeaturesMultiJvmNode1 extends ClusterRemoteFeaturesSpec(new ClusterRemoteFeaturesConfig(true))
|
||||||
|
class ArteryClusterRemoteFeaturesMultiJvmNode2 extends ClusterRemoteFeaturesSpec(new ClusterRemoteFeaturesConfig(true))
|
||||||
|
class ArteryClusterRemoteFeaturesMultiJvmNode3 extends ClusterRemoteFeaturesSpec(new ClusterRemoteFeaturesConfig(true))
|
||||||
|
|
||||||
|
class ClassicClusterRemoteFeaturesMultiJvmNode1
|
||||||
|
extends ClusterRemoteFeaturesSpec(new ClusterRemoteFeaturesConfig(false))
|
||||||
|
class ClassicClusterRemoteFeaturesMultiJvmNode2
|
||||||
|
extends ClusterRemoteFeaturesSpec(new ClusterRemoteFeaturesConfig(false))
|
||||||
|
class ClassicClusterRemoteFeaturesMultiJvmNode3
|
||||||
|
extends ClusterRemoteFeaturesSpec(new ClusterRemoteFeaturesConfig(false))
|
||||||
|
|
||||||
|
abstract class ClusterRemoteFeaturesSpec(multiNodeConfig: ClusterRemoteFeaturesConfig)
|
||||||
|
extends MultiNodeSpec(multiNodeConfig)
|
||||||
|
with MultiNodeClusterSpec
|
||||||
|
with ImplicitSender
|
||||||
|
with ScalaFutures {
|
||||||
|
|
||||||
|
import multiNodeConfig._
|
||||||
|
|
||||||
|
override def initialParticipants: Int = roles.size
|
||||||
|
|
||||||
|
muteDeadLetters(Heartbeat.getClass)()
|
||||||
|
|
||||||
|
protected val provider: RemoteActorRefProvider = RARP(system).provider
|
||||||
|
|
||||||
|
"Remoting with Cluster" must {
|
||||||
|
|
||||||
|
"have the correct settings" in {
|
||||||
|
runOn(first) {
|
||||||
|
system.settings.HasCluster shouldBe true
|
||||||
|
provider.hasClusterOrUseUnsafe shouldBe true
|
||||||
|
provider.transport.system.settings.HasCluster shouldBe true
|
||||||
|
provider.remoteSettings.UseUnsafeRemoteFeaturesWithoutCluster shouldBe false
|
||||||
|
system.settings.ProviderClass shouldEqual classOf[ClusterActorRefProvider].getName
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"create a ClusterRemoteWatcher" in {
|
||||||
|
runOn(roles: _*)(provider.remoteWatcher.isDefined shouldBe true)
|
||||||
|
}
|
||||||
|
|
||||||
|
"create a ClusterDeployer" in {
|
||||||
|
runOn(roles: _*)(provider.deployer.getClass shouldEqual classOf[ClusterDeployer])
|
||||||
|
}
|
||||||
|
|
||||||
|
"have expected `actorOf` behavior" in {
|
||||||
|
awaitClusterUp(first, second)
|
||||||
|
enterBarrier("cluster-up")
|
||||||
|
|
||||||
|
runOn(first) {
|
||||||
|
val actor = system.actorOf(Props[AddressPing], "kattdjur")
|
||||||
|
actor.isInstanceOf[RemoteActorRef] shouldBe true
|
||||||
|
actor.path.address shouldEqual node(second).address
|
||||||
|
actor.path.address.hasGlobalScope shouldBe true
|
||||||
|
|
||||||
|
val secondAddress = node(second).address
|
||||||
|
actor ! "ping"
|
||||||
|
expectMsgType[RemoteActorRef].path.address shouldEqual secondAddress
|
||||||
|
}
|
||||||
|
enterBarrier("CARP-in-cluster-remote-validated")
|
||||||
|
|
||||||
|
def assertIsLocalRef(): Unit = {
|
||||||
|
val actor = system.actorOf(Props[AddressPing], "kattdjur")
|
||||||
|
actor.isInstanceOf[RepointableActorRef] shouldBe true
|
||||||
|
val localAddress = AddressFromURIString(s"akka://${system.name}")
|
||||||
|
actor.path.address shouldEqual localAddress
|
||||||
|
actor.path.address.hasLocalScope shouldBe true
|
||||||
|
|
||||||
|
actor ! "ping"
|
||||||
|
expectMsgType[ActorRef].path.address shouldEqual localAddress
|
||||||
|
}
|
||||||
|
|
||||||
|
runOn(third) {
|
||||||
|
Cluster(system).state.isMemberUp(node(third).address) shouldBe false
|
||||||
|
assertIsLocalRef()
|
||||||
|
}
|
||||||
|
enterBarrier("CARP-outside-cluster-local-validated")
|
||||||
|
|
||||||
|
runOn(second) {
|
||||||
|
assertIsLocalRef()
|
||||||
|
}
|
||||||
|
enterBarrier("CARP-inside-cluster-to-non-member-local-validated")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -34,6 +34,8 @@ class ResetSystemMessageSeqNrSpec extends ArteryMultiNodeSpec("""
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
Cluster(system).state.members.map(_.uniqueAddress) should ===(
|
Cluster(system).state.members.map(_.uniqueAddress) should ===(
|
||||||
Set(Cluster(system).selfUniqueAddress, Cluster(sys2).selfUniqueAddress))
|
Set(Cluster(system).selfUniqueAddress, Cluster(sys2).selfUniqueAddress))
|
||||||
|
|
||||||
|
Cluster(system).state.members.forall(_.status == MemberStatus.Up) shouldBe true
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -67,6 +69,8 @@ class ResetSystemMessageSeqNrSpec extends ArteryMultiNodeSpec("""
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
Cluster(system).state.members.map(_.uniqueAddress) should ===(
|
Cluster(system).state.members.map(_.uniqueAddress) should ===(
|
||||||
Set(Cluster(system).selfUniqueAddress, Cluster(sys3).selfUniqueAddress))
|
Set(Cluster(system).selfUniqueAddress, Cluster(sys3).selfUniqueAddress))
|
||||||
|
|
||||||
|
Cluster(system).state.members.forall(_.status == MemberStatus.Up) shouldBe true
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -189,10 +189,35 @@ are specific to classic search for them in: [`akka-remote/reference.conf`](/akka
|
||||||
The following documents configuration changes and behavior changes where no action is required. In some cases the old
|
The following documents configuration changes and behavior changes where no action is required. In some cases the old
|
||||||
behavior can be restored via configuration.
|
behavior can be restored via configuration.
|
||||||
|
|
||||||
### Remoting dependencies have been made optional
|
### Remoting
|
||||||
|
|
||||||
|
#### Remoting dependencies have been made optional
|
||||||
|
|
||||||
Classic remoting depends on Netty and Artery UDP depends on Aeron. These are now both optional dependencies that need
|
Classic remoting depends on Netty and Artery UDP depends on Aeron. These are now both optional dependencies that need
|
||||||
to be explicitly added. See @ref[classic remoting](../remoting.md) or [artery remoting](../remoting-artery.md) for instructions.
|
to be explicitly added. See @ref[classic remoting](../remoting.md) or @ref[artery remoting](../remoting-artery.md) for instructions.
|
||||||
|
|
||||||
|
#### Remote watch and deployment have been disabled without Cluster use
|
||||||
|
|
||||||
|
By default, these remoting features are disabled when not using Akka Cluster:
|
||||||
|
|
||||||
|
* Remote Deployment: falls back to creating a local actor
|
||||||
|
* Remote Watch: ignores the watch and unwatch request, and `Terminated` will not be delivered when the remote actor is stopped or if a remote node crashes
|
||||||
|
|
||||||
|
When used with Cluster, all previous behavior is the same except a remote watch of an actor is no longer possible before a node joins a cluster, only after.
|
||||||
|
|
||||||
|
To optionally enable them without Cluster, if you understand
|
||||||
|
the @ref[consequences](../remoting-artery.md#quarantine), set
|
||||||
|
```
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on`.
|
||||||
|
```
|
||||||
|
|
||||||
|
When used without Cluster
|
||||||
|
|
||||||
|
* An initial warning is logged on startup of `RemoteActorRefProvider`
|
||||||
|
* A warning will be logged on remote watch attempts, which you can suppress by setting
|
||||||
|
```
|
||||||
|
akka.remote.warn-unsafe-watch-without-cluster = off
|
||||||
|
```
|
||||||
|
|
||||||
## Schedule periodically with fixed-delay vs. fixed-rate
|
## Schedule periodically with fixed-delay vs. fixed-rate
|
||||||
|
|
||||||
|
|
@ -356,5 +381,5 @@ made before finalizing the APIs. Compared to Akka 2.5.x the source incompatible
|
||||||
|
|
||||||
#### Akka Typed Stream API changes
|
#### Akka Typed Stream API changes
|
||||||
|
|
||||||
* `ActorSoruce.actorRef` relying on `PartialFunction` has been replaced in the Java API with a variant more suitable to be called by Java.
|
* `ActorSource.actorRef` relying on `PartialFunction` has been replaced in the Java API with a variant more suitable to be called by Java.
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,7 @@ class RemoteDeploymentDocSpec extends AkkaSpec("""
|
||||||
akka.actor.provider = remote
|
akka.actor.provider = remote
|
||||||
akka.remote.classic.netty.tcp.port = 0
|
akka.remote.classic.netty.tcp.port = 0
|
||||||
akka.remote.artery.canonical.port = 0
|
akka.remote.artery.canonical.port = 0
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""") with ImplicitSender {
|
""") with ImplicitSender {
|
||||||
|
|
||||||
import RemoteDeploymentDocSpec._
|
import RemoteDeploymentDocSpec._
|
||||||
|
|
|
||||||
|
|
@ -21,6 +21,7 @@ class NewRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig {
|
||||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
||||||
akka.remote.log-remote-lifecycle-events = off
|
akka.remote.log-remote-lifecycle-events = off
|
||||||
akka.remote.artery.enabled = $artery
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""").withFallback(RemotingMultiNodeSpec.commonConfig)))
|
""").withFallback(RemotingMultiNodeSpec.commonConfig)))
|
||||||
|
|
||||||
val master = role("master")
|
val master = role("master")
|
||||||
|
|
|
||||||
|
|
@ -24,6 +24,7 @@ class RemoteDeploymentDeathWatchMultiJvmSpec(artery: Boolean) extends MultiNodeC
|
||||||
akka.loglevel = INFO
|
akka.loglevel = INFO
|
||||||
akka.remote.log-remote-lifecycle-events = off
|
akka.remote.log-remote-lifecycle-events = off
|
||||||
akka.remote.artery.enabled = $artery
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
||||||
|
|
||||||
deployOn(second, """/hello.remote = "@third@" """)
|
deployOn(second, """/hello.remote = "@third@" """)
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,351 @@
|
||||||
|
/*
|
||||||
|
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.remote
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
import akka.actor.ActorIdentity
|
||||||
|
import akka.actor.ActorRef
|
||||||
|
import akka.actor.ActorSystemImpl
|
||||||
|
import akka.actor.Address
|
||||||
|
import akka.actor.AddressFromURIString
|
||||||
|
import akka.actor.Identify
|
||||||
|
import akka.actor.Nobody
|
||||||
|
import akka.actor.PoisonPill
|
||||||
|
import akka.actor.Props
|
||||||
|
import akka.remote.RemoteNodeDeathWatchSpec.UnwatchIt
|
||||||
|
import akka.remote.RemoteWatcher.Stats
|
||||||
|
import akka.remote.routing.RemoteRouterConfig
|
||||||
|
import akka.remote.testconductor.RoleName
|
||||||
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
|
import akka.routing.Broadcast
|
||||||
|
import akka.routing.FromConfig
|
||||||
|
import akka.routing.RoundRobinGroup
|
||||||
|
import akka.routing.RoundRobinPool
|
||||||
|
import akka.routing.RoutedActorRef
|
||||||
|
import akka.testkit.TestProbe
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
|
class RemotingFeaturesConfig(val useUnsafe: Boolean, artery: Boolean) extends MultiNodeConfig {
|
||||||
|
|
||||||
|
val first = role("first")
|
||||||
|
val second = role("second")
|
||||||
|
val third = role("third")
|
||||||
|
val fourth = role("fourth")
|
||||||
|
|
||||||
|
val workerInstances = 3
|
||||||
|
val iterationCount = 10
|
||||||
|
|
||||||
|
protected val baseConfig = ConfigFactory.parseString(s"""
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = $useUnsafe
|
||||||
|
akka.remote.log-remote-lifecycle-events = off
|
||||||
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.artery.advanced.flight-recorder.enabled = off
|
||||||
|
""").withFallback(RemotingMultiNodeSpec.commonConfig)
|
||||||
|
|
||||||
|
commonConfig(debugConfig(on = false).withFallback(baseConfig))
|
||||||
|
|
||||||
|
deployOnAll(s"""
|
||||||
|
/sampleActor {
|
||||||
|
remote = "@second@"
|
||||||
|
}
|
||||||
|
/service-hello {
|
||||||
|
router = round-robin-pool
|
||||||
|
nr-of-instances = $workerInstances
|
||||||
|
target.nodes = ["@first@", "@second@", "@third@"]
|
||||||
|
}
|
||||||
|
|
||||||
|
/service-hello2 {
|
||||||
|
router = round-robin-pool
|
||||||
|
target.nodes = ["@first@", "@second@", "@third@"]
|
||||||
|
}
|
||||||
|
|
||||||
|
/service-hello3 {
|
||||||
|
router = round-robin-group
|
||||||
|
routees.paths = [
|
||||||
|
"@first@/user/target-first",
|
||||||
|
"@second@/user/target-second",
|
||||||
|
"@third@/user/target-third"]
|
||||||
|
}
|
||||||
|
""")
|
||||||
|
}
|
||||||
|
|
||||||
|
class RemotingFeaturesSafeMultiJvmNode1 extends RemotingFeaturesSafeSpec
|
||||||
|
class RemotingFeaturesSafeMultiJvmNode2 extends RemotingFeaturesSafeSpec
|
||||||
|
class RemotingFeaturesSafeMultiJvmNode3 extends RemotingFeaturesSafeSpec
|
||||||
|
class RemotingFeaturesSafeMultiJvmNode4 extends RemotingFeaturesSafeSpec
|
||||||
|
|
||||||
|
class RemotingFeaturesUnsafeMultiJvmNode1 extends RemotingFeaturesUnsafeSpec
|
||||||
|
class RemotingFeaturesUnsafeMultiJvmNode2 extends RemotingFeaturesUnsafeSpec
|
||||||
|
class RemotingFeaturesUnsafeMultiJvmNode3 extends RemotingFeaturesUnsafeSpec
|
||||||
|
class RemotingFeaturesUnsafeMultiJvmNode4 extends RemotingFeaturesUnsafeSpec
|
||||||
|
|
||||||
|
abstract class RemotingFeaturesSafeSpec
|
||||||
|
extends RemotingFeaturesSpec(new RemotingFeaturesConfig(useUnsafe = false, artery = true)) {
|
||||||
|
|
||||||
|
import RemoteNodeDeathWatchSpec.ProbeActor
|
||||||
|
import multiNodeConfig._
|
||||||
|
|
||||||
|
"Remoting without Cluster" must {
|
||||||
|
|
||||||
|
"not intercept and send system messages `Watch`/`Unwatch` to `RemoteWatcher` in the provider" in {
|
||||||
|
runOn(second) {
|
||||||
|
val watchee = system.actorOf(Props(classOf[ProbeActor], probe.ref), "watchee")
|
||||||
|
enterBarrier("started")
|
||||||
|
assertWatchNotIntercepted(identify(first, "watcher"), watchee, first)
|
||||||
|
}
|
||||||
|
runOn(first) {
|
||||||
|
val watcher = system.actorOf(Props(classOf[ProbeActor], probe.ref), "watcher")
|
||||||
|
enterBarrier("started")
|
||||||
|
assertWatchNotIntercepted(watcher, identify(second, "watchee"), second)
|
||||||
|
}
|
||||||
|
runOn(third, fourth) {
|
||||||
|
enterBarrier("started")
|
||||||
|
enterBarrier("watch-not-intercepted")
|
||||||
|
}
|
||||||
|
def assertWatchNotIntercepted(watcher: ActorRef, watchee: ActorRef, remoteRole: RoleName): Unit = {
|
||||||
|
val rar = remoteActorRef(remoteRole)
|
||||||
|
rar.isWatchIntercepted(watchee = rar, watcher = watcher) shouldBe false
|
||||||
|
rar.isWatchIntercepted(watchee = watchee, watcher = watcher) shouldBe false
|
||||||
|
enterBarrier("watch-not-intercepted")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"not create a remote actor from deployment config when remote features are disabled" in {
|
||||||
|
runOn(first) {
|
||||||
|
val actor = system.actorOf(Props(classOf[ProbeActor], probe.ref), "sampleActor")
|
||||||
|
actor ! Identify(1)
|
||||||
|
expectMsgType[ActorIdentity].ref.get.path.address.hasGlobalScope shouldBe false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract class RemotingFeaturesUnsafeSpec
|
||||||
|
extends RemotingFeaturesSpec(new RemotingFeaturesConfig(useUnsafe = true, artery = true)) {
|
||||||
|
|
||||||
|
import RemoteNodeDeathWatchSpec.Ack
|
||||||
|
import RemoteNodeDeathWatchSpec.DeathWatchIt
|
||||||
|
import RemoteNodeDeathWatchSpec.ProbeActor
|
||||||
|
import RemoteNodeDeathWatchSpec.WatchIt
|
||||||
|
import multiNodeConfig._
|
||||||
|
|
||||||
|
def stats(watcher: ActorRef, message: DeathWatchIt): Stats = {
|
||||||
|
probe.send(watcher, message)
|
||||||
|
probe.expectMsg(1.second, Ack)
|
||||||
|
provider.remoteWatcher.get ! Stats
|
||||||
|
expectMsgType[Stats]
|
||||||
|
}
|
||||||
|
|
||||||
|
"Remoting with UseUnsafeRemoteFeaturesWithoutCluster enabled" must {
|
||||||
|
|
||||||
|
"intercept and send system messages `Watch`/`Unwatch` to `RemoteWatcher` in the provider" in {
|
||||||
|
runOn(second) {
|
||||||
|
val watchee = system.actorOf(Props(classOf[ProbeActor], probe.ref), "watchee")
|
||||||
|
enterBarrier("watchee-started")
|
||||||
|
enterBarrier("watcher-started")
|
||||||
|
assertWatchIntercepted(identify(first, "watcher"), watchee, first)
|
||||||
|
}
|
||||||
|
runOn(first) {
|
||||||
|
enterBarrier("watchee-started")
|
||||||
|
val watcher = system.actorOf(Props(classOf[ProbeActor], probe.ref), "watcher")
|
||||||
|
enterBarrier("watcher-started")
|
||||||
|
assertWatchIntercepted(watcher, identify(second, "watchee"), second)
|
||||||
|
}
|
||||||
|
runOn(third, fourth) {
|
||||||
|
enterBarrier("watchee-started")
|
||||||
|
enterBarrier("watcher-started")
|
||||||
|
enterBarrier("watch-intercepted")
|
||||||
|
enterBarrier("watch-not-intercepted")
|
||||||
|
}
|
||||||
|
def assertWatchIntercepted(watcher: ActorRef, watchee: ActorRef, remoteRole: RoleName): Unit = {
|
||||||
|
val remoteWatcher = identifyWithPath(remoteRole, "system", "remote-watcher")
|
||||||
|
val rar = remoteActorRef(remoteRole)
|
||||||
|
rar.isWatchIntercepted(watchee = rar, watcher = watcher) shouldBe true
|
||||||
|
rar.isWatchIntercepted(watchee = rar, watcher = remoteWatcher) shouldBe true
|
||||||
|
enterBarrier("watch-intercepted")
|
||||||
|
|
||||||
|
rar.isWatchIntercepted(watchee = watchee, watcher = rar) shouldBe false
|
||||||
|
enterBarrier("watch-not-intercepted")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"create a remote actor from deployment config when remote features are disabled" in {
|
||||||
|
runOn(first) {
|
||||||
|
val secondAddress = node(second).address
|
||||||
|
val actor = system.actorOf(Props(classOf[ProbeActor], probe.ref), "sampleActor")
|
||||||
|
actor.path.address shouldEqual secondAddress
|
||||||
|
actor.isInstanceOf[RemoteActorRef] shouldBe true
|
||||||
|
actor.path.address.hasGlobalScope shouldBe true
|
||||||
|
}
|
||||||
|
enterBarrier("remote-actorOf-validated")
|
||||||
|
}
|
||||||
|
|
||||||
|
"`Watch` and `Unwatch` from `RemoteWatcher`" in {
|
||||||
|
runOn(first) {
|
||||||
|
val watcher = identify(first, "watcher")
|
||||||
|
val watchee = identify(second, "watchee")
|
||||||
|
awaitAssert(stats(watcher, WatchIt(watchee)).watchingRefs == Set((watchee, watcher)), 2.seconds)
|
||||||
|
enterBarrier("system-message1-received-by-remoteWatcher")
|
||||||
|
|
||||||
|
awaitAssert(stats(watcher, UnwatchIt(watchee)).watching == 0, 2.seconds)
|
||||||
|
enterBarrier("system-message2-received-by-remoteWatcher")
|
||||||
|
}
|
||||||
|
runOn(second, third, fourth) {
|
||||||
|
enterBarrier("system-message1-received-by-remoteWatcher")
|
||||||
|
enterBarrier("system-message2-received-by-remoteWatcher")
|
||||||
|
}
|
||||||
|
enterBarrier("done")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract class RemotingFeaturesSpec(val multiNodeConfig: RemotingFeaturesConfig)
|
||||||
|
extends RemotingMultiNodeSpec(multiNodeConfig) {
|
||||||
|
|
||||||
|
import RemoteWatcher._
|
||||||
|
import akka.remote.routing.RemoteRoundRobinSpec._
|
||||||
|
import multiNodeConfig._
|
||||||
|
|
||||||
|
override def initialParticipants: Int = roles.size
|
||||||
|
|
||||||
|
muteDeadLetters(Heartbeat.getClass)()
|
||||||
|
|
||||||
|
protected val probe = TestProbe()
|
||||||
|
|
||||||
|
protected val provider: RemoteActorRefProvider = RARP(system).provider
|
||||||
|
|
||||||
|
protected def remoteActorRef(role: RoleName): RemoteActorRef = {
|
||||||
|
val remotePath = node(role)
|
||||||
|
val rar = new RemoteActorRef(
|
||||||
|
provider.transport,
|
||||||
|
provider.transport.localAddressForRemote(remotePath.address),
|
||||||
|
remotePath,
|
||||||
|
Nobody,
|
||||||
|
None,
|
||||||
|
None)
|
||||||
|
|
||||||
|
rar.start()
|
||||||
|
rar
|
||||||
|
}
|
||||||
|
|
||||||
|
protected def identify(role: RoleName, actorName: String, within: FiniteDuration = 10.seconds): ActorRef =
|
||||||
|
identifyWithPath(role, "user", actorName, within)
|
||||||
|
|
||||||
|
protected def identifyWithPath(
|
||||||
|
role: RoleName,
|
||||||
|
path: String,
|
||||||
|
actorName: String,
|
||||||
|
within: FiniteDuration = 10.seconds): ActorRef = {
|
||||||
|
system.actorSelection(node(role) / path / actorName) ! Identify(actorName)
|
||||||
|
val id = expectMsgType[ActorIdentity](within)
|
||||||
|
assert(id.ref.isDefined, s"Unable to Identify actor [$actorName] on node [$role]")
|
||||||
|
id.ref.get
|
||||||
|
}
|
||||||
|
|
||||||
|
"A remote round robin group" must {
|
||||||
|
"send messages to remote paths" in {
|
||||||
|
|
||||||
|
runOn(first, second, third) {
|
||||||
|
system.actorOf(Props[SomeActor], name = "target-" + myself.name)
|
||||||
|
enterBarrier("start", "end")
|
||||||
|
}
|
||||||
|
|
||||||
|
runOn(fourth) {
|
||||||
|
enterBarrier("start")
|
||||||
|
val actor = system.actorOf(FromConfig.props(), "service-hello3")
|
||||||
|
actor.isInstanceOf[RoutedActorRef] should ===(true)
|
||||||
|
|
||||||
|
for (_ <- 0 until iterationCount; _ <- 0 until workerInstances) {
|
||||||
|
actor ! "hit"
|
||||||
|
}
|
||||||
|
|
||||||
|
val replies: Map[Address, Int] = receiveWhile(5.seconds, messages = workerInstances * iterationCount) {
|
||||||
|
case ref: ActorRef => ref.path.address
|
||||||
|
}.foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) {
|
||||||
|
case (replyMap, address) => replyMap + (address -> (replyMap(address) + 1))
|
||||||
|
}
|
||||||
|
|
||||||
|
enterBarrier("end")
|
||||||
|
replies.values.forall(_ == iterationCount) shouldBe true
|
||||||
|
replies.get(node(fourth).address) should ===(None)
|
||||||
|
}
|
||||||
|
|
||||||
|
enterBarrier("finished")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"A remote round robin pool" must {
|
||||||
|
s"${if (useUnsafe) "be instantiated on remote node and communicate through its RemoteActorRef"
|
||||||
|
else "not be instantiated on remote node and communicate through its LocalActorRef "} " in {
|
||||||
|
|
||||||
|
runOn(first, second, third) {
|
||||||
|
enterBarrier("start", "broadcast-end", "end")
|
||||||
|
}
|
||||||
|
|
||||||
|
runOn(fourth) {
|
||||||
|
enterBarrier("start")
|
||||||
|
val actor = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]), "service-hello")
|
||||||
|
actor.isInstanceOf[RoutedActorRef] should ===(true)
|
||||||
|
|
||||||
|
for (_ <- 0 until iterationCount; _ <- 0 until workerInstances) {
|
||||||
|
actor ! "hit"
|
||||||
|
}
|
||||||
|
|
||||||
|
val replies = receiveWhile(5.seconds, messages = workerInstances * iterationCount) {
|
||||||
|
case ref: ActorRef => ref.path.address
|
||||||
|
}.foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) {
|
||||||
|
case (replyMap, address) =>
|
||||||
|
if (useUnsafe) address.hasLocalScope shouldBe false
|
||||||
|
else address.hasLocalScope shouldBe true
|
||||||
|
replyMap + (address -> (replyMap.getOrElse(address, 0) + 1))
|
||||||
|
}
|
||||||
|
|
||||||
|
if (useUnsafe) {
|
||||||
|
enterBarrier("broadcast-end")
|
||||||
|
actor ! Broadcast(PoisonPill)
|
||||||
|
|
||||||
|
enterBarrier("end")
|
||||||
|
replies.values.foreach { _ should ===(iterationCount) }
|
||||||
|
} else {
|
||||||
|
enterBarrier("broadcast-end")
|
||||||
|
actor ! Broadcast(PoisonPill)
|
||||||
|
|
||||||
|
enterBarrier("end")
|
||||||
|
val (local, remote) = replies.partition { case (address, _) => address.hasLocalScope }
|
||||||
|
local.size shouldEqual 1
|
||||||
|
remote.size shouldEqual 3
|
||||||
|
val others = Set(first, second, third).map(node(_).address)
|
||||||
|
remote.forall { case (address, _) => others.contains(address) } shouldBe true
|
||||||
|
remote.values.forall(_ == 0) shouldBe true
|
||||||
|
local.values.foreach(_ should ===(iterationCount * remote.size))
|
||||||
|
}
|
||||||
|
replies.get(node(fourth).address) should ===(None)
|
||||||
|
system.stop(actor)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
s"Deploy routers with expected behavior if 'akka.remote.use-unsafe-remote-features-without-cluster=$useUnsafe'" must {
|
||||||
|
"deployments" in {
|
||||||
|
runOn(first, second, third, fourth) {
|
||||||
|
val deployment1 = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(List("service-hello"))
|
||||||
|
val deployment2 = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(List("service-hello2"))
|
||||||
|
val deployment3 = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(List("service-hello3"))
|
||||||
|
val deploys = Set(deployment1, deployment2, deployment3)
|
||||||
|
deploys.flatten.size shouldEqual 3
|
||||||
|
deploys.foreach { deploy =>
|
||||||
|
deploy.get.routerConfig match {
|
||||||
|
case RemoteRouterConfig(pool, nodes) =>
|
||||||
|
pool.nrOfInstances(system) == workerInstances && nodes.size == 3 && !nodes.forall(_.hasLocalScope)
|
||||||
|
case RoundRobinGroup(paths, _) =>
|
||||||
|
paths.size == 3 && !paths.forall(AddressFromURIString(_).hasLocalScope)
|
||||||
|
case _ =>
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -29,6 +29,7 @@ class RemoteNodeDeathWatchConfig(artery: Boolean) extends MultiNodeConfig {
|
||||||
## Use a tighter setting than the default, otherwise it takes 20s for DeathWatch to trigger
|
## Use a tighter setting than the default, otherwise it takes 20s for DeathWatch to trigger
|
||||||
akka.remote.watch-failure-detector.acceptable-heartbeat-pause = 3 s
|
akka.remote.watch-failure-detector.acceptable-heartbeat-pause = 3 s
|
||||||
akka.remote.artery.enabled = $artery
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -63,8 +64,9 @@ abstract class RemoteNodeDeathWatchSlowSpec(artery: Boolean)
|
||||||
}
|
}
|
||||||
|
|
||||||
object RemoteNodeDeathWatchSpec {
|
object RemoteNodeDeathWatchSpec {
|
||||||
final case class WatchIt(watchee: ActorRef)
|
sealed trait DeathWatchIt
|
||||||
final case class UnwatchIt(watchee: ActorRef)
|
final case class WatchIt(watchee: ActorRef) extends DeathWatchIt
|
||||||
|
final case class UnwatchIt(watchee: ActorRef) extends DeathWatchIt
|
||||||
case object Ack
|
case object Ack
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -30,6 +30,7 @@ class RemoteNodeRestartDeathWatchConfig(artery: Boolean) extends MultiNodeConfig
|
||||||
akka.remote.classic.transport-failure-detector.heartbeat-interval = 1 s
|
akka.remote.classic.transport-failure-detector.heartbeat-interval = 1 s
|
||||||
akka.remote.classic.transport-failure-detector.acceptable-heartbeat-pause = 3 s
|
akka.remote.classic.transport-failure-detector.acceptable-heartbeat-pause = 3 s
|
||||||
akka.remote.artery.enabled = $artery
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""")))
|
""")))
|
||||||
|
|
||||||
testTransport(on = true)
|
testTransport(on = true)
|
||||||
|
|
|
||||||
|
|
@ -35,6 +35,7 @@ class RemoteReDeploymentConfig(artery: Boolean) extends MultiNodeConfig {
|
||||||
acceptable-heartbeat-pause=2.5s
|
acceptable-heartbeat-pause=2.5s
|
||||||
}
|
}
|
||||||
akka.remote.artery.enabled = $artery
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
akka.loglevel = INFO
|
akka.loglevel = INFO
|
||||||
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -27,6 +27,7 @@ object TransportFailConfig extends MultiNodeConfig {
|
||||||
|
|
||||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
||||||
akka.loglevel = INFO
|
akka.loglevel = INFO
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
akka.remote.classic {
|
akka.remote.classic {
|
||||||
transport-failure-detector {
|
transport-failure-detector {
|
||||||
implementation-class = "akka.remote.TransportFailSpec$$TestFailureDetector"
|
implementation-class = "akka.remote.TransportFailSpec$$TestFailureDetector"
|
||||||
|
|
|
||||||
|
|
@ -30,6 +30,7 @@ object RemoteNodeShutdownAndComesBackSpec extends MultiNodeConfig {
|
||||||
akka.remote.classic.transport-failure-detector.heartbeat-interval = 1 s
|
akka.remote.classic.transport-failure-detector.heartbeat-interval = 1 s
|
||||||
akka.remote.classic.transport-failure-detector.acceptable-heartbeat-pause = 3 s
|
akka.remote.classic.transport-failure-detector.acceptable-heartbeat-pause = 3 s
|
||||||
akka.remote.watch-failure-detector.acceptable-heartbeat-pause = 60 s
|
akka.remote.watch-failure-detector.acceptable-heartbeat-pause = 60 s
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""")))
|
""")))
|
||||||
|
|
||||||
testTransport(on = true)
|
testTransport(on = true)
|
||||||
|
|
|
||||||
|
|
@ -27,6 +27,7 @@ class RemoteRandomConfig(artery: Boolean) extends MultiNodeConfig {
|
||||||
|
|
||||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
||||||
akka.remote.artery.enabled = $artery
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
||||||
|
|
||||||
deployOnAll("""
|
deployOnAll("""
|
||||||
|
|
|
||||||
|
|
@ -26,6 +26,7 @@ class RemoteRoundRobinConfig(artery: Boolean) extends MultiNodeConfig {
|
||||||
|
|
||||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
||||||
akka.remote.artery.enabled = $artery
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
||||||
|
|
||||||
deployOnAll("""
|
deployOnAll("""
|
||||||
|
|
|
||||||
|
|
@ -28,6 +28,7 @@ class RemoteScatterGatherConfig(artery: Boolean) extends MultiNodeConfig {
|
||||||
|
|
||||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
|
||||||
akka.remote.artery.enabled = $artery
|
akka.remote.artery.enabled = $artery
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
""")).withFallback(RemotingMultiNodeSpec.commonConfig))
|
||||||
|
|
||||||
deployOnAll("""
|
deployOnAll("""
|
||||||
|
|
|
||||||
|
|
@ -22,3 +22,7 @@ ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.ArterySet
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.ArterySettings#Advanced.DriverTimeout")
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.ArterySettings#Advanced.DriverTimeout")
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.ArterySettings#Advanced.ConnectionTimeout")
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.ArterySettings#Advanced.ConnectionTimeout")
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.ArterySettings.LogAeronCounters")
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.artery.ArterySettings.LogAeronCounters")
|
||||||
|
|
||||||
|
# Disable remote watch and remote deployment outside Cluster #26176
|
||||||
|
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.RemoteActorRefProvider.remoteWatcher")
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteWatcher.props")
|
||||||
|
|
|
||||||
|
|
@ -181,7 +181,17 @@ akka {
|
||||||
# Using remoting directly is typically not desirable, so a warning will
|
# Using remoting directly is typically not desirable, so a warning will
|
||||||
# be shown to make this clear. Set this setting to 'off' to suppress that
|
# be shown to make this clear. Set this setting to 'off' to suppress that
|
||||||
# warning.
|
# warning.
|
||||||
warn-about-direct-use = "on"
|
warn-about-direct-use = on
|
||||||
|
|
||||||
|
|
||||||
|
# If Cluster is not used, remote watch and deployment are disabled.
|
||||||
|
# To optionally use them while not using Cluster, set to 'on'.
|
||||||
|
use-unsafe-remote-features-without-cluster = off
|
||||||
|
|
||||||
|
# A warning will be logged on remote watch attempts if Cluster
|
||||||
|
# is not in use and 'use-unsafe-remote-features-without-cluster'
|
||||||
|
# is 'off'. Set this to 'off' to suppress these.
|
||||||
|
warn-unsafe-watch-without-cluster = on
|
||||||
|
|
||||||
# Settings for the Phi accrual failure detector (http://www.jaist.ac.jp/~defago/files/pdf/IS_RR_2004_010.pdf
|
# Settings for the Phi accrual failure detector (http://www.jaist.ac.jp/~defago/files/pdf/IS_RR_2004_010.pdf
|
||||||
# [Hayashibara et al]) used for remote death watch.
|
# [Hayashibara et al]) used for remote death watch.
|
||||||
|
|
|
||||||
|
|
@ -4,32 +4,38 @@
|
||||||
|
|
||||||
package akka.remote
|
package akka.remote
|
||||||
|
|
||||||
import akka.Done
|
|
||||||
import akka.actor._
|
|
||||||
import akka.dispatch.sysmsg._
|
|
||||||
import akka.event.{ EventStream, Logging, LoggingAdapter }
|
|
||||||
import akka.event.Logging.Error
|
|
||||||
import akka.pattern.pipe
|
|
||||||
|
|
||||||
import scala.util.control.NonFatal
|
|
||||||
import scala.util.Failure
|
|
||||||
import akka.actor.SystemGuardian.{ RegisterTerminationHook, TerminationHook, TerminationHookDone }
|
|
||||||
|
|
||||||
import scala.util.control.Exception.Catcher
|
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
|
import scala.util.Failure
|
||||||
|
import scala.util.control.Exception.Catcher
|
||||||
|
import scala.util.control.NonFatal
|
||||||
|
|
||||||
import akka.ConfigurationException
|
import akka.ConfigurationException
|
||||||
|
import akka.Done
|
||||||
|
import akka.actor.SystemGuardian.RegisterTerminationHook
|
||||||
|
import akka.actor.SystemGuardian.TerminationHook
|
||||||
|
import akka.actor.SystemGuardian.TerminationHookDone
|
||||||
|
import akka.actor._
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
import akka.dispatch.RequiresMessageQueue
|
||||||
import akka.remote.artery.ArteryTransport
|
import akka.dispatch.UnboundedMessageQueueSemantics
|
||||||
import akka.remote.artery.aeron.ArteryAeronUdpTransport
|
import akka.dispatch.sysmsg._
|
||||||
|
import akka.event.EventStream
|
||||||
|
import akka.event.Logging
|
||||||
|
import akka.event.Logging.Error
|
||||||
|
import akka.event.LoggingAdapter
|
||||||
|
import akka.pattern.pipe
|
||||||
import akka.remote.artery.ArterySettings
|
import akka.remote.artery.ArterySettings
|
||||||
import akka.remote.artery.ArterySettings.AeronUpd
|
import akka.remote.artery.ArterySettings.AeronUpd
|
||||||
import akka.util.{ ErrorMessages, OptionVal }
|
import akka.remote.artery.ArteryTransport
|
||||||
import akka.remote.artery.OutboundEnvelope
|
import akka.remote.artery.OutboundEnvelope
|
||||||
import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope
|
import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope
|
||||||
import akka.remote.serialization.ActorRefResolveThreadLocalCache
|
import akka.remote.artery.aeron.ArteryAeronUdpTransport
|
||||||
import akka.remote.artery.tcp.ArteryTcpTransport
|
import akka.remote.artery.tcp.ArteryTcpTransport
|
||||||
|
import akka.remote.serialization.ActorRefResolveThreadLocalCache
|
||||||
import akka.serialization.Serialization
|
import akka.serialization.Serialization
|
||||||
|
import akka.util.ErrorMessages
|
||||||
|
import akka.util.OptionVal
|
||||||
|
import akka.util.unused
|
||||||
import com.github.ghik.silencer.silent
|
import com.github.ghik.silencer.silent
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -152,6 +158,13 @@ private[akka] class RemoteActorRefProvider(
|
||||||
|
|
||||||
val remoteSettings: RemoteSettings = new RemoteSettings(settings.config)
|
val remoteSettings: RemoteSettings = new RemoteSettings(settings.config)
|
||||||
|
|
||||||
|
private[akka] final val hasClusterOrUseUnsafe = settings.HasCluster || remoteSettings.UseUnsafeRemoteFeaturesWithoutCluster
|
||||||
|
|
||||||
|
private val warnOnUnsafeRemote =
|
||||||
|
!settings.HasCluster &&
|
||||||
|
!remoteSettings.UseUnsafeRemoteFeaturesWithoutCluster &&
|
||||||
|
remoteSettings.WarnUnsafeWatchWithoutCluster
|
||||||
|
|
||||||
override val deployer: Deployer = createDeployer
|
override val deployer: Deployer = createDeployer
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -160,7 +173,7 @@ private[akka] class RemoteActorRefProvider(
|
||||||
*/
|
*/
|
||||||
protected def createDeployer: RemoteDeployer = new RemoteDeployer(settings, dynamicAccess)
|
protected def createDeployer: RemoteDeployer = new RemoteDeployer(settings, dynamicAccess)
|
||||||
|
|
||||||
private val local = new LocalActorRefProvider(
|
private[akka] val local = new LocalActorRefProvider(
|
||||||
systemName,
|
systemName,
|
||||||
settings,
|
settings,
|
||||||
eventStream,
|
eventStream,
|
||||||
|
|
@ -194,10 +207,10 @@ private[akka] class RemoteActorRefProvider(
|
||||||
// This actor ensures the ordering of shutdown between remoteDaemon and the transport
|
// This actor ensures the ordering of shutdown between remoteDaemon and the transport
|
||||||
@volatile private var remotingTerminator: ActorRef = _
|
@volatile private var remotingTerminator: ActorRef = _
|
||||||
|
|
||||||
@volatile private var _remoteWatcher: ActorRef = _
|
@volatile private var _remoteWatcher: Option[ActorRef] = None
|
||||||
private[akka] def remoteWatcher = _remoteWatcher
|
private[akka] def remoteWatcher: Option[ActorRef] = _remoteWatcher
|
||||||
|
|
||||||
@volatile private var remoteDeploymentWatcher: ActorRef = _
|
@volatile private var remoteDeploymentWatcher: Option[ActorRef] = None
|
||||||
|
|
||||||
@volatile private var actorRefResolveThreadLocalCache: ActorRefResolveThreadLocalCache = _
|
@volatile private var actorRefResolveThreadLocalCache: ActorRefResolveThreadLocalCache = _
|
||||||
|
|
||||||
|
|
@ -239,13 +252,14 @@ private[akka] class RemoteActorRefProvider(
|
||||||
|
|
||||||
_log = Logging.withMarker(eventStream, getClass.getName)
|
_log = Logging.withMarker(eventStream, getClass.getName)
|
||||||
|
|
||||||
showDirectUseWarningIfRequired()
|
warnIfDirectUse()
|
||||||
|
warnIfUseUnsafeWithoutCluster()
|
||||||
|
|
||||||
// this enables reception of remote requests
|
// this enables reception of remote requests
|
||||||
transport.start()
|
transport.start()
|
||||||
|
|
||||||
_remoteWatcher = createRemoteWatcher(system)
|
_remoteWatcher = createOrNone[ActorRef](createRemoteWatcher(system))
|
||||||
remoteDeploymentWatcher = createRemoteDeploymentWatcher(system)
|
remoteDeploymentWatcher = createOrNone[ActorRef](createRemoteDeploymentWatcher(system))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def checkNettyOnClassPath(system: ActorSystemImpl): Unit = {
|
private def checkNettyOnClassPath(system: ActorSystemImpl): Unit = {
|
||||||
|
|
@ -280,16 +294,13 @@ private[akka] class RemoteActorRefProvider(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Will call the provided `func` if using Cluster or explicitly enabled unsafe remote features. */
|
||||||
|
private def createOrNone[T](func: => T): Option[T] = if (hasClusterOrUseUnsafe) Some(func) else None
|
||||||
|
|
||||||
protected def createRemoteWatcher(system: ActorSystemImpl): ActorRef = {
|
protected def createRemoteWatcher(system: ActorSystemImpl): ActorRef = {
|
||||||
import remoteSettings._
|
import remoteSettings._
|
||||||
val failureDetector = createRemoteWatcherFailureDetector(system)
|
|
||||||
system.systemActorOf(
|
system.systemActorOf(
|
||||||
configureDispatcher(
|
configureDispatcher(RemoteWatcher.props(remoteSettings, createRemoteWatcherFailureDetector(system))),
|
||||||
RemoteWatcher.props(
|
|
||||||
failureDetector,
|
|
||||||
heartbeatInterval = WatchHeartBeatInterval,
|
|
||||||
unreachableReaperInterval = WatchUnreachableReaperInterval,
|
|
||||||
heartbeatExpectedResponseAfter = WatchHeartbeatExpectedResponseAfter)),
|
|
||||||
"remote-watcher")
|
"remote-watcher")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -309,7 +320,7 @@ private[akka] class RemoteActorRefProvider(
|
||||||
"remote-deployment-watcher")
|
"remote-deployment-watcher")
|
||||||
|
|
||||||
/** Can be overridden when using RemoteActorRefProvider as a superclass rather than directly */
|
/** Can be overridden when using RemoteActorRefProvider as a superclass rather than directly */
|
||||||
protected def showDirectUseWarningIfRequired() = {
|
protected def warnIfDirectUse() = {
|
||||||
if (remoteSettings.WarnAboutDirectUse) {
|
if (remoteSettings.WarnAboutDirectUse) {
|
||||||
log.warning(
|
log.warning(
|
||||||
"Using the 'remote' ActorRefProvider directly, which is a low-level layer. " +
|
"Using the 'remote' ActorRefProvider directly, which is a low-level layer. " +
|
||||||
|
|
@ -317,6 +328,38 @@ private[akka] class RemoteActorRefProvider(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Log on `init` similar to `warnIfDirectUse`.
|
||||||
|
private[akka] def warnIfUseUnsafeWithoutCluster(): Unit =
|
||||||
|
if (!settings.HasCluster) {
|
||||||
|
val msg =
|
||||||
|
if (remoteSettings.UseUnsafeRemoteFeaturesWithoutCluster)
|
||||||
|
"`akka.remote.use-unsafe-remote-features-without-cluster` has been enabled."
|
||||||
|
else
|
||||||
|
"Using Akka Cluster is recommended if you need remote watch and deploy."
|
||||||
|
log.warning(s"Cluster not in use - {}", msg)
|
||||||
|
}
|
||||||
|
|
||||||
|
protected def warnOnUnsafe(message: String): Unit =
|
||||||
|
if (warnOnUnsafeRemote) log.warning(message)
|
||||||
|
else log.debug(message)
|
||||||
|
|
||||||
|
/** Logs if deathwatch message is intentionally dropped. To disable
|
||||||
|
* warnings set `akka.remote.warn-unsafe-watch-without-cluster` to `off`
|
||||||
|
* or use Akka Cluster.
|
||||||
|
*/
|
||||||
|
private[akka] def warnIfUnsafeDeathwatchWithoutCluster(watchee: ActorRef, watcher: ActorRef, action: String): Unit =
|
||||||
|
warnOnUnsafe(s"Dropped remote $action: disabled for [$watcher -> $watchee]")
|
||||||
|
|
||||||
|
/** If `warnOnUnsafeRemote`, this logs a warning if `actorOf` falls back to `LocalActorRef`
|
||||||
|
* versus creating a `RemoteActorRef`. Override to log a more granular reason if using
|
||||||
|
* `RemoteActorRefProvider` as a superclass.
|
||||||
|
*/
|
||||||
|
protected def warnIfNotRemoteActorRef(path: ActorPath): Unit =
|
||||||
|
warnOnUnsafe(s"Remote deploy of [$path] is not allowed, falling back to local.")
|
||||||
|
|
||||||
|
/** Override to add any additional checks if using `RemoteActorRefProvider` as a superclass. */
|
||||||
|
protected def shouldCreateRemoteActorRef(@unused system: ActorSystem, @unused address: Address): Boolean = true
|
||||||
|
|
||||||
def actorOf(
|
def actorOf(
|
||||||
system: ActorSystemImpl,
|
system: ActorSystemImpl,
|
||||||
props: Props,
|
props: Props,
|
||||||
|
|
@ -383,6 +426,11 @@ private[akka] class RemoteActorRefProvider(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def warnThenFallback() = {
|
||||||
|
warnIfNotRemoteActorRef(path)
|
||||||
|
local.actorOf(system, props, supervisor, path, systemService, deployment.headOption, false, async)
|
||||||
|
}
|
||||||
|
|
||||||
(Iterator(props.deploy) ++ deployment.iterator).reduce((a, b) => b.withFallback(a)) match {
|
(Iterator(props.deploy) ++ deployment.iterator).reduce((a, b) => b.withFallback(a)) match {
|
||||||
case d @ Deploy(_, _, _, RemoteScope(address), _, _) =>
|
case d @ Deploy(_, _, _, RemoteScope(address), _, _) =>
|
||||||
if (hasAddress(address)) {
|
if (hasAddress(address)) {
|
||||||
|
|
@ -392,27 +440,29 @@ private[akka] class RemoteActorRefProvider(
|
||||||
s"${ErrorMessages.RemoteDeploymentConfigErrorPrefix} for local-only Props at [$path]")
|
s"${ErrorMessages.RemoteDeploymentConfigErrorPrefix} for local-only Props at [$path]")
|
||||||
} else
|
} else
|
||||||
try {
|
try {
|
||||||
try {
|
if (hasClusterOrUseUnsafe && shouldCreateRemoteActorRef(system, address)) {
|
||||||
// for consistency we check configuration of dispatcher and mailbox locally
|
try {
|
||||||
val dispatcher = system.dispatchers.lookup(props.dispatcher)
|
// for consistency we check configuration of dispatcher and mailbox locally
|
||||||
system.mailboxes.getMailboxType(props, dispatcher.configurator.config)
|
val dispatcher = system.dispatchers.lookup(props.dispatcher)
|
||||||
} catch {
|
system.mailboxes.getMailboxType(props, dispatcher.configurator.config)
|
||||||
case NonFatal(e) =>
|
} catch {
|
||||||
throw new ConfigurationException(
|
case NonFatal(e) =>
|
||||||
s"configuration problem while creating [$path] with dispatcher [${props.dispatcher}] and mailbox [${props.mailbox}]",
|
throw new ConfigurationException(
|
||||||
e)
|
s"configuration problem while creating [$path] with dispatcher [${props.dispatcher}] and mailbox [${props.mailbox}]",
|
||||||
}
|
e)
|
||||||
val localAddress = transport.localAddressForRemote(address)
|
}
|
||||||
val rpath =
|
val localAddress = transport.localAddressForRemote(address)
|
||||||
(RootActorPath(address) / "remote" / localAddress.protocol / localAddress.hostPort / path.elements)
|
val rpath =
|
||||||
.withUid(path.uid)
|
(RootActorPath(address) / "remote" / localAddress.protocol / localAddress.hostPort / path.elements)
|
||||||
new RemoteActorRef(transport, localAddress, rpath, supervisor, Some(props), Some(d))
|
.withUid(path.uid)
|
||||||
|
new RemoteActorRef(transport, localAddress, rpath, supervisor, Some(props), Some(d))
|
||||||
|
} else warnThenFallback()
|
||||||
|
|
||||||
} catch {
|
} catch {
|
||||||
case NonFatal(e) => throw new IllegalArgumentException(s"remote deployment failed for [$path]", e)
|
case NonFatal(e) => throw new IllegalArgumentException(s"remote deployment failed for [$path]", e)
|
||||||
}
|
}
|
||||||
|
|
||||||
case _ =>
|
case _ =>
|
||||||
local.actorOf(system, props, supervisor, path, systemService, deployment.headOption, false, async)
|
warnThenFallback()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -464,7 +514,7 @@ private[akka] class RemoteActorRefProvider(
|
||||||
}
|
}
|
||||||
|
|
||||||
def resolveActorRef(path: String): ActorRef = {
|
def resolveActorRef(path: String): ActorRef = {
|
||||||
// using thread local LRU cache, which will call internalRresolveActorRef
|
// using thread local LRU cache, which will call internalResolveActorRef
|
||||||
// if the value is not cached
|
// if the value is not cached
|
||||||
actorRefResolveThreadLocalCache match {
|
actorRefResolveThreadLocalCache match {
|
||||||
case null => internalResolveActorRef(path) // not initialized yet
|
case null => internalResolveActorRef(path) // not initialized yet
|
||||||
|
|
@ -521,17 +571,20 @@ private[akka] class RemoteActorRefProvider(
|
||||||
/**
|
/**
|
||||||
* Using (checking out) actor on a specific node.
|
* Using (checking out) actor on a specific node.
|
||||||
*/
|
*/
|
||||||
def useActorOnNode(ref: ActorRef, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = {
|
def useActorOnNode(ref: ActorRef, props: Props, deploy: Deploy, supervisor: ActorRef): Unit =
|
||||||
log.debug("[{}] Instantiating Remote Actor [{}]", rootPath, ref.path)
|
remoteDeploymentWatcher match {
|
||||||
|
case Some(watcher) =>
|
||||||
|
log.debug("[{}] Instantiating Remote Actor [{}]", rootPath, ref.path)
|
||||||
|
|
||||||
// we don’t wait for the ACK, because the remote end will process this command before any other message to the new actor
|
// we don’t wait for the ACK, because the remote end will process this command before any other message to the new actor
|
||||||
// actorSelection can't be used here because then it is not guaranteed that the actor is created
|
// actorSelection can't be used here because then it is not guaranteed that the actor is created
|
||||||
// before someone can send messages to it
|
// before someone can send messages to it
|
||||||
resolveActorRef(RootActorPath(ref.path.address) / "remote") !
|
resolveActorRef(RootActorPath(ref.path.address) / "remote") !
|
||||||
DaemonMsgCreate(props, deploy, ref.path.toSerializationFormat, supervisor)
|
DaemonMsgCreate(props, deploy, ref.path.toSerializationFormat, supervisor)
|
||||||
|
|
||||||
remoteDeploymentWatcher ! RemoteDeploymentWatcher.WatchRemote(ref, supervisor)
|
watcher ! RemoteDeploymentWatcher.WatchRemote(ref, supervisor)
|
||||||
}
|
case None => warnIfUseUnsafeWithoutCluster()
|
||||||
|
}
|
||||||
|
|
||||||
def getExternalAddressFor(addr: Address): Option[Address] = {
|
def getExternalAddressFor(addr: Address): Option[Address] = {
|
||||||
addr match {
|
addr match {
|
||||||
|
|
@ -601,8 +654,7 @@ private[akka] class RemoteActorRef private[akka] (
|
||||||
case t: ArteryTransport =>
|
case t: ArteryTransport =>
|
||||||
// detect mistakes such as using "akka.tcp" with Artery
|
// detect mistakes such as using "akka.tcp" with Artery
|
||||||
if (path.address.protocol != t.localAddress.address.protocol)
|
if (path.address.protocol != t.localAddress.address.protocol)
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(s"Wrong protocol of [$path], expected [${t.localAddress.address.protocol}]")
|
||||||
s"Wrong protocol of [${path}], expected [${t.localAddress.address.protocol}]")
|
|
||||||
case _ =>
|
case _ =>
|
||||||
}
|
}
|
||||||
@volatile private[remote] var cachedAssociation: artery.Association = null
|
@volatile private[remote] var cachedAssociation: artery.Association = null
|
||||||
|
|
@ -636,10 +688,12 @@ private[akka] class RemoteActorRef private[akka] (
|
||||||
/**
|
/**
|
||||||
* Determine if a watch/unwatch message must be handled by the remoteWatcher actor, or sent to this remote ref
|
* Determine if a watch/unwatch message must be handled by the remoteWatcher actor, or sent to this remote ref
|
||||||
*/
|
*/
|
||||||
def isWatchIntercepted(watchee: ActorRef, watcher: ActorRef) = {
|
def isWatchIntercepted(watchee: ActorRef, watcher: ActorRef): Boolean = {
|
||||||
// If watchee != this then watcher should == this. This is a reverse watch, and it is not intercepted
|
// If watchee != this then watcher should == this. This is a reverse watch, and it is not intercepted
|
||||||
// If watchee == this, only the watches from remoteWatcher are sent on the wire, on behalf of other watchers
|
// If watchee == this, only the watches from remoteWatcher are sent on the wire, on behalf of other watchers
|
||||||
watcher != provider.remoteWatcher && watchee == this
|
val intercept = provider.remoteWatcher.exists(remoteWatcher => watcher != remoteWatcher) && watchee == this
|
||||||
|
if (intercept) provider.warnIfUnsafeDeathwatchWithoutCluster(watchee, watcher, "remote Watch/Unwatch")
|
||||||
|
intercept
|
||||||
}
|
}
|
||||||
|
|
||||||
def sendSystemMessage(message: SystemMessage): Unit =
|
def sendSystemMessage(message: SystemMessage): Unit =
|
||||||
|
|
@ -647,10 +701,10 @@ private[akka] class RemoteActorRef private[akka] (
|
||||||
//send to remote, unless watch message is intercepted by the remoteWatcher
|
//send to remote, unless watch message is intercepted by the remoteWatcher
|
||||||
message match {
|
message match {
|
||||||
case Watch(watchee, watcher) if isWatchIntercepted(watchee, watcher) =>
|
case Watch(watchee, watcher) if isWatchIntercepted(watchee, watcher) =>
|
||||||
provider.remoteWatcher ! RemoteWatcher.WatchRemote(watchee, watcher)
|
provider.remoteWatcher.foreach(_ ! RemoteWatcher.WatchRemote(watchee, watcher))
|
||||||
//Unwatch has a different signature, need to pattern match arguments against InternalActorRef
|
//Unwatch has a different signature, need to pattern match arguments against InternalActorRef
|
||||||
case Unwatch(watchee: InternalActorRef, watcher: InternalActorRef) if isWatchIntercepted(watchee, watcher) =>
|
case Unwatch(watchee: InternalActorRef, watcher: InternalActorRef) if isWatchIntercepted(watchee, watcher) =>
|
||||||
provider.remoteWatcher ! RemoteWatcher.UnwatchRemote(watchee, watcher)
|
provider.remoteWatcher.foreach(_ ! RemoteWatcher.UnwatchRemote(watchee, watcher))
|
||||||
case _ => remote.send(message, OptionVal.None, this)
|
case _ => remote.send(message, OptionVal.None, this)
|
||||||
}
|
}
|
||||||
} catch handleException(message, Actor.noSender)
|
} catch handleException(message, Actor.noSender)
|
||||||
|
|
|
||||||
|
|
@ -7,9 +7,9 @@ package akka.remote
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.util.Timeout
|
|
||||||
|
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
|
|
||||||
|
import akka.util.Timeout
|
||||||
import akka.util.Helpers.{ toRootLowerCase, ConfigOps, Requiring }
|
import akka.util.Helpers.{ toRootLowerCase, ConfigOps, Requiring }
|
||||||
import akka.japi.Util._
|
import akka.japi.Util._
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
|
|
@ -124,6 +124,11 @@ final class RemoteSettings(val config: Config) {
|
||||||
Timeout(config.getMillisDuration("akka.remote.classic.command-ack-timeout"))
|
Timeout(config.getMillisDuration("akka.remote.classic.command-ack-timeout"))
|
||||||
}.requiring(_.duration > Duration.Zero, "command-ack-timeout must be > 0")
|
}.requiring(_.duration > Duration.Zero, "command-ack-timeout must be > 0")
|
||||||
|
|
||||||
|
val UseUnsafeRemoteFeaturesWithoutCluster: Boolean = getBoolean(
|
||||||
|
"akka.remote.use-unsafe-remote-features-without-cluster")
|
||||||
|
|
||||||
|
val WarnUnsafeWatchWithoutCluster: Boolean = getBoolean("akka.remote.warn-unsafe-watch-without-cluster")
|
||||||
|
|
||||||
val WatchFailureDetectorConfig: Config = getConfig("akka.remote.watch-failure-detector")
|
val WatchFailureDetectorConfig: Config = getConfig("akka.remote.watch-failure-detector")
|
||||||
val WatchFailureDetectorImplementationClass: String = WatchFailureDetectorConfig.getString("implementation-class")
|
val WatchFailureDetectorImplementationClass: String = WatchFailureDetectorConfig.getString("implementation-class")
|
||||||
val WatchHeartBeatInterval: FiniteDuration = {
|
val WatchHeartBeatInterval: FiniteDuration = {
|
||||||
|
|
|
||||||
|
|
@ -4,16 +4,18 @@
|
||||||
|
|
||||||
package akka.remote
|
package akka.remote
|
||||||
|
|
||||||
|
import scala.collection.mutable
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
|
import akka.annotation.InternalApi
|
||||||
import akka.dispatch.sysmsg.{ DeathWatchNotification, Watch }
|
import akka.dispatch.sysmsg.{ DeathWatchNotification, Watch }
|
||||||
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
||||||
import akka.event.AddressTerminatedTopic
|
import akka.event.AddressTerminatedTopic
|
||||||
import akka.remote.artery.ArteryMessage
|
import akka.remote.artery.ArteryMessage
|
||||||
import scala.collection.mutable
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
|
|
||||||
import akka.dispatch.Dispatchers
|
import akka.dispatch.Dispatchers
|
||||||
import akka.remote.artery.ArteryTransport
|
import akka.remote.artery.ArteryTransport
|
||||||
|
import akka.util.unused
|
||||||
import com.github.ghik.silencer.silent
|
import com.github.ghik.silencer.silent
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -24,17 +26,15 @@ private[akka] object RemoteWatcher {
|
||||||
/**
|
/**
|
||||||
* Factory method for `RemoteWatcher` [[akka.actor.Props]].
|
* Factory method for `RemoteWatcher` [[akka.actor.Props]].
|
||||||
*/
|
*/
|
||||||
def props(
|
def props(settings: RemoteSettings, failureDetector: FailureDetectorRegistry[Address]): Props =
|
||||||
failureDetector: FailureDetectorRegistry[Address],
|
|
||||||
heartbeatInterval: FiniteDuration,
|
|
||||||
unreachableReaperInterval: FiniteDuration,
|
|
||||||
heartbeatExpectedResponseAfter: FiniteDuration): Props =
|
|
||||||
Props(
|
Props(
|
||||||
classOf[RemoteWatcher],
|
new RemoteWatcher(
|
||||||
failureDetector,
|
failureDetector,
|
||||||
heartbeatInterval,
|
heartbeatInterval = settings.WatchHeartBeatInterval,
|
||||||
unreachableReaperInterval,
|
unreachableReaperInterval = settings.WatchUnreachableReaperInterval,
|
||||||
heartbeatExpectedResponseAfter).withDispatcher(Dispatchers.InternalDispatcherId).withDeploy(Deploy.local)
|
heartbeatExpectedResponseAfter = settings.WatchHeartbeatExpectedResponseAfter))
|
||||||
|
.withDispatcher(Dispatchers.InternalDispatcherId)
|
||||||
|
.withDeploy(Deploy.local)
|
||||||
|
|
||||||
final case class WatchRemote(watchee: InternalActorRef, watcher: InternalActorRef)
|
final case class WatchRemote(watchee: InternalActorRef, watcher: InternalActorRef)
|
||||||
final case class UnwatchRemote(watchee: InternalActorRef, watcher: InternalActorRef)
|
final case class UnwatchRemote(watchee: InternalActorRef, watcher: InternalActorRef)
|
||||||
|
|
@ -139,7 +139,7 @@ private[akka] class RemoteWatcher(
|
||||||
failureDetectorReaperTask.cancel()
|
failureDetectorReaperTask.cancel()
|
||||||
}
|
}
|
||||||
|
|
||||||
def receive = {
|
def receive: Receive = {
|
||||||
case HeartbeatTick => sendHeartbeat()
|
case HeartbeatTick => sendHeartbeat()
|
||||||
case Heartbeat | ArteryHeartbeat => receiveHeartbeat()
|
case Heartbeat | ArteryHeartbeat => receiveHeartbeat()
|
||||||
case HeartbeatRsp(uid) => receiveHeartbeatRsp(uid.toLong)
|
case HeartbeatRsp(uid) => receiveHeartbeatRsp(uid.toLong)
|
||||||
|
|
@ -202,14 +202,25 @@ private[akka] class RemoteWatcher(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Returns true if either has cluster or `akka.remote.use-unsafe-remote-features-without-cluster`
|
||||||
|
* is enabled. Can be overridden when using RemoteWatcher as a superclass.
|
||||||
|
*/
|
||||||
|
@InternalApi protected def shouldWatch(@unused watchee: InternalActorRef): Boolean = {
|
||||||
|
// In this it is unnecessary if only created by RARP, but cluster needs it.
|
||||||
|
// Cleaner than overriding Cluster watcher addWatch/removeWatch just for one boolean test
|
||||||
|
remoteProvider.hasClusterOrUseUnsafe
|
||||||
|
}
|
||||||
|
|
||||||
def addWatch(watchee: InternalActorRef, watcher: InternalActorRef): Unit = {
|
def addWatch(watchee: InternalActorRef, watcher: InternalActorRef): Unit = {
|
||||||
assert(watcher != self)
|
assert(watcher != self)
|
||||||
log.debug("Watching: [{} -> {}]", watcher, watchee)
|
log.debug("Watching: [{} -> {}]", watcher, watchee)
|
||||||
watching.addBinding(watchee, watcher)
|
if (shouldWatch(watchee)) {
|
||||||
watchNode(watchee)
|
watching.addBinding(watchee, watcher)
|
||||||
|
watchNode(watchee)
|
||||||
|
|
||||||
// add watch from self, this will actually send a Watch to the target when necessary
|
// add watch from self, this will actually send a Watch to the target when necessary
|
||||||
context.watch(watchee)
|
context.watch(watchee)
|
||||||
|
} else remoteProvider.warnIfUnsafeDeathwatchWithoutCluster(watcher, watchee, "Watch")
|
||||||
}
|
}
|
||||||
|
|
||||||
def watchNode(watchee: InternalActorRef): Unit = {
|
def watchNode(watchee: InternalActorRef): Unit = {
|
||||||
|
|
@ -224,20 +235,21 @@ private[akka] class RemoteWatcher(
|
||||||
|
|
||||||
def removeWatch(watchee: InternalActorRef, watcher: InternalActorRef): Unit = {
|
def removeWatch(watchee: InternalActorRef, watcher: InternalActorRef): Unit = {
|
||||||
assert(watcher != self)
|
assert(watcher != self)
|
||||||
log.debug("Unwatching: [{} -> {}]", watcher, watchee)
|
if (shouldWatch(watchee)) {
|
||||||
|
// Could have used removeBinding, but it does not tell if this was the last entry. This saves a contains call.
|
||||||
// Could have used removeBinding, but it does not tell if this was the last entry. This saves a contains call.
|
watching.get(watchee) match {
|
||||||
watching.get(watchee) match {
|
case Some(watchers) =>
|
||||||
case Some(watchers) =>
|
watchers -= watcher
|
||||||
watchers -= watcher
|
if (watchers.isEmpty) {
|
||||||
if (watchers.isEmpty) {
|
log.debug("Unwatching: [{} -> {}]", watcher, watchee)
|
||||||
// clean up self watch when no more watchers of this watchee
|
// clean up self watch when no more watchers of this watchee
|
||||||
log.debug("Cleanup self watch of [{}]", watchee.path)
|
log.debug("Cleanup self watch of [{}]", watchee.path)
|
||||||
context.unwatch(watchee)
|
context.unwatch(watchee)
|
||||||
removeWatchee(watchee)
|
removeWatchee(watchee)
|
||||||
}
|
}
|
||||||
case None =>
|
case None =>
|
||||||
}
|
}
|
||||||
|
} else remoteProvider.warnIfUnsafeDeathwatchWithoutCluster(watcher, watchee, "Unwatch")
|
||||||
}
|
}
|
||||||
|
|
||||||
def removeWatchee(watchee: InternalActorRef): Unit = {
|
def removeWatchee(watchee: InternalActorRef): Unit = {
|
||||||
|
|
|
||||||
167
akka-remote/src/test/scala/akka/remote/RemoteFeaturesSpec.scala
Normal file
167
akka-remote/src/test/scala/akka/remote/RemoteFeaturesSpec.scala
Normal file
|
|
@ -0,0 +1,167 @@
|
||||||
|
/*
|
||||||
|
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.remote
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
import akka.actor.Actor
|
||||||
|
import akka.actor.ActorIdentity
|
||||||
|
import akka.actor.AddressFromURIString
|
||||||
|
import akka.actor.Identify
|
||||||
|
import akka.actor.InternalActorRef
|
||||||
|
import akka.actor.Props
|
||||||
|
import akka.actor.RootActorPath
|
||||||
|
import akka.remote.RemoteWatcher.Stats
|
||||||
|
import akka.remote.RemoteWatcher.UnwatchRemote
|
||||||
|
import akka.remote.RemoteWatcher.WatchRemote
|
||||||
|
import akka.remote.artery.ArteryMultiNodeSpec
|
||||||
|
import akka.remote.artery.ArterySpecSupport
|
||||||
|
import akka.remote.artery.RemoteDeploymentSpec
|
||||||
|
import akka.testkit.EventFilter
|
||||||
|
import akka.testkit.ImplicitSender
|
||||||
|
import akka.testkit.TestProbe
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
|
object RemoteFeaturesSpec {
|
||||||
|
|
||||||
|
val instances = 1
|
||||||
|
|
||||||
|
// string config to pass into `ArteryMultiNodeSpec.extraConfig: Option[String]` for `other` system
|
||||||
|
def common(useUnsafe: Boolean): String = s"""
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = $useUnsafe
|
||||||
|
akka.remote.artery.enabled = on
|
||||||
|
akka.remote.artery.canonical.port = 0
|
||||||
|
akka.remote.artery.advanced.flight-recorder.enabled = off
|
||||||
|
akka.log-dead-letters-during-shutdown = off
|
||||||
|
"""
|
||||||
|
|
||||||
|
def disabled: Config =
|
||||||
|
ConfigFactory.parseString(common(useUnsafe = false)).withFallback(ArterySpecSupport.defaultConfig)
|
||||||
|
def enabled: Config = ConfigFactory.parseString(common(useUnsafe = true))
|
||||||
|
|
||||||
|
class EmptyActor extends Actor {
|
||||||
|
def receive: Receive = Actor.emptyBehavior
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract class RemoteFeaturesSpec(c: Config) extends ArteryMultiNodeSpec(c) with ImplicitSender {
|
||||||
|
import RemoteFeaturesSpec._
|
||||||
|
|
||||||
|
protected final val provider = RARP(system).provider
|
||||||
|
|
||||||
|
protected final val useUnsafe: Boolean = provider.remoteSettings.UseUnsafeRemoteFeaturesWithoutCluster
|
||||||
|
|
||||||
|
protected val remoteSystem1 = newRemoteSystem(name = Some("RS1"), extraConfig = Some(common(useUnsafe)))
|
||||||
|
|
||||||
|
Seq(system, remoteSystem1).foreach(
|
||||||
|
muteDeadLetters(
|
||||||
|
akka.remote.transport.AssociationHandle.Disassociated.getClass,
|
||||||
|
akka.remote.transport.ActorTransportAdapter.DisassociateUnderlying.getClass)(_))
|
||||||
|
|
||||||
|
import akka.remote.artery.RemoteWatcherSpec.TestRemoteWatcher
|
||||||
|
protected val monitor = system.actorOf(Props(new TestRemoteWatcher), "monitor1")
|
||||||
|
|
||||||
|
protected val watcher = system.actorOf(Props(new EmptyActor), "a1").asInstanceOf[InternalActorRef]
|
||||||
|
|
||||||
|
protected val remoteWatchee = createRemoteActor(Props(new EmptyActor), "b1")
|
||||||
|
|
||||||
|
protected def createRemoteActor(props: Props, name: String): InternalActorRef = {
|
||||||
|
remoteSystem1.actorOf(props, name)
|
||||||
|
system.actorSelection(RootActorPath(address(remoteSystem1)) / "user" / name) ! Identify(name)
|
||||||
|
expectMsgType[ActorIdentity].ref.get.asInstanceOf[InternalActorRef]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// all pre-existing remote tests exercise the rest of the unchanged enabled expectations
|
||||||
|
class RARPRemoteFeaturesEnabledSpec extends RemoteFeaturesSpec(RemoteFeaturesSpec.enabled) {
|
||||||
|
"RARP without Cluster: opt-in unsafe enabled" must {
|
||||||
|
|
||||||
|
"have the expected settings" in {
|
||||||
|
provider.transport.system.settings.HasCluster shouldBe false
|
||||||
|
provider.remoteSettings.UseUnsafeRemoteFeaturesWithoutCluster shouldBe true
|
||||||
|
provider.remoteSettings.WarnUnsafeWatchWithoutCluster shouldBe true
|
||||||
|
provider.hasClusterOrUseUnsafe shouldBe true
|
||||||
|
}
|
||||||
|
|
||||||
|
"create a RemoteWatcher" in {
|
||||||
|
provider.remoteWatcher.isDefined shouldBe true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// see the multi-jvm RemoteFeaturesSpec for deployer-router tests
|
||||||
|
class RemoteFeaturesDisabledSpec extends RemoteFeaturesSpec(RemoteFeaturesSpec.disabled) {
|
||||||
|
|
||||||
|
private val actorName = "kattdjur"
|
||||||
|
|
||||||
|
private val port = RARP(system).provider.getDefaultAddress.port.get
|
||||||
|
|
||||||
|
"Remote features without Cluster" must {
|
||||||
|
|
||||||
|
"have the expected settings in a RARP" in {
|
||||||
|
provider.transport.system.settings.HasCluster shouldBe false
|
||||||
|
provider.remoteSettings.UseUnsafeRemoteFeaturesWithoutCluster shouldBe false
|
||||||
|
provider.remoteSettings.WarnUnsafeWatchWithoutCluster shouldBe true
|
||||||
|
provider.hasClusterOrUseUnsafe shouldBe false
|
||||||
|
}
|
||||||
|
|
||||||
|
"not create a RemoteWatcher in a RARP" in {
|
||||||
|
provider.remoteWatcher shouldEqual None
|
||||||
|
}
|
||||||
|
|
||||||
|
"not deathwatch a remote actor" in {
|
||||||
|
EventFilter
|
||||||
|
.warning(pattern = s"Dropped remote Watch: disabled for *", occurrences = 1)
|
||||||
|
.intercept(monitor ! WatchRemote(remoteWatchee, watcher))
|
||||||
|
monitor ! Stats
|
||||||
|
expectMsg(Stats.empty)
|
||||||
|
expectNoMessage(100.millis)
|
||||||
|
|
||||||
|
EventFilter
|
||||||
|
.warning(pattern = s"Dropped remote Unwatch: disabled for *", occurrences = 1)
|
||||||
|
.intercept(monitor ! UnwatchRemote(remoteWatchee, watcher))
|
||||||
|
|
||||||
|
monitor ! Stats
|
||||||
|
expectMsg(Stats.empty)
|
||||||
|
expectNoMessage(100.millis)
|
||||||
|
}
|
||||||
|
|
||||||
|
"fall back to creating local deploy children and supervise children on local node" in {
|
||||||
|
// super.newRemoteSystem adds the new system to shutdown hook
|
||||||
|
val masterSystem = newRemoteSystem(
|
||||||
|
name = Some("RS2"),
|
||||||
|
extraConfig = Some(s"""
|
||||||
|
akka.actor.deployment {
|
||||||
|
/$actorName.remote = "akka://${system.name}@localhost:$port"
|
||||||
|
"/parent*/*".remote = "akka://${system.name}@localhost:$port"
|
||||||
|
}
|
||||||
|
"""))
|
||||||
|
|
||||||
|
val masterRef = masterSystem.actorOf(Props[RemoteDeploymentSpec.Echo1], actorName)
|
||||||
|
masterRef.path shouldEqual RootActorPath(AddressFromURIString(s"akka://${masterSystem.name}")) / "user" / actorName
|
||||||
|
masterRef.path.address.hasLocalScope shouldBe true
|
||||||
|
|
||||||
|
masterSystem.actorSelection(RootActorPath(address(system)) / "user" / actorName) ! Identify(1)
|
||||||
|
expectMsgType[ActorIdentity].ref shouldEqual None
|
||||||
|
|
||||||
|
system.actorSelection(RootActorPath(address(system)) / "user" / actorName) ! Identify(3)
|
||||||
|
expectMsgType[ActorIdentity].ref
|
||||||
|
.forall(_.path == RootActorPath(address(masterSystem)) / "user" / actorName) shouldBe true
|
||||||
|
|
||||||
|
val senderProbe = TestProbe()(masterSystem)
|
||||||
|
masterRef.tell(42, senderProbe.ref)
|
||||||
|
senderProbe.expectMsg(42)
|
||||||
|
EventFilter[Exception]("crash", occurrences = 1).intercept {
|
||||||
|
masterRef ! new Exception("crash")
|
||||||
|
}(masterSystem)
|
||||||
|
senderProbe.expectMsg("preRestart")
|
||||||
|
masterRef.tell(43, senderProbe.ref)
|
||||||
|
senderProbe.expectMsg(43)
|
||||||
|
masterSystem.stop(masterRef)
|
||||||
|
senderProbe.expectMsg("postStop")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -25,6 +25,7 @@ class RemoteRouterSpec extends AkkaSpec(s"""
|
||||||
akka.actor.provider = remote
|
akka.actor.provider = remote
|
||||||
akka.actor.allow-java-serialization = off
|
akka.actor.allow-java-serialization = off
|
||||||
akka.actor.serialize-messages = off
|
akka.actor.serialize-messages = off
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
akka.remote.classic.netty.tcp {
|
akka.remote.classic.netty.tcp {
|
||||||
hostname = localhost
|
hostname = localhost
|
||||||
port = 0
|
port = 0
|
||||||
|
|
|
||||||
|
|
@ -27,6 +27,7 @@ object RemoteDeathWatchSpec {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
test.filter-leeway = 10s
|
test.filter-leeway = 10s
|
||||||
|
remote.use-unsafe-remote-features-without-cluster = on
|
||||||
remote.watch-failure-detector.acceptable-heartbeat-pause = 2s
|
remote.watch-failure-detector.acceptable-heartbeat-pause = 2s
|
||||||
|
|
||||||
# reduce handshake timeout for quicker test of unknownhost, but
|
# reduce handshake timeout for quicker test of unknownhost, but
|
||||||
|
|
|
||||||
|
|
@ -63,6 +63,7 @@ class RemoteDeploymentSpec
|
||||||
extends ArteryMultiNodeSpec(ConfigFactory.parseString("""
|
extends ArteryMultiNodeSpec(ConfigFactory.parseString("""
|
||||||
akka.remote.artery.advanced.inbound-lanes = 10
|
akka.remote.artery.advanced.inbound-lanes = 10
|
||||||
akka.remote.artery.advanced.outbound-lanes = 3
|
akka.remote.artery.advanced.outbound-lanes = 3
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
""").withFallback(ArterySpecSupport.defaultConfig)) {
|
""").withFallback(ArterySpecSupport.defaultConfig)) {
|
||||||
|
|
||||||
import RemoteDeploymentSpec._
|
import RemoteDeploymentSpec._
|
||||||
|
|
@ -97,7 +98,7 @@ class RemoteDeploymentSpec
|
||||||
senderProbe.expectMsg("preRestart")
|
senderProbe.expectMsg("preRestart")
|
||||||
r.tell(43, senderProbe.ref)
|
r.tell(43, senderProbe.ref)
|
||||||
senderProbe.expectMsg(43)
|
senderProbe.expectMsg(43)
|
||||||
system.stop(r)
|
masterSystem.stop(r)
|
||||||
senderProbe.expectMsg("postStop")
|
senderProbe.expectMsg("postStop")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,9 @@ object RemoteRouterSpec {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class RemoteRouterSpec extends AkkaSpec(ConfigFactory.parseString("""
|
class RemoteRouterSpec
|
||||||
|
extends AkkaSpec(ConfigFactory.parseString("""
|
||||||
|
akka.remote.use-unsafe-remote-features-without-cluster = on
|
||||||
akka.actor.deployment {
|
akka.actor.deployment {
|
||||||
/remote-override {
|
/remote-override {
|
||||||
router = round-robin-pool
|
router = round-robin-pool
|
||||||
|
|
@ -36,7 +38,8 @@ class RemoteRouterSpec extends AkkaSpec(ConfigFactory.parseString("""
|
||||||
router = round-robin-pool
|
router = round-robin-pool
|
||||||
nr-of-instances = 6
|
nr-of-instances = 6
|
||||||
}
|
}
|
||||||
}""").withFallback(ArterySpecSupport.defaultConfig)) with FlightRecorderSpecIntegration {
|
}""").withFallback(ArterySpecSupport.defaultConfig))
|
||||||
|
with FlightRecorderSpecIntegration {
|
||||||
|
|
||||||
import RemoteRouterSpec._
|
import RemoteRouterSpec._
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -6,9 +6,11 @@ package akka.remote.artery
|
||||||
|
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.remote._
|
import akka.remote._
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
object RemoteWatcherSpec {
|
object RemoteWatcherSpec {
|
||||||
|
|
||||||
|
|
@ -65,7 +67,12 @@ object RemoteWatcherSpec {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class RemoteWatcherSpec extends ArteryMultiNodeSpec(ArterySpecSupport.defaultConfig) with ImplicitSender {
|
class RemoteWatcherSpec
|
||||||
|
extends ArteryMultiNodeSpec(
|
||||||
|
ConfigFactory
|
||||||
|
.parseString("akka.remote.use-unsafe-remote-features-without-cluster = on")
|
||||||
|
.withFallback(ArterySpecSupport.defaultConfig))
|
||||||
|
with ImplicitSender {
|
||||||
|
|
||||||
import RemoteWatcherSpec._
|
import RemoteWatcherSpec._
|
||||||
import RemoteWatcher._
|
import RemoteWatcher._
|
||||||
|
|
|
||||||
|
|
@ -22,6 +22,7 @@ akka {
|
||||||
/watchers.remote = "akka.tcp://other@localhost:2666"
|
/watchers.remote = "akka.tcp://other@localhost:2666"
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
remote.use-unsafe-remote-features-without-cluster = on
|
||||||
remote.artery.enabled = off
|
remote.artery.enabled = off
|
||||||
remote.classic {
|
remote.classic {
|
||||||
retry-gate-closed-for = 1 s
|
retry-gate-closed-for = 1 s
|
||||||
|
|
|
||||||
|
|
@ -56,6 +56,7 @@ object RemoteDeploymentWhitelistSpec {
|
||||||
actor.provider = remote
|
actor.provider = remote
|
||||||
|
|
||||||
remote {
|
remote {
|
||||||
|
use-unsafe-remote-features-without-cluster = on
|
||||||
classic.enabled-transports = [
|
classic.enabled-transports = [
|
||||||
"akka.remote.test",
|
"akka.remote.test",
|
||||||
"akka.remote.classic.netty.tcp"
|
"akka.remote.classic.netty.tcp"
|
||||||
|
|
|
||||||
|
|
@ -76,6 +76,7 @@ class RemoteWatcherSpec extends AkkaSpec("""akka {
|
||||||
port = 0
|
port = 0
|
||||||
}
|
}
|
||||||
remote.artery.enabled = off
|
remote.artery.enabled = off
|
||||||
|
remote.use-unsafe-remote-features-without-cluster = on
|
||||||
}""") with ImplicitSender {
|
}""") with ImplicitSender {
|
||||||
|
|
||||||
import RemoteWatcher._
|
import RemoteWatcher._
|
||||||
|
|
|
||||||
|
|
@ -86,6 +86,7 @@ object RemotingSpec {
|
||||||
actor.serialize-messages = off
|
actor.serialize-messages = off
|
||||||
|
|
||||||
remote {
|
remote {
|
||||||
|
use-unsafe-remote-features-without-cluster = on
|
||||||
artery.enabled = off
|
artery.enabled = off
|
||||||
classic {
|
classic {
|
||||||
retry-gate-closed-for = 1 s
|
retry-gate-closed-for = 1 s
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue