First step in refactoring of cluster internals to actors, see #2311

* Move clustering code to ClusterCore actor
* More will be done, comitting this for early review
This commit is contained in:
Patrik Nordwall 2012-07-04 11:37:56 +02:00
parent 1ccb9fe7ec
commit c708d2ad8a
11 changed files with 820 additions and 870 deletions

View file

@ -27,9 +27,6 @@ akka {
# network partition. # network partition.
auto-down = off auto-down = off
# the number of gossip daemon actors
nr-of-gossip-daemons = 4
# the number of deputy nodes (the nodes responsible for breaking network partitions) # the number of deputy nodes (the nodes responsible for breaking network partitions)
nr-of-deputy-nodes = 3 nr-of-deputy-nodes = 3
@ -48,6 +45,10 @@ akka {
# how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring? # how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring?
unreachable-nodes-reaper-interval = 1s unreachable-nodes-reaper-interval = 1s
# How often the current state (Gossip) should be published for reading from the outside.
# A value of 0 s can be used to always publish the state, when it happens.
publish-state-interval = 1s
# A joining node stops sending heartbeats to the node to join if it hasn't become member # A joining node stops sending heartbeats to the node to join if it hasn't become member
# of the cluster within this deadline. # of the cluster within this deadline.
join-timeout = 60s join-timeout = 60s

File diff suppressed because it is too large Load diff

View file

@ -32,8 +32,7 @@ class ClusterSettings(val config: Config, val systemName: String) {
final val HeartbeatInterval: Duration = Duration(getMilliseconds("akka.cluster.heartbeat-interval"), MILLISECONDS) final val HeartbeatInterval: Duration = Duration(getMilliseconds("akka.cluster.heartbeat-interval"), MILLISECONDS)
final val LeaderActionsInterval: Duration = Duration(getMilliseconds("akka.cluster.leader-actions-interval"), MILLISECONDS) final val LeaderActionsInterval: Duration = Duration(getMilliseconds("akka.cluster.leader-actions-interval"), MILLISECONDS)
final val UnreachableNodesReaperInterval: Duration = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-interval"), MILLISECONDS) final val UnreachableNodesReaperInterval: Duration = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-interval"), MILLISECONDS)
final val NrOfGossipDaemons: Int = getInt("akka.cluster.nr-of-gossip-daemons") final val PublishStateInterval: Duration = Duration(getMilliseconds("akka.cluster.publish-state-interval"), MILLISECONDS)
final val NrOfDeputyNodes: Int = getInt("akka.cluster.nr-of-deputy-nodes")
final val AutoJoin: Boolean = getBoolean("akka.cluster.auto-join") final val AutoJoin: Boolean = getBoolean("akka.cluster.auto-join")
final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down") final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down")
final val JoinTimeout: Duration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS) final val JoinTimeout: Duration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS)

View file

@ -7,9 +7,9 @@ package akka.cluster
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import akka.actor.Scheduler import akka.actor.Scheduler
import akka.util.Duration import akka.util.Duration
import akka.actor.Cancellable
/** /**
* INTERNAL API * INTERNAL API
@ -27,7 +27,8 @@ private[akka] object FixedRateTask {
* for inaccuracy in scheduler. It will start when constructed, using the * for inaccuracy in scheduler. It will start when constructed, using the
* initialDelay. * initialDelay.
*/ */
private[akka] class FixedRateTask(scheduler: Scheduler, initalDelay: Duration, delay: Duration, task: Runnable) extends Runnable { private[akka] class FixedRateTask(scheduler: Scheduler, initalDelay: Duration, delay: Duration, task: Runnable)
extends Runnable with Cancellable {
private val delayNanos = delay.toNanos private val delayNanos = delay.toNanos
private val cancelled = new AtomicBoolean(false) private val cancelled = new AtomicBoolean(false)
@ -37,9 +38,11 @@ private[akka] class FixedRateTask(scheduler: Scheduler, initalDelay: Duration, d
def cancel(): Unit = cancelled.set(true) def cancel(): Unit = cancelled.set(true)
override final def run(): Unit = if (!cancelled.get) try { def isCancelled: Boolean = cancelled.get
override final def run(): Unit = if (!isCancelled) try {
task.run() task.run()
} finally if (!cancelled.get) { } finally if (!isCancelled) {
val nextTime = startTime + delayNanos * counter.incrementAndGet val nextTime = startTime + delayNanos * counter.incrementAndGet
// it's ok to schedule with negative duration, will run asap // it's ok to schedule with negative duration, will run asap
val nextDelay = Duration(nextTime - System.nanoTime, TimeUnit.NANOSECONDS) val nextDelay = Duration(nextTime - System.nanoTime, TimeUnit.NANOSECONDS)

View file

@ -36,6 +36,10 @@ abstract class JoinSeedNodeSpec
"A cluster with configured seed nodes" must { "A cluster with configured seed nodes" must {
"start the seed nodes sequentially" taggedAs LongRunningTest in { "start the seed nodes sequentially" taggedAs LongRunningTest in {
// without looking up the addresses first there might be
// [akka://JoinSeedNodeSpec/user/TestConductorClient] cannot write GetAddress(RoleName(seed2)) while waiting for seed1
roles foreach address
runOn(seed1) { runOn(seed1) {
startClusterNode() startClusterNode()
} }

View file

@ -36,8 +36,8 @@ object LargeClusterMultiJvmSpec extends MultiNodeConfig {
akka.cluster { akka.cluster {
gossip-interval = 500 ms gossip-interval = 500 ms
auto-join = off auto-join = off
nr-of-gossip-daemons = 2
failure-detector.acceptable-heartbeat-pause = 10s failure-detector.acceptable-heartbeat-pause = 10s
publish-state-interval = 0 s # always, when it happens
} }
akka.loglevel = INFO akka.loglevel = INFO
akka.actor.default-dispatcher.fork-join-executor { akka.actor.default-dispatcher.fork-join-executor {
@ -133,8 +133,10 @@ abstract class LargeClusterSpec
val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet) val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet)
val startGossipCounts = Map.empty[Cluster, Long] ++ val startGossipCounts = Map.empty[Cluster, Long] ++
clusterNodes.map(c (c -> c.receivedGossipCount)) clusterNodes.map(c (c -> c.latestStats.receivedGossipCount))
def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c) def gossipCount(c: Cluster): Long = {
c.latestStats.receivedGossipCount - startGossipCounts(c)
}
val startTime = System.nanoTime val startTime = System.nanoTime
def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms" def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms"
@ -259,8 +261,10 @@ abstract class LargeClusterSpec
within(30.seconds + (3.seconds * liveNodes)) { within(30.seconds + (3.seconds * liveNodes)) {
val startGossipCounts = Map.empty[Cluster, Long] ++ val startGossipCounts = Map.empty[Cluster, Long] ++
systems.map(sys (Cluster(sys) -> Cluster(sys).receivedGossipCount)) systems.map(sys (Cluster(sys) -> Cluster(sys).latestStats.receivedGossipCount))
def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c) def gossipCount(c: Cluster): Long = {
c.latestStats.receivedGossipCount - startGossipCounts(c)
}
val startTime = System.nanoTime val startTime = System.nanoTime
def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms" def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms"
@ -286,7 +290,7 @@ abstract class LargeClusterSpec
runOn(firstDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) { runOn(firstDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) {
Await.ready(latch, remaining) Await.ready(latch, remaining)
awaitCond(systems.forall(Cluster(_).convergence.isDefined)) awaitCond(systems.forall(Cluster(_).convergence.isDefined))
val mergeCount = systems.map(sys Cluster(sys).mergeCount).sum val mergeCount = systems.map(sys Cluster(sys).latestStats.mergeCount).sum
val counts = systems.map(sys gossipCount(Cluster(sys))) val counts = systems.map(sys gossipCount(Cluster(sys)))
val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / nodesPerDatacenter, counts.min, counts.max) 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, merged [{}] times", log.info("Convergence of [{}] nodes reached after failure, it took [{}], received [{}] gossip messages per node, merged [{}] times",

View file

@ -27,6 +27,7 @@ object MultiNodeClusterSpec {
leader-actions-interval = 200 ms leader-actions-interval = 200 ms
unreachable-nodes-reaper-interval = 200 ms unreachable-nodes-reaper-interval = 200 ms
periodic-tasks-initial-delay = 300 ms periodic-tasks-initial-delay = 300 ms
publish-state-interval = 0 s # always, when it happens
} }
akka.test { akka.test {
single-expect-default = 5 s single-expect-default = 5 s

View file

@ -9,8 +9,10 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import akka.actor.Address import akka.actor.Address
import akka.pattern.ask
import akka.remote.testconductor.RoleName import akka.remote.testconductor.RoleName
import MemberStatus._ import MemberStatus._
import InternalClusterAction._
object TransitionMultiJvmSpec extends MultiNodeConfig { object TransitionMultiJvmSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -28,7 +30,8 @@ class TransitionMultiJvmNode3 extends TransitionSpec with FailureDetectorPuppetS
abstract class TransitionSpec abstract class TransitionSpec
extends MultiNodeSpec(TransitionMultiJvmSpec) extends MultiNodeSpec(TransitionMultiJvmSpec)
with MultiNodeClusterSpec { with MultiNodeClusterSpec
with ImplicitSender {
import TransitionMultiJvmSpec._ import TransitionMultiJvmSpec._
@ -67,6 +70,22 @@ abstract class TransitionSpec
memberStatus(address) == status memberStatus(address) == status
} }
def leaderActions(): Unit = {
cluster.clusterCore ! LeaderActionsTick
awaitPing()
}
def reapUnreachable(): Unit = {
cluster.clusterCore ! ReapUnreachableTick
awaitPing()
}
def awaitPing(): Unit = {
val ping = Ping()
cluster.clusterCore ! ping
expectMsgPF() { case pong @ Pong(`ping`, _) pong }
}
// DSL sugar for `role1 gossipTo role2` // DSL sugar for `role1 gossipTo role2`
implicit def roleExtras(role: RoleName): RoleWrapper = new RoleWrapper(role) implicit def roleExtras(role: RoleName): RoleWrapper = new RoleWrapper(role)
var gossipBarrierCounter = 0 var gossipBarrierCounter = 0
@ -83,7 +102,8 @@ abstract class TransitionSpec
} }
runOn(fromRole) { runOn(fromRole) {
enterBarrier("before-gossip-" + gossipBarrierCounter) enterBarrier("before-gossip-" + gossipBarrierCounter)
cluster.gossipTo(toRole) // send gossip // send gossip
cluster.clusterCore ! InternalClusterAction.SendGossipTo(toRole)
// gossip chat will synchronize the views // gossip chat will synchronize the views
awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty) awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty)
enterBarrier("after-gossip-" + gossipBarrierCounter) enterBarrier("after-gossip-" + gossipBarrierCounter)
@ -104,7 +124,7 @@ abstract class TransitionSpec
cluster.isSingletonCluster must be(true) cluster.isSingletonCluster must be(true)
cluster.status must be(Joining) cluster.status must be(Joining)
cluster.convergence.isDefined must be(true) cluster.convergence.isDefined must be(true)
cluster.leaderActions() leaderActions()
cluster.status must be(Up) cluster.status must be(Up)
} }
@ -127,7 +147,7 @@ abstract class TransitionSpec
enterBarrier("convergence-joining-2") enterBarrier("convergence-joining-2")
runOn(leader(first, second)) { runOn(leader(first, second)) {
cluster.leaderActions() leaderActions()
memberStatus(first) must be(Up) memberStatus(first) must be(Up)
memberStatus(second) must be(Up) memberStatus(second) must be(Up)
} }
@ -182,7 +202,7 @@ abstract class TransitionSpec
enterBarrier("convergence-joining-3") enterBarrier("convergence-joining-3")
runOn(leader(first, second, third)) { runOn(leader(first, second, third)) {
cluster.leaderActions() leaderActions()
memberStatus(first) must be(Up) memberStatus(first) must be(Up)
memberStatus(second) must be(Up) memberStatus(second) must be(Up)
memberStatus(third) must be(Up) memberStatus(third) must be(Up)
@ -200,7 +220,8 @@ abstract class TransitionSpec
// first non-leader gossipTo the other non-leader // first non-leader gossipTo the other non-leader
nonLeader(first, second, third).head gossipTo nonLeader(first, second, third).tail.head nonLeader(first, second, third).head gossipTo nonLeader(first, second, third).tail.head
runOn(nonLeader(first, second, third).head) { runOn(nonLeader(first, second, third).head) {
cluster.gossipTo(nonLeader(first, second, third).tail.head) // send gossip
cluster.clusterCore ! InternalClusterAction.SendGossipTo(nonLeader(first, second, third).tail.head)
} }
runOn(nonLeader(first, second, third).tail.head) { runOn(nonLeader(first, second, third).tail.head) {
memberStatus(third) must be(Up) memberStatus(third) must be(Up)
@ -224,7 +245,7 @@ abstract class TransitionSpec
"perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in { "perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in {
runOn(third) { runOn(third) {
markNodeAsUnavailable(second) markNodeAsUnavailable(second)
cluster.reapUnreachableMembers() reapUnreachable()
cluster.latestGossip.overview.unreachable must contain(Member(second, Up)) cluster.latestGossip.overview.unreachable must contain(Member(second, Up))
seenLatestGossip must be(Set(third)) seenLatestGossip must be(Set(third))
} }

View file

@ -9,7 +9,7 @@ import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.Address import akka.actor.Address
import akka.remote.testconductor.{RoleName, Direction} import akka.remote.testconductor.{ RoleName, Direction }
import akka.util.duration._ import akka.util.duration._
object UnreachableNodeRejoinsClusterMultiJvmSpec extends MultiNodeConfig { object UnreachableNodeRejoinsClusterMultiJvmSpec extends MultiNodeConfig {
@ -26,7 +26,6 @@ class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode2 extend
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy
@ -41,7 +40,6 @@ abstract class UnreachableNodeRejoinsClusterSpec
roles.filterNot(_ == role) roles.filterNot(_ == role)
} }
lazy val sortedRoles = roles.sorted lazy val sortedRoles = roles.sorted
lazy val master = sortedRoles(0) lazy val master = sortedRoles(0)
lazy val victim = sortedRoles(1) lazy val victim = sortedRoles(1)
@ -55,14 +53,14 @@ abstract class UnreachableNodeRejoinsClusterSpec
"A cluster of " + roles.size + " members" must { "A cluster of " + roles.size + " members" must {
"reach initial convergence" taggedAs LongRunningTest in { "reach initial convergence" taggedAs LongRunningTest in {
awaitClusterUp(roles:_*) awaitClusterUp(roles: _*)
endBarrier endBarrier
} }
"mark a node as UNREACHABLE when we pull the network" taggedAs LongRunningTest in { "mark a node as UNREACHABLE when we pull the network" taggedAs LongRunningTest in {
runOn(first) { runOn(first) {
// pull network for victim node from all nodes // pull network for victim node from all nodes
allBut(victim).foreach { roleName => allBut(victim).foreach { roleName
testConductor.blackhole(victim, roleName, Direction.Both).await testConductor.blackhole(victim, roleName, Direction.Both).await
} }
} }
@ -74,24 +72,28 @@ abstract class UnreachableNodeRejoinsClusterSpec
allButVictim.foreach(markNodeAsUnavailable(_)) allButVictim.foreach(markNodeAsUnavailable(_))
within(30 seconds) { within(30 seconds) {
// victim becomes all alone // victim becomes all alone
awaitCond({ val gossip = cluster.latestGossip awaitCond({
val gossip = cluster.latestGossip
gossip.overview.unreachable.size == (roles.size - 1) && gossip.overview.unreachable.size == (roles.size - 1) &&
gossip.members.size == 1 && gossip.members.size == 1 &&
gossip.members.forall(_.status == MemberStatus.Up) }) gossip.members.forall(_.status == MemberStatus.Up)
})
cluster.latestGossip.overview.unreachable.map(_.address) must be((allButVictim map address).toSet) cluster.latestGossip.overview.unreachable.map(_.address) must be((allButVictim map address).toSet)
cluster.convergence.isDefined must be(false) cluster.convergence.isDefined must be(false)
} }
} }
runOn(allButVictim:_*) { runOn(allButVictim: _*) {
markNodeAsUnavailable(victim) markNodeAsUnavailable(victim)
within(30 seconds) { within(30 seconds) {
// victim becomes unreachable // victim becomes unreachable
awaitCond({ val gossip = cluster.latestGossip awaitCond({
val gossip = cluster.latestGossip
gossip.overview.unreachable.size == 1 && gossip.overview.unreachable.size == 1 &&
gossip.members.size == (roles.size - 1) && gossip.members.size == (roles.size - 1) &&
gossip.members.forall(_.status == MemberStatus.Up) }) gossip.members.forall(_.status == MemberStatus.Up)
awaitSeenSameState(allButVictim map address:_*) })
awaitSeenSameState(allButVictim map address: _*)
// still one unreachable // still one unreachable
cluster.latestGossip.overview.unreachable.size must be(1) cluster.latestGossip.overview.unreachable.size must be(1)
cluster.latestGossip.overview.unreachable.head.address must be(node(victim).address) cluster.latestGossip.overview.unreachable.head.address must be(node(victim).address)
@ -108,7 +110,7 @@ abstract class UnreachableNodeRejoinsClusterSpec
cluster down victim cluster down victim
} }
runOn(allBut(victim):_*) { runOn(allBut(victim): _*) {
awaitUpConvergence(roles.size - 1, Seq(victim)) awaitUpConvergence(roles.size - 1, Seq(victim))
} }
@ -118,7 +120,7 @@ abstract class UnreachableNodeRejoinsClusterSpec
"allow node to REJOIN when the network is plugged back in" taggedAs LongRunningTest in { "allow node to REJOIN when the network is plugged back in" taggedAs LongRunningTest in {
runOn(first) { runOn(first) {
// put the network back in // put the network back in
allBut(victim).foreach { roleName => allBut(victim).foreach { roleName
testConductor.passThrough(victim, roleName, Direction.Both).await testConductor.passThrough(victim, roleName, Direction.Both).await
} }
} }

View file

@ -29,8 +29,8 @@ class ClusterConfigSpec extends AkkaSpec {
HeartbeatInterval must be(1 second) HeartbeatInterval must be(1 second)
LeaderActionsInterval must be(1 second) LeaderActionsInterval must be(1 second)
UnreachableNodesReaperInterval must be(1 second) UnreachableNodesReaperInterval must be(1 second)
PublishStateInterval must be(1 second)
JoinTimeout must be(60 seconds) JoinTimeout must be(60 seconds)
NrOfGossipDaemons must be(4)
AutoJoin must be(true) AutoJoin must be(true)
AutoDown must be(false) AutoDown must be(false)
UseDispatcher must be(Dispatchers.DefaultDispatcherId) UseDispatcher must be(Dispatchers.DefaultDispatcherId)

View file

@ -50,6 +50,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
// 3 deputy nodes (addresses index 1, 2, 3) // 3 deputy nodes (addresses index 1, 2, 3)
override def seedNodes = addresses.slice(1, 4) override def seedNodes = addresses.slice(1, 4)
/* FIXME This way of mocking is not possible any more...
override def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] = { override def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] = {
if (addresses.isEmpty) None if (addresses.isEmpty) None
else Some(addresses.toSeq(deterministicRandom.getAndIncrement % addresses.size)) else Some(addresses.toSeq(deterministicRandom.getAndIncrement % addresses.size))
@ -71,24 +72,29 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
else _gossipToDeputyProbablity else _gossipToDeputyProbablity
} }
*/
} }
def memberStatus(address: Address): Option[MemberStatus] = def memberStatus(address: Address): Option[MemberStatus] =
cluster.latestGossip.members.collectFirst { case m if m.address == address m.status } cluster.latestGossip.members.collectFirst { case m if m.address == address m.status }
before { before {
/* FIXME
cluster._gossipToDeputyProbablity = 0.0 cluster._gossipToDeputyProbablity = 0.0
*/
addresses foreach failureDetector.remove addresses foreach failureDetector.remove
deterministicRandom.set(0) deterministicRandom.set(0)
} }
/* FIXME ignored due to actor refactoring, must be done in other way
"A Cluster" must { "A Cluster" must {
"use the address of the remote transport" in { "use the address of the remote transport" ignore {
cluster.selfAddress must be(selfAddress) cluster.selfAddress must be(selfAddress)
} }
"initially become singleton cluster when joining itself and reach convergence" in { "initially become singleton cluster when joining itself and reach convergence" ignore {
cluster.isSingletonCluster must be(false) // auto-join = off cluster.isSingletonCluster must be(false) // auto-join = off
cluster.join(selfAddress) cluster.join(selfAddress)
awaitCond(cluster.isSingletonCluster) awaitCond(cluster.isSingletonCluster)
@ -96,11 +102,13 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress)) cluster.latestGossip.members.map(_.address) must be(Set(selfAddress))
memberStatus(selfAddress) must be(Some(MemberStatus.Joining)) memberStatus(selfAddress) must be(Some(MemberStatus.Joining))
cluster.convergence.isDefined must be(true) cluster.convergence.isDefined must be(true)
/* FIXME
cluster.leaderActions() cluster.leaderActions()
*/
memberStatus(selfAddress) must be(Some(MemberStatus.Up)) memberStatus(selfAddress) must be(Some(MemberStatus.Up))
} }
"accept a joining node" in { "accept a joining node" ignore {
cluster.joining(addresses(1)) cluster.joining(addresses(1))
cluster.latestGossip.members.map(_.address) must be(Set(selfAddress, addresses(1))) cluster.latestGossip.members.map(_.address) must be(Set(selfAddress, addresses(1)))
memberStatus(addresses(1)) must be(Some(MemberStatus.Joining)) memberStatus(addresses(1)) must be(Some(MemberStatus.Joining))
@ -108,7 +116,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
expectMsg(GossipTo(addresses(1))) expectMsg(GossipTo(addresses(1)))
} }
"accept a few more joining nodes" in { "accept a few more joining nodes" ignore {
for (a addresses.drop(2)) { for (a addresses.drop(2)) {
cluster.joining(a) cluster.joining(a)
memberStatus(a) must be(Some(MemberStatus.Joining)) memberStatus(a) must be(Some(MemberStatus.Joining))
@ -117,12 +125,12 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
cluster.latestGossip.members.map(_.address) must be(addresses.toSet) cluster.latestGossip.members.map(_.address) must be(addresses.toSet)
} }
"order members by host and port" in { "order members by host and port" ignore {
// note the importance of using toSeq before map, otherwise it will not preserve the order // note the importance of using toSeq before map, otherwise it will not preserve the order
cluster.latestGossip.members.toSeq.map(_.address) must be(addresses.toSeq) cluster.latestGossip.members.toSeq.map(_.address) must be(addresses.toSeq)
} }
"gossip to random live node" in { "gossip to random live node" ignore {
cluster.gossip() cluster.gossip()
cluster.gossip() cluster.gossip()
cluster.gossip() cluster.gossip()
@ -136,7 +144,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
expectNoMsg(1 second) expectNoMsg(1 second)
} }
"use certain probability for gossiping to deputy node depending on the number of unreachable and live nodes" in { "use certain probability for gossiping to deputy node depending on the number of unreachable and live nodes" ignore {
cluster._gossipToDeputyProbablity = -1.0 // use real impl cluster._gossipToDeputyProbablity = -1.0 // use real impl
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(9, 1, 2)) cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(9, 1, 2))
cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 2)) cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 2))
@ -150,7 +158,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
cluster.gossipToDeputyProbablity(3, 7, 4) must be(1.0 plusOrMinus (0.0001)) cluster.gossipToDeputyProbablity(3, 7, 4) must be(1.0 plusOrMinus (0.0001))
} }
"gossip to duputy node" in { "gossip to duputy node" ignore {
cluster._gossipToDeputyProbablity = 1.0 // always cluster._gossipToDeputyProbablity = 1.0 // always
// we have configured 3 deputy nodes (seedNodes) // we have configured 3 deputy nodes (seedNodes)
@ -170,7 +178,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
} }
"gossip to random deputy node if number of live nodes is less than number of deputy nodes" in { "gossip to random deputy node if number of live nodes is less than number of deputy nodes" ignore {
cluster._gossipToDeputyProbablity = -1.0 // real impl cluster._gossipToDeputyProbablity = -1.0 // real impl
// 0 and 2 still alive // 0 and 2 still alive
val dead = Set(addresses(1), addresses(3), addresses(4), addresses(5)) val dead = Set(addresses(1), addresses(3), addresses(4), addresses(5))
@ -190,4 +198,5 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter {
} }
} }
*/
} }