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:
Patrik Nordwall 2017-01-04 15:25:12 +01:00
parent 2a9fa234a1
commit 3e7ffd6b96
18 changed files with 2408 additions and 98 deletions

View file

@ -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;

View file

@ -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.

View file

@ -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)
}
}
}

View file

@ -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 =

View file

@ -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

View file

@ -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&lt;TwoPhaseSet&gt;`
*/
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.

View file

@ -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)
}
}

View file

@ -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

View file

@ -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
}
}
}
}

View file

@ -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;
}
}

View file

@ -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)
}
}
}
}
}

View file

@ -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 {

View file

@ -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 {

View file

@ -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()),