Merge pull request #1443 from akka/wip-3359-auto-join-patriknw

Remove auto-join config, derive from seed-nodes, see #3359
This commit is contained in:
Patrik Nordwall 2013-05-17 04:57:07 -07:00
commit 8f04b53ac7
25 changed files with 53 additions and 94 deletions

View file

@ -9,10 +9,10 @@ akka {
cluster { cluster {
# Initial contact points of the cluster. # Initial contact points of the cluster.
# The nodes to join at startup if auto-join = on. # The nodes to join automatically at startup.
# Comma separated full URIs defined by a string on the form of # Comma separated full URIs defined by a string on the form of
# "akka://system@hostname:port" # "akka://system@hostname:port"
# Leave as empty if the node should be a singleton cluster. # Leave as empty if the node is supposed to be joined manually.
seed-nodes = [] seed-nodes = []
# how long to wait for one of the seed nodes to reply to initial join request # how long to wait for one of the seed nodes to reply to initial join request
@ -22,10 +22,6 @@ akka {
# Disable join retry by specifying "off". # Disable join retry by specifying "off".
retry-unsuccessful-join-after = 10s retry-unsuccessful-join-after = 10s
# Automatic join the seed-nodes at startup.
# If seed-nodes is empty it will join itself and become a single node cluster.
auto-join = on
# Should the 'leader' in the cluster be allowed to automatically mark # Should the 'leader' in the cluster be allowed to automatically mark
# unreachable nodes as DOWN? # unreachable nodes as DOWN?
# Using auto-down implies that two separate clusters will automatically be # Using auto-down implies that two separate clusters will automatically be

View file

@ -260,13 +260,16 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
// start periodic publish of current stats // start periodic publish of current stats
val publishStatsTask: Option[Cancellable] = PublishStatsInterval match { val publishStatsTask: Option[Cancellable] = PublishStatsInterval match {
case Duration.Zero | Duration.Undefined | Duration.Inf None case Duration.Zero | _: Duration.Infinite None
case d: FiniteDuration case d: FiniteDuration
Some(scheduler.schedule(PeriodicTasksInitialDelay.max(d), d, self, PublishStatsTick)) Some(scheduler.schedule(PeriodicTasksInitialDelay.max(d), d, self, PublishStatsTick))
} }
override def preStart(): Unit = { override def preStart(): Unit = {
if (AutoJoin) self ! JoinSeedNodes(SeedNodes) if (SeedNodes.isEmpty)
log.info("No seed-nodes configured, manual cluster join required")
else
self ! JoinSeedNodes(SeedNodes)
} }
override def postStop(): Unit = { override def postStop(): Unit = {
@ -296,7 +299,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
case _: Tick case _: Tick
if (deadline.exists(_.isOverdue)) { if (deadline.exists(_.isOverdue)) {
context.become(uninitialized) context.become(uninitialized)
if (AutoJoin) joinSeedNodes(SeedNodes) if (SeedNodes.nonEmpty) joinSeedNodes(SeedNodes)
else join(joinWith) else join(joinWith)
} }
} }
@ -336,17 +339,19 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
def joinSeedNodes(seedNodes: immutable.IndexedSeq[Address]): Unit = { def joinSeedNodes(seedNodes: immutable.IndexedSeq[Address]): Unit = {
require(seedNodeProcess.isEmpty, "Join seed nodes is already in progress") require(seedNodeProcess.isEmpty, "Join seed nodes is already in progress")
seedNodeProcess = if (seedNodes.nonEmpty) {
if (seedNodes.isEmpty || seedNodes == immutable.IndexedSeq(selfAddress)) { seedNodeProcess =
self ! ClusterUserAction.JoinTo(selfAddress) if (seedNodes == immutable.IndexedSeq(selfAddress)) {
None self ! ClusterUserAction.JoinTo(selfAddress)
} else if (seedNodes.head == selfAddress) { None
Some(context.actorOf(Props(classOf[FirstSeedNodeProcess], seedNodes). } else if (seedNodes.head == selfAddress) {
withDispatcher(UseDispatcher), name = "firstSeedNodeProcess")) Some(context.actorOf(Props(classOf[FirstSeedNodeProcess], seedNodes).
} else { withDispatcher(UseDispatcher), name = "firstSeedNodeProcess"))
Some(context.actorOf(Props(classOf[JoinSeedNodeProcess], seedNodes). } else {
withDispatcher(UseDispatcher), name = "joinSeedNodeProcess")) Some(context.actorOf(Props(classOf[JoinSeedNodeProcess], seedNodes).
} withDispatcher(UseDispatcher), name = "joinSeedNodeProcess"))
}
}
} }
/** /**
@ -383,8 +388,8 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
joining(selfUniqueAddress, cluster.selfRoles) joining(selfUniqueAddress, cluster.selfRoles)
} else { } else {
val joinDeadline = RetryUnsuccessfulJoinAfter match { val joinDeadline = RetryUnsuccessfulJoinAfter match {
case Duration.Undefined | Duration.Inf None case d: FiniteDuration Some(Deadline.now + d)
case d: FiniteDuration Some(Deadline.now + d) case _ None
} }
context.become(tryingToJoin(address, joinDeadline)) context.become(tryingToJoin(address, joinDeadline))
clusterCore(address) ! Join(selfUniqueAddress, cluster.selfRoles) clusterCore(address) ! Join(selfUniqueAddress, cluster.selfRoles)

View file

@ -62,7 +62,6 @@ class ClusterSettings(val config: Config, val systemName: String) {
case _ Duration(cc.getMilliseconds(key), MILLISECONDS) requiring (_ >= Duration.Zero, key + " >= 0s, or off") case _ Duration(cc.getMilliseconds(key), MILLISECONDS) requiring (_ >= Duration.Zero, key + " >= 0s, or off")
} }
} }
final val AutoJoin: Boolean = cc.getBoolean("auto-join")
final val AutoDown: Boolean = cc.getBoolean("auto-down") final val AutoDown: Boolean = cc.getBoolean("auto-down")
final val Roles: Set[String] = immutableSeq(cc.getStringList("roles")).toSet final val Roles: Set[String] = immutableSeq(cc.getStringList("roles")).toSet
final val MinNrOfMembers: Int = { final val MinNrOfMembers: Int = {

View file

@ -20,7 +20,6 @@ object JoinSeedNodeMultiJvmSpec extends MultiNodeConfig {
val ordinary2 = role("ordinary2") val ordinary2 = role("ordinary2")
commonConfig(debugConfig(on = false). commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString("akka.cluster.auto-join = off")).
withFallback(MultiNodeClusterSpec.clusterConfig)) withFallback(MultiNodeClusterSpec.clusterConfig))
} }

View file

@ -34,7 +34,6 @@ object MultiNodeClusterSpec {
def clusterConfig: Config = ConfigFactory.parseString(""" def clusterConfig: Config = ConfigFactory.parseString("""
akka.actor.provider = akka.cluster.ClusterActorRefProvider akka.actor.provider = akka.cluster.ClusterActorRefProvider
akka.cluster { akka.cluster {
auto-join = off
auto-down = off auto-down = off
jmx.enabled = off jmx.enabled = off
gossip-interval = 200 ms gossip-interval = 200 ms

View file

@ -26,7 +26,6 @@ object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false). commonConfig(debugConfig(on = false).
withFallback(ConfigFactory.parseString(""" withFallback(ConfigFactory.parseString("""
akka.cluster { akka.cluster {
auto-join = off
auto-down = on auto-down = on
} }
""")).withFallback(MultiNodeClusterSpec.clusterConfig)) """)).withFallback(MultiNodeClusterSpec.clusterConfig))

View file

@ -23,8 +23,6 @@ case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) exten
""")). """)).
withFallback(MultiNodeClusterSpec.clusterConfig(failureDetectorPuppet))) withFallback(MultiNodeClusterSpec.clusterConfig(failureDetectorPuppet)))
nodeConfig(first)(ConfigFactory.parseString("akka.cluster.auto-join = on"))
} }
class SingletonClusterWithFailureDetectorPuppetMultiJvmNode1 extends SingletonClusterSpec(failureDetectorPuppet = true) class SingletonClusterWithFailureDetectorPuppetMultiJvmNode1 extends SingletonClusterSpec(failureDetectorPuppet = true)
@ -45,8 +43,9 @@ abstract class SingletonClusterSpec(multiNodeConfig: SingletonClusterMultiNodeCo
"A cluster of 2 nodes" must { "A cluster of 2 nodes" must {
"become singleton cluster when started with 'auto-join=on' and 'seed-nodes=[]'" taggedAs LongRunningTest in { "become singleton cluster when started with seed-nodes" taggedAs LongRunningTest in {
runOn(first) { runOn(first) {
cluster.joinSeedNodes(Vector(first))
awaitMembersUp(1) awaitMembersUp(1)
clusterView.isSingletonCluster must be(true) clusterView.isSingletonCluster must be(true)
} }

View file

@ -119,7 +119,6 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
akka.actor.provider = akka.cluster.ClusterActorRefProvider akka.actor.provider = akka.cluster.ClusterActorRefProvider
akka.cluster { akka.cluster {
auto-join = off
auto-down = on auto-down = on
publish-stats-interval = 0 s # always, when it happens publish-stats-interval = 0 s # always, when it happens
} }

View file

@ -25,9 +25,6 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
// not MultiNodeClusterSpec.clusterConfig // not MultiNodeClusterSpec.clusterConfig
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.actor.provider = akka.cluster.ClusterActorRefProvider akka.actor.provider = akka.cluster.ClusterActorRefProvider
akka.cluster {
auto-join = off
}
akka.loggers = ["akka.testkit.TestEventListener"] akka.loggers = ["akka.testkit.TestEventListener"]
akka.loglevel = INFO akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off

View file

@ -37,7 +37,6 @@ class ClusterConfigSpec extends AkkaSpec {
LeaderActionsInterval must be(1 second) LeaderActionsInterval must be(1 second)
UnreachableNodesReaperInterval must be(1 second) UnreachableNodesReaperInterval must be(1 second)
PublishStatsInterval must be(10 second) PublishStatsInterval must be(10 second)
AutoJoin must be(true)
AutoDown must be(false) AutoDown must be(false)
MinNrOfMembers must be(1) MinNrOfMembers must be(1)
MinNrOfMembersOfRole must be === Map.empty MinNrOfMembersOfRole must be === Map.empty

View file

@ -19,7 +19,6 @@ import akka.actor.ActorRef
object ClusterSpec { object ClusterSpec {
val config = """ val config = """
akka.cluster { akka.cluster {
auto-join = off
auto-down = off auto-down = off
periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks
publish-stats-interval = 0 s # always, when it happens publish-stats-interval = 0 s # always, when it happens
@ -60,7 +59,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
} }
"initially become singleton cluster when joining itself and reach convergence" in { "initially become singleton cluster when joining itself and reach convergence" in {
clusterView.members.size must be(0) // auto-join = off clusterView.members.size must be(0)
cluster.join(selfAddress) cluster.join(selfAddress)
leaderActions() // Joining -> Up leaderActions() // Joining -> Up
awaitCond(clusterView.isSingletonCluster) awaitCond(clusterView.isSingletonCluster)

View file

@ -29,7 +29,6 @@ object ClusterClientSpec extends MultiNodeConfig {
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-join = off
akka.cluster.auto-down = on akka.cluster.auto-down = on
""")) """))

View file

@ -40,7 +40,6 @@ object ClusterSingletonManagerChaosSpec extends MultiNodeConfig {
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-join = off
akka.cluster.auto-down = on akka.cluster.auto-down = on
""")) """))

View file

@ -42,7 +42,6 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-join = off
akka.cluster.auto-down = on akka.cluster.auto-down = on
""")) """))

View file

@ -28,7 +28,6 @@ object DistributedPubSubMediatorSpec extends MultiNodeConfig {
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-join = off
akka.cluster.auto-down = on akka.cluster.auto-down = on
""")) """))

View file

@ -68,16 +68,6 @@ partitioning of the application are decoupled. A node could be a member of a
cluster without hosting any actors. cluster without hosting any actors.
Singleton Cluster
-----------------
If a node does not have a preconfigured contact point to join in the Akka
configuration, then it is considered a singleton cluster (single node cluster)
and will automatically transition from ``joining`` to ``up``. Singleton clusters
can later explicitly send a ``Join`` message to another node to form a N-node
cluster. It is also possible to link multiple N-node clusters by ``joining`` them.
Gossip Gossip
------ ------
@ -187,7 +177,8 @@ The seed nodes are configured contact points for inital join of the cluster.
When a new node is started started it sends a message to all seed nodes and When a new node is started started it sends a message to all seed nodes and
then sends join command to the one that answers first. then sends join command to the one that answers first.
It is possible to turn off automatic join. It is possible to not use seed nodes and instead join any node in the cluster
manually.
Gossip Protocol Gossip Protocol

View file

@ -93,13 +93,12 @@ and register it as subscriber of certain cluster events. It gets notified with
an snapshot event, ``CurrentClusterState`` that holds full state information of an snapshot event, ``CurrentClusterState`` that holds full state information of
the cluster. After that it receives events for changes that happen in the cluster. the cluster. After that it receives events for changes that happen in the cluster.
Automatic vs. Manual Joining Joining to Seed Nodes
^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^
You may decide if joining to the cluster should be done automatically or manually. You may decide if joining to the cluster should be done manually or automatically
By default it is automatic and you need to define the seed nodes in configuration to configured initial contact points, so-called seed nodes. When a new node is started
so that a new node has an initial contact point. When a new node is started it it sends a message to all seed nodes and then sends join command to the one that
sends a message to all seed nodes and then sends join command to the one that
answers first. If no one of the seed nodes replied (might not be started yet) answers first. If no one of the seed nodes replied (might not be started yet)
it retries this procedure until successful or shutdown. it retries this procedure until successful or shutdown.
@ -115,22 +114,17 @@ Once more than two seed nodes have been started it is no problem to shut down th
seed node. If the first seed node is restarted it will first try join the other seed node. If the first seed node is restarted it will first try join the other
seed nodes in the existing cluster. seed nodes in the existing cluster.
You can disable automatic joining with configuration:: If you don't configure the seed nodes you need to join manually, using :ref:`cluster_jmx_java`
or :ref:`cluster_command_line_java`. You can join to any node in the cluster. It doesn't
akka.cluster.auto-join = off have to be configured as a seed node.
Then you need to join manually, using :ref:`cluster_jmx_java` or :ref:`cluster_command_line_java`.
You can join to any node in the cluster. It doesn't have to be configured as
seed node. If you are not using auto-join there is no need to configure
seed nodes at all.
Joining can also be performed programatically with ``Cluster.get(system).join(address)``. Joining can also be performed programatically with ``Cluster.get(system).join(address)``.
Unsuccessful join attempts are automatically retried after the time period defined in Unsuccessful join attempts are automatically retried after the time period defined in
configuration property ``retry-unsuccessful-join-after``. When using auto-joining with configuration property ``retry-unsuccessful-join-after``. When using ``seed-nodes`` this
``seed-nodes`` this means that a new seed node is picked. When joining manually or means that a new seed node is picked. When joining manually or programatically this means
programatically this means that the last join request is retried. Retries can be disabled by that the last join request is retried. Retries can be disabled by setting the property to
setting the property to ``off``. ``off``.
An actor system can only join a cluster once. Additional attempts will be ignored. An actor system can only join a cluster once. Additional attempts will be ignored.
When it has successfully joined it must be restarted to be able to join another When it has successfully joined it must be restarted to be able to join another

View file

@ -86,13 +86,12 @@ and register it as subscriber of certain cluster events. It gets notified with
an snapshot event, ``CurrentClusterState`` that holds full state information of an snapshot event, ``CurrentClusterState`` that holds full state information of
the cluster. After that it receives events for changes that happen in the cluster. the cluster. After that it receives events for changes that happen in the cluster.
Automatic vs. Manual Joining Joining to Seed Nodes
^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^
You may decide if joining to the cluster should be done automatically or manually. You may decide if joining to the cluster should be done manually or automatically
By default it is automatic and you need to define the seed nodes in configuration to configured initial contact points, so-called seed nodes. When a new node is started
so that a new node has an initial contact point. When a new node is started it it sends a message to all seed nodes and then sends join command to the one that
sends a message to all seed nodes and then sends join command to the one that
answers first. If no one of the seed nodes replied (might not be started yet) answers first. If no one of the seed nodes replied (might not be started yet)
it retries this procedure until successful or shutdown. it retries this procedure until successful or shutdown.
@ -108,22 +107,17 @@ Once more than two seed nodes have been started it is no problem to shut down th
seed node. If the first seed node is restarted it will first try join the other seed node. If the first seed node is restarted it will first try join the other
seed nodes in the existing cluster. seed nodes in the existing cluster.
You can disable automatic joining with configuration:: If you don't configure the seed nodes you need to join manually, using :ref:`cluster_jmx_scala`
or :ref:`cluster_command_line_scala`. You can join to any node in the cluster. It doesn't
akka.cluster.auto-join = off have to be configured as a seed node.
Then you need to join manually, using :ref:`cluster_jmx_scala` or :ref:`cluster_command_line_scala`.
You can join to any node in the cluster. It doesn't have to be configured as
seed node. If you are not using auto-join there is no need to configure
seed nodes at all.
Joining can also be performed programatically with ``Cluster(system).join(address)``. Joining can also be performed programatically with ``Cluster(system).join(address)``.
Unsuccessful join attempts are automatically retried after the time period defined in Unsuccessful join attempts are automatically retried after the time period defined in
configuration property ``retry-unsuccessful-join-after``. When using auto-joining with configuration property ``retry-unsuccessful-join-after``. When using ``seed-nodes`` this
``seed-nodes`` this means that a new seed node is picked. When joining manually or means that a new seed node is picked. When joining manually or programatically this means
programatically this means that the last join request is retried. Retries can be disabled by that the last join request is retried. Retries can be disabled by setting the property to
setting the property to ``off``. ``off``.
An actor system can only join a cluster once. Additional attempts will be ignored. An actor system can only join a cluster once. Additional attempts will be ignored.
When it has successfully joined it must be restarted to be able to join another When it has successfully joined it must be restarted to be able to join another

View file

@ -32,7 +32,6 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig {
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.roles = [compute] akka.cluster.roles = [compute]
akka.cluster.auto-join = off
# don't use sigar for tests, native lib not in path # don't use sigar for tests, native lib not in path
akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector
#//#router-deploy-config #//#router-deploy-config

View file

@ -27,7 +27,6 @@ object StatsSampleSpecConfig extends MultiNodeConfig {
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.roles = [compute] akka.cluster.roles = [compute]
akka.cluster.auto-join = off
# don't use sigar for tests, native lib not in path # don't use sigar for tests, native lib not in path
akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector
#//#router-lookup-config #//#router-lookup-config

View file

@ -31,7 +31,6 @@ object StatsSampleJapiSpecConfig extends MultiNodeConfig {
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.roles = [compute] akka.cluster.roles = [compute]
akka.cluster.auto-join = off
# don't use sigar for tests, native lib not in path # don't use sigar for tests, native lib not in path
akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector
akka.actor.deployment { akka.actor.deployment {

View file

@ -34,7 +34,6 @@ object StatsSampleSingleMasterJapiSpecConfig extends MultiNodeConfig {
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.roles = [compute] akka.cluster.roles = [compute]
akka.cluster.auto-join = off
# don't use sigar for tests, native lib not in path # don't use sigar for tests, native lib not in path
akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector
akka.actor.deployment { akka.actor.deployment {

View file

@ -28,7 +28,6 @@ object TransformationSampleSpecConfig extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-join = off
# don't use sigar for tests, native lib not in path # don't use sigar for tests, native lib not in path
akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector
""")) """))

View file

@ -29,7 +29,6 @@ object TransformationSampleJapiSpecConfig extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString(""" commonConfig(ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-join = off
# don't use sigar for tests, native lib not in path # don't use sigar for tests, native lib not in path
akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector
""")) """))

View file

@ -64,7 +64,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers {
try { try {
var locker = Seq.empty[DeadLetter] var locker = Seq.empty[DeadLetter]
implicit val timeout = TestKitExtension(system).DefaultTimeout implicit val timeout = TestKitExtension(system).DefaultTimeout
implicit val davyJones = otherSystem.actorOf(Props(new Actor { val davyJones = otherSystem.actorOf(Props(new Actor {
def receive = { def receive = {
case m: DeadLetter locker :+= m case m: DeadLetter locker :+= m
case "Die!" sender ! "finally gone"; context.stop(self) case "Die!" sender ! "finally gone"; context.stop(self)