+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:
parent
bf28260cd0
commit
cbe5dd2cf5
69 changed files with 40036 additions and 3 deletions
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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()
|
||||
}
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Loading…
Add table
Add a link
Reference in a new issue