+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()
}

View file

@ -0,0 +1,187 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.LWWMap
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.LWWMapKey
object ReplicatedCacheSpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
object ReplicatedCache {
import akka.cluster.ddata.Replicator._
def props: Props = Props[ReplicatedCache]
private final case class Request(key: String, replyTo: ActorRef)
final case class PutInCache(key: String, value: Any)
final case class GetFromCache(key: String)
final case class Cached(key: String, value: Option[Any])
final case class Evict(key: String)
}
class ReplicatedCache() extends Actor {
import akka.cluster.ddata.Replicator._
import ReplicatedCache._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
def dataKey(entryKey: String): LWWMapKey[Any] =
LWWMapKey("cache-" + math.abs(entryKey.hashCode) % 100)
def receive = {
case PutInCache(key, value)
replicator ! Update(dataKey(key), LWWMap(), WriteLocal)(_ + (key -> value))
case Evict(key)
replicator ! Update(dataKey(key), LWWMap(), WriteLocal)(_ - key)
case GetFromCache(key)
replicator ! Get(dataKey(key), ReadLocal, Some(Request(key, sender())))
case g @ GetSuccess(LWWMapKey(_), Some(Request(key, replyTo)))
g.dataValue match {
case data: LWWMap[_] data.get(key) match {
case Some(value) replyTo ! Cached(key, Some(value))
case None replyTo ! Cached(key, None)
}
}
case NotFound(_, Some(Request(key, replyTo)))
replyTo ! Cached(key, None)
case _: UpdateResponse[_] // ok
}
}
class ReplicatedCacheSpecMultiJvmNode1 extends ReplicatedCacheSpec
class ReplicatedCacheSpecMultiJvmNode2 extends ReplicatedCacheSpec
class ReplicatedCacheSpecMultiJvmNode3 extends ReplicatedCacheSpec
class ReplicatedCacheSpec extends MultiNodeSpec(ReplicatedCacheSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatedCacheSpec._
import ReplicatedCache._
override def initialParticipants = roles.size
val cluster = Cluster(system)
val replicatedCache = system.actorOf(ReplicatedCache.props)
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated cache" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"replicate cached entry" in within(10.seconds) {
runOn(node1) {
replicatedCache ! PutInCache("key1", "A")
}
awaitAssert {
val probe = TestProbe()
replicatedCache.tell(GetFromCache("key1"), probe.ref)
probe.expectMsg(Cached("key1", Some("A")))
}
enterBarrier("after-2")
}
"replicate many cached entries" in within(10.seconds) {
runOn(node1) {
for (i 100 to 200)
replicatedCache ! PutInCache("key" + i, i)
}
awaitAssert {
val probe = TestProbe()
for (i 100 to 200) {
replicatedCache.tell(GetFromCache("key" + i), probe.ref)
probe.expectMsg(Cached("key" + i, Some(i)))
}
}
enterBarrier("after-3")
}
"replicate evicted entry" in within(15.seconds) {
runOn(node1) {
replicatedCache ! PutInCache("key2", "B")
}
awaitAssert {
val probe = TestProbe()
replicatedCache.tell(GetFromCache("key2"), probe.ref)
probe.expectMsg(Cached("key2", Some("B")))
}
enterBarrier("key2-replicated")
runOn(node3) {
replicatedCache ! Evict("key2")
}
awaitAssert {
val probe = TestProbe()
replicatedCache.tell(GetFromCache("key2"), probe.ref)
probe.expectMsg(Cached("key2", None))
}
enterBarrier("after-4")
}
"replicate updated cached entry" in within(10.seconds) {
runOn(node2) {
replicatedCache ! PutInCache("key1", "A2")
replicatedCache ! PutInCache("key1", "A3")
}
awaitAssert {
val probe = TestProbe()
replicatedCache.tell(GetFromCache("key1"), probe.ref)
probe.expectMsg(Cached("key1", Some("A3")))
}
enterBarrier("after-5")
}
}
}

View file

@ -0,0 +1,200 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import java.lang.management.ManagementFactory
import java.lang.management.MemoryMXBean
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.Address
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ClusterEvent.{ InitialStateAsEvents, MemberUp, MemberRemoved }
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.LWWMap
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.LWWMapKey
object ReplicatedMetricsSpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
object ReplicatedMetrics {
import akka.cluster.ddata.Replicator._
def props(measureInterval: FiniteDuration, cleanupInterval: FiniteDuration): Props =
Props(new ReplicatedMetrics(measureInterval, cleanupInterval))
def props: Props = props(1.second, 1.minute)
private case object Tick
private case object Cleanup
case class UsedHeap(percentPerNode: Map[String, Double]) {
override def toString =
percentPerNode.toSeq.sortBy(_._1).map {
case (key, value) key + " --> " + value + " %"
}.mkString("\n")
}
def nodeKey(address: Address): String = address.host.get + ":" + address.port.get
}
class ReplicatedMetrics(measureInterval: FiniteDuration, cleanupInterval: FiniteDuration)
extends Actor with ActorLogging {
import akka.cluster.ddata.Replicator._
import ReplicatedMetrics._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
val node = nodeKey(cluster.selfAddress)
val tickTask = context.system.scheduler.schedule(measureInterval, measureInterval,
self, Tick)(context.dispatcher)
val cleanupTask = context.system.scheduler.schedule(cleanupInterval, cleanupInterval,
self, Cleanup)(context.dispatcher)
val memoryMBean: MemoryMXBean = ManagementFactory.getMemoryMXBean
val UsedHeapKey = LWWMapKey[Long]("usedHeap")
val MaxHeapKey = LWWMapKey[Long]("maxHeap")
replicator ! Subscribe(UsedHeapKey, self)
replicator ! Subscribe(MaxHeapKey, self)
cluster.subscribe(self, InitialStateAsEvents, classOf[MemberUp], classOf[MemberRemoved])
override def postStop(): Unit = {
tickTask.cancel()
cluster.unsubscribe(self)
super.postStop()
}
var maxHeap = Map.empty[String, Long]
var nodesInCluster = Set.empty[String]
def receive = {
case Tick
val heap = memoryMBean.getHeapMemoryUsage
val used = heap.getUsed
val max = heap.getMax
replicator ! Update(UsedHeapKey, LWWMap.empty[Long], WriteLocal)(_ + (node -> used))
replicator ! Update(MaxHeapKey, LWWMap.empty[Long], WriteLocal) { data
data.get(node) match {
case Some(`max`) data // unchanged
case _ data + (node -> max)
}
}
case c @ Changed(MaxHeapKey)
maxHeap = c.get(MaxHeapKey).entries
case c @ Changed(UsedHeapKey)
val usedHeapPercent = UsedHeap(c.get(UsedHeapKey).entries.collect {
case (key, value) if maxHeap.contains(key)
(key -> (value.toDouble / maxHeap(key)) * 100.0)
})
log.debug("Node {} observed:\n{}", node, usedHeapPercent)
context.system.eventStream.publish(usedHeapPercent)
case _: UpdateResponse[_] // ok
case MemberUp(m)
nodesInCluster += nodeKey(m.address)
case MemberRemoved(m, _)
nodesInCluster -= nodeKey(m.address)
case Cleanup
def cleanupRemoved(data: LWWMap[Long]): LWWMap[Long] =
(data.entries.keySet -- nodesInCluster).foldLeft(data) { case (d, key) d - key }
replicator ! Update(UsedHeapKey, LWWMap.empty[Long], WriteLocal)(cleanupRemoved)
replicator ! Update(MaxHeapKey, LWWMap.empty[Long], WriteLocal)(cleanupRemoved)
}
}
class ReplicatedMetricsSpecMultiJvmNode1 extends ReplicatedMetricsSpec
class ReplicatedMetricsSpecMultiJvmNode2 extends ReplicatedMetricsSpec
class ReplicatedMetricsSpecMultiJvmNode3 extends ReplicatedMetricsSpec
class ReplicatedMetricsSpec extends MultiNodeSpec(ReplicatedMetricsSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatedMetricsSpec._
import ReplicatedMetrics._
override def initialParticipants = roles.size
val cluster = Cluster(system)
val replicatedMetrics = system.actorOf(ReplicatedMetrics.props(1.second, 3.seconds))
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated metrics" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"replicate metrics" in within(10.seconds) {
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[UsedHeap])
awaitAssert {
probe.expectMsgType[UsedHeap].percentPerNode.size should be(3)
}
probe.expectMsgType[UsedHeap].percentPerNode.size should be(3)
probe.expectMsgType[UsedHeap].percentPerNode.size should be(3)
enterBarrier("after-2")
}
"cleanup removed node" in within(15.seconds) {
val node3Address = node(node3).address
runOn(node1) {
cluster.leave(node3Address)
}
runOn(node1, node2) {
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[UsedHeap])
awaitAssert {
probe.expectMsgType[UsedHeap].percentPerNode.size should be(2)
}
probe.expectMsgType[UsedHeap].percentPerNode should not contain (
nodeKey(node3Address))
}
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,267 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.actor.Terminated
import akka.cluster.Cluster
import akka.cluster.ClusterEvent
import akka.cluster.ClusterEvent.LeaderChanged
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.GSet
import akka.cluster.ddata.ORSet
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.GSetKey
import akka.cluster.ddata.ORSetKey
import akka.cluster.ddata.Key
object ReplicatedServiceRegistrySpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
class Service extends Actor {
def receive = {
case s: String sender() ! self.path.name + ": " + s
}
}
}
object ReplicatedServiceRegistry {
import akka.cluster.ddata.Replicator._
val props: Props = Props[ReplicatedServiceRegistry]
/**
* Register a `service` with a `name`. Several services
* can be registered with the same `name`.
* It will be removed when it is terminated.
*/
final case class Register(name: String, service: ActorRef)
/**
* Lookup services registered for a `name`. [[Bindings]] will
* be sent to `sender()`.
*/
final case class Lookup(name: String)
/**
* Reply for [[Lookup]]
*/
final case class Bindings(name: String, services: Set[ActorRef])
/**
* Published to `System.eventStream` when services are changed.
*/
final case class BindingChanged(name: String, services: Set[ActorRef])
final case class ServiceKey(serviceName: String) extends Key[ORSet[ActorRef]](serviceName)
private val AllServicesKey = GSetKey[ServiceKey]("service-keys")
}
class ReplicatedServiceRegistry() extends Actor with ActorLogging {
import akka.cluster.ddata.Replicator._
import ReplicatedServiceRegistry._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
var keys = Set.empty[ServiceKey]
var services = Map.empty[String, Set[ActorRef]]
var leader = false
def serviceKey(serviceName: String): ServiceKey =
ServiceKey("service:" + serviceName)
override def preStart(): Unit = {
replicator ! Subscribe(AllServicesKey, self)
cluster.subscribe(self, ClusterEvent.InitialStateAsEvents, classOf[ClusterEvent.LeaderChanged])
}
override def postStop(): Unit = {
cluster.unsubscribe(self)
}
def receive = {
case Register(name, service)
val dKey = serviceKey(name)
// store the service names in a separate GSet to be able to
// get notifications of new names
if (!keys(dKey))
replicator ! Update(AllServicesKey, GSet(), WriteLocal)(_ + dKey)
// add the service
replicator ! Update(dKey, ORSet(), WriteLocal)(_ + service)
case Lookup(key)
sender() ! Bindings(key, services.getOrElse(key, Set.empty))
case c @ Changed(AllServicesKey)
val newKeys = c.get(AllServicesKey).elements
log.debug("Services changed, added: {}, all: {}", (newKeys -- keys), newKeys)
(newKeys -- keys).foreach { dKey
// subscribe to get notifications of when services with this name are added or removed
replicator ! Subscribe(dKey, self)
}
keys = newKeys
case c @ Changed(ServiceKey(serviceName))
val name = serviceName.split(":").tail.mkString
val newServices = c.get(serviceKey(name)).elements
log.debug("Services changed for name [{}]: {}", name, newServices)
services = services.updated(name, newServices)
context.system.eventStream.publish(BindingChanged(name, newServices))
if (leader)
newServices.foreach(context.watch) // watch is idempotent
case LeaderChanged(node)
// Let one node (the leader) be responsible for removal of terminated services
// to avoid redundant work and too many death watch notifications.
// It is not critical to only do it from one node.
val wasLeader = leader
leader = node.exists(_ == cluster.selfAddress)
// when used with many (> 500) services you must increase the system message buffer
// `akka.remote.system-message-buffer-size`
if (!wasLeader && leader)
for (refs services.valuesIterator; ref refs)
context.watch(ref)
else if (wasLeader && !leader)
for (refs services.valuesIterator; ref refs)
context.unwatch(ref)
case Terminated(ref)
val names = services.collect { case (name, refs) if refs.contains(ref) name }
names.foreach { name
log.debug("Service with name [{}] terminated: {}", name, ref)
replicator ! Update(serviceKey(name), ORSet(), WriteLocal)(_ - ref)
}
case _: UpdateResponse[_] // ok
}
}
class ReplicatedServiceRegistrySpecMultiJvmNode1 extends ReplicatedServiceRegistrySpec
class ReplicatedServiceRegistrySpecMultiJvmNode2 extends ReplicatedServiceRegistrySpec
class ReplicatedServiceRegistrySpecMultiJvmNode3 extends ReplicatedServiceRegistrySpec
class ReplicatedServiceRegistrySpec extends MultiNodeSpec(ReplicatedServiceRegistrySpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatedServiceRegistrySpec._
import ReplicatedServiceRegistry._
override def initialParticipants = roles.size
val cluster = Cluster(system)
val registry = system.actorOf(ReplicatedServiceRegistry.props)
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated service registry" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"replicate service entry" in within(10.seconds) {
runOn(node1) {
val a1 = system.actorOf(Props[Service], name = "a1")
registry ! Register("a", a1)
}
awaitAssert {
val probe = TestProbe()
registry.tell(Lookup("a"), probe.ref)
probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1"))
}
enterBarrier("after-2")
}
"replicate updated service entry, and publish to even bus" in {
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[BindingChanged])
runOn(node2) {
val a2 = system.actorOf(Props[Service], name = "a2")
registry ! Register("a", a2)
}
probe.within(10.seconds) {
probe.expectMsgType[BindingChanged].services.map(_.path.name) should be(Set("a1", "a2"))
registry.tell(Lookup("a"), probe.ref)
probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1", "a2"))
}
enterBarrier("after-4")
}
"remove terminated service" in {
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[BindingChanged])
runOn(node2) {
registry.tell(Lookup("a"), probe.ref)
val a2 = probe.expectMsgType[Bindings].services.find(_.path.name == "a2").get
a2 ! PoisonPill
}
probe.within(10.seconds) {
probe.expectMsgType[BindingChanged].services.map(_.path.name) should be(Set("a1"))
registry.tell(Lookup("a"), probe.ref)
probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1"))
}
enterBarrier("after-5")
}
"replicate many service entries" in within(10.seconds) {
for (i 100 until 200) {
val service = system.actorOf(Props[Service], name = myself.name + "_" + i)
registry ! Register("a" + i, service)
}
awaitAssert {
val probe = TestProbe()
for (i 100 until 200) {
registry.tell(Lookup("a" + i), probe.ref)
probe.expectMsgType[Bindings].services.map(_.path.name) should be(roles.map(_.name + "_" + i).toSet)
}
}
enterBarrier("after-6")
}
}
}

View file

@ -0,0 +1,214 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.LWWMap
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.LWWMapKey
object ReplicatedShoppingCartSpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
object ShoppingCart {
import akka.cluster.ddata.Replicator._
def props(userId: String): Props = Props(new ShoppingCart(userId))
case object GetCart
final case class AddItem(item: LineItem)
final case class RemoveItem(productId: String)
final case class Cart(items: Set[LineItem])
final case class LineItem(productId: String, title: String, quantity: Int)
//#read-write-majority
private val timeout = 3.seconds
private val readMajority = ReadMajority(timeout)
private val writeMajority = WriteMajority(timeout)
//#read-write-majority
}
class ShoppingCart(userId: String) extends Actor {
import ShoppingCart._
import akka.cluster.ddata.Replicator._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
val DataKey = LWWMapKey[LineItem]("cart-" + userId)
def receive = receiveGetCart
.orElse[Any, Unit](receiveAddItem)
.orElse[Any, Unit](receiveRemoveItem)
.orElse[Any, Unit](receiveOther)
//#get-cart
def receiveGetCart: Receive = {
case GetCart
replicator ! Get(DataKey, readMajority, Some(sender()))
case g @ GetSuccess(DataKey, Some(replyTo: ActorRef))
val data = g.get(DataKey)
val cart = Cart(data.entries.values.toSet)
replyTo ! cart
case NotFound(DataKey, Some(replyTo: ActorRef))
replyTo ! Cart(Set.empty)
case GetFailure(DataKey, Some(replyTo: ActorRef))
// ReadMajority failure, try again with local read
replicator ! Get(DataKey, ReadLocal, Some(replyTo))
}
//#get-cart
//#add-item
def receiveAddItem: Receive = {
case cmd @ AddItem(item)
val update = Update(DataKey, LWWMap.empty[LineItem], writeMajority, Some(cmd)) {
cart updateCart(cart, item)
}
replicator ! update
case GetFailure(DataKey, Some(AddItem(item)))
// ReadMajority of Update failed, fall back to best effort local value
replicator ! Update(DataKey, LWWMap.empty[LineItem], writeMajority, None) {
cart updateCart(cart, item)
}
}
//#add-item
//#remove-item
def receiveRemoveItem: Receive = {
case cmd @ RemoveItem(productId)
// Try to fetch latest from a majority of nodes first, since ORMap
// remove must have seen the item to be able to remove it.
replicator ! Get(DataKey, readMajority, Some(cmd))
case GetSuccess(DataKey, Some(RemoveItem(productId)))
replicator ! Update(DataKey, LWWMap(), writeMajority, None) {
_ - productId
}
case GetFailure(DataKey, Some(RemoveItem(productId)))
// ReadMajority failed, fall back to best effort local value
replicator ! Update(DataKey, LWWMap(), writeMajority, None) {
_ - productId
}
case NotFound(DataKey, Some(RemoveItem(productId)))
// nothing to remove
}
//#remove-item
def receiveOther: Receive = {
case _: UpdateSuccess[_] | _: UpdateTimeout[_]
// UpdateTimeout, will eventually be replicated
case e: UpdateFailure[_] throw new IllegalStateException("Unexpected failure: " + e)
}
def updateCart(data: LWWMap[LineItem], item: LineItem): LWWMap[LineItem] =
data.get(item.productId) match {
case Some(LineItem(_, _, existingQuantity))
data + (item.productId -> item.copy(quantity = existingQuantity + item.quantity))
case None data + (item.productId -> item)
}
}
class ReplicatedShoppingCartSpecMultiJvmNode1 extends ReplicatedShoppingCartSpec
class ReplicatedShoppingCartSpecMultiJvmNode2 extends ReplicatedShoppingCartSpec
class ReplicatedShoppingCartSpecMultiJvmNode3 extends ReplicatedShoppingCartSpec
class ReplicatedShoppingCartSpec extends MultiNodeSpec(ReplicatedShoppingCartSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatedShoppingCartSpec._
import ShoppingCart._
override def initialParticipants = roles.size
val cluster = Cluster(system)
val shoppingCart = system.actorOf(ShoppingCart.props("user-1"))
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated shopping cart" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"handle updates directly after start" in within(15.seconds) {
runOn(node2) {
shoppingCart ! ShoppingCart.AddItem(LineItem("1", "Apples", quantity = 2))
shoppingCart ! ShoppingCart.AddItem(LineItem("2", "Oranges", quantity = 3))
}
enterBarrier("updates-done")
awaitAssert {
shoppingCart ! ShoppingCart.GetCart
val cart = expectMsgType[Cart]
cart.items should be(Set(LineItem("1", "Apples", quantity = 2), LineItem("2", "Oranges", quantity = 3)))
}
enterBarrier("after-2")
}
"handle updates from different nodes" in within(5.seconds) {
runOn(node2) {
shoppingCart ! ShoppingCart.AddItem(LineItem("1", "Apples", quantity = 5))
shoppingCart ! ShoppingCart.RemoveItem("2")
}
runOn(node3) {
shoppingCart ! ShoppingCart.AddItem(LineItem("3", "Bananas", quantity = 4))
}
enterBarrier("updates-done")
awaitAssert {
shoppingCart ! ShoppingCart.GetCart
val cart = expectMsgType[Cart]
cart.items should be(Set(LineItem("1", "Apples", quantity = 7), LineItem("3", "Bananas", quantity = 4)))
}
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,184 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.Flag
import akka.cluster.ddata.PNCounterMap
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.FlagKey
import akka.cluster.ddata.PNCounterMapKey
object VotingContestSpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
object VotingService {
case object Open
case object OpenAck
case object Close
case object CloseAck
final case class Vote(participant: String)
case object GetVotes
final case class Votes(result: Map[String, BigInt], open: Boolean)
private final case class GetVotesReq(replyTo: ActorRef)
}
class VotingService extends Actor {
import akka.cluster.ddata.Replicator._
import VotingService._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
val OpenedKey = FlagKey("contestOpened")
val ClosedKey = FlagKey("contestClosed")
val CountersKey = PNCounterMapKey("contestCounters")
replicator ! Subscribe(OpenedKey, self)
def receive = {
case Open
replicator ! Update(OpenedKey, Flag(), WriteAll(5.seconds))(_.switchOn)
becomeOpen()
case c @ Changed(OpenedKey) if c.get(OpenedKey).enabled
becomeOpen()
case GetVotes
sender() ! Votes(Map.empty, open = false)
}
def becomeOpen(): Unit = {
replicator ! Unsubscribe(OpenedKey, self)
replicator ! Subscribe(ClosedKey, self)
context.become(open orElse getVotes(open = true))
}
def open: Receive = {
case v @ Vote(participant)
val update = Update(CountersKey, PNCounterMap(), WriteLocal, request = Some(v)) {
_.increment(participant, 1)
}
replicator ! update
case _: UpdateSuccess[_]
case Close
replicator ! Update(ClosedKey, Flag(), WriteAll(5.seconds))(_.switchOn)
context.become(getVotes(open = false))
case c @ Changed(ClosedKey) if c.get(ClosedKey).enabled
context.become(getVotes(open = false))
}
def getVotes(open: Boolean): Receive = {
case GetVotes
replicator ! Get(CountersKey, ReadAll(3.seconds), Some(GetVotesReq(sender())))
case g @ GetSuccess(CountersKey, Some(GetVotesReq(replyTo)))
val data = g.get(CountersKey)
replyTo ! Votes(data.entries, open)
case NotFound(CountersKey, Some(GetVotesReq(replyTo)))
replyTo ! Votes(Map.empty, open)
case _: GetFailure[_]
case _: UpdateSuccess[_]
}
}
class VotingContestSpecMultiJvmNode1 extends VotingContestSpec
class VotingContestSpecMultiJvmNode2 extends VotingContestSpec
class VotingContestSpecMultiJvmNode3 extends VotingContestSpec
class VotingContestSpec extends MultiNodeSpec(VotingContestSpec) with STMultiNodeSpec with ImplicitSender {
import VotingContestSpec._
override def initialParticipants = roles.size
val cluster = Cluster(system)
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated voting" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"count votes correctly" in within(15.seconds) {
import VotingService._
val votingService = system.actorOf(Props[VotingService], "votingService")
val N = 1000
runOn(node1) {
votingService ! Open
for (n 1 to N) {
votingService ! Vote("#" + ((n % 20) + 1))
}
}
runOn(node2, node3) {
// wait for it to open
val p = TestProbe()
awaitAssert {
votingService.tell(GetVotes, p.ref)
p.expectMsgPF(3.seconds) { case Votes(_, true) true }
}
for (n 1 to N) {
votingService ! Vote("#" + ((n % 20) + 1))
}
}
enterBarrier("voting-done")
runOn(node3) {
votingService ! Close
}
val expected = (1 to 20).map(n "#" + n -> BigInt(3L * N / 20)).toMap
awaitAssert {
votingService ! GetVotes
expectMsg(3.seconds, Votes(expected, false))
}
enterBarrier("after-2")
}
}
}