Add support for delta-CRDT, #21875
* delta GCounter and PNCounter * first stab at delta propagation protocol * send delta in the direct write * possibility to turn off delta propagation * tests * protobuf serializer for DeltaPropagation * documentation
This commit is contained in:
parent
2a9fa234a1
commit
3e7ffd6b96
18 changed files with 2408 additions and 98 deletions
|
|
@ -0,0 +1,185 @@
|
|||
/**
|
||||
* Copyright (C) 2017 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.cluster.ddata
|
||||
|
||||
import akka.actor.Address
|
||||
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
|
||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||
import org.scalactic.TypeCheckedTripleEquals
|
||||
import org.scalatest.Matchers
|
||||
import org.scalatest.WordSpec
|
||||
|
||||
object DeltaPropagationSelectorSpec {
|
||||
class TestSelector(override val allNodes: Vector[Address]) extends DeltaPropagationSelector {
|
||||
override val divisor = 5
|
||||
override def createDeltaPropagation(deltas: Map[String, ReplicatedData]): DeltaPropagation =
|
||||
DeltaPropagation(deltas.mapValues(d ⇒ DataEnvelope(d)))
|
||||
}
|
||||
|
||||
val deltaA = GSet.empty[String] + "a"
|
||||
val deltaB = GSet.empty[String] + "b"
|
||||
val deltaC = GSet.empty[String] + "c"
|
||||
}
|
||||
|
||||
class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheckedTripleEquals {
|
||||
import DeltaPropagationSelectorSpec._
|
||||
val nodes = (2500 until 2600).map(n ⇒ Address("akka", "Sys", "localhost", n)).toVector
|
||||
|
||||
"DeltaPropagationSelector" must {
|
||||
"collect none when no nodes" in {
|
||||
val selector = new TestSelector(Vector.empty)
|
||||
selector.update("A", deltaA)
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(false)
|
||||
}
|
||||
|
||||
"collect 1 when one node" in {
|
||||
val selector = new TestSelector(nodes.take(1))
|
||||
selector.update("A", deltaA)
|
||||
selector.update("B", deltaB)
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(true)
|
||||
selector.hasDeltaEntries("B") should ===(true)
|
||||
val expected = DeltaPropagation(Map("A" → DataEnvelope(deltaA), "B" → DataEnvelope(deltaB)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) → expected))
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(false)
|
||||
selector.hasDeltaEntries("B") should ===(false)
|
||||
}
|
||||
|
||||
"collect 2+1 when three nodes" in {
|
||||
val selector = new TestSelector(nodes.take(3))
|
||||
selector.update("A", deltaA)
|
||||
selector.update("B", deltaB)
|
||||
val expected = DeltaPropagation(Map("A" → DataEnvelope(deltaA), "B" → DataEnvelope(deltaB)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) → expected, nodes(1) → expected))
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(true)
|
||||
selector.hasDeltaEntries("B") should ===(true)
|
||||
selector.collectPropagations() should ===(Map(nodes(2) → expected))
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(false)
|
||||
selector.hasDeltaEntries("B") should ===(false)
|
||||
}
|
||||
|
||||
"keep track of deltas per node" in {
|
||||
val selector = new TestSelector(nodes.take(3))
|
||||
selector.update("A", deltaA)
|
||||
selector.update("B", deltaB)
|
||||
val expected1 = DeltaPropagation(Map("A" → DataEnvelope(deltaA), "B" → DataEnvelope(deltaB)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) → expected1, nodes(1) → expected1))
|
||||
// new update before previous was propagated to all nodes
|
||||
selector.update("C", deltaC)
|
||||
val expected2 = DeltaPropagation(Map("A" → DataEnvelope(deltaA), "B" → DataEnvelope(deltaB),
|
||||
"C" → DataEnvelope(deltaC)))
|
||||
val expected3 = DeltaPropagation(Map("C" → DataEnvelope(deltaC)))
|
||||
selector.collectPropagations() should ===(Map(nodes(2) → expected2, nodes(0) → expected3))
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("A") should ===(false)
|
||||
selector.hasDeltaEntries("B") should ===(false)
|
||||
selector.hasDeltaEntries("C") should ===(true)
|
||||
selector.collectPropagations() should ===(Map(nodes(1) → expected3))
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
selector.cleanupDeltaEntries()
|
||||
selector.hasDeltaEntries("C") should ===(false)
|
||||
}
|
||||
|
||||
"merge updates that occur within same tick" in {
|
||||
val delta1 = GSet.empty[String] + "a1"
|
||||
val delta2 = GSet.empty[String] + "a2"
|
||||
val delta3 = GSet.empty[String] + "a3"
|
||||
val selector = new TestSelector(nodes.take(1))
|
||||
selector.update("A", delta1)
|
||||
selector.update("A", delta2)
|
||||
val expected1 = DeltaPropagation(Map("A" → DataEnvelope(delta1.merge(delta2))))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) → expected1))
|
||||
selector.update("A", delta3)
|
||||
val expected2 = DeltaPropagation(Map("A" → DataEnvelope(delta3)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) → expected2))
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
}
|
||||
|
||||
"merge deltas" in {
|
||||
val delta1 = GSet.empty[String] + "a1"
|
||||
val delta2 = GSet.empty[String] + "a2"
|
||||
val delta3 = GSet.empty[String] + "a3"
|
||||
val selector = new TestSelector(nodes.take(3)) {
|
||||
override def nodesSliceSize(allNodesSize: Int): Int = 1
|
||||
}
|
||||
selector.update("A", delta1)
|
||||
val expected1 = DeltaPropagation(Map("A" → DataEnvelope(delta1)))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) → expected1))
|
||||
|
||||
selector.update("A", delta2)
|
||||
val expected2 = DeltaPropagation(Map("A" → DataEnvelope(delta1.merge(delta2))))
|
||||
selector.collectPropagations() should ===(Map(nodes(1) → expected2))
|
||||
|
||||
selector.update("A", delta3)
|
||||
val expected3 = DeltaPropagation(Map("A" → DataEnvelope(delta1.merge(delta2).merge(delta3))))
|
||||
selector.collectPropagations() should ===(Map(nodes(2) → expected3))
|
||||
|
||||
val expected4 = DeltaPropagation(Map("A" → DataEnvelope(delta2.merge(delta3))))
|
||||
selector.collectPropagations() should ===(Map(nodes(0) → expected4))
|
||||
|
||||
val expected5 = DeltaPropagation(Map("A" → DataEnvelope(delta3)))
|
||||
selector.collectPropagations() should ===(Map(nodes(1) → expected5))
|
||||
|
||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||
}
|
||||
|
||||
"calcualte right slice size" in {
|
||||
val selector = new TestSelector(nodes)
|
||||
selector.nodesSliceSize(0) should ===(0)
|
||||
selector.nodesSliceSize(1) should ===(1)
|
||||
(2 to 9).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(2)
|
||||
}
|
||||
}
|
||||
(10 to 14).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(3)
|
||||
}
|
||||
}
|
||||
(15 to 19).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(4)
|
||||
}
|
||||
}
|
||||
(20 to 24).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(5)
|
||||
}
|
||||
}
|
||||
(25 to 29).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(6)
|
||||
}
|
||||
}
|
||||
(30 to 34).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(7)
|
||||
}
|
||||
}
|
||||
(35 to 39).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(8)
|
||||
}
|
||||
}
|
||||
(40 to 44).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(9)
|
||||
}
|
||||
}
|
||||
(45 to 200).foreach { n ⇒
|
||||
withClue(s"n=$n") {
|
||||
selector.nodesSliceSize(n) should ===(10)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -11,9 +11,9 @@ import org.scalatest.Matchers
|
|||
import org.scalatest.WordSpec
|
||||
|
||||
class GCounterSpec extends WordSpec with Matchers {
|
||||
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
|
||||
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
|
||||
val node3 = UniqueAddress(node1.address.copy(port = Some(2553)), 3)
|
||||
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1L)
|
||||
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2L)
|
||||
val node3 = UniqueAddress(node1.address.copy(port = Some(2553)), 3L)
|
||||
|
||||
"A GCounter" must {
|
||||
|
||||
|
|
@ -25,10 +25,14 @@ class GCounterSpec extends WordSpec with Matchers {
|
|||
|
||||
val c4 = c3 increment node2
|
||||
val c5 = c4 increment node2
|
||||
val c6 = c5 increment node2
|
||||
val c6 = c5.resetDelta increment node2
|
||||
|
||||
c6.state(node1) should be(2)
|
||||
c6.state(node2) should be(3)
|
||||
|
||||
c2.delta.state(node1) should be(1)
|
||||
c3.delta.state(node1) should be(2)
|
||||
c6.delta.state(node2) should be(3)
|
||||
}
|
||||
|
||||
"be able to increment each node's record by arbitrary delta" in {
|
||||
|
|
@ -74,7 +78,7 @@ class GCounterSpec extends WordSpec with Matchers {
|
|||
c16.state(node2) should be(10)
|
||||
c16.value should be(17)
|
||||
|
||||
// counter 1
|
||||
// counter 2
|
||||
val c21 = GCounter()
|
||||
val c22 = c21 increment (node1, 2)
|
||||
val c23 = c22 increment (node1, 2)
|
||||
|
|
@ -91,11 +95,13 @@ class GCounterSpec extends WordSpec with Matchers {
|
|||
merged1.state(node1) should be(7)
|
||||
merged1.state(node2) should be(10)
|
||||
merged1.value should be(17)
|
||||
merged1.delta should be(GCounter.empty)
|
||||
|
||||
val merged2 = c26 merge c16
|
||||
merged2.state(node1) should be(7)
|
||||
merged2.state(node2) should be(10)
|
||||
merged2.value should be(17)
|
||||
merged2.delta should be(GCounter.empty)
|
||||
}
|
||||
|
||||
"be able to have its history correctly merged with another GCounter 2" in {
|
||||
|
|
|
|||
|
|
@ -11,8 +11,8 @@ import org.scalatest.Matchers
|
|||
import org.scalatest.WordSpec
|
||||
|
||||
class PNCounterSpec extends WordSpec with Matchers {
|
||||
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
|
||||
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
|
||||
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1L)
|
||||
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2L)
|
||||
|
||||
"A PNCounter" must {
|
||||
|
||||
|
|
@ -24,10 +24,18 @@ class PNCounterSpec extends WordSpec with Matchers {
|
|||
|
||||
val c4 = c3 increment node2
|
||||
val c5 = c4 increment node2
|
||||
val c6 = c5 increment node2
|
||||
val c6 = c5.resetDelta increment node2
|
||||
|
||||
c6.increments.state(node1) should be(2)
|
||||
c6.increments.state(node2) should be(3)
|
||||
|
||||
c2.delta.value.toLong should be(1)
|
||||
c2.delta.increments.state(node1) should be(1)
|
||||
c3.delta.value should be(2)
|
||||
c3.delta.increments.state(node1) should be(2)
|
||||
|
||||
c6.delta.value should be(3)
|
||||
c6.delta.increments.state(node2) should be(3)
|
||||
}
|
||||
|
||||
"be able to decrement each node's record by one" in {
|
||||
|
|
@ -38,10 +46,16 @@ class PNCounterSpec extends WordSpec with Matchers {
|
|||
|
||||
val c4 = c3 decrement node2
|
||||
val c5 = c4 decrement node2
|
||||
val c6 = c5 decrement node2
|
||||
val c6 = c5.resetDelta decrement node2
|
||||
|
||||
c6.decrements.state(node1) should be(2)
|
||||
c6.decrements.state(node2) should be(3)
|
||||
|
||||
c3.delta.value should be(-2)
|
||||
c3.delta.decrements.state(node1) should be(2)
|
||||
|
||||
c6.delta.value should be(-3)
|
||||
c6.delta.decrements.state(node2) should be(3)
|
||||
}
|
||||
|
||||
"be able to increment each node's record by arbitrary delta" in {
|
||||
|
|
|
|||
|
|
@ -24,6 +24,7 @@ import akka.cluster.UniqueAddress
|
|||
import akka.remote.RARP
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
||||
import akka.cluster.ddata.GCounter
|
||||
|
||||
class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem(
|
||||
"ReplicatorMessageSerializerSpec",
|
||||
|
|
@ -58,6 +59,8 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem(
|
|||
"serialize Replicator messages" in {
|
||||
val ref1 = system.actorOf(Props.empty, "ref1")
|
||||
val data1 = GSet.empty[String] + "a"
|
||||
val delta1 = GCounter.empty.increment(address1, 17).increment(address2, 2)
|
||||
val delta2 = delta1.increment(address2, 1)
|
||||
|
||||
checkSerialization(Get(keyA, ReadLocal))
|
||||
checkSerialization(Get(keyA, ReadMajority(2.seconds), Some("x")))
|
||||
|
|
@ -84,6 +87,9 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem(
|
|||
checkSerialization(Gossip(Map(
|
||||
"A" → DataEnvelope(data1),
|
||||
"B" → DataEnvelope(GSet() + "b" + "c")), sendBack = true))
|
||||
checkSerialization(DeltaPropagation(Map(
|
||||
"A" → DataEnvelope(delta1),
|
||||
"B" → DataEnvelope(delta2))))
|
||||
checkSerialization(new DurableDataEnvelope(data1))
|
||||
checkSerialization(new DurableDataEnvelope(DataEnvelope(data1, pruning = Map(
|
||||
address1 → PruningPerformed(System.currentTimeMillis()),
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue