causal delivery of deltas, #22188
* keep track of delta interval versions and skip deltas that are not consequtive, i.e. when some delta message was lost * send the delta versions in the full state gossip to sync up the expected versions after dropped deltas * implementation of deltas for ORSet * refactoring of the delta types to allow for different type for the delta and the full state * extensive tests * mima filter * performance optimizations * simple pruning of deltas * Java API * update documentation * KeyId type alias * Use InternalApi annotation
This commit is contained in:
parent
94afbee179
commit
b700b840d1
41 changed files with 5010 additions and 1950 deletions
|
|
@ -38,6 +38,8 @@ object PerformanceSpec extends MultiNodeConfig {
|
|||
#akka.cluster.distributed-data.durable.keys = ["*"]
|
||||
#akka.cluster.distributed-data.durable.lmdb.dir = target/PerformanceSpec-${System.currentTimeMillis}-ddata
|
||||
#akka.cluster.distributed-data.durable.lmdb.write-behind-interval = 200ms
|
||||
|
||||
#akka.cluster.distributed-data.delta-crdt.enabled = off
|
||||
"""))
|
||||
|
||||
def countDownProps(latch: TestLatch): Props = Props(new CountDown(latch)).withDeploy(Deploy.local)
|
||||
|
|
|
|||
|
|
@ -22,6 +22,7 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
|||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(ConfigFactory.parseString("""
|
||||
akka.loglevel = DEBUG
|
||||
akka.actor.provider = "cluster"
|
||||
akka.log-dead-letters-during-shutdown = off
|
||||
"""))
|
||||
|
|
@ -32,6 +33,8 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
|||
final case class Delay(n: Int) extends Op
|
||||
final case class Incr(key: PNCounterKey, n: Int, consistency: WriteConsistency) extends Op
|
||||
final case class Decr(key: PNCounterKey, n: Int, consistency: WriteConsistency) extends Op
|
||||
final case class Add(key: ORSetKey[String], elem: String, consistency: WriteConsistency) extends Op
|
||||
final case class Remove(key: ORSetKey[String], elem: String, consistency: WriteConsistency) extends Op
|
||||
|
||||
val timeout = 5.seconds
|
||||
val writeTwo = WriteTo(2, timeout)
|
||||
|
|
@ -40,8 +43,11 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
|||
val KeyA = PNCounterKey("A")
|
||||
val KeyB = PNCounterKey("B")
|
||||
val KeyC = PNCounterKey("C")
|
||||
val KeyD = ORSetKey[String]("D")
|
||||
val KeyE = ORSetKey[String]("E")
|
||||
val KeyF = ORSetKey[String]("F")
|
||||
|
||||
def generateOperations(): Vector[Op] = {
|
||||
def generateOperations(onNode: RoleName): Vector[Op] = {
|
||||
val rnd = ThreadLocalRandom.current()
|
||||
|
||||
def consistency(): WriteConsistency = {
|
||||
|
|
@ -52,7 +58,7 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
|||
}
|
||||
}
|
||||
|
||||
def key(): PNCounterKey = {
|
||||
def rndPnCounterkey(): PNCounterKey = {
|
||||
rnd.nextInt(3) match {
|
||||
case 0 ⇒ KeyA
|
||||
case 1 ⇒ KeyB
|
||||
|
|
@ -60,11 +66,43 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
|||
}
|
||||
}
|
||||
|
||||
(0 to (20 + rnd.nextInt(10))).map { _ ⇒
|
||||
def rndOrSetkey(): ORSetKey[String] = {
|
||||
rnd.nextInt(3) match {
|
||||
case 0 ⇒ KeyD
|
||||
case 1 ⇒ KeyE
|
||||
case 2 ⇒ KeyF
|
||||
}
|
||||
}
|
||||
|
||||
var availableForRemove = Set.empty[String]
|
||||
|
||||
def rndAddElement(): String = {
|
||||
// lower case a - j
|
||||
val s = (97 + rnd.nextInt(10)).toChar.toString
|
||||
availableForRemove += s
|
||||
s
|
||||
}
|
||||
|
||||
def rndRemoveElement(): String = {
|
||||
if (availableForRemove.isEmpty)
|
||||
"a"
|
||||
else
|
||||
availableForRemove.toVector(rnd.nextInt(availableForRemove.size))
|
||||
}
|
||||
|
||||
(0 to (30 + rnd.nextInt(10))).map { _ ⇒
|
||||
rnd.nextInt(4) match {
|
||||
case 0 ⇒ Delay(rnd.nextInt(500))
|
||||
case 1 ⇒ Incr(key(), rnd.nextInt(100), consistency())
|
||||
case 2 ⇒ Decr(key(), rnd.nextInt(10), consistency())
|
||||
case 1 ⇒ Incr(rndPnCounterkey(), rnd.nextInt(100), consistency())
|
||||
case 2 ⇒ Decr(rndPnCounterkey(), rnd.nextInt(10), consistency())
|
||||
case 3 ⇒
|
||||
// ORSet
|
||||
val key = rndOrSetkey()
|
||||
// only removals for KeyF on node first
|
||||
if (key == KeyF && onNode == first && rnd.nextBoolean())
|
||||
Remove(key, rndRemoveElement(), consistency())
|
||||
else
|
||||
Add(key, rndAddElement(), consistency())
|
||||
}
|
||||
}.toVector
|
||||
}
|
||||
|
|
@ -136,21 +174,32 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
|||
enterBarrier("ready")
|
||||
|
||||
runOn(first) {
|
||||
fullStateReplicator ! Update(KeyA, PNCounter.empty, WriteLocal)(_ + 1)
|
||||
deltaReplicator ! Update(KeyA, PNCounter.empty, WriteLocal)(_ + 1)
|
||||
// by setting something for each key we don't have to worry about NotFound
|
||||
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
||||
fullStateReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ + 1)
|
||||
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ + 1)
|
||||
}
|
||||
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||
fullStateReplicator ! Update(key, ORSet.empty[String], WriteLocal)(_ + "a")
|
||||
deltaReplicator ! Update(key, ORSet.empty[String], WriteLocal)(_ + "a")
|
||||
}
|
||||
}
|
||||
enterBarrier("updated-1")
|
||||
|
||||
within(5.seconds) {
|
||||
awaitAssert {
|
||||
val p = TestProbe()
|
||||
deltaReplicator.tell(Get(KeyA, ReadLocal), p.ref)
|
||||
p.expectMsgType[GetSuccess[PNCounter]].dataValue.getValue.intValue should be(1)
|
||||
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
||||
fullStateReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||
p.expectMsgType[GetSuccess[PNCounter]].dataValue.getValue.intValue should be(1)
|
||||
}
|
||||
}
|
||||
awaitAssert {
|
||||
val p = TestProbe()
|
||||
deltaReplicator.tell(Get(KeyA, ReadLocal), p.ref)
|
||||
p.expectMsgType[GetSuccess[PNCounter]].dataValue.getValue.intValue should be(1)
|
||||
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||
fullStateReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||
p.expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements should ===(Set("a"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -158,7 +207,7 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
|||
}
|
||||
|
||||
"be eventually consistent" in {
|
||||
val operations = generateOperations()
|
||||
val operations = generateOperations(onNode = myself)
|
||||
log.debug(s"random operations on [${myself.name}]: ${operations.mkString(", ")}")
|
||||
try {
|
||||
// perform random operations with both delta and full-state replicators
|
||||
|
|
@ -170,10 +219,22 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
|||
case Delay(d) ⇒ Thread.sleep(d)
|
||||
case Incr(key, n, consistency) ⇒
|
||||
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ + n)
|
||||
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ + n)
|
||||
deltaReplicator ! Update(key, PNCounter.empty, consistency)(_ + n)
|
||||
case Decr(key, n, consistency) ⇒
|
||||
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ - n)
|
||||
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ - n)
|
||||
deltaReplicator ! Update(key, PNCounter.empty, consistency)(_ - n)
|
||||
case Add(key, elem, consistency) ⇒
|
||||
// to have an deterministic result when mixing add/remove we can only perform
|
||||
// the ORSet operations from one node
|
||||
runOn((if (key == KeyF) List(first) else List(first, second, third)): _*) {
|
||||
fullStateReplicator ! Update(key, ORSet.empty[String], consistency)(_ + elem)
|
||||
deltaReplicator ! Update(key, ORSet.empty[String], consistency)(_ + elem)
|
||||
}
|
||||
case Remove(key, elem, consistency) ⇒
|
||||
runOn(first) {
|
||||
fullStateReplicator ! Update(key, ORSet.empty[String], consistency)(_ - elem)
|
||||
deltaReplicator ! Update(key, ORSet.empty[String], consistency)(_ - elem)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -192,6 +253,19 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
|||
}
|
||||
}
|
||||
|
||||
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||
within(5.seconds) {
|
||||
awaitAssert {
|
||||
val p = TestProbe()
|
||||
fullStateReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||
val fullStateValue = p.expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements
|
||||
deltaReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||
val deltaValue = p.expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements
|
||||
deltaValue should ===(fullStateValue)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
enterBarrierAfterTestStep()
|
||||
} catch {
|
||||
case e: Throwable ⇒
|
||||
|
|
|
|||
|
|
@ -0,0 +1,163 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.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 ReplicatorORSetDeltaSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
|
||||
commonConfig(ConfigFactory.parseString("""
|
||||
akka.loglevel = DEBUG
|
||||
akka.actor.provider = "cluster"
|
||||
akka.log-dead-letters-during-shutdown = off
|
||||
"""))
|
||||
|
||||
testTransport(on = true)
|
||||
}
|
||||
|
||||
class ReplicatorORSetDeltaSpecMultiJvmNode1 extends ReplicatorORSetDeltaSpec
|
||||
class ReplicatorORSetDeltaSpecMultiJvmNode2 extends ReplicatorORSetDeltaSpec
|
||||
class ReplicatorORSetDeltaSpecMultiJvmNode3 extends ReplicatorORSetDeltaSpec
|
||||
|
||||
class ReplicatorORSetDeltaSpec extends MultiNodeSpec(ReplicatorORSetDeltaSpec) with STMultiNodeSpec with ImplicitSender {
|
||||
import Replicator._
|
||||
import ReplicatorORSetDeltaSpec._
|
||||
|
||||
override def initialParticipants = roles.size
|
||||
|
||||
implicit val cluster = Cluster(system)
|
||||
val replicator = system.actorOf(Replicator.props(
|
||||
ReplicatorSettings(system).withGossipInterval(1.second)), "replicator")
|
||||
val timeout = 3.seconds.dilated
|
||||
|
||||
val KeyA = ORSetKey[String]("A")
|
||||
val KeyB = ORSetKey[String]("B")
|
||||
val KeyC = ORSetKey[String]("C")
|
||||
|
||||
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: ORSet[_] ⇒ c.elements
|
||||
}
|
||||
}
|
||||
value should be(expected)
|
||||
}
|
||||
}
|
||||
|
||||
"ORSet delta" must {
|
||||
|
||||
"replicate data in initial phase" in {
|
||||
join(first, first)
|
||||
join(second, first)
|
||||
join(third, first)
|
||||
|
||||
replicator ! Replicator.Internal.TestFullStateGossip(enabled = false)
|
||||
|
||||
within(10.seconds) {
|
||||
awaitAssert {
|
||||
replicator ! GetReplicaCount
|
||||
expectMsg(ReplicaCount(3))
|
||||
}
|
||||
}
|
||||
|
||||
runOn(first) {
|
||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "a")
|
||||
expectMsg(UpdateSuccess(KeyA, None))
|
||||
}
|
||||
|
||||
enterBarrier("initial-updates-done")
|
||||
|
||||
assertValue(KeyA, Set("a"))
|
||||
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"be propagated with causal consistency during network split" in {
|
||||
runOn(first) {
|
||||
// third is isolated
|
||||
testConductor.blackhole(first, third, Direction.Both).await
|
||||
testConductor.blackhole(second, third, Direction.Both).await
|
||||
}
|
||||
enterBarrier("split")
|
||||
|
||||
runOn(first) {
|
||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "b")
|
||||
expectMsg(UpdateSuccess(KeyA, None))
|
||||
}
|
||||
runOn(second) {
|
||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "d")
|
||||
expectMsg(UpdateSuccess(KeyA, None))
|
||||
}
|
||||
runOn(first, second) {
|
||||
assertValue(KeyA, Set("a", "b", "d"))
|
||||
Thread.sleep(2000) // all deltas sent
|
||||
}
|
||||
enterBarrier("added-b-and-d")
|
||||
|
||||
runOn(first) {
|
||||
testConductor.passThrough(first, third, Direction.Both).await
|
||||
testConductor.passThrough(second, third, Direction.Both).await
|
||||
}
|
||||
enterBarrier("healed")
|
||||
|
||||
runOn(first) {
|
||||
// delta for "c" will be sent to third, but it has not received the previous delta for "b"
|
||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "c")
|
||||
expectMsg(UpdateSuccess(KeyA, None))
|
||||
// let the delta be propagated (will not fail if it takes longer)
|
||||
Thread.sleep(1000)
|
||||
}
|
||||
enterBarrier("added-c")
|
||||
|
||||
runOn(first, second) {
|
||||
assertValue(KeyA, Set("a", "b", "c", "d"))
|
||||
}
|
||||
runOn(third) {
|
||||
// the delta for "c" should not be applied because it has not received previous delta for "b"
|
||||
// and full gossip is turned off so far
|
||||
assertValue(KeyA, Set("a"))
|
||||
}
|
||||
enterBarrier("verified-before-full-gossip")
|
||||
|
||||
replicator ! Replicator.Internal.TestFullStateGossip(enabled = true)
|
||||
assertValue(KeyA, Set("a", "b", "c", "d"))
|
||||
enterBarrier("verified-after-full-gossip")
|
||||
|
||||
replicator ! Replicator.Internal.TestFullStateGossip(enabled = false)
|
||||
|
||||
// and now the delta seqNr should be in sync again
|
||||
runOn(first) {
|
||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "e")
|
||||
expectMsg(UpdateSuccess(KeyA, None))
|
||||
}
|
||||
assertValue(KeyA, Set("a", "b", "c", "d", "e"))
|
||||
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -22,6 +22,7 @@ object ReplicatorSpec extends MultiNodeConfig {
|
|||
akka.loglevel = INFO
|
||||
akka.actor.provider = "cluster"
|
||||
akka.log-dead-letters-during-shutdown = off
|
||||
#akka.cluster.distributed-data.delta-crdt.enabled = off
|
||||
"""))
|
||||
|
||||
testTransport(on = true)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue