+cdd #16799 Add Distributed Data module

Previously know as [patriknw/akka-data-replication](https://github.com/patriknw/akka-data-replication),
which was originally inspired by [jboner/akka-crdt](https://github.com/jboner/akka-crdt).

The functionality is very similar to akka-data-replication 0.11.

Here is a list of the most important changes:

* The package name changed to `akka.cluster.ddata`
* The extension was renamed to `DistributedData`
* The keys changed from strings to classes with unique identifiers and type information of the data values,
  e.g. `ORSetKey[Int]("set2")`
* The optional read consistency parameter was removed from the `Update` message. If you need to read from
  other replicas before performing the update you have to first send a `Get` message and then continue with
  the ``Update`` when the ``GetSuccess`` is received.
* `BigInt` is used in `GCounter` and `PNCounter` instead of `Long`
* Improvements of java api
* Better documentation
This commit is contained in:
Patrik Nordwall 2015-05-17 12:28:47 +02:00
parent bf28260cd0
commit cbe5dd2cf5
69 changed files with 40036 additions and 3 deletions

View file

@ -0,0 +1,283 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
import com.typesafe.config.ConfigFactory
object JepsenInspiredInsertSpec extends MultiNodeConfig {
val controller = role("controller")
val n1 = role("n1")
val n2 = role("n2")
val n3 = role("n3")
val n4 = role("n4")
val n5 = role("n5")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters = off
akka.log-dead-letters-during-shutdown = off
akka.remote.log-remote-lifecycle-events = ERROR
akka.testconductor.barrier-timeout = 60 s
"""))
testTransport(on = true)
}
class JepsenInspiredInsertSpecMultiJvmNode1 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode2 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode3 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode4 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode5 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode6 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) with STMultiNodeSpec with ImplicitSender {
import JepsenInspiredInsertSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val replicator = DistributedData(system).replicator
val nodes = roles.drop(1) // controller not part of active nodes
val nodeCount = nodes.size
val timeout = 3.seconds.dilated
val delayMillis = 0
val totalCount = 200
// val delayMillis = 20
// val totalCount = 2000
val expectedData = (0 until totalCount).toSet
val data: Map[RoleName, Seq[Int]] = {
val nodeIndex = nodes.zipWithIndex.map { case (n, i) i -> n }.toMap
(0 until totalCount).groupBy(i nodeIndex(i % nodeCount))
}
lazy val myData: Seq[Int] = data(myself)
def sleepDelay(): Unit =
if (delayMillis != 0) {
val rndDelay = ThreadLocalRandom.current().nextInt(delayMillis)
if (rndDelay != 0) Thread.sleep(delayMillis)
}
def sleepBeforePartition(): Unit = {
if (delayMillis != 0)
Thread.sleep(delayMillis * totalCount / nodeCount / 10)
}
def sleepDuringPartition(): Unit =
Thread.sleep(math.max(5000, delayMillis * totalCount / nodeCount / 2))
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Insert from 5 nodes" must {
"setup cluster" in {
runOn(nodes: _*) {
nodes.foreach { join(_, n1) }
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(nodes.size))
}
}
}
runOn(controller) {
nodes.foreach { n enterBarrier(n.name + "-joined") }
}
enterBarrier("after-setup")
}
}
"replicate values when all nodes connected" in {
val key = ORSetKey[Int]("A")
runOn(nodes: _*) {
val writeProbe = TestProbe()
val writeAcks = myData.map { i
sleepDelay()
replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ + i), writeProbe.ref)
writeProbe.receiveOne(3.seconds)
}
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] success }
val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure[_] fail }
successWriteAcks.map(_.request.get).toSet should be(myData.toSet)
successWriteAcks.size should be(myData.size)
failureWriteAcks should be(Nil)
(successWriteAcks.size + failureWriteAcks.size) should be(myData.size)
// eventually all nodes will have the data
within(15.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.elements should be(expectedData)
}
}
}
enterBarrier("after-test-1")
}
"write/read to majority when all nodes connected" in {
val key = ORSetKey[Int]("B")
val readMajority = ReadMajority(timeout)
val writeMajority = WriteMajority(timeout)
runOn(nodes: _*) {
val writeProbe = TestProbe()
val writeAcks = myData.map { i
sleepDelay()
replicator.tell(Update(key, ORSet(), writeMajority, Some(i))(_ + i), writeProbe.ref)
writeProbe.receiveOne(timeout + 1.second)
}
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] success }
val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure[_] fail }
successWriteAcks.map(_.request.get).toSet should be(myData.toSet)
successWriteAcks.size should be(myData.size)
failureWriteAcks should be(Nil)
(successWriteAcks.size + failureWriteAcks.size) should be(myData.size)
enterBarrier("data-written-2")
// read from majority of nodes, which is enough to retrieve all data
val readProbe = TestProbe()
replicator.tell(Get(key, readMajority), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
val survivors = result.elements.size
result.elements should be(expectedData)
}
runOn(controller) {
enterBarrier("data-written-2")
}
enterBarrier("after-test-2")
}
"replicate values after partition" in {
val key = ORSetKey[Int]("C")
runOn(controller) {
sleepBeforePartition()
for (a List(n1, n4, n5); b List(n2, n3))
testConductor.blackhole(a, b, Direction.Both).await
sleepDuringPartition()
for (a List(n1, n4, n5); b List(n2, n3))
testConductor.passThrough(a, b, Direction.Both).await
enterBarrier("partition-healed-3")
}
runOn(nodes: _*) {
val writeProbe = TestProbe()
val writeAcks = myData.map { i
sleepDelay()
replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ + i), writeProbe.ref)
writeProbe.receiveOne(3.seconds)
}
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] success }
val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure[_] fail }
successWriteAcks.map(_.request.get).toSet should be(myData.toSet)
successWriteAcks.size should be(myData.size)
failureWriteAcks should be(Nil)
(successWriteAcks.size + failureWriteAcks.size) should be(myData.size)
enterBarrier("partition-healed-3")
// eventually all nodes will have the data
within(15.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.elements should be(expectedData)
}
}
}
enterBarrier("after-test-3")
}
"write to majority during 3+2 partition and read from majority after partition" in {
val key = ORSetKey[Int]("D")
val readMajority = ReadMajority(timeout)
val writeMajority = WriteMajority(timeout)
runOn(controller) {
sleepBeforePartition()
for (a List(n1, n4, n5); b List(n2, n3))
testConductor.blackhole(a, b, Direction.Both).await
sleepDuringPartition()
for (a List(n1, n4, n5); b List(n2, n3))
testConductor.passThrough(a, b, Direction.Both).await
enterBarrier("partition-healed-4")
}
runOn(nodes: _*) {
val writeProbe = TestProbe()
val writeAcks = myData.map { i
sleepDelay()
replicator.tell(Update(key, ORSet(), writeMajority, Some(i))(_ + i), writeProbe.ref)
writeProbe.receiveOne(timeout + 1.second)
}
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] success }
val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure[_] fail }
runOn(n1, n4, n5) {
successWriteAcks.map(_.request.get).toSet should be(myData.toSet)
successWriteAcks.size should be(myData.size)
failureWriteAcks should be(Nil)
}
runOn(n2, n3) {
// without delays all could teoretically have been written before the blackhole
if (delayMillis != 0)
failureWriteAcks should not be (Nil)
}
(successWriteAcks.size + failureWriteAcks.size) should be(myData.size)
enterBarrier("partition-healed-4")
// on the 2 node side, read from majority of nodes is enough to read all writes
runOn(n2, n3) {
val readProbe = TestProbe()
replicator.tell(Get(key, readMajority), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
val survivors = result.elements.size
result.elements should be(expectedData)
}
// but on the 3 node side, read from majority doesn't mean that we are guaranteed to see
// the writes from the other side, yet
// eventually all nodes will have the data
within(15.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.elements should be(expectedData)
}
}
}
enterBarrier("after-test-4")
}
}

View file

@ -0,0 +1,233 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Deploy
import akka.actor.Props
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
object PerformanceSpec extends MultiNodeConfig {
val n1 = role("n1")
val n2 = role("n2")
val n3 = role("n3")
val n4 = role("n4")
val n5 = role("n5")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = ERROR
akka.stdout-loglevel = ERROR
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters = off
akka.log-dead-letters-during-shutdown = off
akka.remote.log-remote-lifecycle-events = ERROR
akka.remote.log-frame-size-exceeding=1000b
akka.testconductor.barrier-timeout = 60 s
akka.cluster.distributed-data.gossip-interval = 1 s
akka.actor.serialize-messages = off
"""))
def countDownProps(latch: TestLatch): Props = Props(new CountDown(latch)).withDeploy(Deploy.local)
class CountDown(latch: TestLatch) extends Actor {
def receive = {
case _
latch.countDown()
if (latch.isOpen)
context.stop(self)
}
}
}
class PerformanceSpecMultiJvmNode1 extends PerformanceSpec
class PerformanceSpecMultiJvmNode2 extends PerformanceSpec
class PerformanceSpecMultiJvmNode3 extends PerformanceSpec
class PerformanceSpecMultiJvmNode4 extends PerformanceSpec
class PerformanceSpecMultiJvmNode5 extends PerformanceSpec
class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpec with ImplicitSender {
import PerformanceSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val replicator = DistributedData(system).replicator
val timeout = 3.seconds.dilated
val factor = 1 // use 3 here for serious tuning
val repeatCount = 3 // use at least 10 here for serious tuning
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
def repeat(description: String, keys: Iterable[ORSetKey[Int]], n: Int, expectedAfterReplication: Option[Set[Int]] = None)(
block: (ORSetKey[Int], Int, ActorRef) Unit, afterEachKey: ORSetKey[Int] Unit = _ ()): Unit = {
keys.foreach { key
val startTime = System.nanoTime()
runOn(n1) {
val latch = TestLatch(n)
val replyTo = system.actorOf(countDownProps(latch))
var i = 0
while (i < n) {
block(key, i, replyTo)
i += 1
}
Await.ready(latch, 5.seconds + (1.second * factor))
}
expectedAfterReplication.foreach { expected
enterBarrier("repeat-" + key + "-before-awaitReplicated")
awaitReplicated(key, expected)
enterBarrier("repeat-" + key + "-after-awaitReplicated")
}
runOn(n1) {
val endTime = System.nanoTime()
val durationMs = (endTime - startTime).nanos.toMillis
val tps = (n * 1000.0 / durationMs).toInt
println(s"## $n $description took $durationMs ms, $tps TPS")
}
afterEachKey(key)
enterBarrier("repeat-" + key + "-done")
}
}
def awaitReplicated(keys: Iterable[ORSetKey[Int]], expectedData: Set[Int]): Unit =
keys.foreach { key awaitReplicated(key, expectedData) }
def awaitReplicated(key: ORSetKey[Int], expectedData: Set[Int]): Unit = {
within(20.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.elements should be(expectedData)
}
}
}
"Performance" must {
"setup cluster" in {
roles.foreach { join(_, n1) }
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
}
enterBarrier("after-setup")
}
"be great for ORSet Update WriteLocal" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("A" + n))
val n = 1000 * factor
val expectedData = (0 until n).toSet
repeat("ORSet Update WriteLocal", keys, n)({ (key, i, replyTo)
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + i), replyTo)
}, key awaitReplicated(key, expectedData))
enterBarrier("after-1")
}
"be blazingly fast for ORSet Get ReadLocal" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("A" + n))
repeat("Get ReadLocal", keys, 100000 * factor) { (key, i, replyTo)
replicator.tell(Get(key, ReadLocal), replyTo)
}
enterBarrier("after-2")
}
"be good for ORSet Update WriteLocal and gossip replication" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("B" + n))
val n = 200 * factor
val expected = Some((0 until n).toSet)
repeat("ORSet Update WriteLocal + gossip", keys, n, expected) { (key, i, replyTo)
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + i), replyTo)
}
enterBarrier("after-3")
}
"be good for ORSet Update WriteLocal and gossip of existing keys" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("B" + n))
val n = 200 * factor
val expected = Some((0 until n).toSet ++ (0 until n).map(-_).toSet)
repeat("ORSet Update WriteLocal existing + gossip", keys, n, expected) { (key, i, replyTo)
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + (-i)), replyTo)
}
enterBarrier("after-4")
}
"be good for ORSet Update WriteTwo and gossip replication" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("C" + n))
val n = 200 * factor
val expected = Some((0 until n).toSet)
val writeTwo = WriteTo(2, timeout)
repeat("ORSet Update WriteTwo + gossip", keys, n, expected) { (key, i, replyTo)
replicator.tell(Update(key, ORSet(), writeTwo)(_ + i), replyTo)
}
enterBarrier("after-5")
}
"be awesome for GCounter Update WriteLocal" in {
val startTime = System.nanoTime()
val n = 1000 * factor
val key = GCounterKey("D")
runOn(n1, n2, n3) {
val latch = TestLatch(n)
val replyTo = system.actorOf(countDownProps(latch))
for (_ 0 until n)
replicator.tell(Update(key, GCounter(), WriteLocal)(_ + 1), replyTo)
Await.ready(latch, 5.seconds + (1.second * factor))
enterBarrier("update-done-6")
runOn(n1) {
val endTime = System.nanoTime()
val durationMs = (endTime - startTime).nanos.toMillis
val tps = (3 * n * 1000.0 / durationMs).toInt
println(s"## ${3 * n} GCounter Update took $durationMs ms, $tps TPS")
}
}
runOn(n4, n5) {
enterBarrier("update-done-6")
}
within(20.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.value should be(3 * n)
}
}
enterBarrier("replication-done-6")
runOn(n1) {
val endTime = System.nanoTime()
val durationMs = (endTime - startTime).nanos.toMillis
val tps = (n * 1000.0 / durationMs).toInt
println(s"## $n GCounter Update + gossip took $durationMs ms, $tps TPS")
}
enterBarrier("after-6")
}
}
}

View file

@ -0,0 +1,234 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.duration._
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
import com.typesafe.config.ConfigFactory
object ReplicatorChaosSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
val fourth = role("fourth")
val fifth = role("fifth")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.cluster.roles = ["backend"]
akka.log-dead-letters-during-shutdown = off
"""))
testTransport(on = true)
}
class ReplicatorChaosSpecMultiJvmNode1 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode2 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode3 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode4 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode5 extends ReplicatorChaosSpec
class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatorChaosSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val replicator = system.actorOf(Replicator.props(
ReplicatorSettings(system).withRole("backend").withGossipInterval(1.second)), "replicator")
val timeout = 3.seconds.dilated
val KeyA = GCounterKey("A")
val KeyB = PNCounterKey("B")
val KeyC = GCounterKey("C")
val KeyD = GCounterKey("D")
val KeyE = GSetKey[String]("E")
val KeyF = ORSetKey[String]("F")
val KeyX = GCounterKey("X")
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
def assertValue(key: Key[ReplicatedData], expected: Any): Unit =
within(10.seconds) {
awaitAssert {
replicator ! Get(key, ReadLocal)
val value = expectMsgPF() {
case g @ GetSuccess(`key`, _) g.dataValue match {
case c: GCounter c.value
case c: PNCounter c.value
case c: GSet[_] c.elements
case c: ORSet[_] c.elements
}
}
value should be(expected)
}
}
def assertDeleted(key: Key[ReplicatedData]): Unit =
within(5.seconds) {
awaitAssert {
replicator ! Get(key, ReadLocal)
expectMsg(DataDeleted(key))
}
}
"Replicator in chaotic cluster" must {
"replicate data in initial phase" in {
join(first, first)
join(second, first)
join(third, first)
join(fourth, first)
join(fifth, first)
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(5))
}
}
runOn(first) {
(0 until 5).foreach { i
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
replicator ! Update(KeyB, PNCounter(), WriteLocal)(_ - 1)
replicator ! Update(KeyC, GCounter(), WriteAll(timeout))(_ + 1)
}
receiveN(15).map(_.getClass).toSet should be(Set(classOf[UpdateSuccess[_]]))
}
runOn(second) {
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 20)
replicator ! Update(KeyB, PNCounter(), WriteTo(2, timeout))(_ + 20)
replicator ! Update(KeyC, GCounter(), WriteAll(timeout))(_ + 20)
receiveN(3).toSet should be(Set(UpdateSuccess(KeyA, None),
UpdateSuccess(KeyB, None), UpdateSuccess(KeyC, None)))
replicator ! Update(KeyE, GSet(), WriteLocal)(_ + "e1" + "e2")
expectMsg(UpdateSuccess(KeyE, None))
replicator ! Update(KeyF, ORSet(), WriteLocal)(_ + "e1" + "e2")
expectMsg(UpdateSuccess(KeyF, None))
}
runOn(fourth) {
replicator ! Update(KeyD, GCounter(), WriteLocal)(_ + 40)
expectMsg(UpdateSuccess(KeyD, None))
replicator ! Update(KeyE, GSet(), WriteLocal)(_ + "e2" + "e3")
expectMsg(UpdateSuccess(KeyE, None))
replicator ! Update(KeyF, ORSet(), WriteLocal)(_ + "e2" + "e3")
expectMsg(UpdateSuccess(KeyF, None))
}
runOn(fifth) {
replicator ! Update(KeyX, GCounter(), WriteTo(2, timeout))(_ + 50)
expectMsg(UpdateSuccess(KeyX, None))
replicator ! Delete(KeyX, WriteLocal)
expectMsg(DeleteSuccess(KeyX))
}
enterBarrier("initial-updates-done")
assertValue(KeyA, 25)
assertValue(KeyB, 15)
assertValue(KeyC, 25)
assertValue(KeyD, 40)
assertValue(KeyE, Set("e1", "e2", "e3"))
assertValue(KeyF, Set("e1", "e2", "e3"))
assertDeleted(KeyX)
enterBarrier("after-1")
}
"be available during network split" in {
val side1 = Seq(first, second)
val side2 = Seq(third, fourth, fifth)
runOn(first) {
for (a side1; b side2)
testConductor.blackhole(a, b, Direction.Both).await
}
enterBarrier("split")
runOn(first) {
replicator ! Update(KeyA, GCounter(), WriteTo(2, timeout))(_ + 1)
expectMsg(UpdateSuccess(KeyA, None))
}
runOn(third) {
replicator ! Update(KeyA, GCounter(), WriteTo(2, timeout))(_ + 2)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Update(KeyE, GSet(), WriteTo(2, timeout))(_ + "e4")
expectMsg(UpdateSuccess(KeyE, None))
replicator ! Update(KeyF, ORSet(), WriteTo(2, timeout))(_ - "e2")
expectMsg(UpdateSuccess(KeyF, None))
}
runOn(fourth) {
replicator ! Update(KeyD, GCounter(), WriteTo(2, timeout))(_ + 1)
expectMsg(UpdateSuccess(KeyD, None))
}
enterBarrier("update-during-split")
runOn(side1: _*) {
assertValue(KeyA, 26)
assertValue(KeyB, 15)
assertValue(KeyD, 40)
assertValue(KeyE, Set("e1", "e2", "e3"))
assertValue(KeyF, Set("e1", "e2", "e3"))
}
runOn(side2: _*) {
assertValue(KeyA, 27)
assertValue(KeyB, 15)
assertValue(KeyD, 41)
assertValue(KeyE, Set("e1", "e2", "e3", "e4"))
assertValue(KeyF, Set("e1", "e3"))
}
enterBarrier("update-during-split-verified")
runOn(first) {
testConductor.exit(fourth, 0).await
}
enterBarrier("after-2")
}
"converge after partition" in {
val side1 = Seq(first, second)
val side2 = Seq(third, fifth) // fourth was shutdown
runOn(first) {
for (a side1; b side2)
testConductor.passThrough(a, b, Direction.Both).await
}
enterBarrier("split-repaired")
assertValue(KeyA, 28)
assertValue(KeyB, 15)
assertValue(KeyC, 25)
assertValue(KeyD, 41)
assertValue(KeyE, Set("e1", "e2", "e3", "e4"))
assertValue(KeyF, Set("e1", "e3"))
assertDeleted(KeyX)
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,197 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.duration._
import akka.cluster.Cluster
import akka.cluster.ClusterEvent.InitialStateAsEvents
import akka.cluster.ClusterEvent.MemberUp
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
object ReplicatorPruningSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
class ReplicatorPruningSpecMultiJvmNode1 extends ReplicatorPruningSpec
class ReplicatorPruningSpecMultiJvmNode2 extends ReplicatorPruningSpec
class ReplicatorPruningSpecMultiJvmNode3 extends ReplicatorPruningSpec
class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatorPruningSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val maxPruningDissemination = 3.seconds
val replicator = system.actorOf(Replicator.props(
ReplicatorSettings(system).withGossipInterval(1.second)
.withPruning(pruningInterval = 1.second, maxPruningDissemination)), "replicator")
val timeout = 2.seconds.dilated
val KeyA = GCounterKey("A")
val KeyB = ORSetKey[String]("B")
val KeyC = PNCounterMapKey("C")
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Pruning of CRDT" must {
"move data from removed node" in {
join(first, first)
join(second, first)
join(third, first)
within(5.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(3))
}
}
// we need the UniqueAddress
val memberProbe = TestProbe()
cluster.subscribe(memberProbe.ref, initialStateMode = InitialStateAsEvents, classOf[MemberUp])
val thirdUniqueAddress = {
val member = memberProbe.fishForMessage(3.seconds) {
case MemberUp(m) if m.address == node(third).address true
case _ false
}.asInstanceOf[MemberUp].member
member.uniqueAddress
}
replicator ! Update(KeyA, GCounter(), WriteAll(timeout))(_ + 3)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Update(KeyB, ORSet(), WriteAll(timeout))(_ + "a" + "b" + "c")
expectMsg(UpdateSuccess(KeyB, None))
replicator ! Update(KeyC, PNCounterMap(), WriteAll(timeout))(_ increment "x" increment "y")
expectMsg(UpdateSuccess(KeyC, None))
enterBarrier("updates-done")
replicator ! Get(KeyA, ReadLocal)
val oldCounter = expectMsgType[GetSuccess[GCounter]].dataValue
oldCounter.value should be(9)
replicator ! Get(KeyB, ReadLocal)
val oldSet = expectMsgType[GetSuccess[ORSet[String]]].dataValue
oldSet.elements should be(Set("a", "b", "c"))
replicator ! Get(KeyC, ReadLocal)
val oldMap = expectMsgType[GetSuccess[PNCounterMap]].dataValue
oldMap.get("x") should be(Some(3))
oldMap.get("y") should be(Some(3))
enterBarrier("get-old")
runOn(first) {
cluster.leave(node(third).address)
}
runOn(first, second) {
within(15.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(2))
}
}
}
enterBarrier("third-removed")
runOn(first, second) {
within(15.seconds) {
awaitAssert {
replicator ! Get(KeyA, ReadLocal)
expectMsgPF() {
case g @ GetSuccess(KeyA, _)
g.get(KeyA).value should be(9)
g.get(KeyA).needPruningFrom(thirdUniqueAddress) should be(false)
}
}
}
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyB, ReadLocal)
expectMsgPF() {
case g @ GetSuccess(KeyB, _)
g.get(KeyB).elements should be(Set("a", "b", "c"))
g.get(KeyB).needPruningFrom(thirdUniqueAddress) should be(false)
}
}
}
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyC, ReadLocal)
expectMsgPF() {
case g @ GetSuccess(KeyC, _)
g.get(KeyC).entries should be(Map("x" -> 3L, "y" -> 3L))
g.get(KeyC).needPruningFrom(thirdUniqueAddress) should be(false)
}
}
}
}
enterBarrier("pruning-done")
// on one of the nodes the data has been updated by the pruning,
// client can update anyway
def updateAfterPruning(expectedValue: Int): Unit = {
replicator ! Update(KeyA, GCounter(), WriteAll(timeout), None)(_ + 1)
expectMsgPF() {
case UpdateSuccess(KeyA, _)
replicator ! Get(KeyA, ReadLocal)
val retrieved = expectMsgType[GetSuccess[GCounter]].dataValue
retrieved.value should be(expectedValue)
}
}
runOn(first) {
updateAfterPruning(expectedValue = 10)
}
enterBarrier("update-first-after-pruning")
runOn(second) {
updateAfterPruning(expectedValue = 11)
}
enterBarrier("update-second-after-pruning")
// after pruning performed and maxDissemination it is tombstoned
// and we should still not be able to update with data from removed node
expectNoMsg(maxPruningDissemination + 3.seconds)
runOn(first) {
updateAfterPruning(expectedValue = 12)
}
enterBarrier("update-first-after-tombstone")
runOn(second) {
updateAfterPruning(expectedValue = 13)
}
enterBarrier("update-second-after-tombstone")
enterBarrier("after-1")
}
}
}

View file

@ -0,0 +1,503 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.duration._
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
import com.typesafe.config.ConfigFactory
object ReplicatorSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
testTransport(on = true)
}
class ReplicatorSpecMultiJvmNode1 extends ReplicatorSpec
class ReplicatorSpecMultiJvmNode2 extends ReplicatorSpec
class ReplicatorSpecMultiJvmNode3 extends ReplicatorSpec
class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatorSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val replicator = system.actorOf(Replicator.props(
ReplicatorSettings(system).withGossipInterval(1.second).withMaxDeltaElements(10)), "replicator")
val timeout = 2.seconds.dilated
val writeTwo = WriteTo(2, timeout)
val writeMajority = WriteMajority(timeout)
val writeAll = WriteAll(timeout)
val readTwo = ReadFrom(2, timeout)
val readAll = ReadAll(timeout)
val readMajority = ReadMajority(timeout)
val KeyA = GCounterKey("A")
val KeyB = GCounterKey("B")
val KeyC = GCounterKey("C")
val KeyD = GCounterKey("D")
val KeyE = GCounterKey("E")
val KeyE2 = GCounterKey("E2")
val KeyF = GCounterKey("F")
val KeyG = ORSetKey[String]("G")
val KeyX = GCounterKey("X")
val KeyY = GCounterKey("Y")
val KeyZ = GCounterKey("Z")
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Cluster CRDT" must {
"work in single node cluster" in {
join(first, first)
runOn(first) {
within(5.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(1))
}
}
val changedProbe = TestProbe()
replicator ! Subscribe(KeyA, changedProbe.ref)
replicator ! Subscribe(KeyX, changedProbe.ref)
replicator ! Get(KeyA, ReadLocal)
expectMsg(NotFound(KeyA, None))
val c3 = GCounter() + 3
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 3)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Get(KeyA, ReadLocal)
expectMsg(GetSuccess(KeyA, None)(c3)).dataValue should be(c3)
changedProbe.expectMsg(Changed(KeyA)(c3)).dataValue should be(c3)
val changedProbe2 = TestProbe()
replicator ! Subscribe(KeyA, changedProbe2.ref)
changedProbe2.expectMsg(Changed(KeyA)(c3)).dataValue should be(c3)
val c4 = c3 + 1
// too strong consistency level
replicator ! Update(KeyA, GCounter(), writeTwo)(_ + 1)
expectMsg(UpdateTimeout(KeyA, None))
replicator ! Get(KeyA, ReadLocal)
expectMsg(GetSuccess(KeyA, None)(c4)).dataValue should be(c4)
changedProbe.expectMsg(Changed(KeyA)(c4)).dataValue should be(c4)
val c5 = c4 + 1
// too strong consistency level
replicator ! Update(KeyA, GCounter(), writeMajority)(_ + 1)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Get(KeyA, readMajority)
expectMsg(GetSuccess(KeyA, None)(c5)).dataValue should be(c5)
changedProbe.expectMsg(Changed(KeyA)(c5)).dataValue should be(c5)
val c6 = c5 + 1
replicator ! Update(KeyA, GCounter(), writeAll)(_ + 1)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Get(KeyA, readAll)
expectMsg(GetSuccess(KeyA, None)(c6)).dataValue should be(c6)
changedProbe.expectMsg(Changed(KeyA)(c6)).dataValue should be(c6)
val c9 = GCounter() + 9
replicator ! Update(KeyX, GCounter(), WriteLocal)(_ + 9)
expectMsg(UpdateSuccess(KeyX, None))
changedProbe.expectMsg(Changed(KeyX)(c9)).dataValue should be(c9)
replicator ! Delete(KeyX, WriteLocal)
expectMsg(DeleteSuccess(KeyX))
changedProbe.expectMsg(DataDeleted(KeyX))
replicator ! Get(KeyX, ReadLocal)
expectMsg(DataDeleted(KeyX))
replicator ! Get(KeyX, readAll)
expectMsg(DataDeleted(KeyX))
replicator ! Update(KeyX, GCounter(), WriteLocal)(_ + 1)
expectMsg(DataDeleted(KeyX))
replicator ! Delete(KeyX, WriteLocal)
expectMsg(DataDeleted(KeyX))
replicator ! GetKeyIds
expectMsg(GetKeyIdsResult(Set("A")))
}
enterBarrier("after-1")
}
}
"reply with ModifyFailure if exception is thrown by modify function" in {
val e = new RuntimeException("errr")
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ throw e)
expectMsgType[ModifyFailure[_]].cause should be(e)
}
"replicate values to new node" in {
join(second, first)
runOn(first, second) {
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(2))
}
}
}
enterBarrier("2-nodes")
runOn(second) {
val changedProbe = TestProbe()
replicator ! Subscribe(KeyA, changedProbe.ref)
// "A" should be replicated via gossip to the new node
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyA, ReadLocal)
val c = expectMsgPF() { case g @ GetSuccess(KeyA, _) g.get(KeyA) }
c.value should be(6)
}
}
val c = changedProbe.expectMsgPF() { case c @ Changed(KeyA) c.get(KeyA) }
c.value should be(6)
}
enterBarrier("after-2")
}
"work in 2 node cluster" in {
runOn(first, second) {
// start with 20 on both nodes
replicator ! Update(KeyB, GCounter(), WriteLocal)(_ + 20)
expectMsg(UpdateSuccess(KeyB, None))
// add 1 on both nodes using WriteTwo
replicator ! Update(KeyB, GCounter(), writeTwo)(_ + 1)
expectMsg(UpdateSuccess(KeyB, None))
// the total, after replication should be 42
awaitAssert {
replicator ! Get(KeyB, readTwo)
val c = expectMsgPF() { case g @ GetSuccess(KeyB, _) g.get(KeyB) }
c.value should be(42)
}
}
enterBarrier("update-42")
runOn(first, second) {
// add 1 on both nodes using WriteAll
replicator ! Update(KeyB, GCounter(), writeAll)(_ + 1)
expectMsg(UpdateSuccess(KeyB, None))
// the total, after replication should be 44
awaitAssert {
replicator ! Get(KeyB, readAll)
val c = expectMsgPF() { case g @ GetSuccess(KeyB, _) g.get(KeyB) }
c.value should be(44)
}
}
enterBarrier("update-44")
runOn(first, second) {
// add 1 on both nodes using WriteMajority
replicator ! Update(KeyB, GCounter(), writeMajority)(_ + 1)
expectMsg(UpdateSuccess(KeyB, None))
// the total, after replication should be 46
awaitAssert {
replicator ! Get(KeyB, readMajority)
val c = expectMsgPF() { case g @ GetSuccess(KeyB, _) g.get(KeyB) }
c.value should be(46)
}
}
enterBarrier("after-3")
}
"be replicated after succesful update" in {
val changedProbe = TestProbe()
runOn(first, second) {
replicator ! Subscribe(KeyC, changedProbe.ref)
}
runOn(first) {
replicator ! Update(KeyC, GCounter(), writeTwo)(_ + 30)
expectMsg(UpdateSuccess(KeyC, None))
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(30)
replicator ! Update(KeyY, GCounter(), writeTwo)(_ + 30)
expectMsg(UpdateSuccess(KeyY, None))
replicator ! Update(KeyZ, GCounter(), writeMajority)(_ + 30)
expectMsg(UpdateSuccess(KeyZ, None))
}
enterBarrier("update-c30")
runOn(second) {
replicator ! Get(KeyC, ReadLocal)
val c30 = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c30.value should be(30)
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(30)
// replicate with gossip after WriteLocal
replicator ! Update(KeyC, GCounter(), WriteLocal)(_ + 1)
expectMsg(UpdateSuccess(KeyC, None))
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(31)
replicator ! Delete(KeyY, WriteLocal)
expectMsg(DeleteSuccess(KeyY))
replicator ! Get(KeyZ, readMajority)
expectMsgPF() { case g @ GetSuccess(KeyZ, _) g.get(KeyZ).value } should be(30)
}
enterBarrier("update-c31")
runOn(first) {
// KeyC and deleted KeyY should be replicated via gossip to the other node
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyC, ReadLocal)
val c = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c.value should be(31)
replicator ! Get(KeyY, ReadLocal)
expectMsg(DataDeleted(KeyY))
}
}
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(31)
}
enterBarrier("verified-c31")
// and also for concurrent updates
runOn(first, second) {
replicator ! Get(KeyC, ReadLocal)
val c31 = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c31.value should be(31)
val c32 = c31 + 1
replicator ! Update(KeyC, GCounter(), WriteLocal)(_ + 1)
expectMsg(UpdateSuccess(KeyC, None))
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyC, ReadLocal)
val c = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c.value should be(33)
}
}
}
enterBarrier("after-4")
}
"converge after partition" in {
runOn(first) {
replicator ! Update(KeyD, GCounter(), writeTwo)(_ + 40)
expectMsg(UpdateSuccess(KeyD, None))
testConductor.blackhole(first, second, Direction.Both).await
}
enterBarrier("blackhole-first-second")
runOn(first, second) {
replicator ! Get(KeyD, ReadLocal)
val c40 = expectMsgPF() { case g @ GetSuccess(KeyD, _) g.get(KeyD) }
c40.value should be(40)
replicator ! Update(KeyD, GCounter() + 1, writeTwo)(_ + 1)
expectMsg(UpdateTimeout(KeyD, None))
replicator ! Update(KeyD, GCounter(), writeTwo)(_ + 1)
expectMsg(UpdateTimeout(KeyD, None))
}
runOn(first) {
for (n 1 to 30) {
val KeyDn = GCounterKey("D" + n)
replicator ! Update(KeyDn, GCounter(), WriteLocal)(_ + n)
expectMsg(UpdateSuccess(KeyDn, None))
}
}
enterBarrier("updates-during-partion")
runOn(first) {
testConductor.passThrough(first, second, Direction.Both).await
}
enterBarrier("passThrough-first-second")
runOn(first, second) {
replicator ! Get(KeyD, readTwo)
val c44 = expectMsgPF() { case g @ GetSuccess(KeyD, _) g.get(KeyD) }
c44.value should be(44)
within(10.seconds) {
awaitAssert {
for (n 1 to 30) {
val KeyDn = GCounterKey("D" + n)
replicator ! Get(KeyDn, ReadLocal)
expectMsgPF() { case g @ GetSuccess(KeyDn, _) g.get(KeyDn) }.value should be(n)
}
}
}
}
enterBarrier("after-5")
}
"support majority quorum write and read with 3 nodes with 1 unreachable" in {
join(third, first)
runOn(first, second, third) {
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(3))
}
}
}
enterBarrier("3-nodes")
runOn(first, second, third) {
replicator ! Update(KeyE, GCounter(), writeMajority)(_ + 50)
expectMsg(UpdateSuccess(KeyE, None))
}
enterBarrier("write-inital-majority")
runOn(first, second, third) {
replicator ! Get(KeyE, readMajority)
val c150 = expectMsgPF() { case g @ GetSuccess(KeyE, _) g.get(KeyE) }
c150.value should be(150)
}
enterBarrier("read-inital-majority")
runOn(first) {
testConductor.blackhole(first, third, Direction.Both).await
testConductor.blackhole(second, third, Direction.Both).await
}
enterBarrier("blackhole-third")
runOn(second) {
replicator ! Update(KeyE, GCounter(), WriteLocal)(_ + 1)
expectMsg(UpdateSuccess(KeyE, None))
}
enterBarrier("local-update-from-second")
runOn(first) {
// ReadMajority should retrive the previous update from second, before applying the modification
val probe1 = TestProbe()
val probe2 = TestProbe()
replicator.tell(Get(KeyE, readMajority), probe2.ref)
probe2.expectMsgType[GetSuccess[_]]
replicator.tell(Update(KeyE, GCounter(), writeMajority, None) { data
probe1.ref ! data.value
data + 1
}, probe2.ref)
// verify read your own writes, without waiting for the UpdateSuccess reply
// note that the order of the replies are not defined, and therefore we use separate probes
val probe3 = TestProbe()
replicator.tell(Get(KeyE, readMajority), probe3.ref)
probe1.expectMsg(151)
probe2.expectMsg(UpdateSuccess(KeyE, None))
val c152 = probe3.expectMsgPF() { case g @ GetSuccess(KeyE, _) g.get(KeyE) }
c152.value should be(152)
}
enterBarrier("majority-update-from-first")
runOn(second) {
val probe1 = TestProbe()
replicator.tell(Get(KeyE, readMajority), probe1.ref)
probe1.expectMsgType[GetSuccess[_]]
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(153))(_ + 1), probe1.ref)
// verify read your own writes, without waiting for the UpdateSuccess reply
// note that the order of the replies are not defined, and therefore we use separate probes
val probe2 = TestProbe()
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(154))(_ + 1), probe2.ref)
val probe3 = TestProbe()
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(155))(_ + 1), probe3.ref)
val probe5 = TestProbe()
replicator.tell(Get(KeyE, readMajority), probe5.ref)
probe1.expectMsg(UpdateSuccess(KeyE, Some(153)))
probe2.expectMsg(UpdateSuccess(KeyE, Some(154)))
probe3.expectMsg(UpdateSuccess(KeyE, Some(155)))
val c155 = probe5.expectMsgPF() { case g @ GetSuccess(KeyE, _) g.get(KeyE) }
c155.value should be(155)
}
enterBarrier("majority-update-from-second")
runOn(first, second) {
replicator ! Get(KeyE2, readAll, Some(998))
expectMsg(GetFailure(KeyE2, Some(998)))
replicator ! Get(KeyE2, ReadLocal)
expectMsg(NotFound(KeyE2, None))
}
enterBarrier("read-all-fail-update")
runOn(first) {
testConductor.passThrough(first, third, Direction.Both).await
testConductor.passThrough(second, third, Direction.Both).await
}
enterBarrier("passThrough-third")
runOn(third) {
replicator ! Get(KeyE, readMajority)
val c155 = expectMsgPF() { case g @ GetSuccess(KeyE, _) g.get(KeyE) }
c155.value should be(155)
}
enterBarrier("after-6")
}
"converge after many concurrent updates" in within(10.seconds) {
runOn(first, second, third) {
var c = GCounter()
for (i 0 until 100) {
c += 1
replicator ! Update(KeyF, GCounter(), writeTwo)(_ + 1)
}
val results = receiveN(100)
results.map(_.getClass).toSet should be(Set(classOf[UpdateSuccess[_]]))
}
enterBarrier("100-updates-done")
runOn(first, second, third) {
replicator ! Get(KeyF, readTwo)
val c = expectMsgPF() { case g @ GetSuccess(KeyF, _) g.get(KeyF) }
c.value should be(3 * 100)
}
enterBarrier("after-7")
}
"read-repair happens before GetSuccess" in {
runOn(first) {
replicator ! Update(KeyG, ORSet(), writeTwo)(_ + "a" + "b")
expectMsgType[UpdateSuccess[_]]
}
enterBarrier("a-b-added-to-G")
runOn(second) {
replicator ! Get(KeyG, readAll)
expectMsgPF() { case g @ GetSuccess(KeyG, _) g.get(KeyG).elements } should be(Set("a", "b"))
replicator ! Get(KeyG, ReadLocal)
expectMsgPF() { case g @ GetSuccess(KeyG, _) g.get(KeyG).elements } should be(Set("a", "b"))
}
enterBarrier("after-8")
}
}

View file

@ -0,0 +1,20 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.remote.testkit.MultiNodeSpecCallbacks
import org.scalatest.{ BeforeAndAfterAll, WordSpecLike }
import org.scalatest.Matchers
/**
* Hooks up MultiNodeSpec with ScalaTest
*/
trait STMultiNodeSpec extends MultiNodeSpecCallbacks
with WordSpecLike with Matchers with BeforeAndAfterAll {
override def beforeAll() = multiNodeSpecBeforeAll()
override def afterAll() = multiNodeSpecAfterAll()
}