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
File diff suppressed because it is too large
Load diff
|
|
@ -96,6 +96,15 @@ message Gossip {
|
|||
repeated Entry entries = 2;
|
||||
}
|
||||
|
||||
message DeltaPropagation {
|
||||
message Entry {
|
||||
required string key = 1;
|
||||
required DataEnvelope envelope = 2;
|
||||
}
|
||||
|
||||
repeated Entry entries = 1;
|
||||
}
|
||||
|
||||
message UniqueAddress {
|
||||
required Address address = 1;
|
||||
required sfixed32 uid = 2;
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ akka.cluster.distributed-data {
|
|||
|
||||
# How often the Replicator should send out gossip information
|
||||
gossip-interval = 2 s
|
||||
|
||||
|
||||
# How often the subscribers will be notified of changes, if any
|
||||
notify-subscribers-interval = 500 ms
|
||||
|
||||
|
|
@ -58,6 +58,12 @@ akka.cluster.distributed-data {
|
|||
# after this duration.
|
||||
serializer-cache-time-to-live = 10s
|
||||
|
||||
# Settings for delta-CRDT
|
||||
delta-crdt {
|
||||
# enable or disable delta-CRDT replication
|
||||
enabled = on
|
||||
}
|
||||
|
||||
durable {
|
||||
# List of keys that are durable. Prefix matching is supported by using * at the
|
||||
# end of a key.
|
||||
|
|
|
|||
|
|
@ -0,0 +1,170 @@
|
|||
/**
|
||||
* Copyright (C) 2017 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.cluster.ddata
|
||||
|
||||
import scala.collection.immutable.TreeMap
|
||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||
import akka.actor.Address
|
||||
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
|
||||
|
||||
/**
|
||||
* INTERNAL API: Used by the Replicator actor.
|
||||
* Extracted to separate trait to make it easy to test.
|
||||
*/
|
||||
private[akka] trait DeltaPropagationSelector {
|
||||
|
||||
private var _propagationCount = 0L
|
||||
def propagationCount: Long = _propagationCount
|
||||
private var deltaCounter = Map.empty[String, Long]
|
||||
private var deltaEntries = Map.empty[String, TreeMap[Long, ReplicatedData]]
|
||||
private var deltaSentToNode = Map.empty[String, Map[Address, Long]]
|
||||
private var deltaNodeRoundRobinCounter = 0L
|
||||
|
||||
def divisor: Int
|
||||
|
||||
def allNodes: Vector[Address]
|
||||
|
||||
def createDeltaPropagation(deltas: Map[String, ReplicatedData]): DeltaPropagation
|
||||
|
||||
def update(key: String, delta: ReplicatedData): Unit = {
|
||||
val c = deltaCounter.get(key) match {
|
||||
case Some(c) ⇒ c
|
||||
case None ⇒
|
||||
deltaCounter = deltaCounter.updated(key, 1L)
|
||||
1L
|
||||
}
|
||||
val deltaEntriesForKey = deltaEntries.getOrElse(key, TreeMap.empty[Long, ReplicatedData])
|
||||
val updatedEntriesForKey =
|
||||
deltaEntriesForKey.get(c) match {
|
||||
case Some(existingDelta) ⇒
|
||||
deltaEntriesForKey.updated(c, existingDelta.merge(delta.asInstanceOf[existingDelta.T]))
|
||||
case None ⇒
|
||||
deltaEntriesForKey.updated(c, delta)
|
||||
}
|
||||
deltaEntries = deltaEntries.updated(key, updatedEntriesForKey)
|
||||
}
|
||||
|
||||
def delete(key: String): Unit = {
|
||||
deltaEntries -= key
|
||||
deltaCounter -= key
|
||||
deltaSentToNode -= key
|
||||
}
|
||||
|
||||
def nodesSliceSize(allNodesSize: Int): Int = {
|
||||
// 2 - 10 nodes
|
||||
math.min(math.max((allNodesSize / divisor) + 1, 2), math.min(allNodesSize, 10))
|
||||
}
|
||||
|
||||
def collectPropagations(): Map[Address, DeltaPropagation] = {
|
||||
_propagationCount += 1
|
||||
val all = allNodes
|
||||
if (all.isEmpty)
|
||||
Map.empty
|
||||
else {
|
||||
// For each tick we pick a few nodes in round-robin fashion, 2 - 10 nodes for each tick.
|
||||
// Normally the delta is propagated to all nodes within the gossip tick, so that
|
||||
// full state gossip is not needed.
|
||||
val sliceSize = nodesSliceSize(all.size)
|
||||
val slice = {
|
||||
if (all.size <= sliceSize)
|
||||
all
|
||||
else {
|
||||
val i = (deltaNodeRoundRobinCounter % all.size).toInt
|
||||
val first = all.slice(i, i + sliceSize)
|
||||
if (first.size == sliceSize) first
|
||||
else first ++ all.take(sliceSize - first.size)
|
||||
}
|
||||
}
|
||||
deltaNodeRoundRobinCounter += sliceSize
|
||||
|
||||
var result = Map.empty[Address, DeltaPropagation]
|
||||
|
||||
slice.foreach { node ⇒
|
||||
// collect the deltas that have not already been sent to the node and merge
|
||||
// them into a delta group
|
||||
var deltas = Map.empty[String, ReplicatedData]
|
||||
deltaEntries.foreach {
|
||||
case (key, entries) ⇒
|
||||
val deltaSentToNodeForKey = deltaSentToNode.getOrElse(key, TreeMap.empty[Address, Long])
|
||||
val j = deltaSentToNodeForKey.getOrElse(node, 0L)
|
||||
val deltaEntriesAfterJ = deltaEntriesAfter(entries, j)
|
||||
if (deltaEntriesAfterJ.nonEmpty) {
|
||||
val deltaGroup = deltaEntriesAfterJ.valuesIterator.reduceLeft {
|
||||
(d1, d2) ⇒ d1.merge(d2.asInstanceOf[d1.T])
|
||||
}
|
||||
deltas = deltas.updated(key, deltaGroup)
|
||||
deltaSentToNode = deltaSentToNode.updated(key, deltaSentToNodeForKey.updated(node, deltaEntriesAfterJ.lastKey))
|
||||
}
|
||||
}
|
||||
|
||||
if (deltas.nonEmpty) {
|
||||
// Important to include the pruning state in the deltas. For example if the delta is based
|
||||
// on an entry that has been pruned but that has not yet been performed on the target node.
|
||||
val deltaPropagation = createDeltaPropagation(deltas)
|
||||
result = result.updated(node, deltaPropagation)
|
||||
}
|
||||
}
|
||||
|
||||
// increase the counter
|
||||
deltaCounter = deltaCounter.map {
|
||||
case (key, value) ⇒
|
||||
if (deltaEntries.contains(key))
|
||||
key → (value + 1)
|
||||
else
|
||||
key → value
|
||||
}
|
||||
|
||||
result
|
||||
}
|
||||
}
|
||||
|
||||
private def deltaEntriesAfter(entries: TreeMap[Long, ReplicatedData], version: Long): TreeMap[Long, ReplicatedData] =
|
||||
entries.from(version) match {
|
||||
case ntrs if ntrs.isEmpty ⇒ ntrs
|
||||
case ntrs if ntrs.firstKey == version ⇒ ntrs.tail // exclude first, i.e. version j that was already sent
|
||||
case ntrs ⇒ ntrs
|
||||
}
|
||||
|
||||
def hasDeltaEntries(key: String): Boolean = {
|
||||
deltaEntries.get(key) match {
|
||||
case Some(m) ⇒ m.nonEmpty
|
||||
case None ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
private def findSmallestVersionPropagatedToAllNodes(key: String, all: Vector[Address]): Long = {
|
||||
deltaSentToNode.get(key) match {
|
||||
case None ⇒ 0L
|
||||
case Some(deltaSentToNodeForKey) ⇒
|
||||
if (deltaSentToNodeForKey.isEmpty) 0L
|
||||
else if (all.exists(node ⇒ !deltaSentToNodeForKey.contains(node))) 0L
|
||||
else deltaSentToNodeForKey.valuesIterator.min
|
||||
}
|
||||
}
|
||||
|
||||
def cleanupDeltaEntries(): Unit = {
|
||||
val all = allNodes
|
||||
if (all.isEmpty)
|
||||
deltaEntries = Map.empty
|
||||
else {
|
||||
deltaEntries = deltaEntries.map {
|
||||
case (key, entries) ⇒
|
||||
val minVersion = findSmallestVersionPropagatedToAllNodes(key, all)
|
||||
|
||||
val deltaEntriesAfterMin = deltaEntriesAfter(entries, minVersion)
|
||||
|
||||
// TODO perhaps also remove oldest when deltaCounter are too far ahead (e.g. 10 cylces)
|
||||
|
||||
key → deltaEntriesAfterMin
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def cleanupRemovedNode(address: Address): Unit = {
|
||||
deltaSentToNode = deltaSentToNode.map {
|
||||
case (key, deltaSentToNodeForKey) ⇒
|
||||
key → (deltaSentToNodeForKey - address)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -39,8 +39,9 @@ object GCounter {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final class GCounter private[akka] (
|
||||
private[akka] val state: Map[UniqueAddress, BigInt] = Map.empty)
|
||||
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning with FastMerge {
|
||||
private[akka] val state: Map[UniqueAddress, BigInt] = Map.empty,
|
||||
private[akka] val _delta: Option[GCounter] = None)
|
||||
extends DeltaReplicatedData with ReplicatedDataSerialization with RemovedNodePruning with FastMerge {
|
||||
|
||||
import GCounter.Zero
|
||||
|
||||
|
|
@ -57,17 +58,17 @@ final class GCounter private[akka] (
|
|||
def getValue: BigInteger = value.bigInteger
|
||||
|
||||
/**
|
||||
* Increment the counter with the delta specified.
|
||||
* Increment the counter with the delta `n` specified.
|
||||
* The delta must be zero or positive.
|
||||
*/
|
||||
def +(delta: Long)(implicit node: Cluster): GCounter = increment(node, delta)
|
||||
def +(n: Long)(implicit node: Cluster): GCounter = increment(node, n)
|
||||
|
||||
/**
|
||||
* Increment the counter with the delta specified.
|
||||
* The delta must be zero or positive.
|
||||
* Increment the counter with the delta `n` specified.
|
||||
* The delta `n` must be zero or positive.
|
||||
*/
|
||||
def increment(node: Cluster, delta: Long = 1): GCounter =
|
||||
increment(node.selfUniqueAddress, delta)
|
||||
def increment(node: Cluster, n: Long = 1): GCounter =
|
||||
increment(node.selfUniqueAddress, n)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -77,14 +78,19 @@ final class GCounter private[akka] (
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def increment(key: UniqueAddress, delta: BigInt): GCounter = {
|
||||
require(delta >= 0, "Can't decrement a GCounter")
|
||||
if (delta == 0) this
|
||||
else state.get(key) match {
|
||||
case Some(v) ⇒
|
||||
val tot = v + delta
|
||||
assignAncestor(new GCounter(state + (key → tot)))
|
||||
case None ⇒ assignAncestor(new GCounter(state + (key → delta)))
|
||||
private[akka] def increment(key: UniqueAddress, n: BigInt): GCounter = {
|
||||
require(n >= 0, "Can't decrement a GCounter")
|
||||
if (n == 0) this
|
||||
else {
|
||||
val nextValue = state.get(key) match {
|
||||
case Some(v) ⇒ v + n
|
||||
case None ⇒ n
|
||||
}
|
||||
val newDelta = _delta match {
|
||||
case Some(d) ⇒ Some(new GCounter(d.state + (key → nextValue)))
|
||||
case None ⇒ Some(new GCounter(Map(key → nextValue)))
|
||||
}
|
||||
assignAncestor(new GCounter(state + (key → nextValue), newDelta))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -102,6 +108,13 @@ final class GCounter private[akka] (
|
|||
new GCounter(merged)
|
||||
}
|
||||
|
||||
override def delta: GCounter = _delta match {
|
||||
case Some(d) ⇒ d
|
||||
case None ⇒ GCounter.empty
|
||||
}
|
||||
|
||||
override def resetDelta: GCounter = new GCounter(state)
|
||||
|
||||
override def modifiedByNodes: Set[UniqueAddress] = state.keySet
|
||||
|
||||
override def needPruningFrom(removedNode: UniqueAddress): Boolean =
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ object PNCounter {
|
|||
@SerialVersionUID(1L)
|
||||
final class PNCounter private[akka] (
|
||||
private[akka] val increments: GCounter, private[akka] val decrements: GCounter)
|
||||
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
|
||||
extends DeltaReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
|
||||
|
||||
type T = PNCounter
|
||||
|
||||
|
|
@ -54,39 +54,39 @@ final class PNCounter private[akka] (
|
|||
def getValue: BigInteger = value.bigInteger
|
||||
|
||||
/**
|
||||
* Increment the counter with the delta specified.
|
||||
* Increment the counter with the delta `n` specified.
|
||||
* If the delta is negative then it will decrement instead of increment.
|
||||
*/
|
||||
def +(delta: Long)(implicit node: Cluster): PNCounter = increment(node, delta)
|
||||
def +(n: Long)(implicit node: Cluster): PNCounter = increment(node, n)
|
||||
|
||||
/**
|
||||
* Increment the counter with the delta specified.
|
||||
* Increment the counter with the delta `n` specified.
|
||||
* If the delta is negative then it will decrement instead of increment.
|
||||
*/
|
||||
def increment(node: Cluster, delta: Long = 1): PNCounter =
|
||||
increment(node.selfUniqueAddress, delta)
|
||||
def increment(node: Cluster, n: Long = 1): PNCounter =
|
||||
increment(node.selfUniqueAddress, n)
|
||||
|
||||
/**
|
||||
* Decrement the counter with the delta specified.
|
||||
* Decrement the counter with the delta `n` specified.
|
||||
* If the delta is negative then it will increment instead of decrement.
|
||||
*/
|
||||
def -(delta: Long)(implicit node: Cluster): PNCounter = decrement(node, delta)
|
||||
def -(n: Long)(implicit node: Cluster): PNCounter = decrement(node, n)
|
||||
|
||||
/**
|
||||
* Decrement the counter with the delta specified.
|
||||
* If the delta is negative then it will increment instead of decrement.
|
||||
* Decrement the counter with the delta `n` specified.
|
||||
* If the delta `n` is negative then it will increment instead of decrement.
|
||||
*/
|
||||
def decrement(node: Cluster, delta: Long = 1): PNCounter =
|
||||
decrement(node.selfUniqueAddress, delta)
|
||||
def decrement(node: Cluster, n: Long = 1): PNCounter =
|
||||
decrement(node.selfUniqueAddress, n)
|
||||
|
||||
private[akka] def increment(key: UniqueAddress, delta: Long): PNCounter = change(key, delta)
|
||||
private[akka] def increment(key: UniqueAddress, n: Long): PNCounter = change(key, n)
|
||||
private[akka] def increment(key: UniqueAddress): PNCounter = increment(key, 1)
|
||||
private[akka] def decrement(key: UniqueAddress, delta: Long): PNCounter = change(key, -delta)
|
||||
private[akka] def decrement(key: UniqueAddress, n: Long): PNCounter = change(key, -n)
|
||||
private[akka] def decrement(key: UniqueAddress): PNCounter = decrement(key, 1)
|
||||
|
||||
private[akka] def change(key: UniqueAddress, delta: Long): PNCounter =
|
||||
if (delta > 0) copy(increments = increments.increment(key, delta))
|
||||
else if (delta < 0) copy(decrements = decrements.increment(key, -delta))
|
||||
private[akka] def change(key: UniqueAddress, n: Long): PNCounter =
|
||||
if (n > 0) copy(increments = increments.increment(key, n))
|
||||
else if (n < 0) copy(decrements = decrements.increment(key, -n))
|
||||
else this
|
||||
|
||||
override def merge(that: PNCounter): PNCounter =
|
||||
|
|
@ -94,6 +94,10 @@ final class PNCounter private[akka] (
|
|||
increments = that.increments.merge(this.increments),
|
||||
decrements = that.decrements.merge(this.decrements))
|
||||
|
||||
override def delta: PNCounter = new PNCounter(increments.delta, decrements.delta)
|
||||
|
||||
override def resetDelta: PNCounter = new PNCounter(increments.resetDelta, decrements.resetDelta)
|
||||
|
||||
override def modifiedByNodes: Set[UniqueAddress] =
|
||||
increments.modifiedByNodes union decrements.modifiedByNodes
|
||||
|
||||
|
|
|
|||
|
|
@ -22,6 +22,9 @@ import akka.cluster.UniqueAddress
|
|||
*
|
||||
* ReplicatedData types should be immutable, i.e. "modifying" methods should return
|
||||
* a new instance.
|
||||
*
|
||||
* Implement the additional methods of [[DeltaReplicatedData]] if
|
||||
* it has support for delta-CRDT replication.
|
||||
*/
|
||||
trait ReplicatedData {
|
||||
/**
|
||||
|
|
@ -37,6 +40,42 @@ trait ReplicatedData {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* [[ReplicatedData]] with additional support for delta-CRDT replication.
|
||||
* delta-CRDT is a way to reduce the need for sending the full state
|
||||
* for updates. For example adding element 'c' and 'd' to set {'a', 'b'} would
|
||||
* result in sending the delta {'c', 'd'} and merge that with the state on the
|
||||
* receiving side, resulting in set {'a', 'b', 'c', 'd'}.
|
||||
*
|
||||
* Learn more about this in the paper
|
||||
* <a href="paper http://arxiv.org/abs/1603.01529">Delta State Replicated Data Types</a>.
|
||||
*/
|
||||
trait DeltaReplicatedData extends ReplicatedData {
|
||||
|
||||
/**
|
||||
* The accumulated delta of mutator operations since previous
|
||||
* [[#resetDelta]]. When the `Replicator` invokes the `modify` function
|
||||
* of the `Update` message and the user code is invoking one or more mutator
|
||||
* operations the data is collecting the delta of the operations and makes
|
||||
* it available for the `Replicator` with the [[#delta]] accessor. The
|
||||
* `modify` function shall still return the full state in the same way as
|
||||
* `ReplicatedData` without support for deltas.
|
||||
*/
|
||||
def delta: T
|
||||
|
||||
/**
|
||||
* Reset collection of deltas from mutator operations. When the `Replicator`
|
||||
* invokes the `modify` function of the `Update` message the delta is always
|
||||
* "reset" and when the user code is invoking one or more mutator operations the
|
||||
* data is collecting the delta of the operations and makes it available for
|
||||
* the `Replicator` with the [[#delta]] accessor. When the `Replicator` has
|
||||
* grabbed the `delta` it will invoke this method to get a clean data instance
|
||||
* without the delta.
|
||||
*/
|
||||
def resetDelta: T
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Interface for implementing a [[ReplicatedData]] in Java.
|
||||
*
|
||||
|
|
@ -61,6 +100,17 @@ abstract class AbstractReplicatedData[D <: AbstractReplicatedData[D]] extends Re
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Interface for implementing a [[DeltaReplicatedData]] in Java.
|
||||
*
|
||||
* The type parameter `D` is a self-recursive type to be defined by the
|
||||
* concrete implementation.
|
||||
* E.g. `class TwoPhaseSet extends AbstractDeltaReplicatedData<TwoPhaseSet>`
|
||||
*/
|
||||
abstract class AbstractDeltaReplicatedData[D <: AbstractDeltaReplicatedData[D]]
|
||||
extends AbstractReplicatedData[D] with DeltaReplicatedData {
|
||||
}
|
||||
|
||||
/**
|
||||
* [[ReplicatedData]] that has support for pruning of data
|
||||
* belonging to a specific node may implement this interface.
|
||||
|
|
|
|||
|
|
@ -44,6 +44,7 @@ import akka.actor.OneForOneStrategy
|
|||
import akka.actor.ActorInitializationException
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.util.Helpers.toRootLowerCase
|
||||
import akka.actor.Cancellable
|
||||
|
||||
object ReplicatorSettings {
|
||||
|
||||
|
|
@ -81,7 +82,8 @@ object ReplicatorSettings {
|
|||
durableStoreProps = Left((config.getString("durable.store-actor-class"), config.getConfig("durable"))),
|
||||
durableKeys = config.getStringList("durable.keys").asScala.toSet,
|
||||
pruningMarkerTimeToLive = config.getDuration("pruning-marker-time-to-live", MILLISECONDS).millis,
|
||||
durablePruningMarkerTimeToLive = config.getDuration("durable.pruning-marker-time-to-live", MILLISECONDS).millis)
|
||||
durablePruningMarkerTimeToLive = config.getDuration("durable.pruning-marker-time-to-live", MILLISECONDS).millis,
|
||||
deltaCrdtEnabled = config.getBoolean("delta-crdt.enabled"))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -128,20 +130,21 @@ final class ReplicatorSettings(
|
|||
val durableStoreProps: Either[(String, Config), Props],
|
||||
val durableKeys: Set[String],
|
||||
val pruningMarkerTimeToLive: FiniteDuration,
|
||||
val durablePruningMarkerTimeToLive: FiniteDuration) {
|
||||
val durablePruningMarkerTimeToLive: FiniteDuration,
|
||||
val deltaCrdtEnabled: Boolean) {
|
||||
|
||||
// For backwards compatibility
|
||||
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
|
||||
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration) =
|
||||
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
|
||||
maxPruningDissemination, Right(Props.empty), Set.empty, 6.hours, 10.days)
|
||||
maxPruningDissemination, Right(Props.empty), Set.empty, 6.hours, 10.days, true)
|
||||
|
||||
// For backwards compatibility
|
||||
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
|
||||
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration,
|
||||
durableStoreProps: Either[(String, Config), Props], durableKeys: Set[String]) =
|
||||
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
|
||||
maxPruningDissemination, durableStoreProps, durableKeys, 6.hours, 10.days)
|
||||
maxPruningDissemination, durableStoreProps, durableKeys, 6.hours, 10.days, true)
|
||||
|
||||
def withRole(role: String): ReplicatorSettings = copy(role = ReplicatorSettings.roleOption(role))
|
||||
|
||||
|
|
@ -191,6 +194,9 @@ final class ReplicatorSettings(
|
|||
withDurableKeys(durableKeys.asScala.toSet)
|
||||
}
|
||||
|
||||
def withDeltaCrdtEnabled(deltaCrdtEnabled: Boolean): ReplicatorSettings =
|
||||
copy(deltaCrdtEnabled = deltaCrdtEnabled)
|
||||
|
||||
private def copy(
|
||||
role: Option[String] = role,
|
||||
gossipInterval: FiniteDuration = gossipInterval,
|
||||
|
|
@ -202,10 +208,11 @@ final class ReplicatorSettings(
|
|||
durableStoreProps: Either[(String, Config), Props] = durableStoreProps,
|
||||
durableKeys: Set[String] = durableKeys,
|
||||
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
|
||||
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive): ReplicatorSettings =
|
||||
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
|
||||
deltaCrdtEnabled: Boolean = deltaCrdtEnabled): ReplicatorSettings =
|
||||
new ReplicatorSettings(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher,
|
||||
pruningInterval, maxPruningDissemination, durableStoreProps, durableKeys,
|
||||
pruningMarkerTimeToLive, durablePruningMarkerTimeToLive)
|
||||
pruningMarkerTimeToLive, durablePruningMarkerTimeToLive, deltaCrdtEnabled)
|
||||
}
|
||||
|
||||
object Replicator {
|
||||
|
|
@ -572,6 +579,7 @@ object Replicator {
|
|||
private[akka] object Internal {
|
||||
|
||||
case object GossipTick
|
||||
case object DeltaPropagationTick
|
||||
case object RemovedNodePruningTick
|
||||
case object ClockTick
|
||||
final case class Write(key: String, envelope: DataEnvelope) extends ReplicatorMessage
|
||||
|
|
@ -581,6 +589,8 @@ object Replicator {
|
|||
final case class ReadResult(envelope: Option[DataEnvelope]) extends ReplicatorMessage with DeadLetterSuppression
|
||||
final case class ReadRepair(key: String, envelope: DataEnvelope)
|
||||
case object ReadRepairAck
|
||||
// for testing purposes
|
||||
final case class TestFullStateGossip(enabled: Boolean)
|
||||
|
||||
// Gossip Status message contains SHA-1 digests of the data to determine when
|
||||
// to send the full data
|
||||
|
|
@ -691,6 +701,8 @@ object Replicator {
|
|||
}
|
||||
final case class Gossip(updatedData: Map[String, DataEnvelope], sendBack: Boolean) extends ReplicatorMessage
|
||||
|
||||
final case class DeltaPropagation(deltas: Map[String, DataEnvelope]) extends ReplicatorMessage
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -704,8 +716,8 @@ object Replicator {
|
|||
* The data types must be convergent CRDTs and implement [[ReplicatedData]], i.e.
|
||||
* they provide a monotonic merge function and the state changes always converge.
|
||||
*
|
||||
* You can use your own custom [[ReplicatedData]] types, and several types are provided
|
||||
* by this package, such as:
|
||||
* You can use your own custom [[ReplicatedData]] or [[DeltaReplicatedData]] types,
|
||||
* and several types are provided by this package, such as:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Counters: [[GCounter]], [[PNCounter]]</li>
|
||||
|
|
@ -726,7 +738,24 @@ object Replicator {
|
|||
* The `Replicator` actor must be started on each node in the cluster, or group of
|
||||
* nodes tagged with a specific role. It communicates with other `Replicator` instances
|
||||
* with the same path (without address) that are running on other nodes . For convenience it
|
||||
* can be used with the [[DistributedData]] extension.
|
||||
* can be used with the [[DistributedData]] extension but it can also be started as an ordinary
|
||||
* actor using the `Replicator.props`. If it is started as an ordinary actor it is important
|
||||
* that it is given the same name, started on same path, on all nodes.
|
||||
*
|
||||
* <a href="paper http://arxiv.org/abs/1603.01529">Delta State Replicated Data Types</a>
|
||||
* is supported. delta-CRDT is a way to reduce the need for sending the full state
|
||||
* for updates. For example adding element 'c' and 'd' to set {'a', 'b'} would
|
||||
* result in sending the delta {'c', 'd'} and merge that with the state on the
|
||||
* receiving side, resulting in set {'a', 'b', 'c', 'd'}.
|
||||
*
|
||||
* Current protocol for replicating the deltas does not support causal consistency.
|
||||
* It is only eventually consistent. This means that if elements 'c' and 'd' are
|
||||
* added in two separate `Update` operations these deltas may occasionally be propagated
|
||||
* to nodes in different order than the causal order of the updates. For this example it
|
||||
* can result in that set {'a', 'b', 'd'} can be seen before element 'c' is seen. Eventually
|
||||
* it will be {'a', 'b', 'c', 'd'}. If causal consistency is needed the delta propagation
|
||||
* should be disabled with configuration property
|
||||
* `akka.cluster.distributed-data.delta-crdt.enabled=off`.
|
||||
*
|
||||
* == Update ==
|
||||
*
|
||||
|
|
@ -910,6 +939,34 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
} else
|
||||
context.system.deadLetters // not used
|
||||
|
||||
val deltaPropagationSelector = new DeltaPropagationSelector {
|
||||
override val divisor = 5
|
||||
override def allNodes: Vector[Address] = {
|
||||
// TODO optimize, by maintaining a sorted instance variable instead
|
||||
nodes.union(weaklyUpNodes).toVector.sorted
|
||||
}
|
||||
|
||||
override def createDeltaPropagation(deltas: Map[String, ReplicatedData]): DeltaPropagation = {
|
||||
// Important to include the pruning state in the deltas. For example if the delta is based
|
||||
// on an entry that has been pruned but that has not yet been performed on the target node.
|
||||
DeltaPropagation(deltas.map {
|
||||
case (key, d) ⇒ getData(key) match {
|
||||
case Some(envelope) ⇒ key → envelope.copy(data = d)
|
||||
case None ⇒ key → DataEnvelope(d)
|
||||
}
|
||||
}(collection.breakOut))
|
||||
}
|
||||
}
|
||||
val deltaPropagationTask: Option[Cancellable] =
|
||||
if (deltaCrdtEnabled) {
|
||||
// Derive the deltaPropagationInterval from the gossipInterval.
|
||||
// Normally the delta is propagated to all nodes within the gossip tick, so that
|
||||
// full state gossip is not needed.
|
||||
val deltaPropagationInterval = (gossipInterval / deltaPropagationSelector.divisor).max(200.millis)
|
||||
Some(context.system.scheduler.schedule(deltaPropagationInterval, deltaPropagationInterval,
|
||||
self, DeltaPropagationTick))
|
||||
} else None
|
||||
|
||||
// cluster nodes, doesn't contain selfAddress
|
||||
var nodes: Set[Address] = Set.empty
|
||||
|
||||
|
|
@ -933,6 +990,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
// for splitting up gossip in chunks
|
||||
var statusCount = 0L
|
||||
var statusTotChunks = 0
|
||||
// possibility to disable Gossip for testing purpose
|
||||
var fullStateGossipEnabled = true
|
||||
|
||||
val subscribers = new mutable.HashMap[String, mutable.Set[ActorRef]] with mutable.MultiMap[String, ActorRef]
|
||||
val newSubscribers = new mutable.HashMap[String, mutable.Set[ActorRef]] with mutable.MultiMap[String, ActorRef]
|
||||
|
|
@ -965,6 +1024,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
override def postStop(): Unit = {
|
||||
cluster.unsubscribe(self)
|
||||
gossipTask.cancel()
|
||||
deltaPropagationTask.foreach(_.cancel())
|
||||
notifyTask.cancel()
|
||||
pruningTask.foreach(_.cancel())
|
||||
clockTask.cancel()
|
||||
|
|
@ -1011,7 +1071,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
case (key, d) ⇒
|
||||
write(key, d.dataEnvelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (newEnvelope.data ne d.dataEnvelope.data)
|
||||
if (newEnvelope ne d.dataEnvelope)
|
||||
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
|
||||
case None ⇒
|
||||
}
|
||||
|
|
@ -1030,6 +1090,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
case RemovedNodePruningTick | FlushChanges | GossipTick ⇒
|
||||
// ignore scheduled ticks when loading durable data
|
||||
case TestFullStateGossip(enabled) ⇒
|
||||
fullStateGossipEnabled = enabled
|
||||
case m @ (_: Read | _: Write | _: Status | _: Gossip) ⇒
|
||||
// ignore gossip and replication when loading durable data
|
||||
log.debug("ignoring message [{}] when loading durable data", m.getClass.getName)
|
||||
|
|
@ -1046,7 +1108,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
case Read(key) ⇒ receiveRead(key)
|
||||
case Write(key, envelope) ⇒ receiveWrite(key, envelope)
|
||||
case ReadRepair(key, envelope) ⇒ receiveReadRepair(key, envelope)
|
||||
case DeltaPropagation(deltas) ⇒ receiveDeltaPropagation(deltas)
|
||||
case FlushChanges ⇒ receiveFlushChanges()
|
||||
case DeltaPropagationTick ⇒ receiveDeltaPropagationTick()
|
||||
case GossipTick ⇒ receiveGossipTick()
|
||||
case ClockTick ⇒ receiveClockTick()
|
||||
case Status(otherDigests, chunk, totChunks) ⇒ receiveStatus(otherDigests, chunk, totChunks)
|
||||
|
|
@ -1066,6 +1130,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
case Delete(key, consistency, req) ⇒ receiveDelete(key, consistency, req)
|
||||
case RemovedNodePruningTick ⇒ receiveRemovedNodePruningTick()
|
||||
case GetReplicaCount ⇒ receiveGetReplicaCount()
|
||||
case TestFullStateGossip(enabled) ⇒ fullStateGossipEnabled = enabled
|
||||
}
|
||||
|
||||
def receiveGet(key: KeyR, consistency: ReadConsistency, req: Option[Any]): Unit = {
|
||||
|
|
@ -1103,13 +1168,28 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
localValue match {
|
||||
case Some(DataEnvelope(DeletedData, _)) ⇒ throw new DataDeleted(key, req)
|
||||
case Some(envelope @ DataEnvelope(existing, _)) ⇒
|
||||
envelope.merge(modify(Some(existing)).asInstanceOf[existing.T])
|
||||
case None ⇒ DataEnvelope(modify(None))
|
||||
modify(Some(existing)) match {
|
||||
case d: DeltaReplicatedData if deltaCrdtEnabled ⇒
|
||||
(envelope.merge(d.resetDelta.asInstanceOf[existing.T]), Some(d.delta))
|
||||
case d ⇒
|
||||
(envelope.merge(d.asInstanceOf[existing.T]), None)
|
||||
}
|
||||
case None ⇒ modify(None) match {
|
||||
case d: DeltaReplicatedData if deltaCrdtEnabled ⇒ (DataEnvelope(d.resetDelta), Some(d.delta))
|
||||
case d ⇒ (DataEnvelope(d), None)
|
||||
}
|
||||
}
|
||||
} match {
|
||||
case Success(envelope) ⇒
|
||||
log.debug("Received Update for key [{}], old data [{}], new data [{}]", key, localValue, envelope.data)
|
||||
case Success((envelope, delta)) ⇒
|
||||
log.debug("Received Update for key [{}], old data [{}], new data [{}], delta [{}]", key, localValue, envelope.data, delta)
|
||||
setData(key.id, envelope)
|
||||
|
||||
// handle the delta
|
||||
delta match {
|
||||
case Some(d) ⇒ deltaPropagationSelector.update(key.id, d)
|
||||
case None ⇒ // not DeltaReplicatedData
|
||||
}
|
||||
|
||||
val durable = isDurable(key.id)
|
||||
if (isLocalUpdate(writeConsistency)) {
|
||||
if (durable)
|
||||
|
|
@ -1118,8 +1198,12 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
else
|
||||
replyTo ! UpdateSuccess(key, req)
|
||||
} else {
|
||||
val writeEnvelope = delta match {
|
||||
case Some(d) ⇒ DataEnvelope(d)
|
||||
case None ⇒ envelope
|
||||
}
|
||||
val writeAggregator =
|
||||
context.actorOf(WriteAggregator.props(key, envelope, writeConsistency, req, nodes, unreachable, replyTo, durable)
|
||||
context.actorOf(WriteAggregator.props(key, writeEnvelope, writeConsistency, req, nodes, unreachable, replyTo, durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(key.id, new DurableDataEnvelope(envelope),
|
||||
|
|
@ -1176,13 +1260,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
Some(writeEnvelope2)
|
||||
}
|
||||
|
||||
def receiveReadRepair(key: String, writeEnvelope: DataEnvelope): Unit = {
|
||||
def writeAndStore(key: String, writeEnvelope: DataEnvelope): Unit = {
|
||||
write(key, writeEnvelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (isDurable(key))
|
||||
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
|
||||
case None ⇒
|
||||
}
|
||||
}
|
||||
|
||||
def receiveReadRepair(key: String, writeEnvelope: DataEnvelope): Unit = {
|
||||
writeAndStore(key, writeEnvelope)
|
||||
replyTo ! ReadRepairAck
|
||||
}
|
||||
|
||||
|
|
@ -1231,6 +1319,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
else LazyDigest
|
||||
|
||||
dataEntries = dataEntries.updated(key, (envelope, dig))
|
||||
if (envelope.data == DeletedData)
|
||||
deltaPropagationSelector.delete(key)
|
||||
}
|
||||
|
||||
def getDigest(key: String): Digest = {
|
||||
|
|
@ -1282,7 +1372,28 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
changed = Set.empty[String]
|
||||
}
|
||||
|
||||
def receiveGossipTick(): Unit = selectRandomNode(nodes.union(weaklyUpNodes).toVector) foreach gossipTo
|
||||
def receiveDeltaPropagationTick(): Unit = {
|
||||
deltaPropagationSelector.collectPropagations().foreach {
|
||||
case (node, deltaPropagation) ⇒
|
||||
// TODO split it to several DeltaPropagation if too many entries
|
||||
replica(node) ! deltaPropagation
|
||||
}
|
||||
if (deltaPropagationSelector.propagationCount % deltaPropagationSelector.divisor == 0)
|
||||
deltaPropagationSelector.cleanupDeltaEntries()
|
||||
}
|
||||
|
||||
def receiveDeltaPropagation(deltas: Map[String, DataEnvelope]): Unit = {
|
||||
if (log.isDebugEnabled)
|
||||
log.debug("Received DeltaPropagation from [{}], containing [{}]", sender().path.address, deltas.keys.mkString(", "))
|
||||
deltas.foreach {
|
||||
case (key, envelope) ⇒ writeAndStore(key, envelope)
|
||||
}
|
||||
}
|
||||
|
||||
def receiveGossipTick(): Unit = {
|
||||
if (fullStateGossipEnabled)
|
||||
selectRandomNode(nodes.union(weaklyUpNodes).toVector) foreach gossipTo
|
||||
}
|
||||
|
||||
def gossipTo(address: Address): Unit = {
|
||||
val to = replica(address)
|
||||
|
|
@ -1353,12 +1464,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
updatedData.foreach {
|
||||
case (key, envelope) ⇒
|
||||
val hadData = dataEntries.contains(key)
|
||||
write(key, envelope) match {
|
||||
case Some(newEnvelope) ⇒
|
||||
if (isDurable(key))
|
||||
durableStore ! Store(key, new DurableDataEnvelope(newEnvelope), None)
|
||||
case None ⇒
|
||||
}
|
||||
writeAndStore(key, envelope)
|
||||
if (sendBack) getData(key) match {
|
||||
case Some(d) ⇒
|
||||
if (hadData || d.pruning.nonEmpty)
|
||||
|
|
@ -1426,6 +1532,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
log.debug("adding removed node [{}] from MemberRemoved", m.uniqueAddress)
|
||||
removedNodes = removedNodes.updated(m.uniqueAddress, allReachableClockTime)
|
||||
unreachable -= m.address
|
||||
deltaPropagationSelector.cleanupRemovedNode(m.address)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -174,6 +174,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
val GossipManifest = "N"
|
||||
val WriteNackManifest = "O"
|
||||
val DurableDataEnvelopeManifest = "P"
|
||||
val DeltaPropagationManifest = "Q"
|
||||
|
||||
private val fromBinaryMap = collection.immutable.HashMap[String, Array[Byte] ⇒ AnyRef](
|
||||
GetManifest → getFromBinary,
|
||||
|
|
@ -190,6 +191,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
ReadResultManifest → readResultFromBinary,
|
||||
StatusManifest → statusFromBinary,
|
||||
GossipManifest → gossipFromBinary,
|
||||
DeltaPropagationManifest → deltaPropagationFromBinary,
|
||||
WriteNackManifest → (_ ⇒ WriteNack),
|
||||
DurableDataEnvelopeManifest → durableDataEnvelopeFromBinary)
|
||||
|
||||
|
|
@ -199,6 +201,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
case WriteAck ⇒ WriteAckManifest
|
||||
case _: Read ⇒ ReadManifest
|
||||
case _: ReadResult ⇒ ReadResultManifest
|
||||
case _: DeltaPropagation ⇒ DeltaPropagationManifest
|
||||
case _: Status ⇒ StatusManifest
|
||||
case _: Get[_] ⇒ GetManifest
|
||||
case _: GetSuccess[_] ⇒ GetSuccessManifest
|
||||
|
|
@ -221,6 +224,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
case m: Read ⇒ readCache.getOrAdd(m)
|
||||
case m: ReadResult ⇒ readResultToProto(m).toByteArray
|
||||
case m: Status ⇒ statusToProto(m).toByteArray
|
||||
case m: DeltaPropagation ⇒ deltaPropagationToProto(m).toByteArray
|
||||
case m: Get[_] ⇒ getToProto(m).toByteArray
|
||||
case m: GetSuccess[_] ⇒ getSuccessToProto(m).toByteArray
|
||||
case m: DurableDataEnvelope ⇒ durableDataEnvelopeToProto(m).toByteArray
|
||||
|
|
@ -281,6 +285,24 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
sendBack = gossip.getSendBack)
|
||||
}
|
||||
|
||||
private def deltaPropagationToProto(deltaPropagation: DeltaPropagation): dm.DeltaPropagation = {
|
||||
val b = dm.DeltaPropagation.newBuilder()
|
||||
val entries = deltaPropagation.deltas.foreach {
|
||||
case (key, data) ⇒
|
||||
b.addEntries(dm.DeltaPropagation.Entry.newBuilder().
|
||||
setKey(key).
|
||||
setEnvelope(dataEnvelopeToProto(data)))
|
||||
}
|
||||
b.build()
|
||||
}
|
||||
|
||||
private def deltaPropagationFromBinary(bytes: Array[Byte]): DeltaPropagation = {
|
||||
val deltaPropagation = dm.DeltaPropagation.parseFrom(bytes)
|
||||
DeltaPropagation(
|
||||
deltaPropagation.getEntriesList.asScala.map(e ⇒
|
||||
e.getKey → dataEnvelopeFromProto(e.getEnvelope))(breakOut))
|
||||
}
|
||||
|
||||
private def getToProto(get: Get[_]): dm.Get = {
|
||||
val consistencyValue = get.consistency match {
|
||||
case ReadLocal ⇒ 1
|
||||
|
|
|
|||
|
|
@ -0,0 +1,205 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.cluster.ddata
|
||||
|
||||
import java.util.concurrent.ThreadLocalRandom
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.ddata.Replicator._
|
||||
import akka.remote.testconductor.RoleName
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
object ReplicatorDeltaSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(ConfigFactory.parseString("""
|
||||
akka.actor.provider = "cluster"
|
||||
akka.log-dead-letters-during-shutdown = off
|
||||
"""))
|
||||
|
||||
testTransport(on = true)
|
||||
|
||||
sealed trait Op
|
||||
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
|
||||
|
||||
val timeout = 5.seconds
|
||||
val writeTwo = WriteTo(2, timeout)
|
||||
val writeMajority = WriteMajority(timeout)
|
||||
|
||||
val KeyA = PNCounterKey("A")
|
||||
val KeyB = PNCounterKey("B")
|
||||
val KeyC = PNCounterKey("C")
|
||||
|
||||
def generateOperations(): Vector[Op] = {
|
||||
val rnd = ThreadLocalRandom.current()
|
||||
|
||||
def consistency(): WriteConsistency = {
|
||||
rnd.nextInt(100) match {
|
||||
case n if n < 90 ⇒ WriteLocal
|
||||
case n if n < 95 ⇒ writeTwo
|
||||
case n if n < 100 ⇒ writeMajority
|
||||
}
|
||||
}
|
||||
|
||||
def key(): PNCounterKey = {
|
||||
rnd.nextInt(3) match {
|
||||
case 0 ⇒ KeyA
|
||||
case 1 ⇒ KeyB
|
||||
case 2 ⇒ KeyC
|
||||
}
|
||||
}
|
||||
|
||||
(0 to (20 + rnd.nextInt(10))).map { _ ⇒
|
||||
rnd.nextInt(3) match {
|
||||
case 0 ⇒ Delay(rnd.nextInt(500))
|
||||
case 1 ⇒ Incr(key(), rnd.nextInt(100), consistency())
|
||||
case 2 ⇒ Decr(key(), rnd.nextInt(10), consistency())
|
||||
}
|
||||
}.toVector
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class ReplicatorDeltaSpecMultiJvmNode1 extends ReplicatorDeltaSpec
|
||||
class ReplicatorDeltaSpecMultiJvmNode2 extends ReplicatorDeltaSpec
|
||||
class ReplicatorDeltaSpecMultiJvmNode3 extends ReplicatorDeltaSpec
|
||||
class ReplicatorDeltaSpecMultiJvmNode4 extends ReplicatorDeltaSpec
|
||||
|
||||
class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMultiNodeSpec with ImplicitSender {
|
||||
import Replicator._
|
||||
import ReplicatorDeltaSpec._
|
||||
|
||||
override def initialParticipants = roles.size
|
||||
|
||||
implicit val cluster = Cluster(system)
|
||||
val fullStateReplicator = system.actorOf(Replicator.props(
|
||||
ReplicatorSettings(system).withGossipInterval(1.second).withDeltaCrdtEnabled(false)), "fullStateReplicator")
|
||||
val deltaReplicator = {
|
||||
val r = system.actorOf(Replicator.props(ReplicatorSettings(system)), "deltaReplicator")
|
||||
r ! Replicator.Internal.TestFullStateGossip(enabled = false)
|
||||
r
|
||||
}
|
||||
|
||||
var afterCounter = 0
|
||||
def enterBarrierAfterTestStep(): Unit = {
|
||||
afterCounter += 1
|
||||
enterBarrier("after-" + afterCounter)
|
||||
}
|
||||
|
||||
def join(from: RoleName, to: RoleName): Unit = {
|
||||
runOn(from) {
|
||||
cluster join node(to).address
|
||||
}
|
||||
enterBarrier(from.name + "-joined")
|
||||
}
|
||||
|
||||
"delta-CRDT" must {
|
||||
"join cluster" in {
|
||||
join(first, first)
|
||||
join(second, first)
|
||||
join(third, first)
|
||||
join(fourth, first)
|
||||
|
||||
within(15.seconds) {
|
||||
awaitAssert {
|
||||
fullStateReplicator ! GetReplicaCount
|
||||
expectMsg(ReplicaCount(4))
|
||||
}
|
||||
}
|
||||
|
||||
enterBarrierAfterTestStep()
|
||||
}
|
||||
|
||||
"propagate delta" in {
|
||||
join(first, first)
|
||||
join(second, first)
|
||||
join(third, first)
|
||||
join(fourth, first)
|
||||
|
||||
within(15.seconds) {
|
||||
awaitAssert {
|
||||
fullStateReplicator ! GetReplicaCount
|
||||
expectMsg(ReplicaCount(4))
|
||||
}
|
||||
}
|
||||
enterBarrier("ready")
|
||||
|
||||
runOn(first) {
|
||||
fullStateReplicator ! Update(KeyA, PNCounter.empty, WriteLocal)(_ + 1)
|
||||
deltaReplicator ! Update(KeyA, PNCounter.empty, WriteLocal)(_ + 1)
|
||||
}
|
||||
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)
|
||||
}
|
||||
awaitAssert {
|
||||
val p = TestProbe()
|
||||
deltaReplicator.tell(Get(KeyA, ReadLocal), p.ref)
|
||||
p.expectMsgType[GetSuccess[PNCounter]].dataValue.getValue.intValue should be(1)
|
||||
}
|
||||
}
|
||||
|
||||
enterBarrierAfterTestStep()
|
||||
}
|
||||
|
||||
"be eventually consistent" in {
|
||||
val operations = generateOperations()
|
||||
log.debug(s"random operations on [${myself.name}]: ${operations.mkString(", ")}")
|
||||
try {
|
||||
// perform random operations with both delta and full-state replicators
|
||||
// and compare that the end result is the same
|
||||
|
||||
for (op ← operations) {
|
||||
log.debug("operation: {}", op)
|
||||
op match {
|
||||
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)
|
||||
case Decr(key, n, consistency) ⇒
|
||||
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ - n)
|
||||
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ - n)
|
||||
}
|
||||
}
|
||||
|
||||
enterBarrier("updated-2")
|
||||
|
||||
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
||||
within(5.seconds) {
|
||||
awaitAssert {
|
||||
val p = TestProbe()
|
||||
fullStateReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||
val fullStateValue = p.expectMsgType[GetSuccess[PNCounter]].dataValue
|
||||
deltaReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||
val deltaValue = p.expectMsgType[GetSuccess[PNCounter]].dataValue
|
||||
deltaValue should ===(fullStateValue)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
enterBarrierAfterTestStep()
|
||||
} catch {
|
||||
case e: Throwable ⇒
|
||||
info(s"random operations on [${myself.name}]: ${operations.mkString(", ")}")
|
||||
throw e
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -0,0 +1,45 @@
|
|||
/**
|
||||
* Copyright (C) 2017 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.cluster.ddata;
|
||||
|
||||
import akka.cluster.UniqueAddress;
|
||||
|
||||
public class JavaImplOfDeltaReplicatedData extends AbstractDeltaReplicatedData<JavaImplOfDeltaReplicatedData> implements
|
||||
RemovedNodePruning {
|
||||
|
||||
@Override
|
||||
public JavaImplOfDeltaReplicatedData mergeData(JavaImplOfDeltaReplicatedData other) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaImplOfDeltaReplicatedData delta() {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaImplOfDeltaReplicatedData resetDelta() {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public scala.collection.immutable.Set<UniqueAddress> modifiedByNodes() {
|
||||
return akka.japi.Util.immutableSeq(new java.util.ArrayList<UniqueAddress>()).toSet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needPruningFrom(UniqueAddress removedNode) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaImplOfDeltaReplicatedData prune(UniqueAddress removedNode, UniqueAddress collapseInto) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaImplOfDeltaReplicatedData pruningCleanup(UniqueAddress removedNode) {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
@ -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