Merge with master
This commit is contained in:
commit
e62a0eee1c
15 changed files with 562 additions and 224 deletions
|
|
@ -0,0 +1,293 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
import akka.actor.ActorSystem
|
||||
import akka.util.Deadline
|
||||
import java.util.concurrent.TimeoutException
|
||||
import scala.collection.immutable.SortedSet
|
||||
import akka.dispatch.Await
|
||||
import akka.util.Duration
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.remote.testconductor.RoleName
|
||||
|
||||
object LargeClusterMultiJvmSpec extends MultiNodeConfig {
|
||||
// each jvm simulates a datacenter with many nodes
|
||||
val firstDatacenter = role("first-datacenter")
|
||||
val secondDatacenter = role("second-datacenter")
|
||||
val thirdDatacenter = role("third-datacenter")
|
||||
val fourthDatacenter = role("fourth-datacenter")
|
||||
val fifthDatacenter = role("fifth-datacenter")
|
||||
|
||||
// Note that this test uses default configuration,
|
||||
// not MultiNodeClusterSpec.clusterConfig
|
||||
commonConfig(ConfigFactory.parseString("""
|
||||
# Number of ActorSystems in each jvm, can be specified as
|
||||
# system property when running real tests. Many nodes
|
||||
# will take long time and consume many threads.
|
||||
# 10 => 50 nodes is possible to run on one machine.
|
||||
akka.test.large-cluster-spec.nodes-per-datacenter = 2
|
||||
akka.cluster {
|
||||
gossip-interval = 500 ms
|
||||
auto-join = off
|
||||
failure-detector.threshold = 4
|
||||
}
|
||||
akka.loglevel = INFO
|
||||
akka.actor.default-dispatcher.fork-join-executor.parallelism-max = 2
|
||||
akka.scheduler.tick-duration = 33 ms
|
||||
akka.remote.netty.execution-pool-size = 0
|
||||
|
||||
# don't use testconductor transport in this test, especially not
|
||||
# when using use-dispatcher-for-io
|
||||
akka.remote.transport = "akka.remote.netty.NettyRemoteTransport"
|
||||
|
||||
# Using a separate dispatcher for netty io doesn't reduce number
|
||||
# of needed threads
|
||||
# akka.remote.netty.use-dispatcher-for-io=akka.test.io-dispatcher
|
||||
# akka.test.io-dispatcher.fork-join-executor {
|
||||
# parallelism-min = 100
|
||||
# parallelism-max = 100
|
||||
# }
|
||||
"""))
|
||||
}
|
||||
|
||||
class LargeClusterMultiJvmNode1 extends LargeClusterSpec with AccrualFailureDetectorStrategy
|
||||
class LargeClusterMultiJvmNode2 extends LargeClusterSpec with AccrualFailureDetectorStrategy
|
||||
class LargeClusterMultiJvmNode3 extends LargeClusterSpec with AccrualFailureDetectorStrategy
|
||||
class LargeClusterMultiJvmNode4 extends LargeClusterSpec with AccrualFailureDetectorStrategy
|
||||
class LargeClusterMultiJvmNode5 extends LargeClusterSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class LargeClusterSpec
|
||||
extends MultiNodeSpec(LargeClusterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
|
||||
import LargeClusterMultiJvmSpec._
|
||||
|
||||
var systems: IndexedSeq[ActorSystem] = IndexedSeq(system)
|
||||
val nodesPerDatacenter = system.settings.config.getInt(
|
||||
"akka.test.large-cluster-spec.nodes-per-datacenter")
|
||||
|
||||
/**
|
||||
* Since we start some ActorSystems/Clusters outside of the
|
||||
* MultiNodeClusterSpec control we can't use use the mechanism
|
||||
* defined in MultiNodeClusterSpec to inject failure detector etc.
|
||||
* Use ordinary Cluster extension with default AccrualFailureDetector.
|
||||
*/
|
||||
override def cluster: Cluster = Cluster(system)
|
||||
|
||||
override def atTermination(): Unit = {
|
||||
systems foreach { _.shutdown }
|
||||
val shutdownTimeout = 20.seconds
|
||||
val deadline = Deadline.now + shutdownTimeout
|
||||
systems.foreach { sys ⇒
|
||||
if (sys.isTerminated)
|
||||
() // already done
|
||||
else if (deadline.isOverdue)
|
||||
sys.log.warning("Failed to shutdown [{}] within [{}]", sys.name, shutdownTimeout)
|
||||
else {
|
||||
try sys.awaitTermination(deadline.timeLeft) catch {
|
||||
case _: TimeoutException ⇒ sys.log.warning("Failed to shutdown [{}] within [{}]", sys.name, shutdownTimeout)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def startupSystems(): Unit = {
|
||||
// one system is already started by the multi-node test
|
||||
for (n ← 2 to nodesPerDatacenter)
|
||||
systems :+= ActorSystem(myself.name + "-" + n, system.settings.config)
|
||||
|
||||
// Initialize the Cluster extensions, i.e. startup the clusters
|
||||
systems foreach { Cluster(_) }
|
||||
}
|
||||
|
||||
def expectedMaxDuration(totalNodes: Int): Duration =
|
||||
5.seconds + (2.seconds * totalNodes)
|
||||
|
||||
def joinAll(from: RoleName, to: RoleName, totalNodes: Int, runOnRoles: RoleName*): Unit = {
|
||||
val joiningClusters = systems.map(Cluster(_)).toSet
|
||||
join(joiningClusters, from, to, totalNodes, runOnRoles: _*)
|
||||
}
|
||||
|
||||
def join(joiningClusterNodes: Set[Cluster], from: RoleName, to: RoleName, totalNodes: Int, runOnRoles: RoleName*): Unit = {
|
||||
runOnRoles must contain(from)
|
||||
runOnRoles must contain(to)
|
||||
|
||||
runOn(runOnRoles: _*) {
|
||||
systems.size must be(nodesPerDatacenter) // make sure it is initialized
|
||||
|
||||
val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet)
|
||||
val startGossipCounts = Map.empty[Cluster, Long] ++
|
||||
clusterNodes.map(c ⇒ (c -> c.receivedGossipCount))
|
||||
def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c)
|
||||
val startTime = System.nanoTime
|
||||
def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms"
|
||||
|
||||
val latch = TestLatch(clusterNodes.size)
|
||||
clusterNodes foreach { c ⇒
|
||||
c.registerListener(new MembershipChangeListener {
|
||||
override def notify(members: SortedSet[Member]): Unit = {
|
||||
if (!latch.isOpen && members.size == totalNodes && members.forall(_.status == MemberStatus.Up)) {
|
||||
log.debug("All [{}] nodes Up in [{}], it took [{}], received [{}] gossip messages",
|
||||
totalNodes, c.selfAddress, tookMillis, gossipCount(c))
|
||||
latch.countDown()
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
runOn(from) {
|
||||
clusterNodes foreach { _ join to }
|
||||
}
|
||||
|
||||
Await.ready(latch, remaining)
|
||||
|
||||
awaitCond(clusterNodes.forall(_.convergence.isDefined))
|
||||
val counts = clusterNodes.map(gossipCount(_))
|
||||
val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / clusterNodes.size, counts.min, counts.max)
|
||||
log.info("Convergence of [{}] nodes reached, it took [{}], received [{}] gossip messages per node",
|
||||
totalNodes, tookMillis, formattedStats)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
"A large cluster" must {
|
||||
|
||||
"join all nodes in first-datacenter to first-datacenter" taggedAs LongRunningTest in {
|
||||
runOn(firstDatacenter) {
|
||||
startupSystems()
|
||||
startClusterNode()
|
||||
}
|
||||
enterBarrier("first-datacenter-started")
|
||||
|
||||
val totalNodes = nodesPerDatacenter
|
||||
within(expectedMaxDuration(totalNodes)) {
|
||||
joinAll(from = firstDatacenter, to = firstDatacenter, totalNodes, runOnRoles = firstDatacenter)
|
||||
enterBarrier("first-datacenter-joined")
|
||||
}
|
||||
}
|
||||
|
||||
"join all nodes in second-datacenter to first-datacenter" taggedAs LongRunningTest in {
|
||||
runOn(secondDatacenter) {
|
||||
startupSystems()
|
||||
}
|
||||
enterBarrier("second-datacenter-started")
|
||||
|
||||
val totalNodes = nodesPerDatacenter * 2
|
||||
within(expectedMaxDuration(totalNodes)) {
|
||||
joinAll(from = secondDatacenter, to = firstDatacenter, totalNodes, runOnRoles = firstDatacenter, secondDatacenter)
|
||||
enterBarrier("second-datacenter-joined")
|
||||
}
|
||||
}
|
||||
|
||||
"join all nodes in third-datacenter to first-datacenter" taggedAs LongRunningTest in {
|
||||
runOn(thirdDatacenter) {
|
||||
startupSystems()
|
||||
}
|
||||
enterBarrier("third-datacenter-started")
|
||||
|
||||
val totalNodes = nodesPerDatacenter * 3
|
||||
within(expectedMaxDuration(totalNodes)) {
|
||||
joinAll(from = thirdDatacenter, to = firstDatacenter, totalNodes,
|
||||
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter)
|
||||
enterBarrier("third-datacenter-joined")
|
||||
}
|
||||
}
|
||||
|
||||
"join all nodes in fourth-datacenter to first-datacenter" taggedAs LongRunningTest in {
|
||||
runOn(fourthDatacenter) {
|
||||
startupSystems()
|
||||
}
|
||||
enterBarrier("fourth-datacenter-started")
|
||||
|
||||
val totalNodes = nodesPerDatacenter * 4
|
||||
within(expectedMaxDuration(totalNodes)) {
|
||||
joinAll(from = fourthDatacenter, to = firstDatacenter, totalNodes,
|
||||
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter)
|
||||
enterBarrier("fourth-datacenter-joined")
|
||||
}
|
||||
}
|
||||
|
||||
"join nodes one by one from fifth-datacenter to first-datacenter" taggedAs LongRunningTest in {
|
||||
runOn(fifthDatacenter) {
|
||||
startupSystems()
|
||||
}
|
||||
enterBarrier("fifth-datacenter-started")
|
||||
|
||||
// enough to join a few one-by-one (takes too long time otherwise)
|
||||
val (bulk, oneByOne) = systems.splitAt(systems.size - 3)
|
||||
|
||||
if (bulk.nonEmpty) {
|
||||
val totalNodes = nodesPerDatacenter * 4 + bulk.size
|
||||
within(expectedMaxDuration(totalNodes)) {
|
||||
val joiningClusters = ifNode(fifthDatacenter)(bulk.map(Cluster(_)).toSet)(Set.empty)
|
||||
join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
|
||||
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
|
||||
enterBarrier("fifth-datacenter-joined-" + bulk.size)
|
||||
}
|
||||
}
|
||||
|
||||
for (i ← 0 until oneByOne.size) {
|
||||
val totalNodes = nodesPerDatacenter * 4 + bulk.size + i + 1
|
||||
within(expectedMaxDuration(totalNodes)) {
|
||||
val joiningClusters = ifNode(fifthDatacenter)(Set(Cluster(oneByOne(i))))(Set.empty)
|
||||
join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
|
||||
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
|
||||
enterBarrier("fifth-datacenter-joined-" + (bulk.size + i))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// FIXME sometimes this fails, FD marks nodes from other than second-datacenter as unavailable
|
||||
"detect failure and auto-down crashed nodes in second-datacenter" taggedAs LongRunningTest ignore {
|
||||
val unreachableNodes = nodesPerDatacenter
|
||||
val liveNodes = nodesPerDatacenter * 4
|
||||
|
||||
within(20.seconds + expectedMaxDuration(liveNodes)) {
|
||||
val startGossipCounts = Map.empty[Cluster, Long] ++
|
||||
systems.map(sys ⇒ (Cluster(sys) -> Cluster(sys).receivedGossipCount))
|
||||
def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c)
|
||||
val startTime = System.nanoTime
|
||||
def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms"
|
||||
|
||||
val latch = TestLatch(nodesPerDatacenter)
|
||||
systems foreach { sys ⇒
|
||||
Cluster(sys).registerListener(new MembershipChangeListener {
|
||||
override def notify(members: SortedSet[Member]): Unit = {
|
||||
if (!latch.isOpen && members.size == liveNodes && Cluster(sys).latestGossip.overview.unreachable.size == unreachableNodes) {
|
||||
log.info("Detected [{}] unreachable nodes in [{}], it took [{}], received [{}] gossip messages",
|
||||
unreachableNodes, Cluster(sys).selfAddress, tookMillis, gossipCount(Cluster(sys)))
|
||||
latch.countDown()
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
runOn(firstDatacenter) {
|
||||
testConductor.shutdown(secondDatacenter, 0)
|
||||
}
|
||||
|
||||
enterBarrier("second-datacenter-shutdown")
|
||||
|
||||
runOn(firstDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) {
|
||||
Await.ready(latch, remaining)
|
||||
awaitCond(systems.forall(Cluster(_).convergence.isDefined))
|
||||
val counts = systems.map(sys ⇒ gossipCount(Cluster(sys)))
|
||||
val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / nodesPerDatacenter, counts.min, counts.max)
|
||||
log.info("Convergence of [{}] nodes reached after failure, it took [{}], received [{}] gossip messages per node",
|
||||
liveNodes, tookMillis, formattedStats)
|
||||
}
|
||||
|
||||
enterBarrier("after-6")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -23,8 +23,6 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
|
|||
// not MultiNodeClusterSpec.clusterConfig
|
||||
commonConfig(ConfigFactory.parseString("""
|
||||
akka.cluster {
|
||||
# FIXME remove this (use default) when ticket #2239 has been fixed
|
||||
gossip-interval = 400 ms
|
||||
auto-join = off
|
||||
}
|
||||
akka.loglevel = INFO
|
||||
|
|
|
|||
|
|
@ -16,8 +16,6 @@ object TransitionMultiJvmSpec extends MultiNodeConfig {
|
|||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
val fifth = role("fifth")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("akka.cluster.periodic-tasks-initial-delay = 300 s # turn off all periodic tasks")).
|
||||
|
|
@ -27,8 +25,6 @@ object TransitionMultiJvmSpec extends MultiNodeConfig {
|
|||
class TransitionMultiJvmNode1 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
class TransitionMultiJvmNode2 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
class TransitionMultiJvmNode3 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
class TransitionMultiJvmNode4 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
class TransitionMultiJvmNode5 extends TransitionSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
abstract class TransitionSpec
|
||||
extends MultiNodeSpec(TransitionMultiJvmSpec)
|
||||
|
|
@ -81,11 +77,15 @@ abstract class TransitionSpec
|
|||
val g = cluster.latestGossip
|
||||
enterBarrier("before-gossip-" + gossipBarrierCounter)
|
||||
awaitCond(cluster.latestGossip != g) // received gossip
|
||||
// gossip chat will synchronize the views
|
||||
awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty)
|
||||
enterBarrier("after-gossip-" + gossipBarrierCounter)
|
||||
}
|
||||
runOn(fromRole) {
|
||||
enterBarrier("before-gossip-" + gossipBarrierCounter)
|
||||
cluster.gossipTo(toRole) // send gossip
|
||||
// gossip chat will synchronize the views
|
||||
awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty)
|
||||
enterBarrier("after-gossip-" + gossipBarrierCounter)
|
||||
}
|
||||
runOn(roles.filterNot(r ⇒ r == fromRole || r == toRole): _*) {
|
||||
|
|
@ -116,22 +116,12 @@ abstract class TransitionSpec
|
|||
runOn(second) {
|
||||
cluster.join(first)
|
||||
}
|
||||
runOn(first) {
|
||||
runOn(first, second) {
|
||||
// gossip chat from the join will synchronize the views
|
||||
awaitMembers(first, second)
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Joining)
|
||||
seenLatestGossip must be(Set(first))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
enterBarrier("second-joined")
|
||||
|
||||
first gossipTo second
|
||||
second gossipTo first
|
||||
|
||||
runOn(first, second) {
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Joining)
|
||||
seenLatestGossip must be(Set(first, second))
|
||||
awaitCond(seenLatestGossip == Set(first, second))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
enterBarrier("convergence-joining-2")
|
||||
|
|
@ -144,18 +134,11 @@ abstract class TransitionSpec
|
|||
enterBarrier("leader-actions-2")
|
||||
|
||||
leader(first, second) gossipTo nonLeader(first, second).head
|
||||
runOn(nonLeader(first, second).head) {
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
seenLatestGossip must be(Set(first, second))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
nonLeader(first, second).head gossipTo leader(first, second)
|
||||
runOn(first, second) {
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
// gossip chat will synchronize the views
|
||||
awaitCond(memberStatus(second) == Up)
|
||||
seenLatestGossip must be(Set(first, second))
|
||||
memberStatus(first) must be(Up)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
|
|
@ -167,25 +150,26 @@ abstract class TransitionSpec
|
|||
runOn(third) {
|
||||
cluster.join(second)
|
||||
}
|
||||
runOn(second) {
|
||||
runOn(second, third) {
|
||||
// gossip chat from the join will synchronize the views
|
||||
awaitMembers(first, second, third)
|
||||
cluster.convergence.isDefined must be(false)
|
||||
memberStatus(third) must be(Joining)
|
||||
seenLatestGossip must be(Set(second))
|
||||
awaitCond(seenLatestGossip == Set(second, third))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
enterBarrier("third-joined-second")
|
||||
|
||||
second gossipTo first
|
||||
runOn(first) {
|
||||
members must be(Set(first, second, third))
|
||||
runOn(first, second) {
|
||||
// gossip chat will synchronize the views
|
||||
awaitMembers(first, second, third)
|
||||
memberStatus(third) must be(Joining)
|
||||
seenLatestGossip must be(Set(first, second))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
awaitCond(memberStatus(second) == Up)
|
||||
seenLatestGossip must be(Set(first, second, third))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
first gossipTo third
|
||||
third gossipTo first
|
||||
third gossipTo second
|
||||
runOn(first, second, third) {
|
||||
members must be(Set(first, second, third))
|
||||
memberStatus(first) must be(Up)
|
||||
|
|
@ -224,14 +208,6 @@ abstract class TransitionSpec
|
|||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
// and back again
|
||||
nonLeader(first, second, third).tail.head gossipTo nonLeader(first, second, third).head
|
||||
runOn(nonLeader(first, second, third).head) {
|
||||
memberStatus(third) must be(Up)
|
||||
seenLatestGossip must be(Set(first, second, third))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
// first non-leader gossipTo the leader
|
||||
nonLeader(first, second, third).head gossipTo leader(first, second, third)
|
||||
runOn(first, second, third) {
|
||||
|
|
@ -245,160 +221,36 @@ abstract class TransitionSpec
|
|||
enterBarrier("after-3")
|
||||
}
|
||||
|
||||
"startup a second separated cluster consisting of nodes fourth and fifth" taggedAs LongRunningTest in {
|
||||
runOn(fifth) {
|
||||
startClusterNode()
|
||||
cluster.leaderActions()
|
||||
cluster.status must be(Up)
|
||||
}
|
||||
enterBarrier("fifth-started")
|
||||
|
||||
runOn(fourth) {
|
||||
cluster.join(fifth)
|
||||
}
|
||||
runOn(fifth) {
|
||||
awaitMembers(fourth, fifth)
|
||||
}
|
||||
enterBarrier("fourth-joined")
|
||||
|
||||
fifth gossipTo fourth
|
||||
fourth gossipTo fifth
|
||||
|
||||
runOn(fourth, fifth) {
|
||||
memberStatus(fourth) must be(Joining)
|
||||
memberStatus(fifth) must be(Up)
|
||||
seenLatestGossip must be(Set(fourth, fifth))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"perform correct transitions when second cluster (node fourth) joins first cluster (node third)" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(fourth) {
|
||||
cluster.join(third)
|
||||
}
|
||||
runOn(third) {
|
||||
awaitMembers(first, second, third, fourth)
|
||||
seenLatestGossip must be(Set(third))
|
||||
}
|
||||
enterBarrier("fourth-joined-third")
|
||||
|
||||
third gossipTo second
|
||||
runOn(second) {
|
||||
seenLatestGossip must be(Set(second, third))
|
||||
}
|
||||
|
||||
second gossipTo fourth
|
||||
runOn(fourth) {
|
||||
members must be(roles.toSet)
|
||||
// merge conflict
|
||||
seenLatestGossip must be(Set(fourth))
|
||||
}
|
||||
|
||||
fourth gossipTo first
|
||||
fourth gossipTo second
|
||||
fourth gossipTo third
|
||||
fourth gossipTo fifth
|
||||
runOn(first, second, third, fifth) {
|
||||
members must be(roles.toSet)
|
||||
seenLatestGossip must be(Set(fourth, myself))
|
||||
}
|
||||
|
||||
first gossipTo fifth
|
||||
runOn(fifth) {
|
||||
seenLatestGossip must be(Set(first, fourth, fifth))
|
||||
}
|
||||
|
||||
fifth gossipTo third
|
||||
runOn(third) {
|
||||
seenLatestGossip must be(Set(first, third, fourth, fifth))
|
||||
}
|
||||
|
||||
third gossipTo second
|
||||
runOn(second) {
|
||||
seenLatestGossip must be(roles.toSet)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
second gossipTo first
|
||||
second gossipTo third
|
||||
second gossipTo fourth
|
||||
third gossipTo fifth
|
||||
|
||||
seenLatestGossip must be(roles.toSet)
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
memberStatus(third) must be(Up)
|
||||
memberStatus(fourth) must be(Joining)
|
||||
memberStatus(fifth) must be(Up)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
|
||||
enterBarrier("convergence-joining-3")
|
||||
|
||||
runOn(leader(roles: _*)) {
|
||||
cluster.leaderActions()
|
||||
memberStatus(fourth) must be(Up)
|
||||
seenLatestGossip must be(Set(myself))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
// spread the word
|
||||
for (x :: y :: Nil ← (roles.sorted ++ roles.sorted.dropRight(1)).toList.sliding(2)) {
|
||||
x gossipTo y
|
||||
}
|
||||
|
||||
enterBarrier("spread-5")
|
||||
|
||||
seenLatestGossip must be(roles.toSet)
|
||||
memberStatus(first) must be(Up)
|
||||
memberStatus(second) must be(Up)
|
||||
memberStatus(third) must be(Up)
|
||||
memberStatus(fourth) must be(Up)
|
||||
memberStatus(fifth) must be(Up)
|
||||
cluster.convergence.isDefined must be(true)
|
||||
|
||||
enterBarrier("after-5")
|
||||
}
|
||||
|
||||
"perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in {
|
||||
runOn(fifth) {
|
||||
runOn(third) {
|
||||
markNodeAsUnavailable(second)
|
||||
cluster.reapUnreachableMembers()
|
||||
cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
|
||||
seenLatestGossip must be(Set(fifth))
|
||||
seenLatestGossip must be(Set(third))
|
||||
}
|
||||
|
||||
enterBarrier("after-second-unavailble")
|
||||
|
||||
// spread the word
|
||||
val gossipRound = List(fifth, fourth, third, first, third, fourth, fifth)
|
||||
for (x :: y :: Nil ← gossipRound.sliding(2)) {
|
||||
x gossipTo y
|
||||
}
|
||||
third gossipTo first
|
||||
|
||||
runOn((roles.filterNot(_ == second)): _*) {
|
||||
runOn(first, third) {
|
||||
cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
|
||||
runOn(third) {
|
||||
runOn(first) {
|
||||
cluster.down(second)
|
||||
awaitMemberStatus(second, Down)
|
||||
}
|
||||
|
||||
enterBarrier("after-second-down")
|
||||
|
||||
// spread the word
|
||||
val gossipRound2 = List(third, fourth, fifth, first, third, fourth, fifth)
|
||||
for (x :: y :: Nil ← gossipRound2.sliding(2)) {
|
||||
x gossipTo y
|
||||
}
|
||||
first gossipTo third
|
||||
|
||||
runOn((roles.filterNot(_ == second)): _*) {
|
||||
runOn(first, third) {
|
||||
cluster.latestGossip.overview.unreachable must contain(Member(second, Down))
|
||||
memberStatus(second) must be(Down)
|
||||
seenLatestGossip must be(Set(first, third, fourth, fifth))
|
||||
seenLatestGossip must be(Set(first, third))
|
||||
cluster.convergence.isDefined must be(true)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,137 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster
|
||||
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.Address
|
||||
import akka.remote.testconductor.{RoleName, Direction}
|
||||
import akka.util.duration._
|
||||
|
||||
object UnreachableNodeRejoinsClusterMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
|
||||
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
|
||||
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
|
||||
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
|
||||
|
||||
|
||||
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
|
||||
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
|
||||
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
|
||||
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
|
||||
|
||||
abstract class UnreachableNodeRejoinsClusterSpec
|
||||
extends MultiNodeSpec(UnreachableNodeRejoinsClusterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec {
|
||||
import UnreachableNodeRejoinsClusterMultiJvmSpec._
|
||||
|
||||
def allBut(role: RoleName, roles: Seq[RoleName] = roles): Seq[RoleName] = {
|
||||
roles.filterNot(_ == role)
|
||||
}
|
||||
|
||||
|
||||
lazy val sortedRoles = roles.sorted
|
||||
lazy val master = sortedRoles(0)
|
||||
lazy val victim = sortedRoles(1)
|
||||
|
||||
var endBarrierNumber = 0
|
||||
def endBarrier: Unit = {
|
||||
endBarrierNumber += 1
|
||||
enterBarrier("after_" + endBarrierNumber)
|
||||
}
|
||||
|
||||
"A cluster of " + roles.size + " members" must {
|
||||
|
||||
"reach initial convergence" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(roles:_*)
|
||||
endBarrier
|
||||
}
|
||||
|
||||
"mark a node as UNREACHABLE when we pull the network" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
// pull network for victim node from all nodes
|
||||
allBut(victim).foreach { roleName =>
|
||||
testConductor.blackhole(victim, roleName, Direction.Both).await
|
||||
}
|
||||
}
|
||||
|
||||
enterBarrier("unplug_victim")
|
||||
|
||||
val allButVictim = allBut(victim, sortedRoles)
|
||||
runOn(victim) {
|
||||
allButVictim.foreach(markNodeAsUnavailable(_))
|
||||
within(30 seconds) {
|
||||
// victim becomes all alone
|
||||
awaitCond({ val gossip = cluster.latestGossip
|
||||
gossip.overview.unreachable.size == (roles.size - 1) &&
|
||||
gossip.members.size == 1 &&
|
||||
gossip.members.forall(_.status == MemberStatus.Up) })
|
||||
cluster.latestGossip.overview.unreachable.map(_.address) must be((allButVictim map address).toSet)
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
}
|
||||
|
||||
runOn(allButVictim:_*) {
|
||||
markNodeAsUnavailable(victim)
|
||||
within(30 seconds) {
|
||||
// victim becomes unreachable
|
||||
awaitCond({ val gossip = cluster.latestGossip
|
||||
gossip.overview.unreachable.size == 1 &&
|
||||
gossip.members.size == (roles.size - 1) &&
|
||||
gossip.members.forall(_.status == MemberStatus.Up) })
|
||||
awaitSeenSameState(allButVictim map address:_*)
|
||||
// still one unreachable
|
||||
cluster.latestGossip.overview.unreachable.size must be(1)
|
||||
cluster.latestGossip.overview.unreachable.head.address must be(node(victim).address)
|
||||
// and therefore no convergence
|
||||
cluster.convergence.isDefined must be(false)
|
||||
}
|
||||
}
|
||||
|
||||
endBarrier
|
||||
}
|
||||
|
||||
"mark the node as DOWN" taggedAs LongRunningTest in {
|
||||
runOn(master) {
|
||||
cluster down victim
|
||||
}
|
||||
|
||||
runOn(allBut(victim):_*) {
|
||||
awaitUpConvergence(roles.size - 1, Seq(victim))
|
||||
}
|
||||
|
||||
endBarrier
|
||||
}
|
||||
|
||||
"allow node to REJOIN when the network is plugged back in" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
// put the network back in
|
||||
allBut(victim).foreach { roleName =>
|
||||
testConductor.passThrough(victim, roleName, Direction.Both).await
|
||||
}
|
||||
}
|
||||
|
||||
enterBarrier("plug_in_victim")
|
||||
|
||||
runOn(victim) {
|
||||
cluster join master
|
||||
}
|
||||
|
||||
awaitUpConvergence(roles.size)
|
||||
|
||||
endBarrier
|
||||
}
|
||||
}
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue