Merge with master

This commit is contained in:
Viktor Klang 2012-06-29 16:07:36 +02:00
commit e62a0eee1c
15 changed files with 562 additions and 224 deletions

View file

@ -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")
}
}
}
}

View file

@ -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

View file

@ -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)
}

View file

@ -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
}
}
}