Skip to content

Commit

Permalink
causal delivery of deltas, akka#22188
Browse files Browse the repository at this point in the history
* 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
  • Loading branch information
patriknw committed Feb 22, 2017
1 parent 94afbee commit b700b84
Show file tree
Hide file tree
Showing 41 changed files with 6,898 additions and 3,838 deletions.

Large diffs are not rendered by default.

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,21 @@ message ORSet {
repeated sint32 intElements = 4 [packed=true];
repeated sint64 longElements = 5 [packed=true];
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 {
Expand All @@ -48,14 +62,6 @@ message PNCounter {
required GCounter decrements = 2;
}

message VersionVector {
message Entry {
required UniqueAddress node = 1;
required int64 version = 2;
}
repeated Entry entries = 1;
}

message ORMap {
message Entry {
optional string stringKey = 1;
Expand Down
14 changes: 13 additions & 1 deletion akka-distributed-data/src/main/protobuf/ReplicatorMessages.proto
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ message DataEnvelope {

required OtherMessage data = 1;
repeated PruningEntry pruning = 2;
optional VersionVector deltaVersions = 3;
}

message Status {
Expand Down Expand Up @@ -100,9 +101,12 @@ message DeltaPropagation {
message Entry {
required string key = 1;
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 {
Expand All @@ -117,6 +121,14 @@ message Address {
required uint32 port = 2;
}

message VersionVector {
message Entry {
required UniqueAddress node = 1;
required int64 version = 2;
}
repeated Entry entries = 1;
}

message OtherMessage {
required bytes enclosedMessage = 1;
required int32 serializerId = 2;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,56 +4,61 @@
package akka.cluster.ddata

import scala.collection.immutable.TreeMap
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation

import akka.actor.Address
import akka.cluster.ddata.Replicator.Internal.DataEnvelope
import akka.annotation.InternalApi
import akka.cluster.ddata.Key.KeyId
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation

/**
* INTERNAL API: Used by the Replicator actor.
* Extracted to separate trait to make it easy to test.
*/
private[akka] trait DeltaPropagationSelector {
@InternalApi private[akka] trait DeltaPropagationSelector {

private var _propagationCount = 0L
def propagationCount: Long = _propagationCount
private var deltaCounter = Map.empty[String, Long]
private var deltaEntries = Map.empty[String, TreeMap[Long, ReplicatedData]]
private var deltaSentToNode = Map.empty[String, Map[Address, Long]]
private var deltaCounter = Map.empty[KeyId, Long]
private var deltaEntries = Map.empty[KeyId, TreeMap[Long, ReplicatedData]]
private var deltaSentToNode = Map.empty[KeyId, Map[Address, Long]]
private var deltaNodeRoundRobinCounter = 0L

def divisor: Int
def gossipIntervalDivisor: Int

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 = {
val c = deltaCounter.get(key) match {
case Some(c) c
case None
deltaCounter = deltaCounter.updated(key, 1L)
1L
def currentVersion(key: KeyId): Long = deltaCounter.get(key) match {
case Some(v) v
case None 0L
}

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
}
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)
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: String): Unit = {
def delete(key: KeyId): Unit = {
deltaEntries -= key
deltaCounter -= key
deltaSentToNode -= key
}

def nodesSliceSize(allNodesSize: Int): Int = {
// 2 - 10 nodes
math.min(math.max((allNodesSize / divisor) + 1, 2), math.min(allNodesSize, 10))
math.min(math.max((allNodesSize / gossipIntervalDivisor) + 1, 2), math.min(allNodesSize, 10))
}

def collectPropagations(): Map[Address, DeltaPropagation] = {
Expand All @@ -80,20 +85,32 @@ private[akka] trait DeltaPropagationSelector {

var result = Map.empty[Address, DeltaPropagation]

var cache = Map.empty[(KeyId, Long, Long), ReplicatedData]
slice.foreach { node
// collect the deltas that have not already been sent to the node and merge
// them into a delta group
var deltas = Map.empty[String, ReplicatedData]
var deltas = Map.empty[KeyId, (ReplicatedData, Long, Long)]
deltaEntries.foreach {
case (key, entries)
val deltaSentToNodeForKey = deltaSentToNode.getOrElse(key, TreeMap.empty[Address, Long])
val j = deltaSentToNodeForKey.getOrElse(node, 0L)
val deltaEntriesAfterJ = deltaEntriesAfter(entries, j)
if (deltaEntriesAfterJ.nonEmpty) {
val deltaGroup = deltaEntriesAfterJ.valuesIterator.reduceLeft {
(d1, d2) d1.merge(d2.asInstanceOf[d1.T])
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])
}
cache = cache.updated(cacheKey, group)
group
case Some(group) group
}
deltas = deltas.updated(key, deltaGroup)
deltas = deltas.updated(key, (deltaGroup, fromSeqNr, toSeqNr))
deltaSentToNode = deltaSentToNode.updated(key, deltaSentToNodeForKey.updated(node, deltaEntriesAfterJ.lastKey))
}
}
Expand All @@ -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
}
}
Expand All @@ -126,14 +134,14 @@ private[akka] trait DeltaPropagationSelector {
case ntrs ntrs
}

def hasDeltaEntries(key: String): Boolean = {
def hasDeltaEntries(key: KeyId): Boolean = {
deltaEntries.get(key) match {
case Some(m) m.nonEmpty
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 {
case None 0L
case Some(deltaSentToNodeForKey)
Expand All @@ -154,7 +162,7 @@ private[akka] trait DeltaPropagationSelector {

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
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,9 @@ import akka.actor.ActorRef
import akka.actor.DeadLetterSuppression
import akka.actor.Props
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.ReplicatorMessage
import akka.io.DirectByteBufferPool
import akka.serialization.SerializationExtension
import akka.serialization.SerializerWithStringManifest
Expand Down Expand Up @@ -53,7 +54,7 @@ object DurableStore {
* should be used to signal success or failure of the operation to the contained
* `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)

/**
Expand All @@ -66,7 +67,7 @@ object DurableStore {
* will stop itself and the durable store.
*/
case object LoadAll
final case class LoadData(data: Map[String, DurableDataEnvelope])
final case class LoadData(data: Map[KeyId, DurableDataEnvelope])
case object LoadAllCompleted
class LoadFailed(message: String, cause: Throwable) extends RuntimeException(message, cause) {
def this(message: String) = this(message, null)
Expand Down Expand Up @@ -143,7 +144,7 @@ final class LmdbDurableStore(config: Config) extends Actor with ActorLogging {
}

// 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 = {
super.postRestart(reason)
Expand Down Expand Up @@ -227,7 +228,7 @@ final class LmdbDurableStore(config: Config) extends Actor with ActorLogging {
writeBehind()
}

def dbPut(tx: OptionVal[Txn[ByteBuffer]], key: String, data: DurableDataEnvelope): Unit = {
def dbPut(tx: OptionVal[Txn[ByteBuffer]], key: KeyId, data: DurableDataEnvelope): Unit = {
try {
keyBuffer.put(key.getBytes(ByteString.UTF_8)).flip()
val value = serializer.toBinary(data)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,8 @@
*/
package akka.cluster.ddata

import akka.annotation.InternalApi

/**
* INTERNAL API
*
Expand All @@ -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
* 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

/** 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 = {
newData.ancestor = if (this.ancestor eq null) this else this.ancestor
this.ancestor = null // only one level, for GC
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ package akka.cluster.ddata
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
import java.math.BigInteger
import akka.annotation.InternalApi

object GCounter {
val empty: GCounter = new GCounter
Expand Down Expand Up @@ -39,13 +40,15 @@ object GCounter {
*/
@SerialVersionUID(1L)
final class GCounter private[akka] (
private[akka] val state: Map[UniqueAddress, BigInt] = Map.empty,
private[akka] val _delta: Option[GCounter] = None)
extends DeltaReplicatedData with ReplicatedDataSerialization with RemovedNodePruning with FastMerge {
private[akka] val state: Map[UniqueAddress, BigInt] = Map.empty,
override val delta: Option[GCounter] = None)
extends DeltaReplicatedData with ReplicatedDelta
with ReplicatedDataSerialization with RemovedNodePruning with FastMerge {

import GCounter.Zero

type T = GCounter
type D = GCounter

/**
* Scala API: Current total value of the counter.
Expand Down Expand Up @@ -73,24 +76,24 @@ final class GCounter private[akka] (
/**
* 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
*/
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")
if (n == 0) this
else {
val nextValue = state.get(key) match {
case Some(v) v + n
case None n
}
val newDelta = _delta match {
case Some(d) Some(new GCounter(d.state + (key nextValue)))
case None Some(new GCounter(Map(key nextValue)))
val newDelta = delta match {
case None 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)))
}
}

Expand All @@ -108,12 +111,13 @@ final class GCounter private[akka] (
new GCounter(merged)
}

override def delta: GCounter = _delta match {
case Some(d) d
case None GCounter.empty
}
override def mergeDelta(thatDelta: GCounter): GCounter = merge(thatDelta)

override def zero: GCounter = GCounter.empty

override def resetDelta: GCounter = new GCounter(state)
override def resetDelta: GCounter =
if (delta.isEmpty) this
else assignAncestor(new GCounter(state))

override def modifiedByNodes: Set[UniqueAddress] = state.keySet

Expand Down
Loading

0 comments on commit b700b84

Please sign in to comment.