causal delivery of deltas, #22188
* keep track of delta interval versions and skip deltas that are not consequtive, i.e. when some delta message was lost * send the delta versions in the full state gossip to sync up the expected versions after dropped deltas * implementation of deltas for ORSet * refactoring of the delta types to allow for different type for the delta and the full state * extensive tests * mima filter * performance optimizations * simple pruning of deltas * Java API * update documentation * KeyId type alias * Use InternalApi annotation
This commit is contained in:
parent
94afbee179
commit
b700b840d1
41 changed files with 5010 additions and 1950 deletions
File diff suppressed because it is too large
Load diff
File diff suppressed because it is too large
Load diff
|
|
@ -21,7 +21,21 @@ message ORSet {
|
||||||
repeated sint32 intElements = 4 [packed=true];
|
repeated sint32 intElements = 4 [packed=true];
|
||||||
repeated sint64 longElements = 5 [packed=true];
|
repeated sint64 longElements = 5 [packed=true];
|
||||||
repeated OtherMessage otherElements = 6;
|
repeated OtherMessage otherElements = 6;
|
||||||
|
}
|
||||||
|
|
||||||
|
message ORSetDeltaGroup {
|
||||||
|
message Entry {
|
||||||
|
required ORSetDeltaOp operation = 1;
|
||||||
|
required ORSet underlying = 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
repeated Entry entries = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
enum ORSetDeltaOp {
|
||||||
|
Add = 0;
|
||||||
|
Remove = 1;
|
||||||
|
Full = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
message Flag {
|
message Flag {
|
||||||
|
|
@ -48,14 +62,6 @@ message PNCounter {
|
||||||
required GCounter decrements = 2;
|
required GCounter decrements = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
message VersionVector {
|
|
||||||
message Entry {
|
|
||||||
required UniqueAddress node = 1;
|
|
||||||
required int64 version = 2;
|
|
||||||
}
|
|
||||||
repeated Entry entries = 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
message ORMap {
|
message ORMap {
|
||||||
message Entry {
|
message Entry {
|
||||||
optional string stringKey = 1;
|
optional string stringKey = 1;
|
||||||
|
|
|
||||||
|
|
@ -73,6 +73,7 @@ message DataEnvelope {
|
||||||
|
|
||||||
required OtherMessage data = 1;
|
required OtherMessage data = 1;
|
||||||
repeated PruningEntry pruning = 2;
|
repeated PruningEntry pruning = 2;
|
||||||
|
optional VersionVector deltaVersions = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
message Status {
|
message Status {
|
||||||
|
|
@ -100,9 +101,12 @@ message DeltaPropagation {
|
||||||
message Entry {
|
message Entry {
|
||||||
required string key = 1;
|
required string key = 1;
|
||||||
required DataEnvelope envelope = 2;
|
required DataEnvelope envelope = 2;
|
||||||
|
required int64 fromSeqNr = 3;
|
||||||
|
optional int64 toSeqNr = 4; // if not set then same as fromSequenceNr
|
||||||
}
|
}
|
||||||
|
|
||||||
repeated Entry entries = 1;
|
required UniqueAddress fromNode = 1;
|
||||||
|
repeated Entry entries = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
message UniqueAddress {
|
message UniqueAddress {
|
||||||
|
|
@ -117,6 +121,14 @@ message Address {
|
||||||
required uint32 port = 2;
|
required uint32 port = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message VersionVector {
|
||||||
|
message Entry {
|
||||||
|
required UniqueAddress node = 1;
|
||||||
|
required int64 version = 2;
|
||||||
|
}
|
||||||
|
repeated Entry entries = 1;
|
||||||
|
}
|
||||||
|
|
||||||
message OtherMessage {
|
message OtherMessage {
|
||||||
required bytes enclosedMessage = 1;
|
required bytes enclosedMessage = 1;
|
||||||
required int32 serializerId = 2;
|
required int32 serializerId = 2;
|
||||||
|
|
|
||||||
|
|
@ -4,48 +4,53 @@
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable.TreeMap
|
||||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
|
import akka.annotation.InternalApi
|
||||||
|
import akka.cluster.ddata.Key.KeyId
|
||||||
|
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API: Used by the Replicator actor.
|
* INTERNAL API: Used by the Replicator actor.
|
||||||
* Extracted to separate trait to make it easy to test.
|
* Extracted to separate trait to make it easy to test.
|
||||||
*/
|
*/
|
||||||
private[akka] trait DeltaPropagationSelector {
|
@InternalApi private[akka] trait DeltaPropagationSelector {
|
||||||
|
|
||||||
private var _propagationCount = 0L
|
private var _propagationCount = 0L
|
||||||
def propagationCount: Long = _propagationCount
|
def propagationCount: Long = _propagationCount
|
||||||
private var deltaCounter = Map.empty[String, Long]
|
private var deltaCounter = Map.empty[KeyId, Long]
|
||||||
private var deltaEntries = Map.empty[String, TreeMap[Long, ReplicatedData]]
|
private var deltaEntries = Map.empty[KeyId, TreeMap[Long, ReplicatedData]]
|
||||||
private var deltaSentToNode = Map.empty[String, Map[Address, Long]]
|
private var deltaSentToNode = Map.empty[KeyId, Map[Address, Long]]
|
||||||
private var deltaNodeRoundRobinCounter = 0L
|
private var deltaNodeRoundRobinCounter = 0L
|
||||||
|
|
||||||
def divisor: Int
|
def gossipIntervalDivisor: Int
|
||||||
|
|
||||||
def allNodes: Vector[Address]
|
def allNodes: Vector[Address]
|
||||||
|
|
||||||
def createDeltaPropagation(deltas: Map[String, ReplicatedData]): DeltaPropagation
|
def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation
|
||||||
|
|
||||||
def update(key: String, delta: ReplicatedData): Unit = {
|
def currentVersion(key: KeyId): Long = deltaCounter.get(key) match {
|
||||||
val c = deltaCounter.get(key) match {
|
case Some(v) ⇒ v
|
||||||
case Some(c) ⇒ c
|
case None ⇒ 0L
|
||||||
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 = {
|
def update(key: KeyId, delta: ReplicatedData): Unit = {
|
||||||
|
// bump the counter for each update
|
||||||
|
val version = deltaCounter.get(key) match {
|
||||||
|
case Some(c) ⇒ c + 1
|
||||||
|
case None ⇒ 1L
|
||||||
|
}
|
||||||
|
deltaCounter = deltaCounter.updated(key, version)
|
||||||
|
|
||||||
|
val deltaEntriesForKey = deltaEntries.get(key) match {
|
||||||
|
case Some(m) ⇒ m
|
||||||
|
case None ⇒ TreeMap.empty[Long, ReplicatedData]
|
||||||
|
}
|
||||||
|
|
||||||
|
deltaEntries = deltaEntries.updated(key, deltaEntriesForKey.updated(version, delta))
|
||||||
|
}
|
||||||
|
|
||||||
|
def delete(key: KeyId): Unit = {
|
||||||
deltaEntries -= key
|
deltaEntries -= key
|
||||||
deltaCounter -= key
|
deltaCounter -= key
|
||||||
deltaSentToNode -= key
|
deltaSentToNode -= key
|
||||||
|
|
@ -53,7 +58,7 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
|
|
||||||
def nodesSliceSize(allNodesSize: Int): Int = {
|
def nodesSliceSize(allNodesSize: Int): Int = {
|
||||||
// 2 - 10 nodes
|
// 2 - 10 nodes
|
||||||
math.min(math.max((allNodesSize / divisor) + 1, 2), math.min(allNodesSize, 10))
|
math.min(math.max((allNodesSize / gossipIntervalDivisor) + 1, 2), math.min(allNodesSize, 10))
|
||||||
}
|
}
|
||||||
|
|
||||||
def collectPropagations(): Map[Address, DeltaPropagation] = {
|
def collectPropagations(): Map[Address, DeltaPropagation] = {
|
||||||
|
|
@ -80,20 +85,32 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
|
|
||||||
var result = Map.empty[Address, DeltaPropagation]
|
var result = Map.empty[Address, DeltaPropagation]
|
||||||
|
|
||||||
|
var cache = Map.empty[(KeyId, Long, Long), ReplicatedData]
|
||||||
slice.foreach { node ⇒
|
slice.foreach { node ⇒
|
||||||
// collect the deltas that have not already been sent to the node and merge
|
// collect the deltas that have not already been sent to the node and merge
|
||||||
// them into a delta group
|
// them into a delta group
|
||||||
var deltas = Map.empty[String, ReplicatedData]
|
var deltas = Map.empty[KeyId, (ReplicatedData, Long, Long)]
|
||||||
deltaEntries.foreach {
|
deltaEntries.foreach {
|
||||||
case (key, entries) ⇒
|
case (key, entries) ⇒
|
||||||
val deltaSentToNodeForKey = deltaSentToNode.getOrElse(key, TreeMap.empty[Address, Long])
|
val deltaSentToNodeForKey = deltaSentToNode.getOrElse(key, TreeMap.empty[Address, Long])
|
||||||
val j = deltaSentToNodeForKey.getOrElse(node, 0L)
|
val j = deltaSentToNodeForKey.getOrElse(node, 0L)
|
||||||
val deltaEntriesAfterJ = deltaEntriesAfter(entries, j)
|
val deltaEntriesAfterJ = deltaEntriesAfter(entries, j)
|
||||||
if (deltaEntriesAfterJ.nonEmpty) {
|
if (deltaEntriesAfterJ.nonEmpty) {
|
||||||
val deltaGroup = deltaEntriesAfterJ.valuesIterator.reduceLeft {
|
val fromSeqNr = deltaEntriesAfterJ.head._1
|
||||||
|
val toSeqNr = deltaEntriesAfterJ.last._1
|
||||||
|
// in most cases the delta group merging will be the same for each node,
|
||||||
|
// so we cache the merged results
|
||||||
|
val cacheKey = (key, fromSeqNr, toSeqNr)
|
||||||
|
val deltaGroup = cache.get(cacheKey) match {
|
||||||
|
case None ⇒
|
||||||
|
val group = deltaEntriesAfterJ.valuesIterator.reduceLeft {
|
||||||
(d1, d2) ⇒ d1.merge(d2.asInstanceOf[d1.T])
|
(d1, d2) ⇒ d1.merge(d2.asInstanceOf[d1.T])
|
||||||
}
|
}
|
||||||
deltas = deltas.updated(key, deltaGroup)
|
cache = cache.updated(cacheKey, group)
|
||||||
|
group
|
||||||
|
case Some(group) ⇒ group
|
||||||
|
}
|
||||||
|
deltas = deltas.updated(key, (deltaGroup, fromSeqNr, toSeqNr))
|
||||||
deltaSentToNode = deltaSentToNode.updated(key, deltaSentToNodeForKey.updated(node, deltaEntriesAfterJ.lastKey))
|
deltaSentToNode = deltaSentToNode.updated(key, deltaSentToNodeForKey.updated(node, deltaEntriesAfterJ.lastKey))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -106,15 +123,6 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// increase the counter
|
|
||||||
deltaCounter = deltaCounter.map {
|
|
||||||
case (key, value) ⇒
|
|
||||||
if (deltaEntries.contains(key))
|
|
||||||
key → (value + 1)
|
|
||||||
else
|
|
||||||
key → value
|
|
||||||
}
|
|
||||||
|
|
||||||
result
|
result
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -126,14 +134,14 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
case ntrs ⇒ ntrs
|
case ntrs ⇒ ntrs
|
||||||
}
|
}
|
||||||
|
|
||||||
def hasDeltaEntries(key: String): Boolean = {
|
def hasDeltaEntries(key: KeyId): Boolean = {
|
||||||
deltaEntries.get(key) match {
|
deltaEntries.get(key) match {
|
||||||
case Some(m) ⇒ m.nonEmpty
|
case Some(m) ⇒ m.nonEmpty
|
||||||
case None ⇒ false
|
case None ⇒ false
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def findSmallestVersionPropagatedToAllNodes(key: String, all: Vector[Address]): Long = {
|
private def findSmallestVersionPropagatedToAllNodes(key: KeyId, all: Vector[Address]): Long = {
|
||||||
deltaSentToNode.get(key) match {
|
deltaSentToNode.get(key) match {
|
||||||
case None ⇒ 0L
|
case None ⇒ 0L
|
||||||
case Some(deltaSentToNodeForKey) ⇒
|
case Some(deltaSentToNodeForKey) ⇒
|
||||||
|
|
@ -154,7 +162,7 @@ private[akka] trait DeltaPropagationSelector {
|
||||||
|
|
||||||
val deltaEntriesAfterMin = deltaEntriesAfter(entries, minVersion)
|
val deltaEntriesAfterMin = deltaEntriesAfter(entries, minVersion)
|
||||||
|
|
||||||
// TODO perhaps also remove oldest when deltaCounter are too far ahead (e.g. 10 cylces)
|
// TODO perhaps also remove oldest when deltaCounter is too far ahead (e.g. 10 cycles)
|
||||||
|
|
||||||
key → deltaEntriesAfterMin
|
key → deltaEntriesAfterMin
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -18,8 +18,9 @@ import akka.actor.ActorRef
|
||||||
import akka.actor.DeadLetterSuppression
|
import akka.actor.DeadLetterSuppression
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.ddata.Replicator.ReplicatorMessage
|
import akka.cluster.ddata.Key.KeyId
|
||||||
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
|
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
|
||||||
|
import akka.cluster.ddata.Replicator.ReplicatorMessage
|
||||||
import akka.io.DirectByteBufferPool
|
import akka.io.DirectByteBufferPool
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.serialization.SerializerWithStringManifest
|
import akka.serialization.SerializerWithStringManifest
|
||||||
|
|
@ -53,7 +54,7 @@ object DurableStore {
|
||||||
* should be used to signal success or failure of the operation to the contained
|
* should be used to signal success or failure of the operation to the contained
|
||||||
* `replyTo` actor.
|
* `replyTo` actor.
|
||||||
*/
|
*/
|
||||||
final case class Store(key: String, data: DurableDataEnvelope, reply: Option[StoreReply])
|
final case class Store(key: KeyId, data: DurableDataEnvelope, reply: Option[StoreReply])
|
||||||
final case class StoreReply(successMsg: Any, failureMsg: Any, replyTo: ActorRef)
|
final case class StoreReply(successMsg: Any, failureMsg: Any, replyTo: ActorRef)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -66,7 +67,7 @@ object DurableStore {
|
||||||
* will stop itself and the durable store.
|
* will stop itself and the durable store.
|
||||||
*/
|
*/
|
||||||
case object LoadAll
|
case object LoadAll
|
||||||
final case class LoadData(data: Map[String, DurableDataEnvelope])
|
final case class LoadData(data: Map[KeyId, DurableDataEnvelope])
|
||||||
case object LoadAllCompleted
|
case object LoadAllCompleted
|
||||||
class LoadFailed(message: String, cause: Throwable) extends RuntimeException(message, cause) {
|
class LoadFailed(message: String, cause: Throwable) extends RuntimeException(message, cause) {
|
||||||
def this(message: String) = this(message, null)
|
def this(message: String) = this(message, null)
|
||||||
|
|
@ -143,7 +144,7 @@ final class LmdbDurableStore(config: Config) extends Actor with ActorLogging {
|
||||||
}
|
}
|
||||||
|
|
||||||
// pending write behind
|
// pending write behind
|
||||||
val pending = new java.util.HashMap[String, DurableDataEnvelope]
|
val pending = new java.util.HashMap[KeyId, DurableDataEnvelope]
|
||||||
|
|
||||||
override def postRestart(reason: Throwable): Unit = {
|
override def postRestart(reason: Throwable): Unit = {
|
||||||
super.postRestart(reason)
|
super.postRestart(reason)
|
||||||
|
|
@ -227,7 +228,7 @@ final class LmdbDurableStore(config: Config) extends Actor with ActorLogging {
|
||||||
writeBehind()
|
writeBehind()
|
||||||
}
|
}
|
||||||
|
|
||||||
def dbPut(tx: OptionVal[Txn[ByteBuffer]], key: String, data: DurableDataEnvelope): Unit = {
|
def dbPut(tx: OptionVal[Txn[ByteBuffer]], key: KeyId, data: DurableDataEnvelope): Unit = {
|
||||||
try {
|
try {
|
||||||
keyBuffer.put(key.getBytes(ByteString.UTF_8)).flip()
|
keyBuffer.put(key.getBytes(ByteString.UTF_8)).flip()
|
||||||
val value = serializer.toBinary(data)
|
val value = serializer.toBinary(data)
|
||||||
|
|
|
||||||
|
|
@ -3,6 +3,8 @@
|
||||||
*/
|
*/
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*
|
*
|
||||||
|
|
@ -19,11 +21,11 @@ package akka.cluster.ddata
|
||||||
* i.e. if used outside the Replicator infrastructure, but the worst thing that can happen is that
|
* i.e. if used outside the Replicator infrastructure, but the worst thing that can happen is that
|
||||||
* a full merge is performed instead of the fast forward merge.
|
* a full merge is performed instead of the fast forward merge.
|
||||||
*/
|
*/
|
||||||
private[akka] trait FastMerge { self: ReplicatedData ⇒
|
@InternalApi private[akka] trait FastMerge { self: ReplicatedData ⇒
|
||||||
|
|
||||||
private var ancestor: FastMerge = null
|
private var ancestor: FastMerge = null
|
||||||
|
|
||||||
/** INTERNAL API: should be called from "updating" methods */
|
/** INTERNAL API: should be called from "updating" methods, and `resetDelta` */
|
||||||
private[akka] def assignAncestor(newData: T with FastMerge): T = {
|
private[akka] def assignAncestor(newData: T with FastMerge): T = {
|
||||||
newData.ancestor = if (this.ancestor eq null) this else this.ancestor
|
newData.ancestor = if (this.ancestor eq null) this else this.ancestor
|
||||||
this.ancestor = null // only one level, for GC
|
this.ancestor = null // only one level, for GC
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ package akka.cluster.ddata
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import java.math.BigInteger
|
import java.math.BigInteger
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
object GCounter {
|
object GCounter {
|
||||||
val empty: GCounter = new GCounter
|
val empty: GCounter = new GCounter
|
||||||
|
|
@ -40,12 +41,14 @@ object GCounter {
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final class GCounter private[akka] (
|
final class GCounter private[akka] (
|
||||||
private[akka] val state: Map[UniqueAddress, BigInt] = Map.empty,
|
private[akka] val state: Map[UniqueAddress, BigInt] = Map.empty,
|
||||||
private[akka] val _delta: Option[GCounter] = None)
|
override val delta: Option[GCounter] = None)
|
||||||
extends DeltaReplicatedData with ReplicatedDataSerialization with RemovedNodePruning with FastMerge {
|
extends DeltaReplicatedData with ReplicatedDelta
|
||||||
|
with ReplicatedDataSerialization with RemovedNodePruning with FastMerge {
|
||||||
|
|
||||||
import GCounter.Zero
|
import GCounter.Zero
|
||||||
|
|
||||||
type T = GCounter
|
type T = GCounter
|
||||||
|
type D = GCounter
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Current total value of the counter.
|
* Scala API: Current total value of the counter.
|
||||||
|
|
@ -73,12 +76,12 @@ final class GCounter private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def increment(key: UniqueAddress): GCounter = increment(key, 1)
|
@InternalApi private[akka] def increment(key: UniqueAddress): GCounter = increment(key, 1)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def increment(key: UniqueAddress, n: BigInt): GCounter = {
|
@InternalApi private[akka] def increment(key: UniqueAddress, n: BigInt): GCounter = {
|
||||||
require(n >= 0, "Can't decrement a GCounter")
|
require(n >= 0, "Can't decrement a GCounter")
|
||||||
if (n == 0) this
|
if (n == 0) this
|
||||||
else {
|
else {
|
||||||
|
|
@ -86,11 +89,11 @@ final class GCounter private[akka] (
|
||||||
case Some(v) ⇒ v + n
|
case Some(v) ⇒ v + n
|
||||||
case None ⇒ n
|
case None ⇒ n
|
||||||
}
|
}
|
||||||
val newDelta = _delta match {
|
val newDelta = delta match {
|
||||||
case Some(d) ⇒ Some(new GCounter(d.state + (key → nextValue)))
|
case None ⇒ new GCounter(Map(key → nextValue))
|
||||||
case None ⇒ Some(new GCounter(Map(key → nextValue)))
|
case Some(d) ⇒ new GCounter(d.state + (key → nextValue))
|
||||||
}
|
}
|
||||||
assignAncestor(new GCounter(state + (key → nextValue), newDelta))
|
assignAncestor(new GCounter(state + (key → nextValue), Some(newDelta)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -108,12 +111,13 @@ final class GCounter private[akka] (
|
||||||
new GCounter(merged)
|
new GCounter(merged)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def delta: GCounter = _delta match {
|
override def mergeDelta(thatDelta: GCounter): GCounter = merge(thatDelta)
|
||||||
case Some(d) ⇒ d
|
|
||||||
case None ⇒ GCounter.empty
|
|
||||||
}
|
|
||||||
|
|
||||||
override def resetDelta: GCounter = new GCounter(state)
|
override def zero: GCounter = GCounter.empty
|
||||||
|
|
||||||
|
override def resetDelta: GCounter =
|
||||||
|
if (delta.isEmpty) this
|
||||||
|
else assignAncestor(new GCounter(state))
|
||||||
|
|
||||||
override def modifiedByNodes: Set[UniqueAddress] = state.keySet
|
override def modifiedByNodes: Set[UniqueAddress] = state.keySet
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -29,10 +29,12 @@ object GSet {
|
||||||
* This class is immutable, i.e. "modifying" methods return a new instance.
|
* This class is immutable, i.e. "modifying" methods return a new instance.
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class GSet[A] private (elements: Set[A])(_delta: Option[GSet[A]])
|
final case class GSet[A] private (elements: Set[A])(override val delta: Option[GSet[A]])
|
||||||
extends DeltaReplicatedData with ReplicatedDataSerialization with FastMerge {
|
extends DeltaReplicatedData with ReplicatedDelta
|
||||||
|
with ReplicatedDataSerialization with FastMerge {
|
||||||
|
|
||||||
type T = GSet[A]
|
type T = GSet[A]
|
||||||
|
type D = GSet[A]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API
|
* Java API
|
||||||
|
|
@ -57,7 +59,7 @@ final case class GSet[A] private (elements: Set[A])(_delta: Option[GSet[A]])
|
||||||
* Adds an element to the set
|
* Adds an element to the set
|
||||||
*/
|
*/
|
||||||
def add(element: A): GSet[A] = {
|
def add(element: A): GSet[A] = {
|
||||||
val newDelta = _delta match {
|
val newDelta = delta match {
|
||||||
case Some(e) ⇒ Some(new GSet(e.elements + element)(None))
|
case Some(e) ⇒ Some(new GSet(e.elements + element)(None))
|
||||||
case None ⇒ Some(new GSet[A](Set.apply[A](element))(None))
|
case None ⇒ Some(new GSet[A](Set.apply[A](element))(None))
|
||||||
}
|
}
|
||||||
|
|
@ -72,16 +74,17 @@ final case class GSet[A] private (elements: Set[A])(_delta: Option[GSet[A]])
|
||||||
new GSet[A](elements union that.elements)(None)
|
new GSet[A](elements union that.elements)(None)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def delta: GSet[A] = _delta match {
|
override def mergeDelta(thatDelta: GSet[A]): GSet[A] = merge(thatDelta)
|
||||||
case Some(d) ⇒ d
|
|
||||||
case None ⇒ GSet.empty[A]
|
|
||||||
}
|
|
||||||
|
|
||||||
override def resetDelta: GSet[A] = new GSet[A](elements)(None)
|
override def zero: GSet[A] = GSet.empty
|
||||||
|
|
||||||
|
override def resetDelta: GSet[A] =
|
||||||
|
if (delta.isEmpty) this
|
||||||
|
else assignAncestor(new GSet[A](elements)(None))
|
||||||
|
|
||||||
override def toString: String = s"G$elements"
|
override def toString: String = s"G$elements"
|
||||||
|
|
||||||
def copy(e: Set[A] = elements) = new GSet[A](e)(_delta)
|
def copy(e: Set[A] = elements) = new GSet[A](e)(delta)
|
||||||
}
|
}
|
||||||
|
|
||||||
object GSetKey {
|
object GSetKey {
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,8 @@ object Key {
|
||||||
|
|
||||||
private[akka]type KeyR = Key[ReplicatedData]
|
private[akka]type KeyR = Key[ReplicatedData]
|
||||||
|
|
||||||
|
type KeyId = String
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -21,7 +23,7 @@ object Key {
|
||||||
* Specific classes are provided for the built in data types, e.g. [[ORSetKey]],
|
* Specific classes are provided for the built in data types, e.g. [[ORSetKey]],
|
||||||
* and you can create your own keys.
|
* and you can create your own keys.
|
||||||
*/
|
*/
|
||||||
abstract class Key[+T <: ReplicatedData](val id: String) extends Serializable {
|
abstract class Key[+T <: ReplicatedData](val id: Key.KeyId) extends Serializable {
|
||||||
|
|
||||||
override final def equals(o: Any): Boolean = o match {
|
override final def equals(o: Any): Boolean = o match {
|
||||||
case k: Key[_] ⇒ id == k.id
|
case k: Key[_] ⇒ id == k.id
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,7 @@ package akka.cluster.ddata
|
||||||
|
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
object LWWMap {
|
object LWWMap {
|
||||||
private val _empty: LWWMap[Any, Any] = new LWWMap(ORMap.empty)
|
private val _empty: LWWMap[Any, Any] = new LWWMap(ORMap.empty)
|
||||||
|
|
@ -111,7 +112,7 @@ final class LWWMap[A, B] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def put(node: UniqueAddress, key: A, value: B, clock: Clock[B]): LWWMap[A, B] = {
|
@InternalApi private[akka] def put(node: UniqueAddress, key: A, value: B, clock: Clock[B]): LWWMap[A, B] = {
|
||||||
val newRegister = underlying.get(key) match {
|
val newRegister = underlying.get(key) match {
|
||||||
case Some(r) ⇒ r.withValue(node, value, clock)
|
case Some(r) ⇒ r.withValue(node, value, clock)
|
||||||
case None ⇒ LWWRegister(node, value, clock)
|
case None ⇒ LWWRegister(node, value, clock)
|
||||||
|
|
@ -137,7 +138,7 @@ final class LWWMap[A, B] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def remove(node: UniqueAddress, key: A): LWWMap[A, B] =
|
@InternalApi private[akka] def remove(node: UniqueAddress, key: A): LWWMap[A, B] =
|
||||||
new LWWMap(underlying.remove(node, key))
|
new LWWMap(underlying.remove(node, key))
|
||||||
|
|
||||||
override def merge(that: LWWMap[A, B]): LWWMap[A, B] =
|
override def merge(that: LWWMap[A, B]): LWWMap[A, B] =
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ package akka.cluster.ddata
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import akka.util.HashCode
|
import akka.util.HashCode
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
object LWWRegister {
|
object LWWRegister {
|
||||||
|
|
||||||
|
|
@ -43,7 +44,7 @@ object LWWRegister {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def apply[A](node: UniqueAddress, initialValue: A, clock: Clock[A]): LWWRegister[A] =
|
@InternalApi private[akka] def apply[A](node: UniqueAddress, initialValue: A, clock: Clock[A]): LWWRegister[A] =
|
||||||
new LWWRegister(node, initialValue, clock(0L, initialValue))
|
new LWWRegister(node, initialValue, clock(0L, initialValue))
|
||||||
|
|
||||||
def apply[A](initialValue: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
|
def apply[A](initialValue: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
|
||||||
|
|
@ -148,7 +149,7 @@ final class LWWRegister[A] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def withValue(node: UniqueAddress, value: A, clock: Clock[A]): LWWRegister[A] =
|
@InternalApi private[akka] def withValue(node: UniqueAddress, value: A, clock: Clock[A]): LWWRegister[A] =
|
||||||
new LWWRegister(node, value, clock(timestamp, value))
|
new LWWRegister(node, value, clock(timestamp, value))
|
||||||
|
|
||||||
override def merge(that: LWWRegister[A]): LWWRegister[A] =
|
override def merge(that: LWWRegister[A]): LWWRegister[A] =
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ package akka.cluster.ddata
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import akka.util.HashCode
|
import akka.util.HashCode
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
object ORMap {
|
object ORMap {
|
||||||
private val _empty: ORMap[Any, ReplicatedData] = new ORMap(ORSet.empty, Map.empty)
|
private val _empty: ORMap[Any, ReplicatedData] = new ORMap(ORSet.empty, Map.empty)
|
||||||
|
|
@ -93,7 +94,7 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def put(node: UniqueAddress, key: A, value: B): ORMap[A, B] =
|
@InternalApi private[akka] def put(node: UniqueAddress, key: A, value: B): ORMap[A, B] =
|
||||||
if (value.isInstanceOf[ORSet[_]] && values.contains(key))
|
if (value.isInstanceOf[ORSet[_]] && values.contains(key))
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
"`ORMap.put` must not be used to replace an existing `ORSet` " +
|
"`ORMap.put` must not be used to replace an existing `ORSet` " +
|
||||||
|
|
@ -123,7 +124,7 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def updated(node: UniqueAddress, key: A, initial: B)(modify: B ⇒ B): ORMap[A, B] = {
|
@InternalApi private[akka] def updated(node: UniqueAddress, key: A, initial: B)(modify: B ⇒ B): ORMap[A, B] = {
|
||||||
val newValue = values.get(key) match {
|
val newValue = values.get(key) match {
|
||||||
case Some(old) ⇒ modify(old)
|
case Some(old) ⇒ modify(old)
|
||||||
case _ ⇒ modify(initial)
|
case _ ⇒ modify(initial)
|
||||||
|
|
@ -148,7 +149,7 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def remove(node: UniqueAddress, key: A): ORMap[A, B] = {
|
@InternalApi private[akka] def remove(node: UniqueAddress, key: A): ORMap[A, B] = {
|
||||||
new ORMap(keys.remove(node, key), values - key)
|
new ORMap(keys.remove(node, key), values - key)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,6 +4,7 @@
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import akka.cluster.{ UniqueAddress, Cluster }
|
import akka.cluster.{ UniqueAddress, Cluster }
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
object ORMultiMap {
|
object ORMultiMap {
|
||||||
|
|
||||||
|
|
@ -113,7 +114,7 @@ final class ORMultiMap[A, B] private[akka] (private[akka] val underlying: ORMap[
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def put(node: UniqueAddress, key: A, value: Set[B]): ORMultiMap[A, B] = {
|
@InternalApi private[akka] def put(node: UniqueAddress, key: A, value: Set[B]): ORMultiMap[A, B] = {
|
||||||
val newUnderlying = underlying.updated(node, key, ORSet.empty[B]) { existing ⇒
|
val newUnderlying = underlying.updated(node, key, ORSet.empty[B]) { existing ⇒
|
||||||
value.foldLeft(existing.clear(node)) { (s, element) ⇒ s.add(node, element) }
|
value.foldLeft(existing.clear(node)) { (s, element) ⇒ s.add(node, element) }
|
||||||
}
|
}
|
||||||
|
|
@ -136,7 +137,7 @@ final class ORMultiMap[A, B] private[akka] (private[akka] val underlying: ORMap[
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def remove(node: UniqueAddress, key: A): ORMultiMap[A, B] =
|
@InternalApi private[akka] def remove(node: UniqueAddress, key: A): ORMultiMap[A, B] =
|
||||||
new ORMultiMap(underlying.remove(node, key))
|
new ORMultiMap(underlying.remove(node, key))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -154,7 +155,7 @@ final class ORMultiMap[A, B] private[akka] (private[akka] val underlying: ORMap[
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def addBinding(node: UniqueAddress, key: A, element: B): ORMultiMap[A, B] = {
|
@InternalApi private[akka] def addBinding(node: UniqueAddress, key: A, element: B): ORMultiMap[A, B] = {
|
||||||
val newUnderlying = underlying.updated(node, key, ORSet.empty[B])(_.add(node, element))
|
val newUnderlying = underlying.updated(node, key, ORSet.empty[B])(_.add(node, element))
|
||||||
new ORMultiMap(newUnderlying)
|
new ORMultiMap(newUnderlying)
|
||||||
}
|
}
|
||||||
|
|
@ -176,7 +177,7 @@ final class ORMultiMap[A, B] private[akka] (private[akka] val underlying: ORMap[
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def removeBinding(node: UniqueAddress, key: A, element: B): ORMultiMap[A, B] = {
|
@InternalApi private[akka] def removeBinding(node: UniqueAddress, key: A, element: B): ORMultiMap[A, B] = {
|
||||||
val newUnderlying = {
|
val newUnderlying = {
|
||||||
val u = underlying.updated(node, key, ORSet.empty[B])(_.remove(node, element))
|
val u = underlying.updated(node, key, ORSet.empty[B])(_.remove(node, element))
|
||||||
u.get(key) match {
|
u.get(key) match {
|
||||||
|
|
@ -198,7 +199,7 @@ final class ORMultiMap[A, B] private[akka] (private[akka] val underlying: ORMap[
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def replaceBinding(node: UniqueAddress, key: A, oldElement: B, newElement: B): ORMultiMap[A, B] =
|
@InternalApi private[akka] def replaceBinding(node: UniqueAddress, key: A, oldElement: B, newElement: B): ORMultiMap[A, B] =
|
||||||
if (newElement != oldElement)
|
if (newElement != oldElement)
|
||||||
addBinding(node, key, newElement).removeBinding(node, key, oldElement)
|
addBinding(node, key, newElement).removeBinding(node, key, oldElement)
|
||||||
else
|
else
|
||||||
|
|
|
||||||
|
|
@ -4,10 +4,12 @@
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import akka.util.HashCode
|
import akka.util.HashCode
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
object ORSet {
|
object ORSet {
|
||||||
private val _empty: ORSet[Any] = new ORSet(Map.empty, VersionVector.empty)
|
private val _empty: ORSet[Any] = new ORSet(Map.empty, VersionVector.empty)
|
||||||
|
|
@ -34,7 +36,75 @@ object ORSet {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka]type Dot = VersionVector
|
@InternalApi private[akka]type Dot = VersionVector
|
||||||
|
|
||||||
|
sealed trait DeltaOp extends ReplicatedDelta with RequiresCausalDeliveryOfDeltas {
|
||||||
|
type T = DeltaOp
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*/
|
||||||
|
@InternalApi private[akka] sealed abstract class AtomicDeltaOp[A] extends DeltaOp {
|
||||||
|
def underlying: ORSet[A]
|
||||||
|
override def zero: ORSet[A] = ORSet.empty
|
||||||
|
}
|
||||||
|
|
||||||
|
/** INTERNAL API */
|
||||||
|
@InternalApi private[akka] final case class AddDeltaOp[A](underlying: ORSet[A]) extends AtomicDeltaOp[A] {
|
||||||
|
|
||||||
|
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||||
|
case AddDeltaOp(u) ⇒
|
||||||
|
// Note that we only merge deltas originating from the same node
|
||||||
|
AddDeltaOp(new ORSet(
|
||||||
|
concatElementsMap(u.elementsMap.asInstanceOf[Map[A, Dot]]),
|
||||||
|
underlying.vvector.merge(u.vvector)))
|
||||||
|
case _: AtomicDeltaOp[A] ⇒ DeltaGroup(Vector(this, that))
|
||||||
|
case DeltaGroup(ops) ⇒ DeltaGroup(this +: ops)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def concatElementsMap(thatMap: Map[A, Dot]): Map[A, Dot] = {
|
||||||
|
if (thatMap.size == 1) {
|
||||||
|
val head = thatMap.head
|
||||||
|
underlying.elementsMap.updated(head._1, head._2)
|
||||||
|
} else
|
||||||
|
underlying.elementsMap ++ thatMap
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** INTERNAL API */
|
||||||
|
@InternalApi private[akka] final case class RemoveDeltaOp[A](underlying: ORSet[A]) extends AtomicDeltaOp[A] {
|
||||||
|
if (underlying.size != 1)
|
||||||
|
throw new IllegalArgumentException(s"RemoveDeltaOp should contain one removed element, but was $underlying")
|
||||||
|
|
||||||
|
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||||
|
case _: AtomicDeltaOp[A] ⇒ DeltaGroup(Vector(this, that)) // keep it simple for removals
|
||||||
|
case DeltaGroup(ops) ⇒ DeltaGroup(this +: ops)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** INTERNAL API: Used for `clear` but could be used for other cases also */
|
||||||
|
@InternalApi private[akka] final case class FullStateDeltaOp[A](underlying: ORSet[A]) extends AtomicDeltaOp[A] {
|
||||||
|
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||||
|
case _: AtomicDeltaOp[A] ⇒ DeltaGroup(Vector(this, that))
|
||||||
|
case DeltaGroup(ops) ⇒ DeltaGroup(this +: ops)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final case class DeltaGroup[A](ops: immutable.IndexedSeq[DeltaOp]) extends DeltaOp {
|
||||||
|
override def merge(that: DeltaOp): DeltaOp = that match {
|
||||||
|
case thatAdd: AddDeltaOp[A] ⇒
|
||||||
|
// merge AddDeltaOp into last AddDeltaOp in the group, if possible
|
||||||
|
ops.last match {
|
||||||
|
case thisAdd: AddDeltaOp[A] ⇒ DeltaGroup(ops.dropRight(1) :+ thisAdd.merge(thatAdd))
|
||||||
|
case _ ⇒ DeltaGroup(ops :+ thatAdd)
|
||||||
|
}
|
||||||
|
case DeltaGroup(thatOps) ⇒ DeltaGroup(ops ++ thatOps)
|
||||||
|
case _ ⇒ DeltaGroup(ops :+ that)
|
||||||
|
}
|
||||||
|
|
||||||
|
override def zero: ORSet[A] = ORSet.empty
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -45,7 +115,7 @@ object ORSet {
|
||||||
* [{a, 4}, {b, 1}, {c, 1}, {d, 14}, {e, 5}, {f, 2}] =
|
* [{a, 4}, {b, 1}, {c, 1}, {d, 14}, {e, 5}, {f, 2}] =
|
||||||
* [{b, 2}, {g, 22}]
|
* [{b, 2}, {g, 22}]
|
||||||
*/
|
*/
|
||||||
private[akka] def subtractDots(dot: Dot, vvector: VersionVector): Dot = {
|
@InternalApi private[akka] def subtractDots(dot: Dot, vvector: VersionVector): Dot = {
|
||||||
|
|
||||||
@tailrec def dropDots(remaining: List[(UniqueAddress, Long)], acc: List[(UniqueAddress, Long)]): List[(UniqueAddress, Long)] =
|
@tailrec def dropDots(remaining: List[(UniqueAddress, Long)], acc: List[(UniqueAddress, Long)]): List[(UniqueAddress, Long)] =
|
||||||
remaining match {
|
remaining match {
|
||||||
|
|
@ -80,7 +150,7 @@ object ORSet {
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
* @see [[ORSet#merge]]
|
* @see [[ORSet#merge]]
|
||||||
*/
|
*/
|
||||||
private[akka] def mergeCommonKeys[A](commonKeys: Set[A], lhs: ORSet[A], rhs: ORSet[A]): Map[A, ORSet.Dot] =
|
@InternalApi private[akka] def mergeCommonKeys[A](commonKeys: Set[A], lhs: ORSet[A], rhs: ORSet[A]): Map[A, ORSet.Dot] =
|
||||||
mergeCommonKeys(commonKeys.iterator, lhs, rhs)
|
mergeCommonKeys(commonKeys.iterator, lhs, rhs)
|
||||||
|
|
||||||
private def mergeCommonKeys[A](commonKeys: Iterator[A], lhs: ORSet[A], rhs: ORSet[A]): Map[A, ORSet.Dot] = {
|
private def mergeCommonKeys[A](commonKeys: Iterator[A], lhs: ORSet[A], rhs: ORSet[A]): Map[A, ORSet.Dot] = {
|
||||||
|
|
@ -149,7 +219,8 @@ object ORSet {
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
* @see [[ORSet#merge]]
|
* @see [[ORSet#merge]]
|
||||||
*/
|
*/
|
||||||
private[akka] def mergeDisjointKeys[A](keys: Set[A], elementsMap: Map[A, ORSet.Dot], vvector: VersionVector,
|
@InternalApi private[akka] def mergeDisjointKeys[A](
|
||||||
|
keys: Set[A], elementsMap: Map[A, ORSet.Dot], vvector: VersionVector,
|
||||||
accumulator: Map[A, ORSet.Dot]): Map[A, ORSet.Dot] =
|
accumulator: Map[A, ORSet.Dot]): Map[A, ORSet.Dot] =
|
||||||
mergeDisjointKeys(keys.iterator, elementsMap, vvector, accumulator)
|
mergeDisjointKeys(keys.iterator, elementsMap, vvector, accumulator)
|
||||||
|
|
||||||
|
|
@ -201,10 +272,15 @@ object ORSet {
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final class ORSet[A] private[akka] (
|
final class ORSet[A] private[akka] (
|
||||||
private[akka] val elementsMap: Map[A, ORSet.Dot],
|
private[akka] val elementsMap: Map[A, ORSet.Dot],
|
||||||
private[akka] val vvector: VersionVector)
|
private[akka] val vvector: VersionVector,
|
||||||
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning with FastMerge {
|
override val delta: Option[ORSet.DeltaOp] = None)
|
||||||
|
extends DeltaReplicatedData
|
||||||
|
with ReplicatedDataSerialization with RemovedNodePruning with FastMerge {
|
||||||
|
|
||||||
|
import ORSet.{ AddDeltaOp, RemoveDeltaOp }
|
||||||
|
|
||||||
type T = ORSet[A]
|
type T = ORSet[A]
|
||||||
|
type D = ORSet.DeltaOp
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API
|
* Scala API
|
||||||
|
|
@ -238,10 +314,18 @@ final class ORSet[A] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def add(node: UniqueAddress, element: A): ORSet[A] = {
|
@InternalApi private[akka] def add(node: UniqueAddress, element: A): ORSet[A] = {
|
||||||
val newVvector = vvector + node
|
val newVvector = vvector + node
|
||||||
val newDot = VersionVector(node, newVvector.versionAt(node))
|
val newDot = VersionVector(node, newVvector.versionAt(node))
|
||||||
assignAncestor(new ORSet(elementsMap = elementsMap.updated(element, newDot), vvector = newVvector))
|
val newDelta = delta match {
|
||||||
|
case None ⇒
|
||||||
|
ORSet.AddDeltaOp(new ORSet(Map(element → newDot), newDot))
|
||||||
|
case Some(existing: ORSet.AddDeltaOp[A]) ⇒
|
||||||
|
existing.merge(ORSet.AddDeltaOp(new ORSet(Map(element → newDot), newDot)))
|
||||||
|
case Some(d) ⇒
|
||||||
|
d.merge(ORSet.AddDeltaOp(new ORSet(Map(element → newDot), newDot)))
|
||||||
|
}
|
||||||
|
assignAncestor(new ORSet(elementsMap.updated(element, newDot), newVvector, Some(newDelta)))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -257,8 +341,15 @@ final class ORSet[A] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def remove(node: UniqueAddress, element: A): ORSet[A] =
|
@InternalApi private[akka] def remove(node: UniqueAddress, element: A): ORSet[A] = {
|
||||||
assignAncestor(copy(elementsMap = elementsMap - element))
|
val deltaDot = VersionVector(node, vvector.versionAt(node))
|
||||||
|
val rmOp = ORSet.RemoveDeltaOp(new ORSet(Map(element → deltaDot), vvector))
|
||||||
|
val newDelta = delta match {
|
||||||
|
case None ⇒ rmOp
|
||||||
|
case Some(d) ⇒ d.merge(rmOp)
|
||||||
|
}
|
||||||
|
assignAncestor(copy(elementsMap = elementsMap - element, delta = Some(newDelta)))
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Removes all elements from the set, but keeps the history.
|
* Removes all elements from the set, but keeps the history.
|
||||||
|
|
@ -270,8 +361,15 @@ final class ORSet[A] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def clear(node: UniqueAddress): ORSet[A] =
|
@InternalApi private[akka] def clear(node: UniqueAddress): ORSet[A] = {
|
||||||
assignAncestor(copy(elementsMap = Map.empty))
|
val newFullState = new ORSet[A](elementsMap = Map.empty, vvector)
|
||||||
|
val clearOp = ORSet.FullStateDeltaOp(newFullState)
|
||||||
|
val newDelta = delta match {
|
||||||
|
case None ⇒ clearOp
|
||||||
|
case Some(d) ⇒ d.merge(clearOp)
|
||||||
|
}
|
||||||
|
assignAncestor(newFullState.copy(delta = Some(newDelta)))
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* When element is in this Set but not in that Set:
|
* When element is in this Set but not in that Set:
|
||||||
|
|
@ -289,15 +387,24 @@ final class ORSet[A] private[akka] (
|
||||||
override def merge(that: ORSet[A]): ORSet[A] = {
|
override def merge(that: ORSet[A]): ORSet[A] = {
|
||||||
if ((this eq that) || that.isAncestorOf(this)) this.clearAncestor()
|
if ((this eq that) || that.isAncestorOf(this)) this.clearAncestor()
|
||||||
else if (this.isAncestorOf(that)) that.clearAncestor()
|
else if (this.isAncestorOf(that)) that.clearAncestor()
|
||||||
else {
|
else dryMerge(that, addDeltaOp = false)
|
||||||
|
}
|
||||||
|
|
||||||
|
// share merge impl between full state merge and AddDeltaOp merge
|
||||||
|
private def dryMerge(that: ORSet[A], addDeltaOp: Boolean): ORSet[A] = {
|
||||||
val commonKeys =
|
val commonKeys =
|
||||||
if (this.elementsMap.size < that.elementsMap.size)
|
if (this.elementsMap.size < that.elementsMap.size)
|
||||||
this.elementsMap.keysIterator.filter(that.elementsMap.contains)
|
this.elementsMap.keysIterator.filter(that.elementsMap.contains)
|
||||||
else
|
else
|
||||||
that.elementsMap.keysIterator.filter(this.elementsMap.contains)
|
that.elementsMap.keysIterator.filter(this.elementsMap.contains)
|
||||||
val entries00 = ORSet.mergeCommonKeys(commonKeys, this, that)
|
val entries00 = ORSet.mergeCommonKeys(commonKeys, this, that)
|
||||||
|
val entries0 =
|
||||||
|
if (addDeltaOp)
|
||||||
|
entries00 ++ this.elementsMap.filter { case (elem, _) ⇒ !that.elementsMap.contains(elem) }
|
||||||
|
else {
|
||||||
val thisUniqueKeys = this.elementsMap.keysIterator.filterNot(that.elementsMap.contains)
|
val thisUniqueKeys = this.elementsMap.keysIterator.filterNot(that.elementsMap.contains)
|
||||||
val entries0 = ORSet.mergeDisjointKeys(thisUniqueKeys, this.elementsMap, that.vvector, entries00)
|
ORSet.mergeDisjointKeys(thisUniqueKeys, this.elementsMap, that.vvector, entries00)
|
||||||
|
}
|
||||||
val thatUniqueKeys = that.elementsMap.keysIterator.filterNot(this.elementsMap.contains)
|
val thatUniqueKeys = that.elementsMap.keysIterator.filterNot(this.elementsMap.contains)
|
||||||
val entries = ORSet.mergeDisjointKeys(thatUniqueKeys, that.elementsMap, this.vvector, entries0)
|
val entries = ORSet.mergeDisjointKeys(thatUniqueKeys, that.elementsMap, this.vvector, entries0)
|
||||||
val mergedVvector = this.vvector.merge(that.vvector)
|
val mergedVvector = this.vvector.merge(that.vvector)
|
||||||
|
|
@ -305,7 +412,46 @@ final class ORSet[A] private[akka] (
|
||||||
clearAncestor()
|
clearAncestor()
|
||||||
new ORSet(entries, mergedVvector)
|
new ORSet(entries, mergedVvector)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def mergeDelta(thatDelta: ORSet.DeltaOp): ORSet[A] = {
|
||||||
|
thatDelta match {
|
||||||
|
case d: ORSet.AddDeltaOp[A] ⇒ dryMerge(d.underlying, addDeltaOp = true)
|
||||||
|
case d: ORSet.RemoveDeltaOp[A] ⇒ mergeRemoveDelta(d)
|
||||||
|
case d: ORSet.FullStateDeltaOp[A] ⇒ dryMerge(d.underlying, addDeltaOp = false)
|
||||||
|
case ORSet.DeltaGroup(ops) ⇒
|
||||||
|
ops.foldLeft(this) {
|
||||||
|
case (acc, op: ORSet.AddDeltaOp[A]) ⇒ acc.dryMerge(op.underlying, addDeltaOp = true)
|
||||||
|
case (acc, op: ORSet.RemoveDeltaOp[A]) ⇒ acc.mergeRemoveDelta(op)
|
||||||
|
case (acc, op: ORSet.FullStateDeltaOp[A]) ⇒ acc.dryMerge(op.underlying, addDeltaOp = false)
|
||||||
|
case (acc, op: ORSet.DeltaGroup[A]) ⇒
|
||||||
|
throw new IllegalArgumentException("ORSet.DeltaGroup should not be nested")
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def mergeRemoveDelta(thatDelta: ORSet.RemoveDeltaOp[A]): ORSet[A] = {
|
||||||
|
val that = thatDelta.underlying
|
||||||
|
val (elem, thatDot) = that.elementsMap.head
|
||||||
|
val newElementsMap =
|
||||||
|
if (that.vvector > vvector || that.vvector == vvector)
|
||||||
|
elementsMap - elem
|
||||||
|
else {
|
||||||
|
elementsMap.get(elem) match {
|
||||||
|
case Some(thisDot) ⇒
|
||||||
|
if (thatDot == thisDot || thatDot > thisDot) elementsMap - elem
|
||||||
|
else elementsMap
|
||||||
|
case None ⇒
|
||||||
|
elementsMap
|
||||||
|
}
|
||||||
|
}
|
||||||
|
clearAncestor()
|
||||||
|
val newVvector = vvector.merge(that.vvector)
|
||||||
|
new ORSet(newElementsMap, newVvector)
|
||||||
|
}
|
||||||
|
|
||||||
|
override def resetDelta: ORSet[A] =
|
||||||
|
if (delta.isEmpty) this
|
||||||
|
else assignAncestor(new ORSet(elementsMap, vvector))
|
||||||
|
|
||||||
override def modifiedByNodes: Set[UniqueAddress] =
|
override def modifiedByNodes: Set[UniqueAddress] =
|
||||||
vvector.modifiedByNodes
|
vvector.modifiedByNodes
|
||||||
|
|
@ -339,8 +485,9 @@ final class ORSet[A] private[akka] (
|
||||||
new ORSet(updated, vvector.pruningCleanup(removedNode))
|
new ORSet(updated, vvector.pruningCleanup(removedNode))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def copy(elementsMap: Map[A, ORSet.Dot] = this.elementsMap, vvector: VersionVector = this.vvector): ORSet[A] =
|
private def copy(elementsMap: Map[A, ORSet.Dot] = this.elementsMap, vvector: VersionVector = this.vvector,
|
||||||
new ORSet(elementsMap, vvector)
|
delta: Option[ORSet.DeltaOp] = this.delta): ORSet[A] =
|
||||||
|
new ORSet(elementsMap, vvector, delta)
|
||||||
|
|
||||||
// this class cannot be a `case class` because we need different `unapply`
|
// this class cannot be a `case class` because we need different `unapply`
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -39,9 +39,11 @@ object PNCounter {
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final class PNCounter private[akka] (
|
final class PNCounter private[akka] (
|
||||||
private[akka] val increments: GCounter, private[akka] val decrements: GCounter)
|
private[akka] val increments: GCounter, private[akka] val decrements: GCounter)
|
||||||
extends DeltaReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
|
extends DeltaReplicatedData with ReplicatedDelta
|
||||||
|
with ReplicatedDataSerialization with RemovedNodePruning {
|
||||||
|
|
||||||
type T = PNCounter
|
type T = PNCounter
|
||||||
|
type D = PNCounter
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Current total value of the counter.
|
* Scala API: Current total value of the counter.
|
||||||
|
|
@ -94,9 +96,29 @@ final class PNCounter private[akka] (
|
||||||
increments = that.increments.merge(this.increments),
|
increments = that.increments.merge(this.increments),
|
||||||
decrements = that.decrements.merge(this.decrements))
|
decrements = that.decrements.merge(this.decrements))
|
||||||
|
|
||||||
override def delta: PNCounter = new PNCounter(increments.delta, decrements.delta)
|
override def delta: Option[PNCounter] = {
|
||||||
|
if (increments.delta.isEmpty && decrements.delta.isEmpty)
|
||||||
|
None
|
||||||
|
else {
|
||||||
|
val incrementsDelta = increments.delta match {
|
||||||
|
case Some(d) ⇒ d
|
||||||
|
case None ⇒ GCounter.empty
|
||||||
|
}
|
||||||
|
val decrementsDelta = decrements.delta match {
|
||||||
|
case Some(d) ⇒ d
|
||||||
|
case None ⇒ GCounter.empty
|
||||||
|
}
|
||||||
|
Some(new PNCounter(incrementsDelta, decrementsDelta))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
override def resetDelta: PNCounter = new PNCounter(increments.resetDelta, decrements.resetDelta)
|
override def mergeDelta(thatDelta: PNCounter): PNCounter = merge(thatDelta)
|
||||||
|
|
||||||
|
override def zero: PNCounter = PNCounter.empty
|
||||||
|
|
||||||
|
override def resetDelta: PNCounter =
|
||||||
|
if (increments.delta.isEmpty && decrements.delta.isEmpty) this
|
||||||
|
else new PNCounter(increments.resetDelta, decrements.resetDelta)
|
||||||
|
|
||||||
override def modifiedByNodes: Set[UniqueAddress] =
|
override def modifiedByNodes: Set[UniqueAddress] =
|
||||||
increments.modifiedByNodes union decrements.modifiedByNodes
|
increments.modifiedByNodes union decrements.modifiedByNodes
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ package akka.cluster.ddata
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import java.math.BigInteger
|
import java.math.BigInteger
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
object PNCounterMap {
|
object PNCounterMap {
|
||||||
def empty[A]: PNCounterMap[A] = new PNCounterMap(ORMap.empty)
|
def empty[A]: PNCounterMap[A] = new PNCounterMap(ORMap.empty)
|
||||||
|
|
@ -75,7 +76,7 @@ final class PNCounterMap[A] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def increment(node: UniqueAddress, key: A, delta: Long): PNCounterMap[A] =
|
@InternalApi private[akka] def increment(node: UniqueAddress, key: A, delta: Long): PNCounterMap[A] =
|
||||||
new PNCounterMap(underlying.updated(node, key, PNCounter())(_.increment(node, delta)))
|
new PNCounterMap(underlying.updated(node, key, PNCounter())(_.increment(node, delta)))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -95,7 +96,7 @@ final class PNCounterMap[A] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def decrement(node: UniqueAddress, key: A, delta: Long): PNCounterMap[A] = {
|
@InternalApi private[akka] def decrement(node: UniqueAddress, key: A, delta: Long): PNCounterMap[A] = {
|
||||||
new PNCounterMap(underlying.updated(node, key, PNCounter())(_.decrement(node, delta)))
|
new PNCounterMap(underlying.updated(node, key, PNCounter())(_.decrement(node, delta)))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -117,7 +118,7 @@ final class PNCounterMap[A] private[akka] (
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def remove(node: UniqueAddress, key: A): PNCounterMap[A] =
|
@InternalApi private[akka] def remove(node: UniqueAddress, key: A): PNCounterMap[A] =
|
||||||
new PNCounterMap(underlying.remove(node, key))
|
new PNCounterMap(underlying.remove(node, key))
|
||||||
|
|
||||||
override def merge(that: PNCounterMap[A]): PNCounterMap[A] =
|
override def merge(that: PNCounterMap[A]): PNCounterMap[A] =
|
||||||
|
|
|
||||||
|
|
@ -6,11 +6,12 @@ package akka.cluster.ddata
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.cluster.Member
|
import akka.cluster.Member
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] object PruningState {
|
@InternalApi private[akka] object PruningState {
|
||||||
final case class PruningInitialized(owner: UniqueAddress, seen: Set[Address]) extends PruningState {
|
final case class PruningInitialized(owner: UniqueAddress, seen: Set[Address]) extends PruningState {
|
||||||
override def addSeen(node: Address): PruningState = {
|
override def addSeen(node: Address): PruningState = {
|
||||||
if (seen(node) || owner.address == node) this
|
if (seen(node) || owner.address == node) this
|
||||||
|
|
@ -25,7 +26,7 @@ private[akka] object PruningState {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] sealed trait PruningState {
|
@InternalApi private[akka] sealed trait PruningState {
|
||||||
import PruningState._
|
import PruningState._
|
||||||
|
|
||||||
def merge(that: PruningState): PruningState =
|
def merge(that: PruningState): PruningState =
|
||||||
|
|
|
||||||
|
|
@ -4,6 +4,8 @@
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
|
import scala.compat.java8.OptionConverters._
|
||||||
|
import java.util.Optional
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Interface for implementing a state based convergent
|
* Interface for implementing a state based convergent
|
||||||
|
|
@ -52,6 +54,13 @@ trait ReplicatedData {
|
||||||
*/
|
*/
|
||||||
trait DeltaReplicatedData extends ReplicatedData {
|
trait DeltaReplicatedData extends ReplicatedData {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The type of the delta. To be specified by subclass.
|
||||||
|
* It may be the same type as `T` or a different type if needed.
|
||||||
|
* For example `GSet` uses the same type and `ORSet` uses different types.
|
||||||
|
*/
|
||||||
|
type D <: ReplicatedDelta
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The accumulated delta of mutator operations since previous
|
* The accumulated delta of mutator operations since previous
|
||||||
* [[#resetDelta]]. When the `Replicator` invokes the `modify` function
|
* [[#resetDelta]]. When the `Replicator` invokes the `modify` function
|
||||||
|
|
@ -61,7 +70,14 @@ trait DeltaReplicatedData extends ReplicatedData {
|
||||||
* `modify` function shall still return the full state in the same way as
|
* `modify` function shall still return the full state in the same way as
|
||||||
* `ReplicatedData` without support for deltas.
|
* `ReplicatedData` without support for deltas.
|
||||||
*/
|
*/
|
||||||
def delta: T
|
def delta: Option[D]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* When delta is merged into the full state this method is used.
|
||||||
|
* When the type `D` of the delta is of the same type as the full state `T`
|
||||||
|
* this method can be implemented by delegating to `merge`.
|
||||||
|
*/
|
||||||
|
def mergeDelta(thatDelta: D): T
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reset collection of deltas from mutator operations. When the `Replicator`
|
* Reset collection of deltas from mutator operations. When the `Replicator`
|
||||||
|
|
@ -76,14 +92,36 @@ trait DeltaReplicatedData extends ReplicatedData {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The delta must implement this type.
|
||||||
|
*/
|
||||||
|
trait ReplicatedDelta extends ReplicatedData {
|
||||||
|
/**
|
||||||
|
* The empty full state. This is used when a delta is received
|
||||||
|
* and no existing full state exists on the receiving side. Then
|
||||||
|
* the delta is merged into the `zero` to create the initial full state.
|
||||||
|
*/
|
||||||
|
def zero: DeltaReplicatedData
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marker that specifies that the deltas must be applied in causal order.
|
||||||
|
* There is some overhead of managing the causal delivery so it should only
|
||||||
|
* be used for types that need it.
|
||||||
|
*
|
||||||
|
* Note that if the full state type `T` is different from the delta type `D`
|
||||||
|
* it is the delta `D` that should be marked with this.
|
||||||
|
*/
|
||||||
|
trait RequiresCausalDeliveryOfDeltas extends ReplicatedDelta
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Interface for implementing a [[ReplicatedData]] in Java.
|
* Java API: Interface for implementing a [[ReplicatedData]] in Java.
|
||||||
*
|
*
|
||||||
* The type parameter `D` is a self-recursive type to be defined by the
|
* The type parameter `A` is a self-recursive type to be defined by the
|
||||||
* concrete implementation.
|
* concrete implementation.
|
||||||
* E.g. `class TwoPhaseSet extends AbstractReplicatedData<TwoPhaseSet>`
|
* E.g. `class TwoPhaseSet extends AbstractReplicatedData<TwoPhaseSet>`
|
||||||
*/
|
*/
|
||||||
abstract class AbstractReplicatedData[D <: AbstractReplicatedData[D]] extends ReplicatedData {
|
abstract class AbstractReplicatedData[A <: AbstractReplicatedData[A]] extends ReplicatedData {
|
||||||
|
|
||||||
override type T = ReplicatedData
|
override type T = ReplicatedData
|
||||||
|
|
||||||
|
|
@ -91,24 +129,57 @@ abstract class AbstractReplicatedData[D <: AbstractReplicatedData[D]] extends Re
|
||||||
* Delegates to [[#mergeData]], which must be implemented by subclass.
|
* Delegates to [[#mergeData]], which must be implemented by subclass.
|
||||||
*/
|
*/
|
||||||
final override def merge(that: ReplicatedData): ReplicatedData =
|
final override def merge(that: ReplicatedData): ReplicatedData =
|
||||||
mergeData(that.asInstanceOf[D])
|
mergeData(that.asInstanceOf[A])
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Monotonic merge function.
|
* Java API: Monotonic merge function.
|
||||||
*/
|
*/
|
||||||
def mergeData(that: D): D
|
def mergeData(that: A): A
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Interface for implementing a [[DeltaReplicatedData]] in Java.
|
* Java API: Interface for implementing a [[DeltaReplicatedData]] in Java.
|
||||||
*
|
*
|
||||||
* The type parameter `D` is a self-recursive type to be defined by the
|
* The type parameter `A` is a self-recursive type to be defined by the
|
||||||
* concrete implementation.
|
* concrete implementation.
|
||||||
* E.g. `class TwoPhaseSet extends AbstractDeltaReplicatedData<TwoPhaseSet>`
|
* E.g. `class TwoPhaseSet extends AbstractDeltaReplicatedData<TwoPhaseSet, TwoPhaseSet>`
|
||||||
*/
|
*/
|
||||||
abstract class AbstractDeltaReplicatedData[D <: AbstractDeltaReplicatedData[D]]
|
abstract class AbstractDeltaReplicatedData[A <: AbstractDeltaReplicatedData[A, B], B <: ReplicatedDelta]
|
||||||
extends AbstractReplicatedData[D] with DeltaReplicatedData {
|
extends AbstractReplicatedData[A] with DeltaReplicatedData {
|
||||||
|
|
||||||
|
override type D = ReplicatedDelta
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Delegates to [[#deltaData]], which must be implemented by subclass.
|
||||||
|
*/
|
||||||
|
final override def delta: Option[ReplicatedDelta] =
|
||||||
|
deltaData.asScala
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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 [[#deltaData]] accessor. The
|
||||||
|
* `modify` function shall still return the full state in the same way as
|
||||||
|
* `ReplicatedData` without support for deltas.
|
||||||
|
*/
|
||||||
|
def deltaData: Optional[B]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Delegates to [[#mergeDeltaData]], which must be implemented by subclass.
|
||||||
|
*/
|
||||||
|
final override def mergeDelta(that: ReplicatedDelta): ReplicatedData =
|
||||||
|
mergeDeltaData(that.asInstanceOf[B])
|
||||||
|
|
||||||
|
/**
|
||||||
|
* When delta is merged into the full state this method is used.
|
||||||
|
* When the type `D` of the delta is of the same type as the full state `T`
|
||||||
|
* this method can be implemented by delegating to `mergeData`.
|
||||||
|
*/
|
||||||
|
def mergeDeltaData(that: B): A
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -45,6 +45,9 @@ import akka.actor.ActorInitializationException
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
import akka.util.Helpers.toRootLowerCase
|
import akka.util.Helpers.toRootLowerCase
|
||||||
import akka.actor.Cancellable
|
import akka.actor.Cancellable
|
||||||
|
import scala.util.control.NonFatal
|
||||||
|
import akka.cluster.ddata.Key.KeyId
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
object ReplicatorSettings {
|
object ReplicatorSettings {
|
||||||
|
|
||||||
|
|
@ -89,7 +92,7 @@ object ReplicatorSettings {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def roleOption(role: String): Option[String] =
|
@InternalApi private[akka] def roleOption(role: String): Option[String] =
|
||||||
if (role == "") None else Option(role)
|
if (role == "") None else Option(role)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -128,7 +131,7 @@ final class ReplicatorSettings(
|
||||||
val pruningInterval: FiniteDuration,
|
val pruningInterval: FiniteDuration,
|
||||||
val maxPruningDissemination: FiniteDuration,
|
val maxPruningDissemination: FiniteDuration,
|
||||||
val durableStoreProps: Either[(String, Config), Props],
|
val durableStoreProps: Either[(String, Config), Props],
|
||||||
val durableKeys: Set[String],
|
val durableKeys: Set[KeyId],
|
||||||
val pruningMarkerTimeToLive: FiniteDuration,
|
val pruningMarkerTimeToLive: FiniteDuration,
|
||||||
val durablePruningMarkerTimeToLive: FiniteDuration,
|
val durablePruningMarkerTimeToLive: FiniteDuration,
|
||||||
val deltaCrdtEnabled: Boolean) {
|
val deltaCrdtEnabled: Boolean) {
|
||||||
|
|
@ -183,7 +186,7 @@ final class ReplicatorSettings(
|
||||||
/**
|
/**
|
||||||
* Scala API
|
* Scala API
|
||||||
*/
|
*/
|
||||||
def withDurableKeys(durableKeys: Set[String]): ReplicatorSettings =
|
def withDurableKeys(durableKeys: Set[KeyId]): ReplicatorSettings =
|
||||||
copy(durableKeys = durableKeys)
|
copy(durableKeys = durableKeys)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -206,7 +209,7 @@ final class ReplicatorSettings(
|
||||||
pruningInterval: FiniteDuration = pruningInterval,
|
pruningInterval: FiniteDuration = pruningInterval,
|
||||||
maxPruningDissemination: FiniteDuration = maxPruningDissemination,
|
maxPruningDissemination: FiniteDuration = maxPruningDissemination,
|
||||||
durableStoreProps: Either[(String, Config), Props] = durableStoreProps,
|
durableStoreProps: Either[(String, Config), Props] = durableStoreProps,
|
||||||
durableKeys: Set[String] = durableKeys,
|
durableKeys: Set[KeyId] = durableKeys,
|
||||||
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
|
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
|
||||||
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
|
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
|
||||||
deltaCrdtEnabled: Boolean = deltaCrdtEnabled): ReplicatorSettings =
|
deltaCrdtEnabled: Boolean = deltaCrdtEnabled): ReplicatorSettings =
|
||||||
|
|
@ -270,12 +273,12 @@ object Replicator {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] case object GetKeyIds
|
@InternalApi private[akka] case object GetKeyIds
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] final case class GetKeyIdsResult(keyIds: Set[String]) {
|
@InternalApi private[akka] final case class GetKeyIdsResult(keyIds: Set[KeyId]) {
|
||||||
/**
|
/**
|
||||||
* Java API
|
* Java API
|
||||||
*/
|
*/
|
||||||
|
|
@ -576,18 +579,18 @@ object Replicator {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] object Internal {
|
@InternalApi private[akka] object Internal {
|
||||||
|
|
||||||
case object GossipTick
|
case object GossipTick
|
||||||
case object DeltaPropagationTick
|
case object DeltaPropagationTick
|
||||||
case object RemovedNodePruningTick
|
case object RemovedNodePruningTick
|
||||||
case object ClockTick
|
case object ClockTick
|
||||||
final case class Write(key: String, envelope: DataEnvelope) extends ReplicatorMessage
|
final case class Write(key: KeyId, envelope: DataEnvelope) extends ReplicatorMessage
|
||||||
case object WriteAck extends ReplicatorMessage with DeadLetterSuppression
|
case object WriteAck extends ReplicatorMessage with DeadLetterSuppression
|
||||||
case object WriteNack extends ReplicatorMessage with DeadLetterSuppression
|
case object WriteNack extends ReplicatorMessage with DeadLetterSuppression
|
||||||
final case class Read(key: String) extends ReplicatorMessage
|
final case class Read(key: KeyId) extends ReplicatorMessage
|
||||||
final case class ReadResult(envelope: Option[DataEnvelope]) extends ReplicatorMessage with DeadLetterSuppression
|
final case class ReadResult(envelope: Option[DataEnvelope]) extends ReplicatorMessage with DeadLetterSuppression
|
||||||
final case class ReadRepair(key: String, envelope: DataEnvelope)
|
final case class ReadRepair(key: KeyId, envelope: DataEnvelope)
|
||||||
case object ReadRepairAck
|
case object ReadRepairAck
|
||||||
// for testing purposes
|
// for testing purposes
|
||||||
final case class TestFullStateGossip(enabled: Boolean)
|
final case class TestFullStateGossip(enabled: Boolean)
|
||||||
|
|
@ -604,11 +607,23 @@ object Replicator {
|
||||||
*/
|
*/
|
||||||
final case class DataEnvelope(
|
final case class DataEnvelope(
|
||||||
data: ReplicatedData,
|
data: ReplicatedData,
|
||||||
pruning: Map[UniqueAddress, PruningState] = Map.empty)
|
pruning: Map[UniqueAddress, PruningState] = Map.empty,
|
||||||
|
deltaVersions: VersionVector = VersionVector.empty)
|
||||||
extends ReplicatorMessage {
|
extends ReplicatorMessage {
|
||||||
|
|
||||||
import PruningState._
|
import PruningState._
|
||||||
|
|
||||||
|
def withoutDeltaVersions: DataEnvelope =
|
||||||
|
if (deltaVersions.isEmpty) this
|
||||||
|
else copy(deltaVersions = VersionVector.empty)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* We only use the deltaVersions to track versions per node, not for ordering comparisons,
|
||||||
|
* so we can just remove the entry for the removed node.
|
||||||
|
*/
|
||||||
|
private def cleanedDeltaVersions(from: UniqueAddress): VersionVector =
|
||||||
|
deltaVersions.pruningCleanup(from)
|
||||||
|
|
||||||
def needPruningFrom(removedNode: UniqueAddress): Boolean =
|
def needPruningFrom(removedNode: UniqueAddress): Boolean =
|
||||||
data match {
|
data match {
|
||||||
case r: RemovedNodePruning ⇒ r.needPruningFrom(removedNode)
|
case r: RemovedNodePruning ⇒ r.needPruningFrom(removedNode)
|
||||||
|
|
@ -616,7 +631,9 @@ object Replicator {
|
||||||
}
|
}
|
||||||
|
|
||||||
def initRemovedNodePruning(removed: UniqueAddress, owner: UniqueAddress): DataEnvelope = {
|
def initRemovedNodePruning(removed: UniqueAddress, owner: UniqueAddress): DataEnvelope = {
|
||||||
copy(pruning = pruning.updated(removed, PruningInitialized(owner, Set.empty)))
|
copy(
|
||||||
|
pruning = pruning.updated(removed, PruningInitialized(owner, Set.empty)),
|
||||||
|
deltaVersions = cleanedDeltaVersions(removed))
|
||||||
}
|
}
|
||||||
|
|
||||||
def prune(from: UniqueAddress, pruningPerformed: PruningPerformed): DataEnvelope = {
|
def prune(from: UniqueAddress, pruningPerformed: PruningPerformed): DataEnvelope = {
|
||||||
|
|
@ -626,7 +643,8 @@ object Replicator {
|
||||||
pruning(from) match {
|
pruning(from) match {
|
||||||
case PruningInitialized(owner, _) ⇒
|
case PruningInitialized(owner, _) ⇒
|
||||||
val prunedData = dataWithRemovedNodePruning.prune(from, owner)
|
val prunedData = dataWithRemovedNodePruning.prune(from, owner)
|
||||||
copy(data = prunedData, pruning = pruning.updated(from, pruningPerformed))
|
copy(data = prunedData, pruning = pruning.updated(from, pruningPerformed),
|
||||||
|
deltaVersions = cleanedDeltaVersions(from))
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
this
|
this
|
||||||
}
|
}
|
||||||
|
|
@ -659,13 +677,36 @@ object Replicator {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// cleanup both sides before merging, `merge((otherData: ReplicatedData)` will cleanup other.data
|
// cleanup and merge deltaVersions
|
||||||
copy(data = cleaned(data, filteredMergedPruning), pruning = filteredMergedPruning).merge(other.data)
|
val removedNodes = filteredMergedPruning.keys
|
||||||
|
val cleanedDV = removedNodes.foldLeft(deltaVersions) { (acc, node) ⇒ acc.pruningCleanup(node) }
|
||||||
|
val cleanedOtherDV = removedNodes.foldLeft(other.deltaVersions) { (acc, node) ⇒ acc.pruningCleanup(node) }
|
||||||
|
val mergedDeltaVersions = cleanedDV.merge(cleanedOtherDV)
|
||||||
|
|
||||||
|
// cleanup both sides before merging, `merge(otherData: ReplicatedData)` will cleanup other.data
|
||||||
|
copy(
|
||||||
|
data = cleaned(data, filteredMergedPruning),
|
||||||
|
deltaVersions = mergedDeltaVersions,
|
||||||
|
pruning = filteredMergedPruning).merge(other.data)
|
||||||
}
|
}
|
||||||
|
|
||||||
def merge(otherData: ReplicatedData): DataEnvelope =
|
def merge(otherData: ReplicatedData): DataEnvelope = {
|
||||||
if (otherData == DeletedData) DeletedEnvelope
|
if (otherData == DeletedData) DeletedEnvelope
|
||||||
else copy(data = data merge cleaned(otherData, pruning).asInstanceOf[data.T])
|
else {
|
||||||
|
val mergedData =
|
||||||
|
cleaned(otherData, pruning) match {
|
||||||
|
case d: ReplicatedDelta ⇒ data match {
|
||||||
|
case drd: DeltaReplicatedData ⇒ drd.mergeDelta(d.asInstanceOf[drd.D])
|
||||||
|
case _ ⇒ throw new IllegalArgumentException("Expected DeltaReplicatedData")
|
||||||
|
}
|
||||||
|
case c ⇒ data.merge(c.asInstanceOf[data.T])
|
||||||
|
}
|
||||||
|
if (data.getClass != mergedData.getClass)
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
s"Wrong type, existing type [${data.getClass.getName}], got [${mergedData.getClass.getName}]")
|
||||||
|
copy(data = mergedData)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private def cleaned(c: ReplicatedData, p: Map[UniqueAddress, PruningState]): ReplicatedData = p.foldLeft(c) {
|
private def cleaned(c: ReplicatedData, p: Map[UniqueAddress, PruningState]): ReplicatedData = p.foldLeft(c) {
|
||||||
case (c: RemovedNodePruning, (removed, _: PruningPerformed)) ⇒
|
case (c: RemovedNodePruning, (removed, _: PruningPerformed)) ⇒
|
||||||
|
|
@ -693,15 +734,16 @@ object Replicator {
|
||||||
override def merge(that: ReplicatedData): ReplicatedData = DeletedData
|
override def merge(that: ReplicatedData): ReplicatedData = DeletedData
|
||||||
}
|
}
|
||||||
|
|
||||||
final case class Status(digests: Map[String, Digest], chunk: Int, totChunks: Int) extends ReplicatorMessage {
|
final case class Status(digests: Map[KeyId, Digest], chunk: Int, totChunks: Int) extends ReplicatorMessage {
|
||||||
override def toString: String =
|
override def toString: String =
|
||||||
(digests.map {
|
(digests.map {
|
||||||
case (key, bytes) ⇒ key + " -> " + bytes.map(byte ⇒ f"$byte%02x").mkString("")
|
case (key, bytes) ⇒ key + " -> " + bytes.map(byte ⇒ f"$byte%02x").mkString("")
|
||||||
}).mkString("Status(", ", ", ")")
|
}).mkString("Status(", ", ", ")")
|
||||||
}
|
}
|
||||||
final case class Gossip(updatedData: Map[String, DataEnvelope], sendBack: Boolean) extends ReplicatorMessage
|
final case class Gossip(updatedData: Map[KeyId, DataEnvelope], sendBack: Boolean) extends ReplicatorMessage
|
||||||
|
|
||||||
final case class DeltaPropagation(deltas: Map[String, DataEnvelope]) extends ReplicatorMessage
|
final case class Delta(dataEnvelope: DataEnvelope, fromSeqNr: Long, toSeqNr: Long)
|
||||||
|
final case class DeltaPropagation(fromNode: UniqueAddress, deltas: Map[KeyId, Delta]) extends ReplicatorMessage
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -748,14 +790,13 @@ object Replicator {
|
||||||
* result in sending the delta {'c', 'd'} and merge that with the state on the
|
* result in sending the delta {'c', 'd'} and merge that with the state on the
|
||||||
* receiving side, resulting in set {'a', 'b', 'c', 'd'}.
|
* receiving side, resulting in set {'a', 'b', 'c', 'd'}.
|
||||||
*
|
*
|
||||||
* Current protocol for replicating the deltas does not support causal consistency.
|
* The protocol for replicating the deltas supports causal consistency if the data type
|
||||||
* It is only eventually consistent. This means that if elements 'c' and 'd' are
|
* is marked with [[RequiresCausalDeliveryOfDeltas]]. Otherwise it is only eventually
|
||||||
|
* consistent. Without causal consistency it means that if elements 'c' and 'd' are
|
||||||
* added in two separate `Update` operations these deltas may occasionally be propagated
|
* 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
|
* 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
|
* 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
|
* it will be {'a', 'b', 'c', 'd'}.
|
||||||
* should be disabled with configuration property
|
|
||||||
* `akka.cluster.distributed-data.delta-crdt.enabled=off`.
|
|
||||||
*
|
*
|
||||||
* == Update ==
|
* == Update ==
|
||||||
*
|
*
|
||||||
|
|
@ -940,19 +981,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
context.system.deadLetters // not used
|
context.system.deadLetters // not used
|
||||||
|
|
||||||
val deltaPropagationSelector = new DeltaPropagationSelector {
|
val deltaPropagationSelector = new DeltaPropagationSelector {
|
||||||
override val divisor = 5
|
override val gossipIntervalDivisor = 5
|
||||||
override def allNodes: Vector[Address] = {
|
override def allNodes: Vector[Address] = {
|
||||||
// TODO optimize, by maintaining a sorted instance variable instead
|
// TODO optimize, by maintaining a sorted instance variable instead
|
||||||
nodes.union(weaklyUpNodes).toVector.sorted
|
nodes.union(weaklyUpNodes).diff(unreachable).toVector.sorted
|
||||||
}
|
}
|
||||||
|
|
||||||
override def createDeltaPropagation(deltas: Map[String, ReplicatedData]): DeltaPropagation = {
|
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation = {
|
||||||
// Important to include the pruning state in the deltas. For example if the delta is based
|
// 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.
|
// on an entry that has been pruned but that has not yet been performed on the target node.
|
||||||
DeltaPropagation(deltas.map {
|
DeltaPropagation(selfUniqueAddress, deltas.map {
|
||||||
case (key, d) ⇒ getData(key) match {
|
case (key, (d, fromSeqNr, toSeqNr)) ⇒ getData(key) match {
|
||||||
case Some(envelope) ⇒ key → envelope.copy(data = d)
|
case Some(envelope) ⇒ key → Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
|
||||||
case None ⇒ key → DataEnvelope(d)
|
case None ⇒ key → Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
||||||
}
|
}
|
||||||
}(collection.breakOut))
|
}(collection.breakOut))
|
||||||
}
|
}
|
||||||
|
|
@ -962,7 +1003,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
// Derive the deltaPropagationInterval from the gossipInterval.
|
// Derive the deltaPropagationInterval from the gossipInterval.
|
||||||
// Normally the delta is propagated to all nodes within the gossip tick, so that
|
// Normally the delta is propagated to all nodes within the gossip tick, so that
|
||||||
// full state gossip is not needed.
|
// full state gossip is not needed.
|
||||||
val deltaPropagationInterval = (gossipInterval / deltaPropagationSelector.divisor).max(200.millis)
|
val deltaPropagationInterval = (gossipInterval / deltaPropagationSelector.gossipIntervalDivisor).max(200.millis)
|
||||||
Some(context.system.scheduler.schedule(deltaPropagationInterval, deltaPropagationInterval,
|
Some(context.system.scheduler.schedule(deltaPropagationInterval, deltaPropagationInterval,
|
||||||
self, DeltaPropagationTick))
|
self, DeltaPropagationTick))
|
||||||
} else None
|
} else None
|
||||||
|
|
@ -983,9 +1024,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
var unreachable = Set.empty[Address]
|
var unreachable = Set.empty[Address]
|
||||||
|
|
||||||
// the actual data
|
// the actual data
|
||||||
var dataEntries = Map.empty[String, (DataEnvelope, Digest)]
|
var dataEntries = Map.empty[KeyId, (DataEnvelope, Digest)]
|
||||||
// keys that have changed, Changed event published to subscribers on FlushChanges
|
// keys that have changed, Changed event published to subscribers on FlushChanges
|
||||||
var changed = Set.empty[String]
|
var changed = Set.empty[KeyId]
|
||||||
|
|
||||||
// for splitting up gossip in chunks
|
// for splitting up gossip in chunks
|
||||||
var statusCount = 0L
|
var statusCount = 0L
|
||||||
|
|
@ -993,9 +1034,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
// possibility to disable Gossip for testing purpose
|
// possibility to disable Gossip for testing purpose
|
||||||
var fullStateGossipEnabled = true
|
var fullStateGossipEnabled = true
|
||||||
|
|
||||||
val subscribers = new mutable.HashMap[String, mutable.Set[ActorRef]] with mutable.MultiMap[String, ActorRef]
|
val subscribers = new mutable.HashMap[KeyId, mutable.Set[ActorRef]] with mutable.MultiMap[KeyId, ActorRef]
|
||||||
val newSubscribers = new mutable.HashMap[String, mutable.Set[ActorRef]] with mutable.MultiMap[String, ActorRef]
|
val newSubscribers = new mutable.HashMap[KeyId, mutable.Set[ActorRef]] with mutable.MultiMap[KeyId, ActorRef]
|
||||||
var subscriptionKeys = Map.empty[String, KeyR]
|
var subscriptionKeys = Map.empty[KeyId, KeyR]
|
||||||
|
|
||||||
// To be able to do efficient stashing we use this field instead of sender().
|
// To be able to do efficient stashing we use this field instead of sender().
|
||||||
// Using internal buffer instead of Stash to avoid the overhead of the Stash mailbox.
|
// Using internal buffer instead of Stash to avoid the overhead of the Stash mailbox.
|
||||||
|
|
@ -1108,7 +1149,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
case Read(key) ⇒ receiveRead(key)
|
case Read(key) ⇒ receiveRead(key)
|
||||||
case Write(key, envelope) ⇒ receiveWrite(key, envelope)
|
case Write(key, envelope) ⇒ receiveWrite(key, envelope)
|
||||||
case ReadRepair(key, envelope) ⇒ receiveReadRepair(key, envelope)
|
case ReadRepair(key, envelope) ⇒ receiveReadRepair(key, envelope)
|
||||||
case DeltaPropagation(deltas) ⇒ receiveDeltaPropagation(deltas)
|
case DeltaPropagation(from, deltas) ⇒ receiveDeltaPropagation(from, deltas)
|
||||||
case FlushChanges ⇒ receiveFlushChanges()
|
case FlushChanges ⇒ receiveFlushChanges()
|
||||||
case DeltaPropagationTick ⇒ receiveDeltaPropagationTick()
|
case DeltaPropagationTick ⇒ receiveDeltaPropagationTick()
|
||||||
case GossipTick ⇒ receiveGossipTick()
|
case GossipTick ⇒ receiveGossipTick()
|
||||||
|
|
@ -1138,8 +1179,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
log.debug("Received Get for key [{}], local data [{}]", key, localValue)
|
log.debug("Received Get for key [{}], local data [{}]", key, localValue)
|
||||||
if (isLocalGet(consistency)) {
|
if (isLocalGet(consistency)) {
|
||||||
val reply = localValue match {
|
val reply = localValue match {
|
||||||
case Some(DataEnvelope(DeletedData, _)) ⇒ DataDeleted(key, req)
|
case Some(DataEnvelope(DeletedData, _, _)) ⇒ DataDeleted(key, req)
|
||||||
case Some(DataEnvelope(data, _)) ⇒ GetSuccess(key, req)(data)
|
case Some(DataEnvelope(data, _, _)) ⇒ GetSuccess(key, req)(data)
|
||||||
case None ⇒ NotFound(key, req)
|
case None ⇒ NotFound(key, req)
|
||||||
}
|
}
|
||||||
replyTo ! reply
|
replyTo ! reply
|
||||||
|
|
@ -1155,7 +1196,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
case _ ⇒ false
|
case _ ⇒ false
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveRead(key: String): Unit = {
|
def receiveRead(key: KeyId): Unit = {
|
||||||
replyTo ! ReadResult(getData(key))
|
replyTo ! ReadResult(getData(key))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1166,23 +1207,22 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
val localValue = getData(key.id)
|
val localValue = getData(key.id)
|
||||||
Try {
|
Try {
|
||||||
localValue match {
|
localValue match {
|
||||||
case Some(DataEnvelope(DeletedData, _)) ⇒ throw new DataDeleted(key, req)
|
case Some(DataEnvelope(DeletedData, _, _)) ⇒ throw new DataDeleted(key, req)
|
||||||
case Some(envelope @ DataEnvelope(existing, _)) ⇒
|
case Some(envelope @ DataEnvelope(existing, _, _)) ⇒
|
||||||
modify(Some(existing)) match {
|
modify(Some(existing)) match {
|
||||||
case d: DeltaReplicatedData if deltaCrdtEnabled ⇒
|
case d: DeltaReplicatedData if deltaCrdtEnabled ⇒
|
||||||
(envelope.merge(d.resetDelta.asInstanceOf[existing.T]), Some(d.delta))
|
(envelope.merge(d.resetDelta.asInstanceOf[existing.T]), d.delta)
|
||||||
case d ⇒
|
case d ⇒
|
||||||
(envelope.merge(d.asInstanceOf[existing.T]), None)
|
(envelope.merge(d.asInstanceOf[existing.T]), None)
|
||||||
}
|
}
|
||||||
case None ⇒ modify(None) match {
|
case None ⇒ modify(None) match {
|
||||||
case d: DeltaReplicatedData if deltaCrdtEnabled ⇒ (DataEnvelope(d.resetDelta), Some(d.delta))
|
case d: DeltaReplicatedData if deltaCrdtEnabled ⇒ (DataEnvelope(d.resetDelta), d.delta)
|
||||||
case d ⇒ (DataEnvelope(d), None)
|
case d ⇒ (DataEnvelope(d), None)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} match {
|
} match {
|
||||||
case Success((envelope, delta)) ⇒
|
case Success((envelope, delta)) ⇒
|
||||||
log.debug("Received Update for key [{}], old data [{}], new data [{}], delta [{}]", key, localValue, envelope.data, delta)
|
log.debug("Received Update for key [{}], old data [{}], new data [{}], delta [{}]", key, localValue, envelope.data, delta)
|
||||||
setData(key.id, envelope)
|
|
||||||
|
|
||||||
// handle the delta
|
// handle the delta
|
||||||
delta match {
|
delta match {
|
||||||
|
|
@ -1190,23 +1230,28 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
case None ⇒ // not DeltaReplicatedData
|
case None ⇒ // not DeltaReplicatedData
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// note that it's important to do deltaPropagationSelector.update before setData,
|
||||||
|
// so that the latest delta version is used
|
||||||
|
val newEnvelope = setData(key.id, envelope)
|
||||||
|
|
||||||
val durable = isDurable(key.id)
|
val durable = isDurable(key.id)
|
||||||
if (isLocalUpdate(writeConsistency)) {
|
if (isLocalUpdate(writeConsistency)) {
|
||||||
if (durable)
|
if (durable)
|
||||||
durableStore ! Store(key.id, new DurableDataEnvelope(envelope),
|
durableStore ! Store(key.id, new DurableDataEnvelope(newEnvelope),
|
||||||
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), replyTo)))
|
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), replyTo)))
|
||||||
else
|
else
|
||||||
replyTo ! UpdateSuccess(key, req)
|
replyTo ! UpdateSuccess(key, req)
|
||||||
} else {
|
} else {
|
||||||
val writeEnvelope = delta match {
|
val writeEnvelope = delta match {
|
||||||
|
case Some(d: RequiresCausalDeliveryOfDeltas) ⇒ newEnvelope
|
||||||
case Some(d) ⇒ DataEnvelope(d)
|
case Some(d) ⇒ DataEnvelope(d)
|
||||||
case None ⇒ envelope
|
case None ⇒ newEnvelope
|
||||||
}
|
}
|
||||||
val writeAggregator =
|
val writeAggregator =
|
||||||
context.actorOf(WriteAggregator.props(key, writeEnvelope, writeConsistency, req, nodes, unreachable, replyTo, durable)
|
context.actorOf(WriteAggregator.props(key, writeEnvelope, writeConsistency, req, nodes, unreachable, replyTo, durable)
|
||||||
.withDispatcher(context.props.dispatcher))
|
.withDispatcher(context.props.dispatcher))
|
||||||
if (durable) {
|
if (durable) {
|
||||||
durableStore ! Store(key.id, new DurableDataEnvelope(envelope),
|
durableStore ! Store(key.id, new DurableDataEnvelope(newEnvelope),
|
||||||
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), writeAggregator)))
|
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), writeAggregator)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -1219,7 +1264,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def isDurable(key: String): Boolean =
|
def isDurable(key: KeyId): Boolean =
|
||||||
durable(key) || (durableWildcards.nonEmpty && durableWildcards.exists(key.startsWith))
|
durable(key) || (durableWildcards.nonEmpty && durableWildcards.exists(key.startsWith))
|
||||||
|
|
||||||
def isLocalUpdate(writeConsistency: WriteConsistency): Boolean =
|
def isLocalUpdate(writeConsistency: WriteConsistency): Boolean =
|
||||||
|
|
@ -1229,7 +1274,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
case _ ⇒ false
|
case _ ⇒ false
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveWrite(key: String, envelope: DataEnvelope): Unit = {
|
def receiveWrite(key: KeyId, envelope: DataEnvelope): Unit = {
|
||||||
write(key, envelope) match {
|
write(key, envelope) match {
|
||||||
case Some(newEnvelope) ⇒
|
case Some(newEnvelope) ⇒
|
||||||
if (isDurable(key))
|
if (isDurable(key))
|
||||||
|
|
@ -1240,27 +1285,38 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def write(key: String, writeEnvelope: DataEnvelope): Option[DataEnvelope] =
|
def write(key: KeyId, writeEnvelope: DataEnvelope): Option[DataEnvelope] = {
|
||||||
getData(key) match {
|
getData(key) match {
|
||||||
case Some(DataEnvelope(DeletedData, _)) ⇒ Some(DeletedEnvelope) // already deleted
|
case someEnvelope @ Some(envelope) if envelope eq writeEnvelope ⇒ someEnvelope
|
||||||
case Some(envelope @ DataEnvelope(existing, _)) ⇒
|
case Some(DataEnvelope(DeletedData, _, _)) ⇒ Some(DeletedEnvelope) // already deleted
|
||||||
if (existing.getClass == writeEnvelope.data.getClass || writeEnvelope.data == DeletedData) {
|
case Some(envelope @ DataEnvelope(existing, _, _)) ⇒
|
||||||
|
try {
|
||||||
|
// DataEnvelope will mergeDelta when needed
|
||||||
val merged = envelope.merge(writeEnvelope).addSeen(selfAddress)
|
val merged = envelope.merge(writeEnvelope).addSeen(selfAddress)
|
||||||
setData(key, merged)
|
Some(setData(key, merged))
|
||||||
Some(merged)
|
} catch {
|
||||||
} else {
|
case e: IllegalArgumentException ⇒
|
||||||
log.warning(
|
log.warning(
|
||||||
"Wrong type for writing [{}], existing type [{}], got [{}]",
|
"Couldn't merge [{}], due to: {}", key, e.getMessage)
|
||||||
key, existing.getClass.getName, writeEnvelope.data.getClass.getName)
|
|
||||||
None
|
None
|
||||||
}
|
}
|
||||||
case None ⇒
|
case None ⇒
|
||||||
val writeEnvelope2 = writeEnvelope.addSeen(selfAddress)
|
// no existing data for the key
|
||||||
setData(key, writeEnvelope2)
|
val writeEnvelope2 =
|
||||||
Some(writeEnvelope2)
|
writeEnvelope.data match {
|
||||||
|
case d: ReplicatedDelta ⇒
|
||||||
|
val z = d.zero
|
||||||
|
writeEnvelope.copy(data = z.mergeDelta(d.asInstanceOf[z.D]))
|
||||||
|
case _ ⇒
|
||||||
|
writeEnvelope
|
||||||
}
|
}
|
||||||
|
|
||||||
def writeAndStore(key: String, writeEnvelope: DataEnvelope): Unit = {
|
val writeEnvelope3 = writeEnvelope2.addSeen(selfAddress)
|
||||||
|
Some(setData(key, writeEnvelope3))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def writeAndStore(key: KeyId, writeEnvelope: DataEnvelope): Unit = {
|
||||||
write(key, writeEnvelope) match {
|
write(key, writeEnvelope) match {
|
||||||
case Some(newEnvelope) ⇒
|
case Some(newEnvelope) ⇒
|
||||||
if (isDurable(key))
|
if (isDurable(key))
|
||||||
|
|
@ -1269,21 +1325,21 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveReadRepair(key: String, writeEnvelope: DataEnvelope): Unit = {
|
def receiveReadRepair(key: KeyId, writeEnvelope: DataEnvelope): Unit = {
|
||||||
writeAndStore(key, writeEnvelope)
|
writeAndStore(key, writeEnvelope)
|
||||||
replyTo ! ReadRepairAck
|
replyTo ! ReadRepairAck
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveGetKeyIds(): Unit = {
|
def receiveGetKeyIds(): Unit = {
|
||||||
val keys: Set[String] = dataEntries.collect {
|
val keys: Set[KeyId] = dataEntries.collect {
|
||||||
case (key, (DataEnvelope(data, _), _)) if data != DeletedData ⇒ key
|
case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData ⇒ key
|
||||||
}(collection.breakOut)
|
}(collection.breakOut)
|
||||||
replyTo ! GetKeyIdsResult(keys)
|
replyTo ! GetKeyIdsResult(keys)
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveDelete(key: KeyR, consistency: WriteConsistency, req: Option[Any]): Unit = {
|
def receiveDelete(key: KeyR, consistency: WriteConsistency, req: Option[Any]): Unit = {
|
||||||
getData(key.id) match {
|
getData(key.id) match {
|
||||||
case Some(DataEnvelope(DeletedData, _)) ⇒
|
case Some(DataEnvelope(DeletedData, _, _)) ⇒
|
||||||
// already deleted
|
// already deleted
|
||||||
replyTo ! DataDeleted(key, req)
|
replyTo ! DataDeleted(key, req)
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
|
|
@ -1307,23 +1363,35 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def setData(key: String, envelope: DataEnvelope): Unit = {
|
def setData(key: KeyId, envelope: DataEnvelope): DataEnvelope = {
|
||||||
|
val newEnvelope = {
|
||||||
|
if (deltaCrdtEnabled) {
|
||||||
|
val deltaVersions = envelope.deltaVersions
|
||||||
|
val currVersion = deltaPropagationSelector.currentVersion(key)
|
||||||
|
if (currVersion == 0L || currVersion == deltaVersions.versionAt(selfUniqueAddress))
|
||||||
|
envelope
|
||||||
|
else
|
||||||
|
envelope.copy(deltaVersions = deltaVersions.merge(VersionVector(selfUniqueAddress, currVersion)))
|
||||||
|
} else envelope
|
||||||
|
}
|
||||||
|
|
||||||
val dig =
|
val dig =
|
||||||
if (subscribers.contains(key) && !changed.contains(key)) {
|
if (subscribers.contains(key) && !changed.contains(key)) {
|
||||||
val oldDigest = getDigest(key)
|
val oldDigest = getDigest(key)
|
||||||
val dig = digest(envelope)
|
val dig = digest(newEnvelope)
|
||||||
if (dig != oldDigest)
|
if (dig != oldDigest)
|
||||||
changed += key // notify subscribers, later
|
changed += key // notify subscribers, later
|
||||||
dig
|
dig
|
||||||
} else if (envelope.data == DeletedData) DeletedDigest
|
} else if (newEnvelope.data == DeletedData) DeletedDigest
|
||||||
else LazyDigest
|
else LazyDigest
|
||||||
|
|
||||||
dataEntries = dataEntries.updated(key, (envelope, dig))
|
dataEntries = dataEntries.updated(key, (newEnvelope, dig))
|
||||||
if (envelope.data == DeletedData)
|
if (newEnvelope.data == DeletedData)
|
||||||
deltaPropagationSelector.delete(key)
|
deltaPropagationSelector.delete(key)
|
||||||
|
newEnvelope
|
||||||
}
|
}
|
||||||
|
|
||||||
def getDigest(key: String): Digest = {
|
def getDigest(key: KeyId): Digest = {
|
||||||
dataEntries.get(key) match {
|
dataEntries.get(key) match {
|
||||||
case Some((envelope, LazyDigest)) ⇒
|
case Some((envelope, LazyDigest)) ⇒
|
||||||
val d = digest(envelope)
|
val d = digest(envelope)
|
||||||
|
|
@ -1337,14 +1405,27 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
def digest(envelope: DataEnvelope): Digest =
|
def digest(envelope: DataEnvelope): Digest =
|
||||||
if (envelope.data == DeletedData) DeletedDigest
|
if (envelope.data == DeletedData) DeletedDigest
|
||||||
else {
|
else {
|
||||||
val bytes = serializer.toBinary(envelope)
|
val bytes = serializer.toBinary(envelope.withoutDeltaVersions)
|
||||||
ByteString.fromArray(MessageDigest.getInstance("SHA-1").digest(bytes))
|
ByteString.fromArray(MessageDigest.getInstance("SHA-1").digest(bytes))
|
||||||
}
|
}
|
||||||
|
|
||||||
def getData(key: String): Option[DataEnvelope] = dataEntries.get(key).map { case (envelope, _) ⇒ envelope }
|
def getData(key: KeyId): Option[DataEnvelope] = dataEntries.get(key).map { case (envelope, _) ⇒ envelope }
|
||||||
|
|
||||||
|
def getDeltaSeqNr(key: KeyId, fromNode: UniqueAddress): Long =
|
||||||
|
dataEntries.get(key) match {
|
||||||
|
case Some((DataEnvelope(_, _, deltaVersions), _)) ⇒ deltaVersions.versionAt(fromNode)
|
||||||
|
case None ⇒ 0L
|
||||||
|
}
|
||||||
|
|
||||||
|
def isNodeRemoved(node: UniqueAddress, keys: Iterable[KeyId]): Boolean = {
|
||||||
|
removedNodes.contains(node) || (keys.exists(key ⇒ dataEntries.get(key) match {
|
||||||
|
case Some((DataEnvelope(_, pruning, _), _)) ⇒ pruning.contains(node)
|
||||||
|
case None ⇒ false
|
||||||
|
}))
|
||||||
|
}
|
||||||
|
|
||||||
def receiveFlushChanges(): Unit = {
|
def receiveFlushChanges(): Unit = {
|
||||||
def notify(keyId: String, subs: mutable.Set[ActorRef]): Unit = {
|
def notify(keyId: KeyId, subs: mutable.Set[ActorRef]): Unit = {
|
||||||
val key = subscriptionKeys(keyId)
|
val key = subscriptionKeys(keyId)
|
||||||
getData(keyId) match {
|
getData(keyId) match {
|
||||||
case Some(envelope) ⇒
|
case Some(envelope) ⇒
|
||||||
|
|
@ -1369,7 +1450,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
newSubscribers.clear()
|
newSubscribers.clear()
|
||||||
}
|
}
|
||||||
|
|
||||||
changed = Set.empty[String]
|
changed = Set.empty[KeyId]
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveDeltaPropagationTick(): Unit = {
|
def receiveDeltaPropagationTick(): Unit = {
|
||||||
|
|
@ -1378,15 +1459,53 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
// TODO split it to several DeltaPropagation if too many entries
|
// TODO split it to several DeltaPropagation if too many entries
|
||||||
replica(node) ! deltaPropagation
|
replica(node) ! deltaPropagation
|
||||||
}
|
}
|
||||||
if (deltaPropagationSelector.propagationCount % deltaPropagationSelector.divisor == 0)
|
|
||||||
|
if (deltaPropagationSelector.propagationCount % deltaPropagationSelector.gossipIntervalDivisor == 0)
|
||||||
deltaPropagationSelector.cleanupDeltaEntries()
|
deltaPropagationSelector.cleanupDeltaEntries()
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveDeltaPropagation(deltas: Map[String, DataEnvelope]): Unit = {
|
def receiveDeltaPropagation(fromNode: UniqueAddress, deltas: Map[KeyId, Delta]): Unit =
|
||||||
if (log.isDebugEnabled)
|
if (deltaCrdtEnabled) {
|
||||||
log.debug("Received DeltaPropagation from [{}], containing [{}]", sender().path.address, deltas.keys.mkString(", "))
|
try {
|
||||||
|
val isDebugEnabled = log.isDebugEnabled
|
||||||
|
if (isDebugEnabled)
|
||||||
|
log.debug("Received DeltaPropagation from [{}], containing [{}]", fromNode.address,
|
||||||
|
deltas.collect { case (key, Delta(_, fromSeqNr, toSeqNr)) ⇒ s"$key $fromSeqNr-$toSeqNr" }.mkString(", "))
|
||||||
|
|
||||||
|
if (isNodeRemoved(fromNode, deltas.keys)) {
|
||||||
|
// Late message from a removed node.
|
||||||
|
// Drop it to avoid merging deltas that have been pruned on one side.
|
||||||
|
if (isDebugEnabled) log.debug(
|
||||||
|
"Skipping DeltaPropagation from [{}] because that node has been removed", fromNode.address)
|
||||||
|
} else {
|
||||||
deltas.foreach {
|
deltas.foreach {
|
||||||
case (key, envelope) ⇒ writeAndStore(key, envelope)
|
case (key, Delta(envelope @ DataEnvelope(_: RequiresCausalDeliveryOfDeltas, _, _), fromSeqNr, toSeqNr)) ⇒
|
||||||
|
val currentSeqNr = getDeltaSeqNr(key, fromNode)
|
||||||
|
if (currentSeqNr >= toSeqNr) {
|
||||||
|
if (isDebugEnabled) log.debug(
|
||||||
|
"Skipping DeltaPropagation from [{}] for [{}] because toSeqNr [{}] already handled [{}]",
|
||||||
|
fromNode.address, key, toSeqNr, currentSeqNr)
|
||||||
|
} else if (fromSeqNr > (currentSeqNr + 1)) {
|
||||||
|
if (isDebugEnabled) log.debug(
|
||||||
|
"Skipping DeltaPropagation from [{}] for [{}] because missing deltas between [{}-{}]",
|
||||||
|
fromNode.address, key, currentSeqNr + 1, fromSeqNr - 1)
|
||||||
|
} else {
|
||||||
|
if (isDebugEnabled) log.debug(
|
||||||
|
"Applying DeltaPropagation from [{}] for [{}] with sequence numbers [{}], current was [{}]",
|
||||||
|
fromNode.address, key, s"$fromSeqNr-$toSeqNr", currentSeqNr)
|
||||||
|
val newEnvelope = envelope.copy(deltaVersions = VersionVector(fromNode, toSeqNr))
|
||||||
|
writeAndStore(key, newEnvelope)
|
||||||
|
}
|
||||||
|
case (key, Delta(envelope, _, _)) ⇒
|
||||||
|
// causal delivery of deltas not needed, just apply it
|
||||||
|
writeAndStore(key, envelope)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
case NonFatal(e) ⇒
|
||||||
|
// catching in case we need to support rolling upgrades that are
|
||||||
|
// mixing nodes with incompatible delta-CRDT types
|
||||||
|
log.warning("Couldn't process DeltaPropagation from [] due to {}", fromNode, e)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1424,12 +1543,12 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
def replica(address: Address): ActorSelection =
|
def replica(address: Address): ActorSelection =
|
||||||
context.actorSelection(self.path.toStringWithAddress(address))
|
context.actorSelection(self.path.toStringWithAddress(address))
|
||||||
|
|
||||||
def receiveStatus(otherDigests: Map[String, Digest], chunk: Int, totChunks: Int): Unit = {
|
def receiveStatus(otherDigests: Map[KeyId, Digest], chunk: Int, totChunks: Int): Unit = {
|
||||||
if (log.isDebugEnabled)
|
if (log.isDebugEnabled)
|
||||||
log.debug("Received gossip status from [{}], chunk [{}] of [{}] containing [{}]", replyTo.path.address,
|
log.debug("Received gossip status from [{}], chunk [{}] of [{}] containing [{}]", replyTo.path.address,
|
||||||
(chunk + 1), totChunks, otherDigests.keys.mkString(", "))
|
(chunk + 1), totChunks, otherDigests.keys.mkString(", "))
|
||||||
|
|
||||||
def isOtherDifferent(key: String, otherDigest: Digest): Boolean = {
|
def isOtherDifferent(key: KeyId, otherDigest: Digest): Boolean = {
|
||||||
val d = getDigest(key)
|
val d = getDigest(key)
|
||||||
d != NotFoundDigest && d != otherDigest
|
d != NotFoundDigest && d != otherDigest
|
||||||
}
|
}
|
||||||
|
|
@ -1457,10 +1576,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveGossip(updatedData: Map[String, DataEnvelope], sendBack: Boolean): Unit = {
|
def receiveGossip(updatedData: Map[KeyId, DataEnvelope], sendBack: Boolean): Unit = {
|
||||||
if (log.isDebugEnabled)
|
if (log.isDebugEnabled)
|
||||||
log.debug("Received gossip from [{}], containing [{}]", replyTo.path.address, updatedData.keys.mkString(", "))
|
log.debug("Received gossip from [{}], containing [{}]", replyTo.path.address, updatedData.keys.mkString(", "))
|
||||||
var replyData = Map.empty[String, DataEnvelope]
|
var replyData = Map.empty[KeyId, DataEnvelope]
|
||||||
updatedData.foreach {
|
updatedData.foreach {
|
||||||
case (key, envelope) ⇒
|
case (key, envelope) ⇒
|
||||||
val hadData = dataEntries.contains(key)
|
val hadData = dataEntries.contains(key)
|
||||||
|
|
@ -1568,7 +1687,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
val knownNodes = nodes union weaklyUpNodes union removedNodes.keySet.map(_.address)
|
val knownNodes = nodes union weaklyUpNodes union removedNodes.keySet.map(_.address)
|
||||||
val newRemovedNodes =
|
val newRemovedNodes =
|
||||||
dataEntries.foldLeft(Set.empty[UniqueAddress]) {
|
dataEntries.foldLeft(Set.empty[UniqueAddress]) {
|
||||||
case (acc, (_, (envelope @ DataEnvelope(data: RemovedNodePruning, _), _))) ⇒
|
case (acc, (_, (envelope @ DataEnvelope(data: RemovedNodePruning, _, _), _))) ⇒
|
||||||
acc union data.modifiedByNodes.filterNot(n ⇒ n == selfUniqueAddress || knownNodes(n.address))
|
acc union data.modifiedByNodes.filterNot(n ⇒ n == selfUniqueAddress || knownNodes(n.address))
|
||||||
case (acc, _) ⇒
|
case (acc, _) ⇒
|
||||||
acc
|
acc
|
||||||
|
|
@ -1616,7 +1735,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
val pruningPerformed = PruningPerformed(System.currentTimeMillis() + pruningMarkerTimeToLive.toMillis)
|
val pruningPerformed = PruningPerformed(System.currentTimeMillis() + pruningMarkerTimeToLive.toMillis)
|
||||||
val durablePruningPerformed = PruningPerformed(System.currentTimeMillis() + durablePruningMarkerTimeToLive.toMillis)
|
val durablePruningPerformed = PruningPerformed(System.currentTimeMillis() + durablePruningMarkerTimeToLive.toMillis)
|
||||||
dataEntries.foreach {
|
dataEntries.foreach {
|
||||||
case (key, (envelope @ DataEnvelope(data: RemovedNodePruning, pruning), _)) ⇒
|
case (key, (envelope @ DataEnvelope(data: RemovedNodePruning, pruning, _), _)) ⇒
|
||||||
pruning.foreach {
|
pruning.foreach {
|
||||||
case (removed, PruningInitialized(owner, seen)) if owner == selfUniqueAddress
|
case (removed, PruningInitialized(owner, seen)) if owner == selfUniqueAddress
|
||||||
&& (allNodes.isEmpty || allNodes.forall(seen)) ⇒
|
&& (allNodes.isEmpty || allNodes.forall(seen)) ⇒
|
||||||
|
|
@ -1634,7 +1753,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
def deleteObsoletePruningPerformed(): Unit = {
|
def deleteObsoletePruningPerformed(): Unit = {
|
||||||
val currentTime = System.currentTimeMillis()
|
val currentTime = System.currentTimeMillis()
|
||||||
dataEntries.foreach {
|
dataEntries.foreach {
|
||||||
case (key, (envelope @ DataEnvelope(_: RemovedNodePruning, pruning), _)) ⇒
|
case (key, (envelope @ DataEnvelope(_: RemovedNodePruning, pruning, _), _)) ⇒
|
||||||
val newEnvelope = pruning.foldLeft(envelope) {
|
val newEnvelope = pruning.foldLeft(envelope) {
|
||||||
case (acc, (removed, p: PruningPerformed)) if p.isObsolete(currentTime) ⇒
|
case (acc, (removed, p: PruningPerformed)) if p.isObsolete(currentTime) ⇒
|
||||||
log.debug("Removing obsolete pruning marker for [{}] in [{}]", removed, key)
|
log.debug("Removing obsolete pruning marker for [{}] in [{}]", removed, key)
|
||||||
|
|
@ -1660,7 +1779,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] object ReadWriteAggregator {
|
@InternalApi private[akka] object ReadWriteAggregator {
|
||||||
case object SendToSecondary
|
case object SendToSecondary
|
||||||
val MaxSecondaryNodes = 10
|
val MaxSecondaryNodes = 10
|
||||||
|
|
||||||
|
|
@ -1678,7 +1797,7 @@ private[akka] object ReadWriteAggregator {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] abstract class ReadWriteAggregator extends Actor {
|
@InternalApi private[akka] abstract class ReadWriteAggregator extends Actor {
|
||||||
import ReadWriteAggregator._
|
import ReadWriteAggregator._
|
||||||
|
|
||||||
def timeout: FiniteDuration
|
def timeout: FiniteDuration
|
||||||
|
|
@ -1719,7 +1838,7 @@ private[akka] abstract class ReadWriteAggregator extends Actor {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] object WriteAggregator {
|
@InternalApi private[akka] object WriteAggregator {
|
||||||
def props(
|
def props(
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
envelope: Replicator.Internal.DataEnvelope,
|
envelope: Replicator.Internal.DataEnvelope,
|
||||||
|
|
@ -1736,7 +1855,7 @@ private[akka] object WriteAggregator {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] class WriteAggregator(
|
@InternalApi private[akka] class WriteAggregator(
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
envelope: Replicator.Internal.DataEnvelope,
|
envelope: Replicator.Internal.DataEnvelope,
|
||||||
consistency: Replicator.WriteConsistency,
|
consistency: Replicator.WriteConsistency,
|
||||||
|
|
@ -1826,7 +1945,7 @@ private[akka] class WriteAggregator(
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] object ReadAggregator {
|
@InternalApi private[akka] object ReadAggregator {
|
||||||
def props(
|
def props(
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
consistency: Replicator.ReadConsistency,
|
consistency: Replicator.ReadConsistency,
|
||||||
|
|
@ -1843,7 +1962,7 @@ private[akka] object ReadAggregator {
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] class ReadAggregator(
|
@InternalApi private[akka] class ReadAggregator(
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
consistency: Replicator.ReadConsistency,
|
consistency: Replicator.ReadConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
|
|
|
||||||
|
|
@ -9,6 +9,7 @@ import scala.collection.immutable.TreeMap
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* VersionVector module with helper classes and methods.
|
* VersionVector module with helper classes and methods.
|
||||||
|
|
@ -28,7 +29,7 @@ object VersionVector {
|
||||||
def apply(node: UniqueAddress, version: Long): VersionVector = OneVersionVector(node, version)
|
def apply(node: UniqueAddress, version: Long): VersionVector = OneVersionVector(node, version)
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] def apply(versions: List[(UniqueAddress, Long)]): VersionVector =
|
@InternalApi private[akka] def apply(versions: List[(UniqueAddress, Long)]): VersionVector =
|
||||||
if (versions.isEmpty) empty
|
if (versions.isEmpty) empty
|
||||||
else if (versions.tail.isEmpty) apply(versions.head._1, versions.head._2)
|
else if (versions.tail.isEmpty) apply(versions.head._1, versions.head._2)
|
||||||
else apply(emptyVersions ++ versions)
|
else apply(emptyVersions ++ versions)
|
||||||
|
|
@ -69,7 +70,7 @@ object VersionVector {
|
||||||
def ConcurrentInstance = Concurrent
|
def ConcurrentInstance = Concurrent
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] object Timestamp {
|
@InternalApi private[akka] object Timestamp {
|
||||||
final val Zero = 0L
|
final val Zero = 0L
|
||||||
final val EndMarker = Long.MinValue
|
final val EndMarker = Long.MinValue
|
||||||
val counter = new AtomicLong(1L)
|
val counter = new AtomicLong(1L)
|
||||||
|
|
@ -111,7 +112,7 @@ sealed abstract class VersionVector
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
||||||
*/
|
*/
|
||||||
private[akka] def +(node: UniqueAddress): VersionVector = increment(node)
|
@InternalApi private[akka] def +(node: UniqueAddress): VersionVector = increment(node)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
||||||
|
|
@ -123,23 +124,23 @@ sealed abstract class VersionVector
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def size: Int
|
@InternalApi private[akka] def size: Int
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
||||||
*/
|
*/
|
||||||
private[akka] def increment(node: UniqueAddress): VersionVector
|
@InternalApi private[akka] def increment(node: UniqueAddress): VersionVector
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def versionAt(node: UniqueAddress): Long
|
@InternalApi private[akka] def versionAt(node: UniqueAddress): Long
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def contains(node: UniqueAddress): Boolean
|
@InternalApi private[akka] def contains(node: UniqueAddress): Boolean
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns true if <code>this</code> and <code>that</code> are concurrent else false.
|
* Returns true if <code>this</code> and <code>that</code> are concurrent else false.
|
||||||
|
|
@ -221,7 +222,7 @@ sealed abstract class VersionVector
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def versionsIterator: Iterator[(UniqueAddress, Long)]
|
@InternalApi private[akka] def versionsIterator: Iterator[(UniqueAddress, Long)]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compare two version vectors. The outcome will be one of the following:
|
* Compare two version vectors. The outcome will be one of the following:
|
||||||
|
|
@ -256,26 +257,26 @@ final case class OneVersionVector private[akka] (node: UniqueAddress, version: L
|
||||||
override def isEmpty: Boolean = false
|
override def isEmpty: Boolean = false
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def size: Int = 1
|
@InternalApi private[akka] override def size: Int = 1
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def increment(n: UniqueAddress): VersionVector = {
|
@InternalApi private[akka] override def increment(n: UniqueAddress): VersionVector = {
|
||||||
val v = Timestamp.counter.getAndIncrement()
|
val v = Timestamp.counter.getAndIncrement()
|
||||||
if (n == node) copy(version = v)
|
if (n == node) copy(version = v)
|
||||||
else ManyVersionVector(TreeMap(node → version, n → v))
|
else ManyVersionVector(TreeMap(node → version, n → v))
|
||||||
}
|
}
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def versionAt(n: UniqueAddress): Long =
|
@InternalApi private[akka] override def versionAt(n: UniqueAddress): Long =
|
||||||
if (n == node) version
|
if (n == node) version
|
||||||
else Timestamp.Zero
|
else Timestamp.Zero
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def contains(n: UniqueAddress): Boolean =
|
@InternalApi private[akka] override def contains(n: UniqueAddress): Boolean =
|
||||||
n == node
|
n == node
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def versionsIterator: Iterator[(UniqueAddress, Long)] =
|
@InternalApi private[akka] override def versionsIterator: Iterator[(UniqueAddress, Long)] =
|
||||||
Iterator.single((node, version))
|
Iterator.single((node, version))
|
||||||
|
|
||||||
override def merge(that: VersionVector): VersionVector = {
|
override def merge(that: VersionVector): VersionVector = {
|
||||||
|
|
@ -315,30 +316,32 @@ final case class ManyVersionVector(versions: TreeMap[UniqueAddress, Long]) exten
|
||||||
override def isEmpty: Boolean = versions.isEmpty
|
override def isEmpty: Boolean = versions.isEmpty
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def size: Int = versions.size
|
@InternalApi private[akka] override def size: Int = versions.size
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def increment(node: UniqueAddress): VersionVector = {
|
@InternalApi private[akka] override def increment(node: UniqueAddress): VersionVector = {
|
||||||
val v = Timestamp.counter.getAndIncrement()
|
val v = Timestamp.counter.getAndIncrement()
|
||||||
VersionVector(versions.updated(node, v))
|
VersionVector(versions.updated(node, v))
|
||||||
}
|
}
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def versionAt(node: UniqueAddress): Long = versions.get(node) match {
|
@InternalApi private[akka] override def versionAt(node: UniqueAddress): Long = versions.get(node) match {
|
||||||
case Some(v) ⇒ v
|
case Some(v) ⇒ v
|
||||||
case None ⇒ Timestamp.Zero
|
case None ⇒ Timestamp.Zero
|
||||||
}
|
}
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def contains(node: UniqueAddress): Boolean =
|
@InternalApi private[akka] override def contains(node: UniqueAddress): Boolean =
|
||||||
versions.contains(node)
|
versions.contains(node)
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[akka] override def versionsIterator: Iterator[(UniqueAddress, Long)] =
|
@InternalApi private[akka] override def versionsIterator: Iterator[(UniqueAddress, Long)] =
|
||||||
versions.iterator
|
versions.iterator
|
||||||
|
|
||||||
override def merge(that: VersionVector): VersionVector = {
|
override def merge(that: VersionVector): VersionVector = {
|
||||||
that match {
|
if (that.isEmpty) this
|
||||||
|
else if (this.isEmpty) that
|
||||||
|
else that match {
|
||||||
case ManyVersionVector(vs2) ⇒
|
case ManyVersionVector(vs2) ⇒
|
||||||
var mergedVersions = vs2
|
var mergedVersions = vs2
|
||||||
for ((node, time) ← versions) {
|
for ((node, time) ← versions) {
|
||||||
|
|
@ -366,7 +369,8 @@ final case class ManyVersionVector(versions: TreeMap[UniqueAddress, Long]) exten
|
||||||
VersionVector(versions = versions - removedNode) + collapseInto
|
VersionVector(versions = versions - removedNode) + collapseInto
|
||||||
|
|
||||||
override def pruningCleanup(removedNode: UniqueAddress): VersionVector =
|
override def pruningCleanup(removedNode: UniqueAddress): VersionVector =
|
||||||
VersionVector(versions = versions - removedNode)
|
if (versions.contains(removedNode)) VersionVector(versions = versions - removedNode)
|
||||||
|
else this
|
||||||
|
|
||||||
override def toString: String =
|
override def toString: String =
|
||||||
versions.map { case ((n, v)) ⇒ n + " -> " + v }.mkString("VersionVector(", ", ", ")")
|
versions.map { case ((n, v)) ⇒ n + " -> " + v }.mkString("VersionVector(", ", ", ")")
|
||||||
|
|
|
||||||
|
|
@ -25,6 +25,7 @@ import scala.collection.immutable.TreeMap
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
|
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
|
||||||
|
import akka.cluster.ddata.ORSet.DeltaOp
|
||||||
|
|
||||||
private object ReplicatedDataSerializer {
|
private object ReplicatedDataSerializer {
|
||||||
/*
|
/*
|
||||||
|
|
@ -163,6 +164,10 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
private val GSetKeyManifest = "b"
|
private val GSetKeyManifest = "b"
|
||||||
private val ORSetManifest = "C"
|
private val ORSetManifest = "C"
|
||||||
private val ORSetKeyManifest = "c"
|
private val ORSetKeyManifest = "c"
|
||||||
|
private val ORSetAddManifest = "Ca"
|
||||||
|
private val ORSetRemoveManifest = "Cr"
|
||||||
|
private val ORSetFullManifest = "Cf"
|
||||||
|
private val ORSetDeltaGroupManifest = "Cg"
|
||||||
private val FlagManifest = "D"
|
private val FlagManifest = "D"
|
||||||
private val FlagKeyManifest = "d"
|
private val FlagKeyManifest = "d"
|
||||||
private val LWWRegisterManifest = "E"
|
private val LWWRegisterManifest = "E"
|
||||||
|
|
@ -184,6 +189,10 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
private val fromBinaryMap = collection.immutable.HashMap[String, Array[Byte] ⇒ AnyRef](
|
private val fromBinaryMap = collection.immutable.HashMap[String, Array[Byte] ⇒ AnyRef](
|
||||||
GSetManifest → gsetFromBinary,
|
GSetManifest → gsetFromBinary,
|
||||||
ORSetManifest → orsetFromBinary,
|
ORSetManifest → orsetFromBinary,
|
||||||
|
ORSetAddManifest → orsetAddFromBinary,
|
||||||
|
ORSetRemoveManifest → orsetRemoveFromBinary,
|
||||||
|
ORSetFullManifest → orsetFullFromBinary,
|
||||||
|
ORSetDeltaGroupManifest → orsetDeltaGroupFromBinary,
|
||||||
FlagManifest → flagFromBinary,
|
FlagManifest → flagFromBinary,
|
||||||
LWWRegisterManifest → lwwRegisterFromBinary,
|
LWWRegisterManifest → lwwRegisterFromBinary,
|
||||||
GCounterManifest → gcounterFromBinary,
|
GCounterManifest → gcounterFromBinary,
|
||||||
|
|
@ -208,6 +217,8 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
|
|
||||||
override def manifest(obj: AnyRef): String = obj match {
|
override def manifest(obj: AnyRef): String = obj match {
|
||||||
case _: ORSet[_] ⇒ ORSetManifest
|
case _: ORSet[_] ⇒ ORSetManifest
|
||||||
|
case _: ORSet.AddDeltaOp[_] ⇒ ORSetAddManifest
|
||||||
|
case _: ORSet.RemoveDeltaOp[_] ⇒ ORSetRemoveManifest
|
||||||
case _: GSet[_] ⇒ GSetManifest
|
case _: GSet[_] ⇒ GSetManifest
|
||||||
case _: GCounter ⇒ GCounterManifest
|
case _: GCounter ⇒ GCounterManifest
|
||||||
case _: PNCounter ⇒ PNCounterManifest
|
case _: PNCounter ⇒ PNCounterManifest
|
||||||
|
|
@ -231,12 +242,17 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
case _: PNCounterMapKey[_] ⇒ PNCounterMapKeyManifest
|
case _: PNCounterMapKey[_] ⇒ PNCounterMapKeyManifest
|
||||||
case _: ORMultiMapKey[_, _] ⇒ ORMultiMapKeyManifest
|
case _: ORMultiMapKey[_, _] ⇒ ORMultiMapKeyManifest
|
||||||
|
|
||||||
|
case _: ORSet.DeltaGroup[_] ⇒ ORSetDeltaGroupManifest
|
||||||
|
case _: ORSet.FullStateDeltaOp[_] ⇒ ORSetFullManifest
|
||||||
|
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
|
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
||||||
case m: ORSet[_] ⇒ compress(orsetToProto(m))
|
case m: ORSet[_] ⇒ compress(orsetToProto(m))
|
||||||
|
case m: ORSet.AddDeltaOp[_] ⇒ orsetToProto(m.underlying).toByteArray
|
||||||
|
case m: ORSet.RemoveDeltaOp[_] ⇒ orsetToProto(m.underlying).toByteArray
|
||||||
case m: GSet[_] ⇒ gsetToProto(m).toByteArray
|
case m: GSet[_] ⇒ gsetToProto(m).toByteArray
|
||||||
case m: GCounter ⇒ gcounterToProto(m).toByteArray
|
case m: GCounter ⇒ gcounterToProto(m).toByteArray
|
||||||
case m: PNCounter ⇒ pncounterToProto(m).toByteArray
|
case m: PNCounter ⇒ pncounterToProto(m).toByteArray
|
||||||
|
|
@ -249,6 +265,8 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
case DeletedData ⇒ dm.Empty.getDefaultInstance.toByteArray
|
case DeletedData ⇒ dm.Empty.getDefaultInstance.toByteArray
|
||||||
case m: VersionVector ⇒ versionVectorToProto(m).toByteArray
|
case m: VersionVector ⇒ versionVectorToProto(m).toByteArray
|
||||||
case Key(id) ⇒ keyIdToBinary(id)
|
case Key(id) ⇒ keyIdToBinary(id)
|
||||||
|
case m: ORSet.DeltaGroup[_] ⇒ orsetDeltaGroupToProto(m).toByteArray
|
||||||
|
case m: ORSet.FullStateDeltaOp[_] ⇒ orsetToProto(m.underlying).toByteArray
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
|
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
@ -362,6 +380,52 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
def orsetFromBinary(bytes: Array[Byte]): ORSet[Any] =
|
def orsetFromBinary(bytes: Array[Byte]): ORSet[Any] =
|
||||||
orsetFromProto(rd.ORSet.parseFrom(decompress(bytes)))
|
orsetFromProto(rd.ORSet.parseFrom(decompress(bytes)))
|
||||||
|
|
||||||
|
private def orsetAddFromBinary(bytes: Array[Byte]): ORSet.AddDeltaOp[Any] =
|
||||||
|
new ORSet.AddDeltaOp(orsetFromProto(rd.ORSet.parseFrom(bytes)))
|
||||||
|
|
||||||
|
private def orsetRemoveFromBinary(bytes: Array[Byte]): ORSet.RemoveDeltaOp[Any] =
|
||||||
|
new ORSet.RemoveDeltaOp(orsetFromProto(rd.ORSet.parseFrom(bytes)))
|
||||||
|
|
||||||
|
private def orsetFullFromBinary(bytes: Array[Byte]): ORSet.FullStateDeltaOp[Any] =
|
||||||
|
new ORSet.FullStateDeltaOp(orsetFromProto(rd.ORSet.parseFrom(bytes)))
|
||||||
|
|
||||||
|
private def orsetDeltaGroupToProto(deltaGroup: ORSet.DeltaGroup[_]): rd.ORSetDeltaGroup = {
|
||||||
|
def createEntry(opType: rd.ORSetDeltaOp, u: ORSet[_]) = {
|
||||||
|
rd.ORSetDeltaGroup.Entry.newBuilder()
|
||||||
|
.setOperation(opType)
|
||||||
|
.setUnderlying(orsetToProto(u))
|
||||||
|
}
|
||||||
|
|
||||||
|
val b = rd.ORSetDeltaGroup.newBuilder()
|
||||||
|
deltaGroup.ops.foreach {
|
||||||
|
case ORSet.AddDeltaOp(u) ⇒
|
||||||
|
b.addEntries(createEntry(rd.ORSetDeltaOp.Add, u))
|
||||||
|
case ORSet.RemoveDeltaOp(u) ⇒
|
||||||
|
b.addEntries(createEntry(rd.ORSetDeltaOp.Remove, u))
|
||||||
|
case ORSet.FullStateDeltaOp(u) ⇒
|
||||||
|
b.addEntries(createEntry(rd.ORSetDeltaOp.Full, u))
|
||||||
|
case ORSet.DeltaGroup(u) ⇒
|
||||||
|
throw new IllegalArgumentException("ORSet.DeltaGroup should not be nested")
|
||||||
|
}
|
||||||
|
b.build()
|
||||||
|
}
|
||||||
|
|
||||||
|
private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = {
|
||||||
|
val deltaGroup = rd.ORSetDeltaGroup.parseFrom(bytes)
|
||||||
|
val ops: Vector[ORSet.DeltaOp] =
|
||||||
|
deltaGroup.getEntriesList.asScala.map { entry ⇒
|
||||||
|
if (entry.getOperation == rd.ORSetDeltaOp.Add)
|
||||||
|
ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||||
|
else if (entry.getOperation == rd.ORSetDeltaOp.Remove)
|
||||||
|
ORSet.RemoveDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||||
|
else if (entry.getOperation == rd.ORSetDeltaOp.Full)
|
||||||
|
ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||||
|
else
|
||||||
|
throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}")
|
||||||
|
}(collection.breakOut)
|
||||||
|
ORSet.DeltaGroup(ops)
|
||||||
|
}
|
||||||
|
|
||||||
def orsetFromProto(orset: rd.ORSet): ORSet[Any] = {
|
def orsetFromProto(orset: rd.ORSet): ORSet[Any] = {
|
||||||
val elements: Iterator[Any] =
|
val elements: Iterator[Any] =
|
||||||
(orset.getStringElementsList.iterator.asScala ++
|
(orset.getStringElementsList.iterator.asScala ++
|
||||||
|
|
@ -432,31 +496,6 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
decrements = gcounterFromProto(pncounter.getDecrements))
|
decrements = gcounterFromProto(pncounter.getDecrements))
|
||||||
}
|
}
|
||||||
|
|
||||||
def versionVectorToProto(versionVector: VersionVector): rd.VersionVector = {
|
|
||||||
val b = rd.VersionVector.newBuilder()
|
|
||||||
versionVector.versionsIterator.foreach {
|
|
||||||
case (node, value) ⇒ b.addEntries(rd.VersionVector.Entry.newBuilder().
|
|
||||||
setNode(uniqueAddressToProto(node)).setVersion(value))
|
|
||||||
}
|
|
||||||
b.build()
|
|
||||||
}
|
|
||||||
|
|
||||||
def versionVectorFromBinary(bytes: Array[Byte]): VersionVector =
|
|
||||||
versionVectorFromProto(rd.VersionVector.parseFrom(bytes))
|
|
||||||
|
|
||||||
def versionVectorFromProto(versionVector: rd.VersionVector): VersionVector = {
|
|
||||||
val entries = versionVector.getEntriesList
|
|
||||||
if (entries.isEmpty)
|
|
||||||
VersionVector.empty
|
|
||||||
else if (entries.size == 1)
|
|
||||||
VersionVector(uniqueAddressFromProto(entries.get(0).getNode), entries.get(0).getVersion)
|
|
||||||
else {
|
|
||||||
val versions: TreeMap[UniqueAddress, Long] = versionVector.getEntriesList.asScala.map(entry ⇒
|
|
||||||
uniqueAddressFromProto(entry.getNode) → entry.getVersion)(breakOut)
|
|
||||||
VersionVector(versions)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Convert a Map[A, B] to an Iterable[Entry] where Entry is the protobuf map entry.
|
* Convert a Map[A, B] to an Iterable[Entry] where Entry is the protobuf map entry.
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -26,14 +26,15 @@ import java.util.concurrent.atomic.AtomicInteger
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.concurrent.duration.FiniteDuration
|
import scala.concurrent.duration.FiniteDuration
|
||||||
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
||||||
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
|
import akka.cluster.ddata.VersionVector
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] object ReplicatorMessageSerializer {
|
@InternalApi private[akka] object ReplicatorMessageSerializer {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A cache that is designed for a small number (<= 32) of
|
* A cache that is designed for a small number (<= 32) of
|
||||||
|
|
@ -287,11 +288,16 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
|
|
||||||
private def deltaPropagationToProto(deltaPropagation: DeltaPropagation): dm.DeltaPropagation = {
|
private def deltaPropagationToProto(deltaPropagation: DeltaPropagation): dm.DeltaPropagation = {
|
||||||
val b = dm.DeltaPropagation.newBuilder()
|
val b = dm.DeltaPropagation.newBuilder()
|
||||||
|
.setFromNode(uniqueAddressToProto(deltaPropagation.fromNode))
|
||||||
val entries = deltaPropagation.deltas.foreach {
|
val entries = deltaPropagation.deltas.foreach {
|
||||||
case (key, data) ⇒
|
case (key, Delta(data, fromSeqNr, toSeqNr)) ⇒
|
||||||
b.addEntries(dm.DeltaPropagation.Entry.newBuilder().
|
val b2 = dm.DeltaPropagation.Entry.newBuilder()
|
||||||
setKey(key).
|
.setKey(key)
|
||||||
setEnvelope(dataEnvelopeToProto(data)))
|
.setEnvelope(dataEnvelopeToProto(data))
|
||||||
|
.setFromSeqNr(fromSeqNr)
|
||||||
|
if (toSeqNr != fromSeqNr)
|
||||||
|
b2.setToSeqNr(toSeqNr)
|
||||||
|
b.addEntries(b2)
|
||||||
}
|
}
|
||||||
b.build()
|
b.build()
|
||||||
}
|
}
|
||||||
|
|
@ -299,8 +305,12 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
private def deltaPropagationFromBinary(bytes: Array[Byte]): DeltaPropagation = {
|
private def deltaPropagationFromBinary(bytes: Array[Byte]): DeltaPropagation = {
|
||||||
val deltaPropagation = dm.DeltaPropagation.parseFrom(bytes)
|
val deltaPropagation = dm.DeltaPropagation.parseFrom(bytes)
|
||||||
DeltaPropagation(
|
DeltaPropagation(
|
||||||
deltaPropagation.getEntriesList.asScala.map(e ⇒
|
uniqueAddressFromProto(deltaPropagation.getFromNode),
|
||||||
e.getKey → dataEnvelopeFromProto(e.getEnvelope))(breakOut))
|
deltaPropagation.getEntriesList.asScala.map { e ⇒
|
||||||
|
val fromSeqNr = e.getFromSeqNr
|
||||||
|
val toSeqNr = if (e.hasToSeqNr) e.getToSeqNr else fromSeqNr
|
||||||
|
e.getKey → Delta(dataEnvelopeFromProto(e.getEnvelope), fromSeqNr, toSeqNr)
|
||||||
|
}(breakOut))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def getToProto(get: Get[_]): dm.Get = {
|
private def getToProto(get: Get[_]): dm.Get = {
|
||||||
|
|
@ -434,6 +444,10 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
}
|
}
|
||||||
dataEnvelopeBuilder.addPruning(b)
|
dataEnvelopeBuilder.addPruning(b)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (!dataEnvelope.deltaVersions.isEmpty)
|
||||||
|
dataEnvelopeBuilder.setDeltaVersions(versionVectorToProto(dataEnvelope.deltaVersions))
|
||||||
|
|
||||||
dataEnvelopeBuilder.build()
|
dataEnvelopeBuilder.build()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -443,7 +457,10 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
private def dataEnvelopeFromProto(dataEnvelope: dm.DataEnvelope): DataEnvelope = {
|
private def dataEnvelopeFromProto(dataEnvelope: dm.DataEnvelope): DataEnvelope = {
|
||||||
val data = otherMessageFromProto(dataEnvelope.getData).asInstanceOf[ReplicatedData]
|
val data = otherMessageFromProto(dataEnvelope.getData).asInstanceOf[ReplicatedData]
|
||||||
val pruning = pruningFromProto(dataEnvelope.getPruningList)
|
val pruning = pruningFromProto(dataEnvelope.getPruningList)
|
||||||
DataEnvelope(data, pruning)
|
val deltaVersions =
|
||||||
|
if (dataEnvelope.hasDeltaVersions) versionVectorFromProto(dataEnvelope.getDeltaVersions)
|
||||||
|
else VersionVector.empty
|
||||||
|
DataEnvelope(data, pruning, deltaVersions)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def pruningFromProto(pruningEntries: java.util.List[dm.DataEnvelope.PruningEntry]): Map[UniqueAddress, PruningState] = {
|
private def pruningFromProto(pruningEntries: java.util.List[dm.DataEnvelope.PruningEntry]): Map[UniqueAddress, PruningState] = {
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,9 @@ import java.io.ByteArrayOutputStream
|
||||||
import java.util.zip.GZIPInputStream
|
import java.util.zip.GZIPInputStream
|
||||||
import java.util.zip.GZIPOutputStream
|
import java.util.zip.GZIPOutputStream
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
import scala.collection.immutable.TreeMap
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
import scala.collection.breakOut
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
|
|
@ -19,6 +22,7 @@ import akka.serialization.SerializationExtension
|
||||||
import akka.protobuf.ByteString
|
import akka.protobuf.ByteString
|
||||||
import akka.protobuf.MessageLite
|
import akka.protobuf.MessageLite
|
||||||
import akka.serialization.SerializerWithStringManifest
|
import akka.serialization.SerializerWithStringManifest
|
||||||
|
import akka.cluster.ddata.VersionVector
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Some useful serialization helper methods.
|
* Some useful serialization helper methods.
|
||||||
|
|
@ -101,8 +105,32 @@ trait SerializationSupport {
|
||||||
} else {
|
} else {
|
||||||
// old remote node
|
// old remote node
|
||||||
uniqueAddress.getUid.toLong
|
uniqueAddress.getUid.toLong
|
||||||
|
})
|
||||||
|
|
||||||
|
def versionVectorToProto(versionVector: VersionVector): dm.VersionVector = {
|
||||||
|
val b = dm.VersionVector.newBuilder()
|
||||||
|
versionVector.versionsIterator.foreach {
|
||||||
|
case (node, value) ⇒ b.addEntries(dm.VersionVector.Entry.newBuilder().
|
||||||
|
setNode(uniqueAddressToProto(node)).setVersion(value))
|
||||||
|
}
|
||||||
|
b.build()
|
||||||
|
}
|
||||||
|
|
||||||
|
def versionVectorFromBinary(bytes: Array[Byte]): VersionVector =
|
||||||
|
versionVectorFromProto(dm.VersionVector.parseFrom(bytes))
|
||||||
|
|
||||||
|
def versionVectorFromProto(versionVector: dm.VersionVector): VersionVector = {
|
||||||
|
val entries = versionVector.getEntriesList
|
||||||
|
if (entries.isEmpty)
|
||||||
|
VersionVector.empty
|
||||||
|
else if (entries.size == 1)
|
||||||
|
VersionVector(uniqueAddressFromProto(entries.get(0).getNode), entries.get(0).getVersion)
|
||||||
|
else {
|
||||||
|
val versions: TreeMap[UniqueAddress, Long] = versionVector.getEntriesList.asScala.map(entry ⇒
|
||||||
|
uniqueAddressFromProto(entry.getNode) → entry.getVersion)(breakOut)
|
||||||
|
VersionVector(versions)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
)
|
|
||||||
|
|
||||||
def resolveActorRef(path: String): ActorRef =
|
def resolveActorRef(path: String): ActorRef =
|
||||||
system.provider.resolveActorRef(path)
|
system.provider.resolveActorRef(path)
|
||||||
|
|
|
||||||
|
|
@ -38,6 +38,8 @@ object PerformanceSpec extends MultiNodeConfig {
|
||||||
#akka.cluster.distributed-data.durable.keys = ["*"]
|
#akka.cluster.distributed-data.durable.keys = ["*"]
|
||||||
#akka.cluster.distributed-data.durable.lmdb.dir = target/PerformanceSpec-${System.currentTimeMillis}-ddata
|
#akka.cluster.distributed-data.durable.lmdb.dir = target/PerformanceSpec-${System.currentTimeMillis}-ddata
|
||||||
#akka.cluster.distributed-data.durable.lmdb.write-behind-interval = 200ms
|
#akka.cluster.distributed-data.durable.lmdb.write-behind-interval = 200ms
|
||||||
|
|
||||||
|
#akka.cluster.distributed-data.delta-crdt.enabled = off
|
||||||
"""))
|
"""))
|
||||||
|
|
||||||
def countDownProps(latch: TestLatch): Props = Props(new CountDown(latch)).withDeploy(Deploy.local)
|
def countDownProps(latch: TestLatch): Props = Props(new CountDown(latch)).withDeploy(Deploy.local)
|
||||||
|
|
|
||||||
|
|
@ -22,6 +22,7 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
||||||
val fourth = role("fourth")
|
val fourth = role("fourth")
|
||||||
|
|
||||||
commonConfig(ConfigFactory.parseString("""
|
commonConfig(ConfigFactory.parseString("""
|
||||||
|
akka.loglevel = DEBUG
|
||||||
akka.actor.provider = "cluster"
|
akka.actor.provider = "cluster"
|
||||||
akka.log-dead-letters-during-shutdown = off
|
akka.log-dead-letters-during-shutdown = off
|
||||||
"""))
|
"""))
|
||||||
|
|
@ -32,6 +33,8 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
||||||
final case class Delay(n: Int) extends Op
|
final case class Delay(n: Int) extends Op
|
||||||
final case class Incr(key: PNCounterKey, n: Int, consistency: WriteConsistency) extends Op
|
final case class Incr(key: PNCounterKey, n: Int, consistency: WriteConsistency) extends Op
|
||||||
final case class Decr(key: PNCounterKey, n: Int, consistency: WriteConsistency) extends Op
|
final case class Decr(key: PNCounterKey, n: Int, consistency: WriteConsistency) extends Op
|
||||||
|
final case class Add(key: ORSetKey[String], elem: String, consistency: WriteConsistency) extends Op
|
||||||
|
final case class Remove(key: ORSetKey[String], elem: String, consistency: WriteConsistency) extends Op
|
||||||
|
|
||||||
val timeout = 5.seconds
|
val timeout = 5.seconds
|
||||||
val writeTwo = WriteTo(2, timeout)
|
val writeTwo = WriteTo(2, timeout)
|
||||||
|
|
@ -40,8 +43,11 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
||||||
val KeyA = PNCounterKey("A")
|
val KeyA = PNCounterKey("A")
|
||||||
val KeyB = PNCounterKey("B")
|
val KeyB = PNCounterKey("B")
|
||||||
val KeyC = PNCounterKey("C")
|
val KeyC = PNCounterKey("C")
|
||||||
|
val KeyD = ORSetKey[String]("D")
|
||||||
|
val KeyE = ORSetKey[String]("E")
|
||||||
|
val KeyF = ORSetKey[String]("F")
|
||||||
|
|
||||||
def generateOperations(): Vector[Op] = {
|
def generateOperations(onNode: RoleName): Vector[Op] = {
|
||||||
val rnd = ThreadLocalRandom.current()
|
val rnd = ThreadLocalRandom.current()
|
||||||
|
|
||||||
def consistency(): WriteConsistency = {
|
def consistency(): WriteConsistency = {
|
||||||
|
|
@ -52,7 +58,7 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def key(): PNCounterKey = {
|
def rndPnCounterkey(): PNCounterKey = {
|
||||||
rnd.nextInt(3) match {
|
rnd.nextInt(3) match {
|
||||||
case 0 ⇒ KeyA
|
case 0 ⇒ KeyA
|
||||||
case 1 ⇒ KeyB
|
case 1 ⇒ KeyB
|
||||||
|
|
@ -60,11 +66,43 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
(0 to (20 + rnd.nextInt(10))).map { _ ⇒
|
def rndOrSetkey(): ORSetKey[String] = {
|
||||||
rnd.nextInt(3) match {
|
rnd.nextInt(3) match {
|
||||||
|
case 0 ⇒ KeyD
|
||||||
|
case 1 ⇒ KeyE
|
||||||
|
case 2 ⇒ KeyF
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
var availableForRemove = Set.empty[String]
|
||||||
|
|
||||||
|
def rndAddElement(): String = {
|
||||||
|
// lower case a - j
|
||||||
|
val s = (97 + rnd.nextInt(10)).toChar.toString
|
||||||
|
availableForRemove += s
|
||||||
|
s
|
||||||
|
}
|
||||||
|
|
||||||
|
def rndRemoveElement(): String = {
|
||||||
|
if (availableForRemove.isEmpty)
|
||||||
|
"a"
|
||||||
|
else
|
||||||
|
availableForRemove.toVector(rnd.nextInt(availableForRemove.size))
|
||||||
|
}
|
||||||
|
|
||||||
|
(0 to (30 + rnd.nextInt(10))).map { _ ⇒
|
||||||
|
rnd.nextInt(4) match {
|
||||||
case 0 ⇒ Delay(rnd.nextInt(500))
|
case 0 ⇒ Delay(rnd.nextInt(500))
|
||||||
case 1 ⇒ Incr(key(), rnd.nextInt(100), consistency())
|
case 1 ⇒ Incr(rndPnCounterkey(), rnd.nextInt(100), consistency())
|
||||||
case 2 ⇒ Decr(key(), rnd.nextInt(10), consistency())
|
case 2 ⇒ Decr(rndPnCounterkey(), rnd.nextInt(10), consistency())
|
||||||
|
case 3 ⇒
|
||||||
|
// ORSet
|
||||||
|
val key = rndOrSetkey()
|
||||||
|
// only removals for KeyF on node first
|
||||||
|
if (key == KeyF && onNode == first && rnd.nextBoolean())
|
||||||
|
Remove(key, rndRemoveElement(), consistency())
|
||||||
|
else
|
||||||
|
Add(key, rndAddElement(), consistency())
|
||||||
}
|
}
|
||||||
}.toVector
|
}.toVector
|
||||||
}
|
}
|
||||||
|
|
@ -136,21 +174,32 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
enterBarrier("ready")
|
enterBarrier("ready")
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
fullStateReplicator ! Update(KeyA, PNCounter.empty, WriteLocal)(_ + 1)
|
// by setting something for each key we don't have to worry about NotFound
|
||||||
deltaReplicator ! Update(KeyA, PNCounter.empty, WriteLocal)(_ + 1)
|
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
||||||
|
fullStateReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ + 1)
|
||||||
|
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ + 1)
|
||||||
|
}
|
||||||
|
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||||
|
fullStateReplicator ! Update(key, ORSet.empty[String], WriteLocal)(_ + "a")
|
||||||
|
deltaReplicator ! Update(key, ORSet.empty[String], WriteLocal)(_ + "a")
|
||||||
|
}
|
||||||
}
|
}
|
||||||
enterBarrier("updated-1")
|
enterBarrier("updated-1")
|
||||||
|
|
||||||
within(5.seconds) {
|
within(5.seconds) {
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
val p = TestProbe()
|
val p = TestProbe()
|
||||||
deltaReplicator.tell(Get(KeyA, ReadLocal), p.ref)
|
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
||||||
|
fullStateReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||||
p.expectMsgType[GetSuccess[PNCounter]].dataValue.getValue.intValue should be(1)
|
p.expectMsgType[GetSuccess[PNCounter]].dataValue.getValue.intValue should be(1)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
val p = TestProbe()
|
val p = TestProbe()
|
||||||
deltaReplicator.tell(Get(KeyA, ReadLocal), p.ref)
|
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||||
p.expectMsgType[GetSuccess[PNCounter]].dataValue.getValue.intValue should be(1)
|
fullStateReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||||
|
p.expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements should ===(Set("a"))
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -158,7 +207,7 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
}
|
}
|
||||||
|
|
||||||
"be eventually consistent" in {
|
"be eventually consistent" in {
|
||||||
val operations = generateOperations()
|
val operations = generateOperations(onNode = myself)
|
||||||
log.debug(s"random operations on [${myself.name}]: ${operations.mkString(", ")}")
|
log.debug(s"random operations on [${myself.name}]: ${operations.mkString(", ")}")
|
||||||
try {
|
try {
|
||||||
// perform random operations with both delta and full-state replicators
|
// perform random operations with both delta and full-state replicators
|
||||||
|
|
@ -170,10 +219,22 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
case Delay(d) ⇒ Thread.sleep(d)
|
case Delay(d) ⇒ Thread.sleep(d)
|
||||||
case Incr(key, n, consistency) ⇒
|
case Incr(key, n, consistency) ⇒
|
||||||
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ + n)
|
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ + n)
|
||||||
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ + n)
|
deltaReplicator ! Update(key, PNCounter.empty, consistency)(_ + n)
|
||||||
case Decr(key, n, consistency) ⇒
|
case Decr(key, n, consistency) ⇒
|
||||||
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ - n)
|
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ - n)
|
||||||
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ - n)
|
deltaReplicator ! Update(key, PNCounter.empty, consistency)(_ - n)
|
||||||
|
case Add(key, elem, consistency) ⇒
|
||||||
|
// to have an deterministic result when mixing add/remove we can only perform
|
||||||
|
// the ORSet operations from one node
|
||||||
|
runOn((if (key == KeyF) List(first) else List(first, second, third)): _*) {
|
||||||
|
fullStateReplicator ! Update(key, ORSet.empty[String], consistency)(_ + elem)
|
||||||
|
deltaReplicator ! Update(key, ORSet.empty[String], consistency)(_ + elem)
|
||||||
|
}
|
||||||
|
case Remove(key, elem, consistency) ⇒
|
||||||
|
runOn(first) {
|
||||||
|
fullStateReplicator ! Update(key, ORSet.empty[String], consistency)(_ - elem)
|
||||||
|
deltaReplicator ! Update(key, ORSet.empty[String], consistency)(_ - elem)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -192,6 +253,19 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||||
|
within(5.seconds) {
|
||||||
|
awaitAssert {
|
||||||
|
val p = TestProbe()
|
||||||
|
fullStateReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||||
|
val fullStateValue = p.expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements
|
||||||
|
deltaReplicator.tell(Get(key, ReadLocal), p.ref)
|
||||||
|
val deltaValue = p.expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements
|
||||||
|
deltaValue should ===(fullStateValue)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
enterBarrierAfterTestStep()
|
enterBarrierAfterTestStep()
|
||||||
} catch {
|
} catch {
|
||||||
case e: Throwable ⇒
|
case e: Throwable ⇒
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,163 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
package akka.cluster.ddata
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
import akka.cluster.Cluster
|
||||||
|
import akka.remote.testconductor.RoleName
|
||||||
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
|
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||||
|
import akka.testkit._
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
|
object ReplicatorORSetDeltaSpec extends MultiNodeConfig {
|
||||||
|
val first = role("first")
|
||||||
|
val second = role("second")
|
||||||
|
val third = role("third")
|
||||||
|
|
||||||
|
commonConfig(ConfigFactory.parseString("""
|
||||||
|
akka.loglevel = DEBUG
|
||||||
|
akka.actor.provider = "cluster"
|
||||||
|
akka.log-dead-letters-during-shutdown = off
|
||||||
|
"""))
|
||||||
|
|
||||||
|
testTransport(on = true)
|
||||||
|
}
|
||||||
|
|
||||||
|
class ReplicatorORSetDeltaSpecMultiJvmNode1 extends ReplicatorORSetDeltaSpec
|
||||||
|
class ReplicatorORSetDeltaSpecMultiJvmNode2 extends ReplicatorORSetDeltaSpec
|
||||||
|
class ReplicatorORSetDeltaSpecMultiJvmNode3 extends ReplicatorORSetDeltaSpec
|
||||||
|
|
||||||
|
class ReplicatorORSetDeltaSpec extends MultiNodeSpec(ReplicatorORSetDeltaSpec) with STMultiNodeSpec with ImplicitSender {
|
||||||
|
import Replicator._
|
||||||
|
import ReplicatorORSetDeltaSpec._
|
||||||
|
|
||||||
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
|
implicit val cluster = Cluster(system)
|
||||||
|
val replicator = system.actorOf(Replicator.props(
|
||||||
|
ReplicatorSettings(system).withGossipInterval(1.second)), "replicator")
|
||||||
|
val timeout = 3.seconds.dilated
|
||||||
|
|
||||||
|
val KeyA = ORSetKey[String]("A")
|
||||||
|
val KeyB = ORSetKey[String]("B")
|
||||||
|
val KeyC = ORSetKey[String]("C")
|
||||||
|
|
||||||
|
def join(from: RoleName, to: RoleName): Unit = {
|
||||||
|
runOn(from) {
|
||||||
|
cluster join node(to).address
|
||||||
|
}
|
||||||
|
enterBarrier(from.name + "-joined")
|
||||||
|
}
|
||||||
|
|
||||||
|
def assertValue(key: Key[ReplicatedData], expected: Any): Unit =
|
||||||
|
within(10.seconds) {
|
||||||
|
awaitAssert {
|
||||||
|
replicator ! Get(key, ReadLocal)
|
||||||
|
val value = expectMsgPF() {
|
||||||
|
case g @ GetSuccess(`key`, _) ⇒ g.dataValue match {
|
||||||
|
case c: ORSet[_] ⇒ c.elements
|
||||||
|
}
|
||||||
|
}
|
||||||
|
value should be(expected)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"ORSet delta" must {
|
||||||
|
|
||||||
|
"replicate data in initial phase" in {
|
||||||
|
join(first, first)
|
||||||
|
join(second, first)
|
||||||
|
join(third, first)
|
||||||
|
|
||||||
|
replicator ! Replicator.Internal.TestFullStateGossip(enabled = false)
|
||||||
|
|
||||||
|
within(10.seconds) {
|
||||||
|
awaitAssert {
|
||||||
|
replicator ! GetReplicaCount
|
||||||
|
expectMsg(ReplicaCount(3))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
runOn(first) {
|
||||||
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "a")
|
||||||
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
}
|
||||||
|
|
||||||
|
enterBarrier("initial-updates-done")
|
||||||
|
|
||||||
|
assertValue(KeyA, Set("a"))
|
||||||
|
|
||||||
|
enterBarrier("after-1")
|
||||||
|
}
|
||||||
|
|
||||||
|
"be propagated with causal consistency during network split" in {
|
||||||
|
runOn(first) {
|
||||||
|
// third is isolated
|
||||||
|
testConductor.blackhole(first, third, Direction.Both).await
|
||||||
|
testConductor.blackhole(second, third, Direction.Both).await
|
||||||
|
}
|
||||||
|
enterBarrier("split")
|
||||||
|
|
||||||
|
runOn(first) {
|
||||||
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "b")
|
||||||
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
}
|
||||||
|
runOn(second) {
|
||||||
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "d")
|
||||||
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
}
|
||||||
|
runOn(first, second) {
|
||||||
|
assertValue(KeyA, Set("a", "b", "d"))
|
||||||
|
Thread.sleep(2000) // all deltas sent
|
||||||
|
}
|
||||||
|
enterBarrier("added-b-and-d")
|
||||||
|
|
||||||
|
runOn(first) {
|
||||||
|
testConductor.passThrough(first, third, Direction.Both).await
|
||||||
|
testConductor.passThrough(second, third, Direction.Both).await
|
||||||
|
}
|
||||||
|
enterBarrier("healed")
|
||||||
|
|
||||||
|
runOn(first) {
|
||||||
|
// delta for "c" will be sent to third, but it has not received the previous delta for "b"
|
||||||
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "c")
|
||||||
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
// let the delta be propagated (will not fail if it takes longer)
|
||||||
|
Thread.sleep(1000)
|
||||||
|
}
|
||||||
|
enterBarrier("added-c")
|
||||||
|
|
||||||
|
runOn(first, second) {
|
||||||
|
assertValue(KeyA, Set("a", "b", "c", "d"))
|
||||||
|
}
|
||||||
|
runOn(third) {
|
||||||
|
// the delta for "c" should not be applied because it has not received previous delta for "b"
|
||||||
|
// and full gossip is turned off so far
|
||||||
|
assertValue(KeyA, Set("a"))
|
||||||
|
}
|
||||||
|
enterBarrier("verified-before-full-gossip")
|
||||||
|
|
||||||
|
replicator ! Replicator.Internal.TestFullStateGossip(enabled = true)
|
||||||
|
assertValue(KeyA, Set("a", "b", "c", "d"))
|
||||||
|
enterBarrier("verified-after-full-gossip")
|
||||||
|
|
||||||
|
replicator ! Replicator.Internal.TestFullStateGossip(enabled = false)
|
||||||
|
|
||||||
|
// and now the delta seqNr should be in sync again
|
||||||
|
runOn(first) {
|
||||||
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "e")
|
||||||
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
}
|
||||||
|
assertValue(KeyA, Set("a", "b", "c", "d", "e"))
|
||||||
|
|
||||||
|
enterBarrier("after-2")
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
@ -22,6 +22,7 @@ object ReplicatorSpec extends MultiNodeConfig {
|
||||||
akka.loglevel = INFO
|
akka.loglevel = INFO
|
||||||
akka.actor.provider = "cluster"
|
akka.actor.provider = "cluster"
|
||||||
akka.log-dead-letters-during-shutdown = off
|
akka.log-dead-letters-during-shutdown = off
|
||||||
|
#akka.cluster.distributed-data.delta-crdt.enabled = off
|
||||||
"""))
|
"""))
|
||||||
|
|
||||||
testTransport(on = true)
|
testTransport(on = true)
|
||||||
|
|
|
||||||
|
|
@ -3,10 +3,12 @@
|
||||||
*/
|
*/
|
||||||
package akka.cluster.ddata;
|
package akka.cluster.ddata;
|
||||||
|
|
||||||
import akka.cluster.UniqueAddress;
|
|
||||||
|
|
||||||
public class JavaImplOfDeltaReplicatedData extends AbstractDeltaReplicatedData<JavaImplOfDeltaReplicatedData> implements
|
import java.util.Optional;
|
||||||
RemovedNodePruning {
|
|
||||||
|
// same delta type
|
||||||
|
public class JavaImplOfDeltaReplicatedData extends AbstractDeltaReplicatedData<JavaImplOfDeltaReplicatedData, JavaImplOfDeltaReplicatedData>
|
||||||
|
implements ReplicatedDelta {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaImplOfDeltaReplicatedData mergeData(JavaImplOfDeltaReplicatedData other) {
|
public JavaImplOfDeltaReplicatedData mergeData(JavaImplOfDeltaReplicatedData other) {
|
||||||
|
|
@ -14,32 +16,23 @@ public class JavaImplOfDeltaReplicatedData extends AbstractDeltaReplicatedData<J
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaImplOfDeltaReplicatedData delta() {
|
public JavaImplOfDeltaReplicatedData mergeDeltaData(JavaImplOfDeltaReplicatedData other) {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Optional<JavaImplOfDeltaReplicatedData> deltaData() {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaImplOfDeltaReplicatedData resetDelta() {
|
public JavaImplOfDeltaReplicatedData resetDelta() {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public scala.collection.immutable.Set<UniqueAddress> modifiedByNodes() {
|
public JavaImplOfDeltaReplicatedData zero() {
|
||||||
return akka.japi.Util.immutableSeq(new java.util.ArrayList<UniqueAddress>()).toSet();
|
return new JavaImplOfDeltaReplicatedData();
|
||||||
}
|
}
|
||||||
|
|
||||||
@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,49 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2017 Lightbend Inc. <http://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
package akka.cluster.ddata;
|
||||||
|
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
|
import akka.cluster.UniqueAddress;
|
||||||
|
|
||||||
|
// different delta type
|
||||||
|
public class JavaImplOfDeltaReplicatedData2
|
||||||
|
extends AbstractDeltaReplicatedData<JavaImplOfDeltaReplicatedData2, JavaImplOfDeltaReplicatedData2.Delta> {
|
||||||
|
|
||||||
|
public static class Delta extends AbstractReplicatedData<Delta> implements ReplicatedDelta, RequiresCausalDeliveryOfDeltas {
|
||||||
|
@Override
|
||||||
|
public Delta mergeData(Delta other) {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public JavaImplOfDeltaReplicatedData2 zero() {
|
||||||
|
return new JavaImplOfDeltaReplicatedData2();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public JavaImplOfDeltaReplicatedData2 mergeData(JavaImplOfDeltaReplicatedData2 other) {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public JavaImplOfDeltaReplicatedData2 mergeDeltaData(Delta other) {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Optional<Delta> deltaData() {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public JavaImplOfDeltaReplicatedData2 resetDelta() {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -4,17 +4,24 @@
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
|
import akka.cluster.UniqueAddress
|
||||||
|
import akka.cluster.ddata.Key.KeyId
|
||||||
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
|
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
|
||||||
|
import akka.cluster.ddata.Replicator.Internal.Delta
|
||||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||||
import org.scalactic.TypeCheckedTripleEquals
|
import org.scalactic.TypeCheckedTripleEquals
|
||||||
import org.scalatest.Matchers
|
import org.scalatest.Matchers
|
||||||
import org.scalatest.WordSpec
|
import org.scalatest.WordSpec
|
||||||
|
|
||||||
object DeltaPropagationSelectorSpec {
|
object DeltaPropagationSelectorSpec {
|
||||||
class TestSelector(override val allNodes: Vector[Address]) extends DeltaPropagationSelector {
|
class TestSelector(
|
||||||
override val divisor = 5
|
val selfUniqueAddress: UniqueAddress,
|
||||||
override def createDeltaPropagation(deltas: Map[String, ReplicatedData]): DeltaPropagation =
|
override val allNodes: Vector[Address]) extends DeltaPropagationSelector {
|
||||||
DeltaPropagation(deltas.mapValues(d ⇒ DataEnvelope(d)))
|
override val gossipIntervalDivisor = 5
|
||||||
|
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
|
||||||
|
DeltaPropagation(selfUniqueAddress, deltas.mapValues {
|
||||||
|
case (d, fromSeqNr, toSeqNr) ⇒ Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
val deltaA = GSet.empty[String] + "a"
|
val deltaA = GSet.empty[String] + "a"
|
||||||
|
|
@ -24,11 +31,12 @@ object DeltaPropagationSelectorSpec {
|
||||||
|
|
||||||
class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheckedTripleEquals {
|
class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheckedTripleEquals {
|
||||||
import DeltaPropagationSelectorSpec._
|
import DeltaPropagationSelectorSpec._
|
||||||
|
val selfUniqueAddress = UniqueAddress(Address("akka", "Sys", "localhost", 4999), 1L)
|
||||||
val nodes = (2500 until 2600).map(n ⇒ Address("akka", "Sys", "localhost", n)).toVector
|
val nodes = (2500 until 2600).map(n ⇒ Address("akka", "Sys", "localhost", n)).toVector
|
||||||
|
|
||||||
"DeltaPropagationSelector" must {
|
"DeltaPropagationSelector" must {
|
||||||
"collect none when no nodes" in {
|
"collect none when no nodes" in {
|
||||||
val selector = new TestSelector(Vector.empty)
|
val selector = new TestSelector(selfUniqueAddress, Vector.empty)
|
||||||
selector.update("A", deltaA)
|
selector.update("A", deltaA)
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
|
|
@ -36,13 +44,15 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
}
|
}
|
||||||
|
|
||||||
"collect 1 when one node" in {
|
"collect 1 when one node" in {
|
||||||
val selector = new TestSelector(nodes.take(1))
|
val selector = new TestSelector(selfUniqueAddress, nodes.take(1))
|
||||||
selector.update("A", deltaA)
|
selector.update("A", deltaA)
|
||||||
selector.update("B", deltaB)
|
selector.update("B", deltaB)
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("A") should ===(true)
|
selector.hasDeltaEntries("A") should ===(true)
|
||||||
selector.hasDeltaEntries("B") should ===(true)
|
selector.hasDeltaEntries("B") should ===(true)
|
||||||
val expected = DeltaPropagation(Map("A" → DataEnvelope(deltaA), "B" → DataEnvelope(deltaB)))
|
val expected = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(deltaA), 1L, 1L),
|
||||||
|
"B" → Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(0) → expected))
|
selector.collectPropagations() should ===(Map(nodes(0) → expected))
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
|
|
@ -51,10 +61,12 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
}
|
}
|
||||||
|
|
||||||
"collect 2+1 when three nodes" in {
|
"collect 2+1 when three nodes" in {
|
||||||
val selector = new TestSelector(nodes.take(3))
|
val selector = new TestSelector(selfUniqueAddress, nodes.take(3))
|
||||||
selector.update("A", deltaA)
|
selector.update("A", deltaA)
|
||||||
selector.update("B", deltaB)
|
selector.update("B", deltaB)
|
||||||
val expected = DeltaPropagation(Map("A" → DataEnvelope(deltaA), "B" → DataEnvelope(deltaB)))
|
val expected = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(deltaA), 1L, 1L),
|
||||||
|
"B" → Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(0) → expected, nodes(1) → expected))
|
selector.collectPropagations() should ===(Map(nodes(0) → expected, nodes(1) → expected))
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("A") should ===(true)
|
selector.hasDeltaEntries("A") should ===(true)
|
||||||
|
|
@ -67,16 +79,21 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
}
|
}
|
||||||
|
|
||||||
"keep track of deltas per node" in {
|
"keep track of deltas per node" in {
|
||||||
val selector = new TestSelector(nodes.take(3))
|
val selector = new TestSelector(selfUniqueAddress, nodes.take(3))
|
||||||
selector.update("A", deltaA)
|
selector.update("A", deltaA)
|
||||||
selector.update("B", deltaB)
|
selector.update("B", deltaB)
|
||||||
val expected1 = DeltaPropagation(Map("A" → DataEnvelope(deltaA), "B" → DataEnvelope(deltaB)))
|
val expected1 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(deltaA), 1L, 1L),
|
||||||
|
"B" → Delta(DataEnvelope(deltaB), 1L, 1L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(0) → expected1, nodes(1) → expected1))
|
selector.collectPropagations() should ===(Map(nodes(0) → expected1, nodes(1) → expected1))
|
||||||
// new update before previous was propagated to all nodes
|
// new update before previous was propagated to all nodes
|
||||||
selector.update("C", deltaC)
|
selector.update("C", deltaC)
|
||||||
val expected2 = DeltaPropagation(Map("A" → DataEnvelope(deltaA), "B" → DataEnvelope(deltaB),
|
val expected2 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
"C" → DataEnvelope(deltaC)))
|
"A" → Delta(DataEnvelope(deltaA), 1L, 1L),
|
||||||
val expected3 = DeltaPropagation(Map("C" → DataEnvelope(deltaC)))
|
"B" → Delta(DataEnvelope(deltaB), 1L, 1L),
|
||||||
|
"C" → Delta(DataEnvelope(deltaC), 1L, 1L)))
|
||||||
|
val expected3 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"C" → Delta(DataEnvelope(deltaC), 1L, 1L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(2) → expected2, nodes(0) → expected3))
|
selector.collectPropagations() should ===(Map(nodes(2) → expected2, nodes(0) → expected3))
|
||||||
selector.cleanupDeltaEntries()
|
selector.cleanupDeltaEntries()
|
||||||
selector.hasDeltaEntries("A") should ===(false)
|
selector.hasDeltaEntries("A") should ===(false)
|
||||||
|
|
@ -88,17 +105,22 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
selector.hasDeltaEntries("C") should ===(false)
|
selector.hasDeltaEntries("C") should ===(false)
|
||||||
}
|
}
|
||||||
|
|
||||||
"merge updates that occur within same tick" in {
|
"bump version for each update" in {
|
||||||
val delta1 = GSet.empty[String] + "a1"
|
val delta1 = GSet.empty[String] + "a1"
|
||||||
val delta2 = GSet.empty[String] + "a2"
|
val delta2 = GSet.empty[String] + "a2"
|
||||||
val delta3 = GSet.empty[String] + "a3"
|
val delta3 = GSet.empty[String] + "a3"
|
||||||
val selector = new TestSelector(nodes.take(1))
|
val selector = new TestSelector(selfUniqueAddress, nodes.take(1))
|
||||||
selector.update("A", delta1)
|
selector.update("A", delta1)
|
||||||
|
selector.currentVersion("A") should ===(1L)
|
||||||
selector.update("A", delta2)
|
selector.update("A", delta2)
|
||||||
val expected1 = DeltaPropagation(Map("A" → DataEnvelope(delta1.merge(delta2))))
|
selector.currentVersion("A") should ===(2L)
|
||||||
|
val expected1 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(delta1.merge(delta2)), 1L, 2L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(0) → expected1))
|
selector.collectPropagations() should ===(Map(nodes(0) → expected1))
|
||||||
selector.update("A", delta3)
|
selector.update("A", delta3)
|
||||||
val expected2 = DeltaPropagation(Map("A" → DataEnvelope(delta3)))
|
selector.currentVersion("A") should ===(3L)
|
||||||
|
val expected2 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(delta3), 3L, 3L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(0) → expected2))
|
selector.collectPropagations() should ===(Map(nodes(0) → expected2))
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||||
}
|
}
|
||||||
|
|
@ -107,32 +129,37 @@ class DeltaPropagationSelectorSpec extends WordSpec with Matchers with TypeCheck
|
||||||
val delta1 = GSet.empty[String] + "a1"
|
val delta1 = GSet.empty[String] + "a1"
|
||||||
val delta2 = GSet.empty[String] + "a2"
|
val delta2 = GSet.empty[String] + "a2"
|
||||||
val delta3 = GSet.empty[String] + "a3"
|
val delta3 = GSet.empty[String] + "a3"
|
||||||
val selector = new TestSelector(nodes.take(3)) {
|
val selector = new TestSelector(selfUniqueAddress, nodes.take(3)) {
|
||||||
override def nodesSliceSize(allNodesSize: Int): Int = 1
|
override def nodesSliceSize(allNodesSize: Int): Int = 1
|
||||||
}
|
}
|
||||||
selector.update("A", delta1)
|
selector.update("A", delta1)
|
||||||
val expected1 = DeltaPropagation(Map("A" → DataEnvelope(delta1)))
|
val expected1 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(delta1), 1L, 1L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(0) → expected1))
|
selector.collectPropagations() should ===(Map(nodes(0) → expected1))
|
||||||
|
|
||||||
selector.update("A", delta2)
|
selector.update("A", delta2)
|
||||||
val expected2 = DeltaPropagation(Map("A" → DataEnvelope(delta1.merge(delta2))))
|
val expected2 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(delta1.merge(delta2)), 1L, 2L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(1) → expected2))
|
selector.collectPropagations() should ===(Map(nodes(1) → expected2))
|
||||||
|
|
||||||
selector.update("A", delta3)
|
selector.update("A", delta3)
|
||||||
val expected3 = DeltaPropagation(Map("A" → DataEnvelope(delta1.merge(delta2).merge(delta3))))
|
val expected3 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(delta1.merge(delta2).merge(delta3)), 1L, 3L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(2) → expected3))
|
selector.collectPropagations() should ===(Map(nodes(2) → expected3))
|
||||||
|
|
||||||
val expected4 = DeltaPropagation(Map("A" → DataEnvelope(delta2.merge(delta3))))
|
val expected4 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(delta2.merge(delta3)), 2L, 3L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(0) → expected4))
|
selector.collectPropagations() should ===(Map(nodes(0) → expected4))
|
||||||
|
|
||||||
val expected5 = DeltaPropagation(Map("A" → DataEnvelope(delta3)))
|
val expected5 = DeltaPropagation(selfUniqueAddress, Map(
|
||||||
|
"A" → Delta(DataEnvelope(delta3), 3L, 3L)))
|
||||||
selector.collectPropagations() should ===(Map(nodes(1) → expected5))
|
selector.collectPropagations() should ===(Map(nodes(1) → expected5))
|
||||||
|
|
||||||
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
selector.collectPropagations() should ===(Map.empty[Address, DeltaPropagation])
|
||||||
}
|
}
|
||||||
|
|
||||||
"calcualte right slice size" in {
|
"calcualte right slice size" in {
|
||||||
val selector = new TestSelector(nodes)
|
val selector = new TestSelector(selfUniqueAddress, nodes)
|
||||||
selector.nodesSliceSize(0) should ===(0)
|
selector.nodesSliceSize(0) should ===(0)
|
||||||
selector.nodesSliceSize(1) should ===(1)
|
selector.nodesSliceSize(1) should ===(1)
|
||||||
(2 to 9).foreach { n ⇒
|
(2 to 9).foreach { n ⇒
|
||||||
|
|
|
||||||
|
|
@ -30,9 +30,12 @@ class GCounterSpec extends WordSpec with Matchers {
|
||||||
c6.state(node1) should be(2)
|
c6.state(node1) should be(2)
|
||||||
c6.state(node2) should be(3)
|
c6.state(node2) should be(3)
|
||||||
|
|
||||||
c2.delta.state(node1) should be(1)
|
c2.delta.get.state(node1) should be(1)
|
||||||
c3.delta.state(node1) should be(2)
|
c1.mergeDelta(c2.delta.get) should be(c2)
|
||||||
c6.delta.state(node2) should be(3)
|
c3.delta.get.state(node1) should be(2)
|
||||||
|
c2.mergeDelta(c3.delta.get) should be(c3)
|
||||||
|
c6.delta.get.state(node2) should be(3)
|
||||||
|
c5.mergeDelta(c6.delta.get) should be(c6)
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to increment each node's record by arbitrary delta" in {
|
"be able to increment each node's record by arbitrary delta" in {
|
||||||
|
|
@ -95,13 +98,13 @@ class GCounterSpec extends WordSpec with Matchers {
|
||||||
merged1.state(node1) should be(7)
|
merged1.state(node1) should be(7)
|
||||||
merged1.state(node2) should be(10)
|
merged1.state(node2) should be(10)
|
||||||
merged1.value should be(17)
|
merged1.value should be(17)
|
||||||
merged1.delta should be(GCounter.empty)
|
merged1.delta should ===(None)
|
||||||
|
|
||||||
val merged2 = c26 merge c16
|
val merged2 = c26 merge c16
|
||||||
merged2.state(node1) should be(7)
|
merged2.state(node1) should be(7)
|
||||||
merged2.state(node2) should be(10)
|
merged2.state(node2) should be(10)
|
||||||
merged2.value should be(17)
|
merged2.value should be(17)
|
||||||
merged2.delta should be(GCounter.empty)
|
merged2.delta should ===(None)
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to have its history correctly merged with another GCounter 2" in {
|
"be able to have its history correctly merged with another GCounter 2" in {
|
||||||
|
|
|
||||||
|
|
@ -64,14 +64,14 @@ class GSetSpec extends WordSpec with Matchers {
|
||||||
val c12 = c11 + user1
|
val c12 = c11 + user1
|
||||||
val c13 = c12 + user2
|
val c13 = c12 + user2
|
||||||
|
|
||||||
c12.delta.elements should ===(Set(user1))
|
c12.delta.get.elements should ===(Set(user1))
|
||||||
c13.delta.elements should ===(Set(user1, user2))
|
c13.delta.get.elements should ===(Set(user1, user2))
|
||||||
|
|
||||||
// deltas build state
|
// deltas build state
|
||||||
(c12 merge c13.delta) should ===(c13)
|
(c12 mergeDelta c13.delta.get) should ===(c13)
|
||||||
|
|
||||||
// own deltas are idempotent
|
// own deltas are idempotent
|
||||||
(c13 merge c13.delta) should ===(c13)
|
(c13 mergeDelta c13.delta.get) should ===(c13)
|
||||||
|
|
||||||
// set 2
|
// set 2
|
||||||
val c21 = GSet.empty[String]
|
val c21 = GSet.empty[String]
|
||||||
|
|
@ -79,18 +79,18 @@ class GSetSpec extends WordSpec with Matchers {
|
||||||
val c22 = c21 + user3
|
val c22 = c21 + user3
|
||||||
val c23 = c22.resetDelta + user4
|
val c23 = c22.resetDelta + user4
|
||||||
|
|
||||||
c22.delta.elements should ===(Set(user3))
|
c22.delta.get.elements should ===(Set(user3))
|
||||||
c23.delta.elements should ===(Set(user4))
|
c23.delta.get.elements should ===(Set(user4))
|
||||||
|
|
||||||
c23.elements should ===(Set(user3, user4))
|
c23.elements should ===(Set(user3, user4))
|
||||||
|
|
||||||
val c33 = c13 merge c23
|
val c33 = c13 merge c23
|
||||||
|
|
||||||
// merge both ways
|
// merge both ways
|
||||||
val merged1 = GSet.empty[String] merge c12.delta merge c13.delta merge c22.delta merge c23.delta
|
val merged1 = GSet.empty[String] mergeDelta c12.delta.get mergeDelta c13.delta.get mergeDelta c22.delta.get mergeDelta c23.delta.get
|
||||||
merged1.elements should ===(Set(user1, user2, user3, user4))
|
merged1.elements should ===(Set(user1, user2, user3, user4))
|
||||||
|
|
||||||
val merged2 = GSet.empty[String] merge c23.delta merge c13.delta merge c22.delta
|
val merged2 = GSet.empty[String] mergeDelta c23.delta.get mergeDelta c13.delta.get mergeDelta c22.delta.get
|
||||||
merged2.elements should ===(Set(user1, user2, user3, user4))
|
merged2.elements should ===(Set(user1, user2, user3, user4))
|
||||||
|
|
||||||
merged1 should ===(c33)
|
merged1 should ===(c33)
|
||||||
|
|
|
||||||
|
|
@ -14,17 +14,17 @@ import org.scalatest.WordSpec
|
||||||
|
|
||||||
class ORSetSpec extends WordSpec with Matchers {
|
class ORSetSpec extends WordSpec with Matchers {
|
||||||
|
|
||||||
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
|
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1L)
|
||||||
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
|
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2L)
|
||||||
|
|
||||||
val nodeA = UniqueAddress(Address("akka.tcp", "Sys", "a", 2552), 1)
|
val nodeA = UniqueAddress(Address("akka.tcp", "Sys", "a", 2552), 1L)
|
||||||
val nodeB = UniqueAddress(nodeA.address.copy(host = Some("b")), 2)
|
val nodeB = UniqueAddress(nodeA.address.copy(host = Some("b")), 2L)
|
||||||
val nodeC = UniqueAddress(nodeA.address.copy(host = Some("c")), 3)
|
val nodeC = UniqueAddress(nodeA.address.copy(host = Some("c")), 3L)
|
||||||
val nodeD = UniqueAddress(nodeA.address.copy(host = Some("d")), 4)
|
val nodeD = UniqueAddress(nodeA.address.copy(host = Some("d")), 4L)
|
||||||
val nodeE = UniqueAddress(nodeA.address.copy(host = Some("e")), 5)
|
val nodeE = UniqueAddress(nodeA.address.copy(host = Some("e")), 5L)
|
||||||
val nodeF = UniqueAddress(nodeA.address.copy(host = Some("f")), 6)
|
val nodeF = UniqueAddress(nodeA.address.copy(host = Some("f")), 6L)
|
||||||
val nodeG = UniqueAddress(nodeA.address.copy(host = Some("g")), 7)
|
val nodeG = UniqueAddress(nodeA.address.copy(host = Some("g")), 7L)
|
||||||
val nodeH = UniqueAddress(nodeA.address.copy(host = Some("h")), 8)
|
val nodeH = UniqueAddress(nodeA.address.copy(host = Some("h")), 8L)
|
||||||
|
|
||||||
val user1 = """{"username":"john","password":"coltrane"}"""
|
val user1 = """{"username":"john","password":"coltrane"}"""
|
||||||
val user2 = """{"username":"sonny","password":"rollins"}"""
|
val user2 = """{"username":"sonny","password":"rollins"}"""
|
||||||
|
|
@ -226,6 +226,185 @@ class ORSetSpec extends WordSpec with Matchers {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
"ORSet deltas" must {
|
||||||
|
|
||||||
|
def addDeltaOp(s: ORSet[String]): ORSet.AddDeltaOp[String] =
|
||||||
|
asAddDeltaOp(s.delta.get)
|
||||||
|
|
||||||
|
def asAddDeltaOp(delta: Any): ORSet.AddDeltaOp[String] =
|
||||||
|
delta match {
|
||||||
|
case d: ORSet.AddDeltaOp[String] @unchecked ⇒ d
|
||||||
|
case _ ⇒ throw new IllegalArgumentException("Expected AddDeltaOp")
|
||||||
|
}
|
||||||
|
|
||||||
|
"work for additions" in {
|
||||||
|
val s1 = ORSet.empty[String]
|
||||||
|
val s2 = s1.add(node1, "a")
|
||||||
|
addDeltaOp(s2).underlying.elements should ===(Set("a"))
|
||||||
|
s1.mergeDelta(s2.delta.get) should ===(s2)
|
||||||
|
|
||||||
|
val s3 = s2.resetDelta.add(node1, "b").add(node1, "c")
|
||||||
|
addDeltaOp(s3).underlying.elements should ===(Set("b", "c"))
|
||||||
|
s2.mergeDelta(s3.delta.get) should ===(s3)
|
||||||
|
|
||||||
|
// another node adds "d"
|
||||||
|
val s4 = s3.resetDelta.add(node2, "d")
|
||||||
|
addDeltaOp(s4).underlying.elements should ===(Set("d"))
|
||||||
|
s3.mergeDelta(s4.delta.get) should ===(s4)
|
||||||
|
|
||||||
|
// concurrent update
|
||||||
|
val s5 = s3.resetDelta.add(node1, "e")
|
||||||
|
val s6 = s5.merge(s4)
|
||||||
|
s5.mergeDelta(s4.delta.get) should ===(s6)
|
||||||
|
|
||||||
|
// concurrent add of same element
|
||||||
|
val s7 = s3.resetDelta.add(node1, "d")
|
||||||
|
val s8 = s7.merge(s4)
|
||||||
|
// the dot contains both nodes
|
||||||
|
s8.elementsMap("d").contains(node1)
|
||||||
|
s8.elementsMap("d").contains(node2)
|
||||||
|
// and same result when merging the deltas
|
||||||
|
s7.mergeDelta(s4.delta.get) should ===(s8)
|
||||||
|
s4.mergeDelta(s7.delta.get) should ===(s8)
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle another concurrent add scenario" in {
|
||||||
|
val s1 = ORSet.empty[String]
|
||||||
|
val s2 = s1.add(node1, "a")
|
||||||
|
val s3 = s2.add(node1, "b")
|
||||||
|
val s4 = s2.add(node2, "c")
|
||||||
|
|
||||||
|
// full state merge for reference
|
||||||
|
val s5 = s4.merge(s3)
|
||||||
|
s5.elements should ===(Set("a", "b", "c"))
|
||||||
|
|
||||||
|
val s6 = s4.mergeDelta(s3.delta.get)
|
||||||
|
s6.elements should ===(Set("a", "b", "c"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"merge deltas into delta groups" in {
|
||||||
|
val s1 = ORSet.empty[String]
|
||||||
|
val s2 = s1.add(node1, "a")
|
||||||
|
val d2 = s2.delta.get
|
||||||
|
val s3 = s2.resetDelta.add(node1, "b")
|
||||||
|
val d3 = s3.delta.get
|
||||||
|
val d4 = d2 merge d3
|
||||||
|
asAddDeltaOp(d4).underlying.elements should ===(Set("a", "b"))
|
||||||
|
s1.mergeDelta(d4) should ===(s3)
|
||||||
|
s2.mergeDelta(d4) should ===(s3)
|
||||||
|
|
||||||
|
val s5 = s3.resetDelta.remove(node1, "b")
|
||||||
|
val d5 = s5.delta.get
|
||||||
|
val d6 = (d4 merge d5).asInstanceOf[ORSet.DeltaGroup[String]]
|
||||||
|
d6.ops.last.getClass should ===(classOf[ORSet.RemoveDeltaOp[String]])
|
||||||
|
d6.ops.size should ===(2)
|
||||||
|
s3.mergeDelta(d6) should ===(s5)
|
||||||
|
|
||||||
|
val s7 = s5.resetDelta.add(node1, "c")
|
||||||
|
val s8 = s7.resetDelta.add(node1, "d")
|
||||||
|
val d9 = (d6 merge s7.delta.get merge s8.delta.get).asInstanceOf[ORSet.DeltaGroup[String]]
|
||||||
|
// the add "c" and add "d" are merged into one AddDeltaOp
|
||||||
|
asAddDeltaOp(d9.ops.last).underlying.elements should ===(Set("c", "d"))
|
||||||
|
d9.ops.size should ===(3)
|
||||||
|
s5.mergeDelta(d9) should ===(s8)
|
||||||
|
s5.mergeDelta(s7.delta.get).mergeDelta(s8.delta.get) should ===(s8)
|
||||||
|
}
|
||||||
|
|
||||||
|
"work for removals" in {
|
||||||
|
val s1 = ORSet.empty[String]
|
||||||
|
val s2 = s1.add(node1, "a").add(node1, "b").resetDelta
|
||||||
|
val s3 = s2.remove(node1, "b")
|
||||||
|
s2.merge(s3) should ===(s3)
|
||||||
|
s2.mergeDelta(s3.delta.get) should ===(s3)
|
||||||
|
s2.mergeDelta(s3.delta.get).elements should ===(Set("a"))
|
||||||
|
|
||||||
|
// concurrent update
|
||||||
|
val s4 = s2.add(node2, "c").resetDelta
|
||||||
|
val s5 = s4.merge(s3)
|
||||||
|
s5.elements should ===(Set("a", "c"))
|
||||||
|
s4.mergeDelta(s3.delta.get) should ===(s5)
|
||||||
|
|
||||||
|
// add "b" again
|
||||||
|
val s6 = s5.add(node2, "b")
|
||||||
|
// merging the old delta should not remove it
|
||||||
|
s6.mergeDelta(s3.delta.get) should ===(s6)
|
||||||
|
s6.mergeDelta(s3.delta.get).elements should ===(Set("a", "b", "c"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"work for clear" in {
|
||||||
|
val s1 = ORSet.empty[String]
|
||||||
|
val s2 = s1.add(node1, "a").add(node1, "b")
|
||||||
|
val s3 = s2.resetDelta.clear(node1)
|
||||||
|
val s4 = s3.resetDelta.add(node1, "c")
|
||||||
|
s2.merge(s3) should ===(s3)
|
||||||
|
s2.mergeDelta(s3.delta.get) should ===(s3)
|
||||||
|
val s5 = s2.mergeDelta(s3.delta.get).mergeDelta(s4.delta.get)
|
||||||
|
s5.elements should ===(Set("c"))
|
||||||
|
s5 should ===(s4)
|
||||||
|
|
||||||
|
// concurrent update
|
||||||
|
val s6 = s2.resetDelta.add(node2, "d")
|
||||||
|
val s7 = s6.merge(s3)
|
||||||
|
s7.elements should ===(Set("d"))
|
||||||
|
s6.mergeDelta(s3.delta.get) should ===(s7)
|
||||||
|
|
||||||
|
// add "b" again
|
||||||
|
val s8 = s7.add(node2, "b")
|
||||||
|
// merging the old delta should not remove it
|
||||||
|
s8.mergeDelta(s3.delta.get) should ===(s8)
|
||||||
|
s8.mergeDelta(s3.delta.get).elements should ===(Set("b", "d"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle a mixed add/remove scenario" in {
|
||||||
|
val s1 = ORSet.empty[String]
|
||||||
|
val s2 = s1.resetDelta.remove(node1, "e")
|
||||||
|
val s3 = s2.resetDelta.add(node1, "b")
|
||||||
|
val s4 = s3.resetDelta.add(node1, "a")
|
||||||
|
val s5 = s4.resetDelta.remove(node1, "b")
|
||||||
|
|
||||||
|
val deltaGroup1 = s3.delta.get merge s4.delta.get merge s5.delta.get
|
||||||
|
|
||||||
|
val s7 = s2 mergeDelta deltaGroup1
|
||||||
|
s7.elements should ===(Set("a"))
|
||||||
|
// The above scenario was constructed from failing ReplicatorDeltaSpec,
|
||||||
|
// some more checks...
|
||||||
|
|
||||||
|
val s8 = s2.resetDelta.add(node2, "z") // concurrent update from node2
|
||||||
|
val s9 = s8 mergeDelta deltaGroup1
|
||||||
|
s9.elements should ===(Set("a", "z"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"require causal delivery of deltas" in {
|
||||||
|
// This test illustrates why we need causal delivery of deltas.
|
||||||
|
// Otherwise the following could happen.
|
||||||
|
|
||||||
|
// s0 is the stable state that is initially replicated to all nodes
|
||||||
|
val s0 = ORSet.empty[String].add(node1, "a")
|
||||||
|
|
||||||
|
// add element "b" and "c" at node1
|
||||||
|
val s11 = s0.resetDelta.add(node1, "b")
|
||||||
|
val s12 = s11.resetDelta.add(node1, "c")
|
||||||
|
|
||||||
|
// at the same time, add element "d" at node2
|
||||||
|
val s21 = s0.resetDelta.add(node2, "d")
|
||||||
|
|
||||||
|
// node3 receives delta for "d" and "c", but the delta for "b" is lost
|
||||||
|
val s31 = s0 mergeDelta s21.delta.get mergeDelta s12.delta.get
|
||||||
|
s31.elements should ===(Set("a", "c", "d"))
|
||||||
|
|
||||||
|
// node4 receives all deltas
|
||||||
|
val s41 = s0 mergeDelta s11.delta.get mergeDelta s12.delta.get mergeDelta s21.delta.get
|
||||||
|
s41.elements should ===(Set("a", "b", "c", "d"))
|
||||||
|
|
||||||
|
// node3 and node4 sync with full state gossip
|
||||||
|
val s32 = s31 merge s41
|
||||||
|
// one would expect elements "a", "b", "c", "d", but "b" is removed
|
||||||
|
// because we applied s12.delta without applying s11.delta
|
||||||
|
s32.elements should ===(Set("a", "c", "d"))
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
"ORSet unit test" must {
|
"ORSet unit test" must {
|
||||||
"verify subtractDots" in {
|
"verify subtractDots" in {
|
||||||
val dot = VersionVector(TreeMap(nodeA → 3L, nodeB → 2L, nodeD → 14L, nodeG → 22L))
|
val dot = VersionVector(TreeMap(nodeA → 3L, nodeB → 2L, nodeD → 14L, nodeG → 22L))
|
||||||
|
|
|
||||||
|
|
@ -29,13 +29,13 @@ class PNCounterSpec extends WordSpec with Matchers {
|
||||||
c6.increments.state(node1) should be(2)
|
c6.increments.state(node1) should be(2)
|
||||||
c6.increments.state(node2) should be(3)
|
c6.increments.state(node2) should be(3)
|
||||||
|
|
||||||
c2.delta.value.toLong should be(1)
|
c2.delta.get.value.toLong should be(1)
|
||||||
c2.delta.increments.state(node1) should be(1)
|
c2.delta.get.increments.state(node1) should be(1)
|
||||||
c3.delta.value should be(2)
|
c3.delta.get.value should be(2)
|
||||||
c3.delta.increments.state(node1) should be(2)
|
c3.delta.get.increments.state(node1) should be(2)
|
||||||
|
|
||||||
c6.delta.value should be(3)
|
c6.delta.get.value should be(3)
|
||||||
c6.delta.increments.state(node2) should be(3)
|
c6.delta.get.increments.state(node2) should be(3)
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to decrement each node's record by one" in {
|
"be able to decrement each node's record by one" in {
|
||||||
|
|
@ -51,11 +51,11 @@ class PNCounterSpec extends WordSpec with Matchers {
|
||||||
c6.decrements.state(node1) should be(2)
|
c6.decrements.state(node1) should be(2)
|
||||||
c6.decrements.state(node2) should be(3)
|
c6.decrements.state(node2) should be(3)
|
||||||
|
|
||||||
c3.delta.value should be(-2)
|
c3.delta.get.value should be(-2)
|
||||||
c3.delta.decrements.state(node1) should be(2)
|
c3.delta.get.decrements.state(node1) should be(2)
|
||||||
|
|
||||||
c6.delta.value should be(-3)
|
c6.delta.get.value should be(-3)
|
||||||
c6.delta.decrements.state(node2) should be(3)
|
c6.delta.get.decrements.state(node2) should be(3)
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to increment each node's record by arbitrary delta" in {
|
"be able to increment each node's record by arbitrary delta" in {
|
||||||
|
|
|
||||||
|
|
@ -101,9 +101,17 @@ class ReplicatedDataSerializerSpec extends TestKit(ActorSystem(
|
||||||
checkSameContent(s3.merge(s4), s4.merge(s3))
|
checkSameContent(s3.merge(s4), s4.merge(s3))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
"serialize ORSet delta" in {
|
||||||
|
checkSerialization(ORSet().add(address1, "a").delta.get)
|
||||||
|
checkSerialization(ORSet().add(address1, "a").resetDelta.remove(address2, "a").delta.get)
|
||||||
|
checkSerialization(ORSet().add(address1, "a").remove(address2, "a").delta.get)
|
||||||
|
checkSerialization(ORSet().add(address1, "a").resetDelta.clear(address2).delta.get)
|
||||||
|
checkSerialization(ORSet().add(address1, "a").clear(address2).delta.get)
|
||||||
|
}
|
||||||
|
|
||||||
"serialize large GSet" in {
|
"serialize large GSet" in {
|
||||||
val largeSet = (10000 until 20000).foldLeft(GSet.empty[String]) {
|
val largeSet = (10000 until 20000).foldLeft(GSet.empty[String]) {
|
||||||
case (acc, n) ⇒ acc.add(n.toString)
|
case (acc, n) ⇒ acc.resetDelta.add(n.toString)
|
||||||
}
|
}
|
||||||
val numberOfBytes = checkSerialization(largeSet)
|
val numberOfBytes = checkSerialization(largeSet)
|
||||||
info(s"size of GSet with ${largeSet.size} elements: $numberOfBytes bytes")
|
info(s"size of GSet with ${largeSet.size} elements: $numberOfBytes bytes")
|
||||||
|
|
@ -118,7 +126,7 @@ class ReplicatedDataSerializerSpec extends TestKit(ActorSystem(
|
||||||
case 1 ⇒ address2
|
case 1 ⇒ address2
|
||||||
case 2 ⇒ address3
|
case 2 ⇒ address3
|
||||||
}
|
}
|
||||||
acc.add(address, n.toString)
|
acc.resetDelta.add(address, n.toString)
|
||||||
}
|
}
|
||||||
val numberOfBytes = checkSerialization(largeSet)
|
val numberOfBytes = checkSerialization(largeSet)
|
||||||
// note that ORSet is compressed, and therefore smaller than GSet
|
// note that ORSet is compressed, and therefore smaller than GSet
|
||||||
|
|
|
||||||
|
|
@ -87,9 +87,9 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem(
|
||||||
checkSerialization(Gossip(Map(
|
checkSerialization(Gossip(Map(
|
||||||
"A" → DataEnvelope(data1),
|
"A" → DataEnvelope(data1),
|
||||||
"B" → DataEnvelope(GSet() + "b" + "c")), sendBack = true))
|
"B" → DataEnvelope(GSet() + "b" + "c")), sendBack = true))
|
||||||
checkSerialization(DeltaPropagation(Map(
|
checkSerialization(DeltaPropagation(address1, Map(
|
||||||
"A" → DataEnvelope(delta1),
|
"A" → Delta(DataEnvelope(delta1), 1L, 1L),
|
||||||
"B" → DataEnvelope(delta2))))
|
"B" → Delta(DataEnvelope(delta2), 3L, 5L))))
|
||||||
checkSerialization(new DurableDataEnvelope(data1))
|
checkSerialization(new DurableDataEnvelope(data1))
|
||||||
checkSerialization(new DurableDataEnvelope(DataEnvelope(data1, pruning = Map(
|
checkSerialization(new DurableDataEnvelope(DataEnvelope(data1, pruning = Map(
|
||||||
address1 → PruningPerformed(System.currentTimeMillis()),
|
address1 → PruningPerformed(System.currentTimeMillis()),
|
||||||
|
|
|
||||||
|
|
@ -272,19 +272,22 @@ for updates. For example adding element ``'c'`` and ``'d'`` to set ``{'a', 'b'}`
|
||||||
result in sending the delta ``{'c', 'd'}`` and merge that with the state on the
|
result in sending the delta ``{'c', 'd'}`` and merge that with the state on the
|
||||||
receiving side, resulting in set ``{'a', 'b', 'c', 'd'}``.
|
receiving side, resulting in set ``{'a', 'b', 'c', 'd'}``.
|
||||||
|
|
||||||
Current protocol for replicating the deltas does not support causal consistency.
|
The protocol for replicating the deltas supports causal consistency if the data type
|
||||||
It is only eventually consistent. This means that if elements ``'c'`` and ``'d'`` are
|
is marked with ``RequiresCausalDeliveryOfDeltas``. Otherwise it is only eventually
|
||||||
|
consistent. Without causal consistency it means that if elements ``'c'`` and ``'d'`` are
|
||||||
added in two separate `Update` operations these deltas may occasionally be propagated
|
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
|
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
|
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
|
it will be ``{'a', 'b', 'c', 'd'}``.
|
||||||
should be disabled with configuration property
|
|
||||||
``akka.cluster.distributed-data.delta-crdt.enabled=off``.
|
|
||||||
|
|
||||||
Note that the full state is occasionally also replicated for delta-CRDTs, for example when
|
Note that the full state is occasionally also replicated for delta-CRDTs, for example when
|
||||||
new nodes are added to the cluster or when deltas could not be propagated because
|
new nodes are added to the cluster or when deltas could not be propagated because
|
||||||
of network partitions or similar problems.
|
of network partitions or similar problems.
|
||||||
|
|
||||||
|
The the delta propagation can be disabled with configuration property::
|
||||||
|
|
||||||
|
akka.cluster.distributed-data.delta-crdt.enabled=off
|
||||||
|
|
||||||
Data Types
|
Data Types
|
||||||
==========
|
==========
|
||||||
|
|
||||||
|
|
@ -316,7 +319,8 @@ The value of the counter is the value of the P counter minus the value of the N
|
||||||
|
|
||||||
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#pncounter
|
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#pncounter
|
||||||
|
|
||||||
``GCounter`` and ``PNCounter`` have support for :ref:`delta_crdt_java`.
|
``GCounter`` and ``PNCounter`` have support for :ref:`delta_crdt_java` and don't need causal
|
||||||
|
delivery of deltas.
|
||||||
|
|
||||||
Several related counters can be managed in a map with the ``PNCounterMap`` data type.
|
Several related counters can be managed in a map with the ``PNCounterMap`` data type.
|
||||||
When the counters are placed in a ``PNCounterMap`` as opposed to placing them as separate top level
|
When the counters are placed in a ``PNCounterMap`` as opposed to placing them as separate top level
|
||||||
|
|
@ -334,6 +338,8 @@ Merge is simply the union of the two sets.
|
||||||
|
|
||||||
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#gset
|
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#gset
|
||||||
|
|
||||||
|
``GSet`` has support for :ref:`delta_crdt_java` and it doesn't require causal delivery of deltas.
|
||||||
|
|
||||||
If you need add and remove operations you should use the ``ORSet`` (observed-remove set).
|
If you need add and remove operations you should use the ``ORSet`` (observed-remove set).
|
||||||
Elements can be added and removed any number of times. If an element is concurrently added and
|
Elements can be added and removed any number of times. If an element is concurrently added and
|
||||||
removed, the add will win. You cannot remove an element that you have not seen.
|
removed, the add will win. You cannot remove an element that you have not seen.
|
||||||
|
|
@ -345,6 +351,8 @@ track causality of the operations and resolve concurrent updates.
|
||||||
|
|
||||||
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#orset
|
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#orset
|
||||||
|
|
||||||
|
``ORSet`` has support for :ref:`delta_crdt_java` and it requires causal delivery of deltas.
|
||||||
|
|
||||||
Maps
|
Maps
|
||||||
----
|
----
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,7 @@
|
||||||
|
|
||||||
package docs.serialization {
|
package docs.serialization {
|
||||||
|
|
||||||
import akka.actor.{ExtensionId, ExtensionIdProvider}
|
import akka.actor.{ ExtensionId, ExtensionIdProvider }
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
//#imports
|
//#imports
|
||||||
import akka.actor.{ ActorRef, ActorSystem }
|
import akka.actor.{ ActorRef, ActorSystem }
|
||||||
|
|
|
||||||
|
|
@ -285,19 +285,22 @@ for updates. For example adding element ``'c'`` and ``'d'`` to set ``{'a', 'b'}`
|
||||||
result in sending the delta ``{'c', 'd'}`` and merge that with the state on the
|
result in sending the delta ``{'c', 'd'}`` and merge that with the state on the
|
||||||
receiving side, resulting in set ``{'a', 'b', 'c', 'd'}``.
|
receiving side, resulting in set ``{'a', 'b', 'c', 'd'}``.
|
||||||
|
|
||||||
Current protocol for replicating the deltas does not support causal consistency.
|
The protocol for replicating the deltas supports causal consistency if the data type
|
||||||
It is only eventually consistent. This means that if elements ``'c'`` and ``'d'`` are
|
is marked with ``RequiresCausalDeliveryOfDeltas``. Otherwise it is only eventually
|
||||||
|
consistent. Without causal consistency it means that if elements ``'c'`` and ``'d'`` are
|
||||||
added in two separate `Update` operations these deltas may occasionally be propagated
|
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
|
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
|
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
|
it will be ``{'a', 'b', 'c', 'd'}``.
|
||||||
should be disabled with configuration property
|
|
||||||
``akka.cluster.distributed-data.delta-crdt.enabled=off``.
|
|
||||||
|
|
||||||
Note that the full state is occasionally also replicated for delta-CRDTs, for example when
|
Note that the full state is occasionally also replicated for delta-CRDTs, for example when
|
||||||
new nodes are added to the cluster or when deltas could not be propagated because
|
new nodes are added to the cluster or when deltas could not be propagated because
|
||||||
of network partitions or similar problems.
|
of network partitions or similar problems.
|
||||||
|
|
||||||
|
The the delta propagation can be disabled with configuration property::
|
||||||
|
|
||||||
|
akka.cluster.distributed-data.delta-crdt.enabled=off
|
||||||
|
|
||||||
Data Types
|
Data Types
|
||||||
==========
|
==========
|
||||||
|
|
||||||
|
|
@ -329,7 +332,8 @@ The value of the counter is the value of the P counter minus the value of the N
|
||||||
|
|
||||||
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#pncounter
|
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#pncounter
|
||||||
|
|
||||||
``GCounter`` and ``PNCounter`` have support for :ref:`delta_crdt_scala`.
|
``GCounter`` and ``PNCounter`` have support for :ref:`delta_crdt_scala` and don't need causal
|
||||||
|
delivery of deltas.
|
||||||
|
|
||||||
Several related counters can be managed in a map with the ``PNCounterMap`` data type.
|
Several related counters can be managed in a map with the ``PNCounterMap`` data type.
|
||||||
When the counters are placed in a ``PNCounterMap`` as opposed to placing them as separate top level
|
When the counters are placed in a ``PNCounterMap`` as opposed to placing them as separate top level
|
||||||
|
|
@ -347,6 +351,8 @@ Merge is simply the union of the two sets.
|
||||||
|
|
||||||
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#gset
|
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#gset
|
||||||
|
|
||||||
|
``GSet`` has support for :ref:`delta_crdt_scala` and it doesn't require causal delivery of deltas.
|
||||||
|
|
||||||
If you need add and remove operations you should use the ``ORSet`` (observed-remove set).
|
If you need add and remove operations you should use the ``ORSet`` (observed-remove set).
|
||||||
Elements can be added and removed any number of times. If an element is concurrently added and
|
Elements can be added and removed any number of times. If an element is concurrently added and
|
||||||
removed, the add will win. You cannot remove an element that you have not seen.
|
removed, the add will win. You cannot remove an element that you have not seen.
|
||||||
|
|
@ -358,6 +364,8 @@ track causality of the operations and resolve concurrent updates.
|
||||||
|
|
||||||
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#orset
|
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#orset
|
||||||
|
|
||||||
|
``ORSet`` has support for :ref:`delta_crdt_scala` and it requires causal delivery of deltas.
|
||||||
|
|
||||||
Maps
|
Maps
|
||||||
----
|
----
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -103,6 +103,14 @@ object MiMa extends AutoPlugin {
|
||||||
// #21875 delta-CRDT
|
// #21875 delta-CRDT
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.GCounter.this"),
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.GCounter.this"),
|
||||||
|
|
||||||
|
// #22188 ORSet delta-CRDT
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.ORSet.this"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.protobuf.SerializationSupport.versionVectorToProto"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.protobuf.SerializationSupport.versionVectorFromProto"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.protobuf.SerializationSupport.versionVectorFromBinary"),
|
||||||
|
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.ddata.protobuf.ReplicatedDataSerializer.versionVectorToProto"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.ddata.protobuf.ReplicatedDataSerializer.versionVectorFromProto"),
|
||||||
|
|
||||||
// #22141 sharding minCap
|
// #22141 sharding minCap
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.sharding.DDataShardCoordinator.updatingStateTimeout"),
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.sharding.DDataShardCoordinator.updatingStateTimeout"),
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.sharding.DDataShardCoordinator.waitingForStateTimeout"),
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.sharding.DDataShardCoordinator.waitingForStateTimeout"),
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue