make pruning of CRDT garbage work, #21647
* fix merge issues of DataEnvelope and its pruning * simplify by removing the tombstones, which didn't work in all cases anyway * keep the PruningPerformed markers in the DataEnvelope until configured TTL has elapsed (wall clock) * simplify PruningState structure * also store the pruning markers in durable data * collect removed nodes from the data, listing on MemberRemoved is not enough * possibility to disable pruning altogether * documented caveat for durable data
This commit is contained in:
parent
c5d18c30d6
commit
952be31a7d
28 changed files with 951 additions and 229 deletions
|
|
@ -0,0 +1,70 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.ddata
|
||||
|
||||
import akka.actor.Address
|
||||
import akka.cluster.UniqueAddress
|
||||
import org.scalatest.Matchers
|
||||
import org.scalatest.WordSpec
|
||||
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
|
||||
|
||||
class DataEnvelopeSpec extends WordSpec with Matchers {
|
||||
import PruningState._
|
||||
|
||||
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)
|
||||
val node4 = UniqueAddress(node1.address.copy(port = Some(2554)), 4L)
|
||||
val obsoleteTimeInFuture = System.currentTimeMillis() + 3600 * 1000
|
||||
val oldObsoleteTime = System.currentTimeMillis() - 3600 * 1000
|
||||
|
||||
"DataEnvelope" must {
|
||||
|
||||
"handle pruning transitions" in {
|
||||
val g1 = GCounter.empty.increment(node1, 1)
|
||||
val d1 = DataEnvelope(g1)
|
||||
|
||||
val d2 = d1.initRemovedNodePruning(node1, node2)
|
||||
d2.pruning(node1).isInstanceOf[PruningInitialized] should ===(true)
|
||||
d2.pruning(node1).asInstanceOf[PruningInitialized].owner should ===(node2)
|
||||
|
||||
val d3 = d2.addSeen(node3.address)
|
||||
d3.pruning(node1).asInstanceOf[PruningInitialized].seen should ===(Set(node3.address))
|
||||
|
||||
val d4 = d3.prune(node1, PruningPerformed(obsoleteTimeInFuture))
|
||||
d4.data.asInstanceOf[GCounter].modifiedByNodes should ===(Set(node2))
|
||||
}
|
||||
|
||||
"merge correctly" in {
|
||||
val g1 = GCounter.empty.increment(node1, 1)
|
||||
val d1 = DataEnvelope(g1)
|
||||
val g2 = GCounter.empty.increment(node2, 2)
|
||||
val d2 = DataEnvelope(g2)
|
||||
|
||||
val d3 = d1.merge(d2)
|
||||
d3.data.asInstanceOf[GCounter].value should ===(3)
|
||||
d3.data.asInstanceOf[GCounter].modifiedByNodes should ===(Set(node1, node2))
|
||||
val d4 = d3.initRemovedNodePruning(node1, node2)
|
||||
val d5 = d4.prune(node1, PruningPerformed(obsoleteTimeInFuture))
|
||||
d5.data.asInstanceOf[GCounter].modifiedByNodes should ===(Set(node2))
|
||||
|
||||
// late update from node1
|
||||
val g11 = g1.increment(node1, 10)
|
||||
val d6 = d5.merge(DataEnvelope(g11))
|
||||
d6.data.asInstanceOf[GCounter].value should ===(3)
|
||||
d6.data.asInstanceOf[GCounter].modifiedByNodes should ===(Set(node2))
|
||||
|
||||
// remove obsolete
|
||||
val d7 = d5.copy(pruning = d5.pruning.updated(node1, PruningPerformed(oldObsoleteTime)))
|
||||
val d8 = d5.copy(pruning = Map.empty)
|
||||
d8.merge(d7).pruning should ===(Map.empty)
|
||||
d7.merge(d8).pruning should ===(Map.empty)
|
||||
|
||||
d5.merge(d7).pruning(node1) should ===(PruningPerformed(obsoleteTimeInFuture))
|
||||
d7.merge(d5).pruning(node1) should ===(PruningPerformed(obsoleteTimeInFuture))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -139,18 +139,22 @@ class GCounterSpec extends WordSpec with Matchers {
|
|||
val c1 = GCounter()
|
||||
val c2 = c1 increment node1
|
||||
val c3 = c2 increment node2
|
||||
c2.modifiedByNodes should ===(Set(node1))
|
||||
c2.needPruningFrom(node1) should be(true)
|
||||
c2.needPruningFrom(node2) should be(false)
|
||||
c3.modifiedByNodes should ===(Set(node1, node2))
|
||||
c3.needPruningFrom(node1) should be(true)
|
||||
c3.needPruningFrom(node2) should be(true)
|
||||
c3.value should be(2)
|
||||
|
||||
val c4 = c3.prune(node1, node2)
|
||||
c4.modifiedByNodes should ===(Set(node2))
|
||||
c4.needPruningFrom(node2) should be(true)
|
||||
c4.needPruningFrom(node1) should be(false)
|
||||
c4.value should be(2)
|
||||
|
||||
val c5 = (c4 increment node1).pruningCleanup(node1)
|
||||
c5.modifiedByNodes should ===(Set(node2))
|
||||
c5.needPruningFrom(node1) should be(false)
|
||||
c4.value should be(2)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -143,16 +143,20 @@ class PNCounterSpec extends WordSpec with Matchers {
|
|||
val c1 = PNCounter()
|
||||
val c2 = c1 increment node1
|
||||
val c3 = c2 decrement node2
|
||||
c2.modifiedByNodes should ===(Set(node1))
|
||||
c2.needPruningFrom(node1) should be(true)
|
||||
c2.needPruningFrom(node2) should be(false)
|
||||
c3.modifiedByNodes should ===(Set(node1, node2))
|
||||
c3.needPruningFrom(node1) should be(true)
|
||||
c3.needPruningFrom(node2) should be(true)
|
||||
|
||||
val c4 = c3.prune(node1, node2)
|
||||
c4.modifiedByNodes should ===(Set(node2))
|
||||
c4.needPruningFrom(node2) should be(true)
|
||||
c4.needPruningFrom(node1) should be(false)
|
||||
|
||||
val c5 = (c4 increment node1).pruningCleanup(node1)
|
||||
c5.modifiedByNodes should ===(Set(node2))
|
||||
c5.needPruningFrom(node1) should be(false)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -12,32 +12,37 @@ import org.scalatest.WordSpec
|
|||
class PruningStateSpec extends WordSpec with Matchers {
|
||||
import PruningState._
|
||||
|
||||
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 node4 = UniqueAddress(node1.address.copy(port = Some(2554)), 4)
|
||||
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)
|
||||
val node4 = UniqueAddress(node1.address.copy(port = Some(2554)), 4L)
|
||||
|
||||
"Pruning state" must {
|
||||
|
||||
"merge phase correctly" in {
|
||||
val p1 = PruningState(node1, PruningInitialized(Set.empty))
|
||||
val p2 = PruningState(node1, PruningPerformed)
|
||||
p1.merge(p2).phase should be(PruningPerformed)
|
||||
p2.merge(p1).phase should be(PruningPerformed)
|
||||
"merge state correctly" in {
|
||||
val p1 = PruningInitialized(node1, Set.empty)
|
||||
val p2 = PruningPerformed(System.currentTimeMillis() + 3600 * 1000)
|
||||
p1.merge(p2) should be(p2)
|
||||
p2.merge(p1) should be(p2)
|
||||
|
||||
val p3 = p2.copy(p2.obsoleteTime - 1)
|
||||
p2.merge(p3) should be(p2) // keep greatest obsoleteTime
|
||||
p3.merge(p2) should be(p2)
|
||||
|
||||
}
|
||||
|
||||
"merge owner correctly" in {
|
||||
val p1 = PruningState(node1, PruningInitialized(Set.empty))
|
||||
val p2 = PruningState(node2, PruningInitialized(Set.empty))
|
||||
val expected = PruningState(node1, PruningInitialized(Set.empty))
|
||||
val p1 = PruningInitialized(node1, Set.empty)
|
||||
val p2 = PruningInitialized(node2, Set.empty)
|
||||
val expected = PruningInitialized(node1, Set.empty)
|
||||
p1.merge(p2) should be(expected)
|
||||
p2.merge(p1) should be(expected)
|
||||
}
|
||||
|
||||
"merge seen correctly" in {
|
||||
val p1 = PruningState(node1, PruningInitialized(Set(node2.address)))
|
||||
val p2 = PruningState(node1, PruningInitialized(Set(node4.address)))
|
||||
val expected = PruningState(node1, PruningInitialized(Set(node2.address, node4.address)))
|
||||
val p1 = PruningInitialized(node1, Set(node2.address))
|
||||
val p2 = PruningInitialized(node1, Set(node4.address))
|
||||
val expected = PruningInitialized(node1, Set(node2.address, node4.address))
|
||||
p1.merge(p2) should be(expected)
|
||||
p2.merge(p1) should be(expected)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -70,8 +70,8 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem(
|
|||
checkSerialization(Changed(keyA)(data1))
|
||||
checkSerialization(DataEnvelope(data1))
|
||||
checkSerialization(DataEnvelope(data1, pruning = Map(
|
||||
address1 → PruningState(address2, PruningPerformed),
|
||||
address3 → PruningState(address2, PruningInitialized(Set(address1.address))))))
|
||||
address1 → PruningPerformed(System.currentTimeMillis()),
|
||||
address3 → PruningInitialized(address2, Set(address1.address)))))
|
||||
checkSerialization(Write("A", DataEnvelope(data1)))
|
||||
checkSerialization(WriteAck)
|
||||
checkSerialization(WriteNack)
|
||||
|
|
@ -85,6 +85,9 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem(
|
|||
"A" → DataEnvelope(data1),
|
||||
"B" → DataEnvelope(GSet() + "b" + "c")), sendBack = true))
|
||||
checkSerialization(new DurableDataEnvelope(data1))
|
||||
checkSerialization(new DurableDataEnvelope(DataEnvelope(data1, pruning = Map(
|
||||
address1 → PruningPerformed(System.currentTimeMillis()),
|
||||
address3 → PruningInitialized(address2, Set(address1.address))))))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue