+cdd #16799 Add Distributed Data module

Previously know as [patriknw/akka-data-replication](https://github.com/patriknw/akka-data-replication),
which was originally inspired by [jboner/akka-crdt](https://github.com/jboner/akka-crdt).

The functionality is very similar to akka-data-replication 0.11.

Here is a list of the most important changes:

* The package name changed to `akka.cluster.ddata`
* The extension was renamed to `DistributedData`
* The keys changed from strings to classes with unique identifiers and type information of the data values,
  e.g. `ORSetKey[Int]("set2")`
* The optional read consistency parameter was removed from the `Update` message. If you need to read from
  other replicas before performing the update you have to first send a `Get` message and then continue with
  the ``Update`` when the ``GetSuccess`` is received.
* `BigInt` is used in `GCounter` and `PNCounter` instead of `Long`
* Improvements of java api
* Better documentation
This commit is contained in:
Patrik Nordwall 2015-05-17 12:28:47 +02:00
parent bf28260cd0
commit cbe5dd2cf5
69 changed files with 40036 additions and 3 deletions

View file

@ -0,0 +1,14 @@
import akka.{ AkkaBuild, Dependencies, Formatting, MultiNode, Unidoc, OSGi }
import com.typesafe.tools.mima.plugin.MimaKeys
AkkaBuild.defaultSettings
Formatting.formatSettings
OSGi.distributedData
Dependencies.distributedData
//MimaKeys.previousArtifact := akkaPreviousArtifact("akka-distributed-data").value
enablePlugins(MultiNodeScalaTest)

View file

@ -0,0 +1,92 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata;
option java_package = "akka.cluster.ddata.protobuf.msg";
option optimize_for = SPEED;
import "ReplicatorMessages.proto";
message GSet {
repeated string stringElements = 1;
repeated sint32 intElements = 2 [packed=true];
repeated sint64 longElements = 3 [packed=true];
repeated OtherMessage otherElements = 4;
}
message ORSet {
required VersionVector vvector = 1;
repeated VersionVector dots = 2;
repeated string stringElements = 3;
repeated sint32 intElements = 4 [packed=true];
repeated sint64 longElements = 5 [packed=true];
repeated OtherMessage otherElements = 6;
}
message Flag {
required bool enabled = 1;
}
message LWWRegister {
required sint64 timestamp = 1;
required UniqueAddress node = 2;
required OtherMessage state = 3;
}
message GCounter {
message Entry {
required UniqueAddress node = 1;
required bytes value = 2;
}
repeated Entry entries = 1;
}
message PNCounter {
required GCounter increments = 1;
required GCounter decrements = 2;
}
message VersionVector {
message Entry {
required UniqueAddress node = 1;
required int64 version = 2;
}
repeated Entry entries = 1;
}
message ORMap {
message Entry {
required string key = 1;
required OtherMessage value = 2;
}
required ORSet keys = 1;
repeated Entry entries = 2;
}
message LWWMap {
message Entry {
required string key = 1;
required LWWRegister value = 2;
}
required ORSet keys = 1;
repeated Entry entries = 2;
}
message PNCounterMap {
message Entry {
required string key = 1;
required PNCounter value = 2;
}
required ORSet keys = 1;
repeated Entry entries = 2;
}

View file

@ -0,0 +1,118 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata;
option java_package = "akka.cluster.ddata.protobuf.msg";
option optimize_for = SPEED;
message Get {
required OtherMessage key = 1;
required sint32 consistency = 2;
required uint32 timeout = 3;
optional OtherMessage request = 4;
}
message GetSuccess {
required OtherMessage key = 1;
required OtherMessage data = 2;
optional OtherMessage request = 4;
}
message NotFound {
required OtherMessage key = 1;
optional OtherMessage request = 2;
}
message GetFailure {
required OtherMessage key = 1;
optional OtherMessage request = 2;
}
message Subscribe {
required OtherMessage key = 1;
required string ref = 2;
}
message Unsubscribe {
required OtherMessage key = 1;
required string ref = 2;
}
message Changed {
required OtherMessage key = 1;
required OtherMessage data = 2;
}
message Write {
required string key = 1;
required DataEnvelope envelope = 2;
}
// message WriteAck, via Empty
message Empty {
}
message Read {
required string key = 1;
}
message ReadResult {
optional DataEnvelope envelope = 1;
}
message DataEnvelope {
message PruningEntry {
required UniqueAddress removedAddress = 1;
required UniqueAddress ownerAddress = 2;
required bool performed = 3;
repeated Address seen = 4;
}
required OtherMessage data = 1;
repeated PruningEntry pruning = 2;
}
message Status {
message Entry {
required string key = 1;
required bytes digest = 2;
}
required uint32 chunk = 1;
required uint32 totChunks = 2;
repeated Entry entries = 3;
}
message Gossip {
message Entry {
required string key = 1;
required DataEnvelope envelope = 2;
}
required bool sendBack = 1;
repeated Entry entries = 2;
}
message UniqueAddress {
required Address address = 1;
required sfixed32 uid = 2;
}
message Address {
required string hostname = 1;
required uint32 port = 2;
}
message OtherMessage {
required bytes enclosedMessage = 1;
required int32 serializerId = 2;
optional bytes messageManifest = 4;
}
message StringGSet {
repeated string elements = 1;
}

View file

@ -0,0 +1,61 @@
##############################################
# Akka Distributed DataReference Config File #
##############################################
# This is the reference config file that contains all the default settings.
# Make your edits/overrides in your application.conf.
#//#distributed-data
# Settings for the DistributedData extension
akka.cluster.distributed-data {
# Actor name of the Replicator actor, /system/ddataReplicator
name = ddataReplicator
# Replicas are running on members tagged with this role.
# All members are used if undefined or empty.
role = ""
# How often the Replicator should send out gossip information
gossip-interval = 2 s
# How often the subscribers will be notified of changes, if any
notify-subscribers-interval = 500 ms
# Maximum number of entries to transfer in one gossip message when synchronizing
# the replicas. Next chunk will be transferred in next round of gossip.
max-delta-elements = 1000
# The id of the dispatcher to use for Replicator actors. If not specified
# default dispatcher is used.
# If specified you need to define the settings of the actual dispatcher.
use-dispatcher = ""
# How often the Replicator checks for pruning of data associated with
# removed cluster nodes.
pruning-interval = 30 s
# How long time it takes (worst case) to spread the data to all other replica nodes.
# This is used when initiating and completing the pruning process of data associated
# with removed cluster nodes. The time measurement is stopped when any replica is
# unreachable, so it should be configured to worst case in a healthy cluster.
max-pruning-dissemination = 60 s
}
#//#distributed-data
# Protobuf serializer for cluster DistributedData messages
akka.actor {
serializers {
akka-data-replication = "akka.cluster.ddata.protobuf.ReplicatorMessageSerializer"
akka-replicated-data = "akka.cluster.ddata.protobuf.ReplicatedDataSerializer"
}
serialization-bindings {
"akka.cluster.ddata.Replicator$ReplicatorMessage" = akka-data-replication
"akka.cluster.ddata.ReplicatedDataSerialization" = akka-replicated-data
}
serialization-identifiers {
"akka.cluster.ddata.protobuf.ReplicatedDataSerializer" = 11
"akka.cluster.ddata.protobuf.ReplicatorMessageSerializer" = 12
}
}

View file

@ -0,0 +1,52 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.duration._
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.cluster.Cluster
object DistributedData extends ExtensionId[DistributedData] with ExtensionIdProvider {
override def get(system: ActorSystem): DistributedData = super.get(system)
override def lookup = DistributedData
override def createExtension(system: ExtendedActorSystem): DistributedData =
new DistributedData(system)
}
/**
* Akka extension for convenient configuration and use of the
* [[Replicator]]. Configuration settings are defined in the
* `akka.cluster.ddata` section, see `reference.conf`.
*/
class DistributedData(system: ExtendedActorSystem) extends Extension {
private val config = system.settings.config.getConfig("akka.cluster.distributed-data")
private val settings = ReplicatorSettings(config)
/**
* Returns true if this member is not tagged with the role configured for the
* replicas.
*/
def isTerminated: Boolean = Cluster(system).isTerminated || !settings.role.forall(Cluster(system).selfRoles.contains)
/**
* `ActorRef` of the [[Replicator]] .
*/
val replicator: ActorRef =
if (isTerminated) {
system.log.warning("Replicator points to dead letters: Make sure the cluster node is not terminated and has the proper role!")
system.deadLetters
} else {
val name = config.getString("name")
system.systemActorOf(Replicator.props(settings), name)
}
}

View file

@ -0,0 +1,45 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
object Flag {
/**
* `Flag` that is initialized to `false`.
*/
val empty = new Flag(false)
def apply(): Flag = empty
/**
* Java API: `Flag` that is initialized to `false`.
*/
def create(): Flag = empty
// unapply from case class
}
/**
* Implements a boolean flag CRDT that is initialized to `false` and
* can be switched to `true`. `true` wins over `false` in merge.
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final case class Flag(enabled: Boolean) extends ReplicatedData with ReplicatedDataSerialization {
type T = Flag
def switchOn: Flag =
if (enabled) this
else Flag(true)
override def merge(that: Flag): Flag =
if (that.enabled) that
else this
}
object FlagKey {
def create(id: String): Key[Flag] = FlagKey(id)
}
@SerialVersionUID(1L)
final case class FlagKey(_id: String) extends Key[Flag](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,131 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
import java.math.BigInteger
object GCounter {
val empty: GCounter = new GCounter
def apply(): GCounter = empty
/**
* Java API
*/
def create(): GCounter = empty
/**
* Extract the [[GCounter#value]].
*/
def unapply(c: GCounter): Option[BigInt] = Some(c.value)
private val Zero = BigInt(0)
}
/**
* Implements a 'Growing Counter' CRDT, also called a 'G-Counter'.
*
* It is described in the paper
* <a href="http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf">A comprehensive study of Convergent and Commutative Replicated Data Types</a>.
*
* A G-Counter is a increment-only counter (inspired by vector clocks) in
* which only increment and merge are possible. Incrementing the counter
* adds 1 to the count for the current node. Divergent histories are
* resolved by taking the maximum count for each node (like a vector
* clock merge). The value of the counter is the sum of all node counts.
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final class GCounter private[akka] (
private[akka] val state: Map[UniqueAddress, BigInt] = Map.empty)
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
import GCounter.Zero
type T = GCounter
/**
* Scala API: Current total value of the counter.
*/
def value: BigInt = state.values.foldLeft(Zero) { (acc, v) acc + v }
/**
* Java API: Current total value of the counter.
*/
def getValue: BigInteger = value.bigInteger
/**
* Increment the counter with the delta specified.
* The delta must be zero or positive.
*/
def +(delta: Long)(implicit node: Cluster): GCounter = increment(node, delta)
/**
* Increment the counter with the delta specified.
* The delta must be zero or positive.
*/
def increment(node: Cluster, delta: Long = 1): GCounter =
increment(node.selfUniqueAddress, delta)
/**
* INTERNAL API
*/
private[akka] def increment(key: UniqueAddress): GCounter = increment(key, 1)
/**
* INTERNAL API
*/
private[akka] def increment(key: UniqueAddress, delta: BigInt): GCounter = {
require(delta >= 0, "Can't decrement a GCounter")
if (delta == 0) this
else state.get(key) match {
case Some(v)
val tot = v + delta
new GCounter(state + (key -> tot))
case None new GCounter(state + (key -> delta))
}
}
override def merge(that: GCounter): GCounter = {
var merged = that.state
for ((key, thisValue) state) {
val thatValue = merged.getOrElse(key, Zero)
if (thisValue > thatValue)
merged = merged.updated(key, thisValue)
}
new GCounter(merged)
}
override def needPruningFrom(removedNode: UniqueAddress): Boolean =
state.contains(removedNode)
override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): GCounter =
state.get(removedNode) match {
case Some(value) new GCounter(state - removedNode).increment(collapseInto, value)
case None this
}
override def pruningCleanup(removedNode: UniqueAddress): GCounter =
new GCounter(state - removedNode)
// this class cannot be a `case class` because we need different `unapply`
override def toString: String = s"GCounter($value)"
override def equals(o: Any): Boolean = o match {
case other: GCounter state == other.state
case _ false
}
override def hashCode: Int = state.hashCode
}
object GCounterKey {
def create(id: String): Key[GCounter] = GCounterKey(id)
}
@SerialVersionUID(1L)
final case class GCounterKey(_id: String) extends Key[GCounter](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,66 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
object GSet {
private val _empty: GSet[Any] = new GSet(Set.empty)
def empty[A]: GSet[A] = _empty.asInstanceOf[GSet[A]]
def apply(): GSet[Any] = _empty
/**
* Java API
*/
def create[A](): GSet[A] = empty[A]
// unapply from case class
}
/**
* Implements a 'Add Set' CRDT, also called a 'G-Set'. You can't
* remove elements of a G-Set.
*
* It is described in the paper
* <a href="http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf">A comprehensive study of Convergent and Commutative Replicated Data Types</a>.
*
* A G-Set doesn't accumulate any garbage apart from the elements themselves.
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final case class GSet[A](elements: Set[A]) extends ReplicatedData with ReplicatedDataSerialization {
type T = GSet[A]
/**
* Java API
*/
def getElements(): java.util.Set[A] = {
import scala.collection.JavaConverters._
elements.asJava
}
def contains(a: A): Boolean = elements(a)
def isEmpty: Boolean = elements.isEmpty
def size: Int = elements.size
/**
* Adds an element to the set
*/
def +(element: A): GSet[A] = add(element)
/**
* Adds an element to the set
*/
def add(element: A): GSet[A] = copy(elements + element)
override def merge(that: GSet[A]): GSet[A] = copy(elements ++ that.elements)
}
object GSetKey {
def create[A](id: String): Key[GSet[A]] = GSetKey(id)
}
@SerialVersionUID(1L)
final case class GSetKey[A](_id: String) extends Key[GSet[A]](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,35 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
object Key {
/**
* Extract the [[Key#id]].
*/
def unapply(k: Key[_]): Option[String] = Some(k.id)
private[akka]type KeyR = Key[ReplicatedData]
}
/**
* Key for the key-value data in [[Replicator]]. The type of the data value
* is defined in the key. Keys are compared equal if the `id` strings are equal,
* i.e. use unique identifiers.
*
* Specific classes are provided for the built in data types, e.g. [[ORSetKey]],
* and you can create your own keys.
*/
abstract class Key[+T <: ReplicatedData](val id: String) extends Serializable {
override final def equals(o: Any): Boolean = o match {
case k: Key[_] id == k.id
case _ false
}
override final def hashCode: Int = id.hashCode
override def toString(): String = id
}

View file

@ -0,0 +1,154 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
object LWWMap {
private val _empty: LWWMap[Any] = new LWWMap(ORMap.empty)
def empty[A]: LWWMap[A] = _empty.asInstanceOf[LWWMap[A]]
def apply(): LWWMap[Any] = _empty
/**
* Java API
*/
def create[A](): LWWMap[A] = empty
/**
* Extract the [[LWWMap#entries]].
*/
def unapply[A](m: LWWMap[A]): Option[Map[String, A]] = Some(m.entries)
}
/**
* Specialized [[ORMap]] with [[LWWRegister]] values.
*
* `LWWRegister` relies on synchronized clocks and should only be used when the choice of
* value is not important for concurrent updates occurring within the clock skew.
*
* Instead of using timestamps based on `System.currentTimeMillis()` time it is possible to
* use a timestamp value based on something else, for example an increasing version number
* from a database record that is used for optimistic concurrency control.
*
* For first-write-wins semantics you can use the [[LWWRegister#reverseClock]] instead of the
* [[LWWRegister#defaultClock]]
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final class LWWMap[A] private[akka] (
private[akka] val underlying: ORMap[LWWRegister[A]])
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
import LWWRegister.{ Clock, defaultClock }
type T = LWWMap[A]
def entries: Map[String, A] = underlying.entries.map { case (k, r) k -> r.value }
def get(key: String): Option[A] = underlying.get(key).map(_.value)
def contains(key: String): Boolean = underlying.contains(key)
def isEmpty: Boolean = underlying.isEmpty
def size: Int = underlying.size
/**
* Adds an entry to the map
*/
def +(entry: (String, A))(implicit node: Cluster): LWWMap[A] = {
val (key, value) = entry
put(node, key, value)
}
/**
* Adds an entry to the map
*/
def put(node: Cluster, key: String, value: A): LWWMap[A] =
put(node, key, value, defaultClock[A])
/**
* Adds an entry to the map.
*
* You can provide your `clock` implementation instead of using timestamps based
* on `System.currentTimeMillis()` time. The timestamp can for example be an
* increasing version number from a database record that is used for optimistic
* concurrency control.
*/
def put(node: Cluster, key: String, value: A, clock: Clock[A]): LWWMap[A] =
put(node.selfUniqueAddress, key, value, clock)
/**
* Adds an entry to the map.
*
* You can provide your `clock` implementation instead of using timestamps based
* on `System.currentTimeMillis()` time. The timestamp can for example be an
* increasing version number from a database record that is used for optimistic
* concurrency control.
*/
def put(key: String, value: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWMap[A] =
put(node, key, value, clock)
/**
* INTERNAL API
*/
private[akka] def put(node: UniqueAddress, key: String, value: A, clock: Clock[A]): LWWMap[A] = {
val newRegister = underlying.get(key) match {
case Some(r) r.withValue(node, value, clock)
case None LWWRegister(node, value, clock)
}
new LWWMap(underlying.put(node, key, newRegister))
}
/**
* Removes an entry from the map.
* Note that if there is a conflicting update on another node the entry will
* not be removed after merge.
*/
def -(key: String)(implicit node: Cluster): LWWMap[A] = remove(node, key)
/**
* Removes an entry from the map.
* Note that if there is a conflicting update on another node the entry will
* not be removed after merge.
*/
def remove(node: Cluster, key: String): LWWMap[A] =
remove(node.selfUniqueAddress, key)
/**
* INTERNAL API
*/
private[akka] def remove(node: UniqueAddress, key: String): LWWMap[A] =
new LWWMap(underlying.remove(node, key))
override def merge(that: LWWMap[A]): LWWMap[A] =
new LWWMap(underlying.merge(that.underlying))
override def needPruningFrom(removedNode: UniqueAddress): Boolean =
underlying.needPruningFrom(removedNode)
override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): LWWMap[A] =
new LWWMap(underlying.prune(removedNode, collapseInto))
override def pruningCleanup(removedNode: UniqueAddress): LWWMap[A] =
new LWWMap(underlying.pruningCleanup(removedNode))
// this class cannot be a `case class` because we need different `unapply`
override def toString: String = s"LWW$entries" //e.g. LWWMap(a -> 1, b -> 2)
override def equals(o: Any): Boolean = o match {
case other: LWWMap[_] underlying == other.underlying
case _ false
}
override def hashCode: Int = underlying.hashCode
}
object LWWMapKey {
def create[A](id: String): Key[LWWMap[A]] = LWWMapKey(id)
}
@SerialVersionUID(1L)
final case class LWWMapKey[A](_id: String) extends Key[LWWMap[A]](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,178 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
import akka.util.HashCode
object LWWRegister {
trait Clock[A] {
/**
* @param currentTimestamp the current `timestamp` value of the `LWWRegister`
* @param value the register value to set and associate with the returned timestamp
* @return next timestamp
*/
def apply(currentTimestamp: Long, value: A): Long
}
private val _defaultClock: Clock[Any] = new Clock[Any] {
override def apply(currentTimestamp: Long, value: Any): Long =
math.max(System.currentTimeMillis(), currentTimestamp + 1)
}
/**
* The default [[LWWRegister.Clock]] is using max value of `System.currentTimeMillis()`
* and `currentTimestamp + 1`.
*/
def defaultClock[A]: Clock[A] = _defaultClock.asInstanceOf[Clock[A]]
private val _reverseClock = new Clock[Any] {
override def apply(currentTimestamp: Long, value: Any): Long =
math.min(-System.currentTimeMillis(), currentTimestamp - 1)
}
/**
* This [[LWWRegister.Clock]] can be used for first-write-wins semantics. It is using min value of
* `-System.currentTimeMillis()` and `currentTimestamp + 1`, i.e. it is counting backwards.
*/
def reverseClock[A]: Clock[A] = _reverseClock.asInstanceOf[Clock[A]]
/**
* INTERNAL API
*/
private[akka] def apply[A](node: UniqueAddress, initialValue: A, clock: Clock[A]): LWWRegister[A] =
new LWWRegister(node, initialValue, clock(0L, initialValue))
def apply[A](initialValue: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
apply(node.selfUniqueAddress, initialValue, clock)
/**
* Java API
*/
def create[A](node: Cluster, initialValue: A): LWWRegister[A] =
apply(initialValue)(node)
/**
* Java API
*/
def create[A](node: Cluster, initialValue: A, clock: Clock[A]): LWWRegister[A] =
apply(initialValue)(node, clock)
/**
* Extract the [[LWWRegister#value]].
*/
def unapply[A](c: LWWRegister[A]): Option[A] = Some(c.value)
}
/**
* Implements a 'Last Writer Wins Register' CRDT, also called a 'LWW-Register'.
*
* It is described in the paper
* <a href="http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf">A comprehensive study of Convergent and Commutative Replicated Data Types</a>.
*
* Merge takes the the register with highest timestamp. Note that this
* relies on synchronized clocks. `LWWRegister` should only be used when the choice of
* value is not important for concurrent updates occurring within the clock skew.
*
* Merge takes the register updated by the node with lowest address (`UniqueAddress` is ordered)
* if the timestamps are exactly the same.
*
* Instead of using timestamps based on `System.currentTimeMillis()` time it is possible to
* use a timestamp value based on something else, for example an increasing version number
* from a database record that is used for optimistic concurrency control.
*
* For first-write-wins semantics you can use the [[LWWRegister#reverseClock]] instead of the
* [[LWWRegister#defaultClock]]
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final class LWWRegister[A] private[akka] (
private[akka] val node: UniqueAddress,
val value: A,
val timestamp: Long)
extends ReplicatedData with ReplicatedDataSerialization {
import LWWRegister.{ Clock, defaultClock }
type T = LWWRegister[A]
/**
* Java API
*/
def getValue(): A = value
/**
* Change the value of the register.
*
* You can provide your `clock` implementation instead of using timestamps based
* on `System.currentTimeMillis()` time. The timestamp can for example be an
* increasing version number from a database record that is used for optimistic
* concurrency control.
*/
def withValue(value: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
withValue(node, value)
/**
* Change the value of the register.
*/
def withValue(node: Cluster, value: A): LWWRegister[A] =
withValue(node, value, defaultClock[A])
/**
* Change the value of the register.
*
* You can provide your `clock` implementation instead of using timestamps based
* on `System.currentTimeMillis()` time. The timestamp can for example be an
* increasing version number from a database record that is used for optimistic
* concurrency control.
*/
def withValue(node: Cluster, value: A, clock: Clock[A]): LWWRegister[A] =
withValue(node.selfUniqueAddress, value, clock)
/**
* The current `value` was set by this node.
*/
def updatedBy: UniqueAddress = node
/**
* INTERNAL API
*/
private[akka] def withValue(node: UniqueAddress, value: A, clock: Clock[A]): LWWRegister[A] =
new LWWRegister(node, value, clock(timestamp, value))
override def merge(that: LWWRegister[A]): LWWRegister[A] =
if (that.timestamp > this.timestamp) that
else if (that.timestamp < this.timestamp) this
else if (that.node < this.node) that
else this
// this class cannot be a `case class` because we need different `unapply`
override def toString: String = s"LWWRegister($value)"
override def equals(o: Any): Boolean = o match {
case other: LWWRegister[_]
timestamp == other.timestamp && value == other.value && node == other.node
case _ false
}
override def hashCode: Int = {
var result = HashCode.SEED
result = HashCode.hash(result, timestamp)
result = HashCode.hash(result, node)
result = HashCode.hash(result, value)
result
}
}
object LWWRegisterKey {
def create[A](id: String): Key[LWWRegister[A]] = LWWRegisterKey(id)
}
@SerialVersionUID(1L)
final case class LWWRegisterKey[A](_id: String) extends Key[LWWRegister[A]](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,231 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
import akka.util.HashCode
import akka.japi.function.{ Function JFunction }
object ORMap {
private val _empty: ORMap[ReplicatedData] = new ORMap(ORSet.empty, Map.empty)
def empty[A <: ReplicatedData]: ORMap[A] = _empty.asInstanceOf[ORMap[A]]
def apply(): ORMap[ReplicatedData] = _empty
/**
* Java API
*/
def create[A <: ReplicatedData](): ORMap[A] = empty[A]
/**
* Extract the [[ORMap#entries]].
*/
def unapply[A <: ReplicatedData](m: ORMap[A]): Option[Map[String, A]] = Some(m.entries)
}
/**
* Implements a 'Observed Remove Map' CRDT, also called a 'OR-Map'.
*
* It has similar semantics as an [[ORSet]], but in case of concurrent updates
* the values are merged, and must therefore be [[ReplicatedData]] types themselves.
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final class ORMap[A <: ReplicatedData] private[akka] (
private[akka] val keys: ORSet[String],
private[akka] val values: Map[String, A])
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
type T = ORMap[A]
/**
* Scala API: All entries of the map.
*/
def entries: Map[String, A] = values
/**
* Java API: All entries of the map.
*/
def getEntries(): java.util.Map[String, A] = {
import scala.collection.JavaConverters._
entries.asJava
}
def get(key: String): Option[A] = values.get(key)
/**
* Scala API: Get the value associated with the key if there is one,
* else return the given default.
*/
def getOrElse(key: String, default: A): A = values.getOrElse(key, default)
def contains(key: String): Boolean = values.contains(key)
def isEmpty: Boolean = values.isEmpty
def size: Int = values.size
/**
* Adds an entry to the map
* @see [[#put]]
*/
def +(entry: (String, A))(implicit node: Cluster): ORMap[A] = {
val (key, value) = entry
put(node, key, value)
}
/**
* Adds an entry to the map.
* Note that the new `value` will be merged with existing values
* on other nodes and the outcome depends on what `ReplicatedData`
* type that is used.
*
* Consider using [[#updated]] instead of `put` if you want modify
* existing entry.
*
* `IllegalArgumentException` is thrown if you try to replace an existing `ORSet`
* value, because important history can be lost when replacing the `ORSet` and
* undesired effects of merging will occur.
*/
def put(node: Cluster, key: String, value: A): ORMap[A] = put(node.selfUniqueAddress, key, value)
/**
* INTERNAL API
*/
private[akka] def put(node: UniqueAddress, key: String, value: A): ORMap[A] =
if (value.isInstanceOf[ORSet[_]] && values.contains(key))
throw new IllegalArgumentException(
"`ORMap.put` must not be used to replace an existing `ORSet` " +
"value, because important history can be lost when replacing the `ORSet` and " +
"undesired effects of merging will occur. Use `ORMap.updated` instead.")
else
new ORMap(keys.add(node, key), values.updated(key, value))
/**
* Scala API: Replace a value by applying the `modify` function on the existing value.
*
* If there is no current value for the `key` the `initial` value will be
* passed to the `modify` function.
*/
def updated(node: Cluster, key: String, initial: A)(modify: A A): ORMap[A] =
updated(node.selfUniqueAddress, key, initial)(modify)
/**
* Java API: Replace a value by applying the `modify` function on the existing value.
*
* If there is no current value for the `key` the `initial` value will be
* passed to the `modify` function.
*/
def updated(node: Cluster, key: String, initial: A, modify: java.util.function.Function[A, A]): ORMap[A] =
updated(node, key, initial)(value modify.apply(value))
/**
* INTERNAL API
*/
private[akka] def updated(node: UniqueAddress, key: String, initial: A)(modify: A A): ORMap[A] = {
val newValue = values.get(key) match {
case Some(old) modify(old)
case _ modify(initial)
}
new ORMap(keys.add(node, key), values.updated(key, newValue))
}
/**
* Removes an entry from the map.
* Note that if there is a conflicting update on another node the entry will
* not be removed after merge.
*/
def -(key: String)(implicit node: Cluster): ORMap[A] = remove(node, key)
/**
* Removes an entry from the map.
* Note that if there is a conflicting update on another node the entry will
* not be removed after merge.
*/
def remove(node: Cluster, key: String): ORMap[A] = remove(node.selfUniqueAddress, key)
/**
* INTERNAL API
*/
private[akka] def remove(node: UniqueAddress, key: String): ORMap[A] = {
new ORMap(keys.remove(node, key), values - key)
}
override def merge(that: ORMap[A]): ORMap[A] = {
val mergedKeys = keys.merge(that.keys)
var mergedValues = Map.empty[String, A]
mergedKeys.elementsMap.keysIterator.foreach { key
(this.values.get(key), that.values.get(key)) match {
case (Some(thisValue), Some(thatValue))
if (thisValue.getClass != thatValue.getClass) {
val errMsg = s"Wrong type for merging [$key] in [${getClass.getName}], existing type " +
s"[${thisValue.getClass.getName}], got [${thatValue.getClass.getName}]"
throw new IllegalArgumentException(errMsg)
}
// TODO can we get rid of these (safe) casts?
val mergedValue = thisValue.merge(thatValue.asInstanceOf[thisValue.T]).asInstanceOf[A]
mergedValues = mergedValues.updated(key, mergedValue)
case (Some(thisValue), None)
mergedValues = mergedValues.updated(key, thisValue)
case (None, Some(thatValue))
mergedValues = mergedValues.updated(key, thatValue)
case (None, None) throw new IllegalStateException(s"missing value for $key")
}
}
new ORMap(mergedKeys, mergedValues)
}
override def needPruningFrom(removedNode: UniqueAddress): Boolean = {
keys.needPruningFrom(removedNode) || values.exists {
case (_, data: RemovedNodePruning) data.needPruningFrom(removedNode)
case _ false
}
}
override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): ORMap[A] = {
val prunedKeys = keys.prune(removedNode, collapseInto)
val prunedValues = values.foldLeft(values) {
case (acc, (key, data: RemovedNodePruning)) if data.needPruningFrom(removedNode)
acc.updated(key, data.prune(removedNode, collapseInto).asInstanceOf[A])
case (acc, _) acc
}
new ORMap(prunedKeys, prunedValues)
}
override def pruningCleanup(removedNode: UniqueAddress): ORMap[A] = {
val pruningCleanupedKeys = keys.pruningCleanup(removedNode)
val pruningCleanupedValues = values.foldLeft(values) {
case (acc, (key, data: RemovedNodePruning)) if data.needPruningFrom(removedNode)
acc.updated(key, data.pruningCleanup(removedNode).asInstanceOf[A])
case (acc, _) acc
}
new ORMap(pruningCleanupedKeys, pruningCleanupedValues)
}
// this class cannot be a `case class` because we need different `unapply`
override def toString: String = s"OR$entries"
override def equals(o: Any): Boolean = o match {
case other: ORMap[_] keys == other.keys && values == other.values
case _ false
}
override def hashCode: Int = {
var result = HashCode.SEED
result = HashCode.hash(result, keys)
result = HashCode.hash(result, values)
result
}
}
object ORMapKey {
def create[A <: ReplicatedData](id: String): Key[ORMap[A]] = ORMapKey(id)
}
@SerialVersionUID(1L)
final case class ORMapKey[A <: ReplicatedData](_id: String) extends Key[ORMap[A]](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,299 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.annotation.tailrec
import scala.collection.immutable.TreeMap
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
import akka.util.HashCode
// TODO this class can be optimized, but I wanted to start with correct functionality and comparability with riak_dt_orswot
object ORSet {
private val _empty: ORSet[Any] = new ORSet(Map.empty, VersionVector.empty)
def empty[A]: ORSet[A] = _empty.asInstanceOf[ORSet[A]]
def apply(): ORSet[Any] = _empty
/**
* Java API
*/
def create[A](): ORSet[A] = empty[A]
/**
* Extract the [[ORSet#elements]].
*/
def unapply[A](s: ORSet[A]): Option[Set[A]] = Some(s.elements)
/**
* Extract the [[ORSet#elements]] of an `ORSet`.
*/
def unapply(a: ReplicatedData): Option[Set[Any]] = a match {
case s: ORSet[Any] @unchecked Some(s.elements)
case _ None
}
/**
* INTERNAL API
*/
private[akka]type Dot = VersionVector
/**
* INTERNAL API
* Subtract the `vvector` from the `dot`.
* What this means is that any (node, version) pair in
* `dot` that is &lt;= an entry in `vvector` is removed from `dot`.
* Example [{a, 3}, {b, 2}, {d, 14}, {g, 22}] -
* [{a, 4}, {b, 1}, {c, 1}, {d, 14}, {e, 5}, {f, 2}] =
* [{b, 2}, {g, 22}]
*/
private[akka] def subtractDots(dot: Dot, vvector: VersionVector): Dot = {
@tailrec def dropDots(remaining: List[(UniqueAddress, Long)], acc: List[(UniqueAddress, Long)]): List[(UniqueAddress, Long)] =
remaining match {
case Nil acc
case (d @ (node, v1)) :: rest
vvector.versions.get(node) match {
case Some(v2) if v2 >= v1
// dot is dominated by version vector, drop it
dropDots(rest, acc)
case _
dropDots(rest, d :: acc)
}
}
val newDots = dropDots(dot.versions.toList, Nil)
new VersionVector(versions = TreeMap.empty[UniqueAddress, Long] ++ newDots)
}
/**
* INTERNAL API
* @see [[ORSet#merge]]
*/
private[akka] def mergeCommonKeys[A](commonKeys: Set[A], lhs: ORSet[A], rhs: ORSet[A]): Map[A, ORSet.Dot] = {
commonKeys.foldLeft(Map.empty[A, ORSet.Dot]) {
case (acc, k)
val lhsDots = lhs.elementsMap(k).versions
val rhsDots = rhs.elementsMap(k).versions
val commonDots = lhsDots.filter {
case (thisDotNode, v) rhsDots.get(thisDotNode).exists(_ == v)
}
val commonDotsKeys = commonDots.keys
val lhsUniqueDots = lhsDots -- commonDotsKeys
val rhsUniqueDots = rhsDots -- commonDotsKeys
val lhsKeep = ORSet.subtractDots(new VersionVector(lhsUniqueDots), rhs.vvector)
val rhsKeep = ORSet.subtractDots(new VersionVector(rhsUniqueDots), lhs.vvector)
val merged = lhsKeep.merge(rhsKeep).merge(new VersionVector(versions = commonDots))
// Perfectly possible that an item in both sets should be dropped
if (merged.versions.isEmpty) acc
else acc.updated(k, merged)
}
}
/**
* INTERNAL API
* @see [[ORSet#merge]]
*/
private[akka] def mergeDisjointKeys[A](keys: Set[A], elementsMap: Map[A, ORSet.Dot], vvector: VersionVector,
accumulator: Map[A, ORSet.Dot]): Map[A, ORSet.Dot] = {
keys.foldLeft(accumulator) {
case (acc, k)
val dots = elementsMap(k)
if (vvector > dots || vvector == dots)
acc
else {
// Optimise the set of stored dots to include only those unseen
val newDots = subtractDots(dots, vvector)
acc.updated(k, newDots)
}
}
}
}
/**
* Implements a 'Observed Remove Set' CRDT, also called a 'OR-Set'.
* Elements can be added and removed any number of times. Concurrent add wins
* over remove.
*
* It is not implemented as in the paper
* <a href="http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf">A comprehensive study of Convergent and Commutative Replicated Data Types</a>.
* This is more space efficient and doesn't accumulate garbage for removed elements.
* It is described in the paper
* <a href="https://hal.inria.fr/file/index/docid/738680/filename/RR-8083.pdf">An optimized conflict-free replicated set</a>
* The implementation is inspired by the Riak DT <a href="https://github.com/basho/riak_dt/blob/develop/src/riak_dt_orswot.erl">
* riak_dt_orswot</a>.
*
* The ORSet has a version vector that is incremented when an element is added to
* the set. The `node -&gt; count` pair for that increment is stored against the
* element as its "birth dot". Every time the element is re-added to the set,
* its "birth dot" is updated to that of the `node -&gt; count` version vector entry
* resulting from the add. When an element is removed, we simply drop it, no tombstones.
*
* When an element exists in replica A and not replica B, is it because A added
* it and B has not yet seen that, or that B removed it and A has not yet seen that?
* In this implementation we compare the `dot` of the present element to the version vector
* in the Set it is absent from. If the element dot is not "seen" by the Set version vector,
* that means the other set has yet to see this add, and the item is in the merged
* Set. If the Set version vector dominates the dot, that means the other Set has removed this
* element already, and the item is not in the merged Set.
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final class ORSet[A] private[akka] (
private[akka] val elementsMap: Map[A, ORSet.Dot],
private[akka] val vvector: VersionVector)
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
type T = ORSet[A]
/**
* Scala API
*/
def elements: Set[A] = elementsMap.keySet
/**
* Java API
*/
def getElements(): java.util.Set[A] = {
import scala.collection.JavaConverters._
elements.asJava
}
def contains(a: A): Boolean = elementsMap.contains(a)
def isEmpty: Boolean = elementsMap.isEmpty
def size: Int = elementsMap.size
/**
* Adds an element to the set
*/
def +(element: A)(implicit node: Cluster): ORSet[A] = add(node, element)
/**
* Adds an element to the set
*/
def add(node: Cluster, element: A): ORSet[A] = add(node.selfUniqueAddress, element)
/**
* INTERNAL API
*/
private[akka] def add(node: UniqueAddress, element: A): ORSet[A] = {
val newVvector = vvector + node
val newDot = new VersionVector(versions = TreeMap(node -> newVvector.versions(node)))
new ORSet(elementsMap = elementsMap.updated(element, newDot), vvector = newVvector)
}
/**
* Removes an element from the set.
*/
def -(element: A)(implicit node: Cluster): ORSet[A] = remove(node, element)
/**
* Removes an element from the set.
*/
def remove(node: Cluster, element: A): ORSet[A] = remove(node.selfUniqueAddress, element)
/**
* INTERNAL API
*/
private[akka] def remove(node: UniqueAddress, element: A): ORSet[A] =
copy(elementsMap = elementsMap - element)
/**
* Removes all elements from the set, but keeps the history.
* This has the same result as using [[#remove]] for each
* element, but it is more efficient.
*/
def clear(node: Cluster): ORSet[A] = clear(node.selfUniqueAddress)
/**
* INTERNAL API
*/
private[akka] def clear(node: UniqueAddress): ORSet[A] = copy(elementsMap = Map.empty)
/**
* When element is in this Set but not in that Set:
* Compare the "birth dot" of the present element to the version vector in the Set it is absent from.
* If the element dot is not "seen" by other Set version vector, that means the other set has yet to
* see this add, and the element is to be in the merged Set.
* If the other Set version vector dominates the dot, that means the other Set has removed
* the element already, and the element is not to be in the merged Set.
*
* When element in both this Set and in that Set:
* Some dots may still need to be shed. If this Set has dots that the other Set does not have,
* and the other Set version vector dominates those dots, then we need to drop those dots.
* Keep only common dots, and dots that are not dominated by the other sides version vector
*/
override def merge(that: ORSet[A]): ORSet[A] = {
val thisKeys = elementsMap.keySet
val thatKeys = that.elementsMap.keySet
val commonKeys = thisKeys.intersect(thatKeys)
val thisUniqueKeys = thisKeys -- commonKeys
val thatUniqueKeys = thatKeys -- commonKeys
val entries00 = ORSet.mergeCommonKeys(commonKeys, this, that)
val entries0 = ORSet.mergeDisjointKeys(thisUniqueKeys, this.elementsMap, that.vvector, entries00)
val entries = ORSet.mergeDisjointKeys(thatUniqueKeys, that.elementsMap, this.vvector, entries0)
val mergedVvector = this.vvector.merge(that.vvector)
new ORSet(entries, mergedVvector)
}
override def needPruningFrom(removedNode: UniqueAddress): Boolean =
vvector.needPruningFrom(removedNode)
override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): ORSet[A] = {
val pruned = elementsMap.foldLeft(Map.empty[A, ORSet.Dot]) {
case (acc, (elem, dot))
if (dot.needPruningFrom(removedNode)) acc.updated(elem, dot.prune(removedNode, collapseInto))
else acc
}
if (pruned.isEmpty)
copy(vvector = vvector.prune(removedNode, collapseInto))
else {
// re-add elements that were pruned, to bump dots to right vvector
val newSet = new ORSet(elementsMap = elementsMap ++ pruned, vvector = vvector.prune(removedNode, collapseInto))
pruned.keys.foldLeft(newSet) {
case (s, elem) s.add(collapseInto, elem)
}
}
}
override def pruningCleanup(removedNode: UniqueAddress): ORSet[A] = {
val updated = elementsMap.foldLeft(elementsMap) {
case (acc, (elem, dot))
if (dot.needPruningFrom(removedNode)) acc.updated(elem, dot.pruningCleanup(removedNode))
else acc
}
new ORSet(updated, vvector.pruningCleanup(removedNode))
}
private def copy(elementsMap: Map[A, ORSet.Dot] = this.elementsMap, vvector: VersionVector = this.vvector): ORSet[A] =
new ORSet(elementsMap, vvector)
// this class cannot be a `case class` because we need different `unapply`
override def toString: String = s"OR$elements"
override def equals(o: Any): Boolean = o match {
case other: ORSet[_] vvector == other.vvector && elementsMap == other.elementsMap
case _ false
}
override def hashCode: Int = {
var result = HashCode.SEED
result = HashCode.hash(result, elementsMap)
result = HashCode.hash(result, vvector)
result
}
}
object ORSetKey {
def create[A](id: String): Key[ORSet[A]] = ORSetKey(id)
}
@SerialVersionUID(1L)
final case class ORSetKey[A](_id: String) extends Key[ORSet[A]](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,134 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
import akka.util.HashCode
import java.math.BigInteger
object PNCounter {
val empty: PNCounter = new PNCounter(GCounter.empty, GCounter.empty)
def apply(): PNCounter = empty
/**
* Java API
*/
def create(): PNCounter = empty
/**
* Extract the [[GCounter#value]].
*/
def unapply(c: PNCounter): Option[BigInt] = Some(c.value)
}
/**
* Implements a 'Increment/Decrement Counter' CRDT, also called a 'PN-Counter'.
*
* It is described in the paper
* <a href="http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf">A comprehensive study of Convergent and Commutative Replicated Data Types</a>.
*
* PN-Counters allow the counter to be incremented by tracking the
* increments (P) separate from the decrements (N). Both P and N are represented
* as two internal [[GCounter]]s. Merge is handled by merging the internal P and N
* counters. The value of the counter is the value of the P counter minus
* the value of the N counter.
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final class PNCounter private[akka] (
private[akka] val increments: GCounter, private[akka] val decrements: GCounter)
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
type T = PNCounter
/**
* Scala API: Current total value of the counter.
*/
def value: BigInt = increments.value - decrements.value
/**
* Java API: Current total value of the counter.
*/
def getValue: BigInteger = value.bigInteger
/**
* Increment the counter with the delta specified.
* If the delta is negative then it will decrement instead of increment.
*/
def +(delta: Long)(implicit node: Cluster): PNCounter = increment(node, delta)
/**
* Increment the counter with the delta specified.
* If the delta is negative then it will decrement instead of increment.
*/
def increment(node: Cluster, delta: Long = 1): PNCounter =
increment(node.selfUniqueAddress, delta)
/**
* Decrement the counter with the delta specified.
* If the delta is negative then it will increment instead of decrement.
*/
def -(delta: Long)(implicit node: Cluster): PNCounter = decrement(node, delta)
/**
* Decrement the counter with the delta specified.
* If the delta is negative then it will increment instead of decrement.
*/
def decrement(node: Cluster, delta: Long = 1): PNCounter =
decrement(node.selfUniqueAddress, delta)
private[akka] def increment(key: UniqueAddress, delta: Long): PNCounter = change(key, delta)
private[akka] def increment(key: UniqueAddress): PNCounter = increment(key, 1)
private[akka] def decrement(key: UniqueAddress, delta: Long): PNCounter = change(key, -delta)
private[akka] def decrement(key: UniqueAddress): PNCounter = decrement(key, 1)
private[akka] def change(key: UniqueAddress, delta: Long): PNCounter =
if (delta > 0) copy(increments = increments.increment(key, delta))
else if (delta < 0) copy(decrements = decrements.increment(key, -delta))
else this
override def merge(that: PNCounter): PNCounter =
copy(increments = that.increments.merge(this.increments),
decrements = that.decrements.merge(this.decrements))
override def needPruningFrom(removedNode: UniqueAddress): Boolean =
increments.needPruningFrom(removedNode) || decrements.needPruningFrom(removedNode)
override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): PNCounter =
copy(increments = increments.prune(removedNode, collapseInto),
decrements = decrements.prune(removedNode, collapseInto))
override def pruningCleanup(removedNode: UniqueAddress): PNCounter =
copy(increments = increments.pruningCleanup(removedNode),
decrements = decrements.pruningCleanup(removedNode))
private def copy(increments: GCounter = this.increments, decrements: GCounter = this.decrements): PNCounter =
new PNCounter(increments, decrements)
// this class cannot be a `case class` because we need different `unapply`
override def toString: String = s"PNCounter($value)"
override def equals(o: Any): Boolean = o match {
case other: PNCounter
increments == other.increments && decrements == other.decrements
case _ false
}
override def hashCode: Int = {
var result = HashCode.SEED
result = HashCode.hash(result, increments)
result = HashCode.hash(result, decrements)
result
}
}
object PNCounterKey {
def create[A](id: String): Key[PNCounter] = PNCounterKey(id)
}
@SerialVersionUID(1L)
final case class PNCounterKey(_id: String) extends Key[PNCounter](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,149 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
import java.math.BigInteger
object PNCounterMap {
val empty: PNCounterMap = new PNCounterMap(ORMap.empty)
def apply(): PNCounterMap = empty
/**
* Java API
*/
def create(): PNCounterMap = empty
/**
* Extract the [[PNCounterMap#entries]].
*/
def unapply(m: PNCounterMap): Option[Map[String, BigInt]] = Some(m.entries)
}
/**
* Map of named counters. Specialized [[ORMap]] with [[PNCounter]] values.
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final class PNCounterMap private[akka] (
private[akka] val underlying: ORMap[PNCounter])
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
type T = PNCounterMap
/** Scala API */
def entries: Map[String, BigInt] = underlying.entries.map { case (k, c) k -> c.value }
/** Java API */
def getEntries: Map[String, BigInteger] = underlying.entries.map { case (k, c) k -> c.value.bigInteger }
/**
* Scala API: The count for a key
*/
def get(key: String): Option[BigInt] = underlying.get(key).map(_.value)
/**
* Java API: The count for a key, or `null` if it doesn't exist
*/
def getValue(key: String): BigInteger = underlying.get(key).map(_.value.bigInteger).orNull
def contains(key: String): Boolean = underlying.contains(key)
def isEmpty: Boolean = underlying.isEmpty
def size: Int = underlying.size
/**
* Increment the counter with the delta specified.
* If the delta is negative then it will decrement instead of increment.
*/
def increment(key: String, delta: Long = 1)(implicit node: Cluster): PNCounterMap =
increment(node, key, delta)
/**
* Increment the counter with the delta specified.
* If the delta is negative then it will decrement instead of increment.
*/
def increment(node: Cluster, key: String, delta: Long): PNCounterMap =
increment(node.selfUniqueAddress, key, delta)
/**
* INTERNAL API
*/
private[akka] def increment(node: UniqueAddress, key: String, delta: Long): PNCounterMap =
new PNCounterMap(underlying.updated(node, key, PNCounter())(_.increment(node, delta)))
/**
* Decrement the counter with the delta specified.
* If the delta is negative then it will increment instead of decrement.
*/
def decrement(key: String, delta: Long = 1)(implicit node: Cluster): PNCounterMap =
decrement(node, key, delta)
/**
* Decrement the counter with the delta specified.
* If the delta is negative then it will increment instead of decrement.
*/
def decrement(node: Cluster, key: String, delta: Long): PNCounterMap =
decrement(node.selfUniqueAddress, key, delta)
/**
* INTERNAL API
*/
private[akka] def decrement(node: UniqueAddress, key: String, delta: Long): PNCounterMap = {
new PNCounterMap(underlying.updated(node, key, PNCounter())(_.decrement(node, delta)))
}
/**
* Removes an entry from the map.
* Note that if there is a conflicting update on another node the entry will
* not be removed after merge.
*/
def -(key: String)(implicit node: Cluster): PNCounterMap = remove(node, key)
/**
* Removes an entry from the map.
* Note that if there is a conflicting update on another node the entry will
* not be removed after merge.
*/
def remove(node: Cluster, key: String): PNCounterMap =
remove(node.selfUniqueAddress, key)
/**
* INTERNAL API
*/
private[akka] def remove(node: UniqueAddress, key: String): PNCounterMap =
new PNCounterMap(underlying.remove(node, key))
override def merge(that: PNCounterMap): PNCounterMap =
new PNCounterMap(underlying.merge(that.underlying))
override def needPruningFrom(removedNode: UniqueAddress): Boolean =
underlying.needPruningFrom(removedNode)
override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): PNCounterMap =
new PNCounterMap(underlying.prune(removedNode, collapseInto))
override def pruningCleanup(removedNode: UniqueAddress): PNCounterMap =
new PNCounterMap(underlying.pruningCleanup(removedNode))
// this class cannot be a `case class` because we need different `unapply`
override def toString: String = s"PNCounter$entries"
override def equals(o: Any): Boolean = o match {
case other: PNCounterMap underlying == other.underlying
case _ false
}
override def hashCode: Int = underlying.hashCode
}
object PNCounterMapKey {
def create[A](id: String): Key[PNCounterMap] = PNCounterMapKey(id)
}
@SerialVersionUID(1L)
final case class PNCounterMapKey(_id: String) extends Key[PNCounterMap](_id) with ReplicatedDataSerialization

View file

@ -0,0 +1,45 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.Address
import akka.cluster.Member
import akka.cluster.UniqueAddress
/**
* INTERNAL API
*/
private[akka] object PruningState {
sealed trait PruningPhase
final case class PruningInitialized(seen: Set[Address]) extends PruningPhase
case object PruningPerformed extends PruningPhase
}
/**
* INTERNAL API
*/
private[akka] final case class PruningState(owner: UniqueAddress, phase: PruningState.PruningPhase) {
import PruningState._
def merge(that: PruningState): PruningState =
(this.phase, that.phase) match {
case (PruningPerformed, _) this
case (_, PruningPerformed) that
case (PruningInitialized(thisSeen), PruningInitialized(thatSeen))
if (this.owner == that.owner)
copy(phase = PruningInitialized(thisSeen ++ thatSeen))
else if (Member.addressOrdering.compare(this.owner.address, that.owner.address) > 0)
that
else
this
}
def addSeen(node: Address): PruningState = phase match {
case PruningInitialized(seen)
if (seen(node) || owner.address == node) this
else copy(phase = PruningInitialized(seen + node))
case _ this
}
}

View file

@ -0,0 +1,82 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.UniqueAddress
/**
* Interface for implementing a state based convergent
* replicated data type (CvRDT).
*
* ReplicatedData types must be serializable with an Akka
* Serializer. It is highly recommended to implement a serializer with
* Protobuf or similar. The built in data types are marked with
* [[ReplicatedDataSerialization]] and serialized with
* [[akka.cluster.ddata.protobuf.ReplicatedDataSerializer]].
*
* Serialization of the data types are used in remote messages and also
* for creating message digests (SHA-1) to detect changes. Therefore it is
* important that the serialization produce the same bytes for the same content.
* For example sets and maps should be sorted deterministically in the serialization.
*
* ReplicatedData types should be immutable, i.e. "modifying" methods should return
* a new instance.
*/
trait ReplicatedData {
type T <: ReplicatedData
/**
* Monotonic merge function.
*/
def merge(that: T): T
}
/**
* Java API: Interface for implementing a [[ReplicatedData]] in
* Java.
*/
abstract class AbstractReplicatedData extends ReplicatedData {
// it is not possible to use a more strict type, because it is erased somehow, and
// the implementation is anyway required to implement
// merge(that: ReplicatedData): ReplicatedData
type T = AbstractReplicatedData
}
/**
* [[ReplicatedData]] that has support for pruning of data
* belonging to a specific node may implement this interface.
* When a node is removed from the cluster these methods will be
* used by the [[Replicator]] to collapse data from the removed node
* into some other node in the cluster.
*/
trait RemovedNodePruning { this: ReplicatedData
/**
* Does it have any state changes from a specific node,
* which has been removed from the cluster.
*/
def needPruningFrom(removedNode: UniqueAddress): Boolean
/**
* When the `removed` node has been removed from the cluster the state
* changes from that node will be pruned by collapsing the data entries
* to another node.
*/
def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): T
/**
* Remove data entries from a node that has been removed from the cluster
* and already been pruned.
*/
def pruningCleanup(removedNode: UniqueAddress): T
}
/**
* Marker trait for `ReplicatedData` serialized by
* [[akka.cluster.ddata.protobuf.ReplicatedDataSerializer]].
*/
trait ReplicatedDataSerialization extends Serializable

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,226 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import java.util.concurrent.atomic.AtomicLong
import scala.annotation.tailrec
import scala.collection.immutable.TreeMap
import akka.cluster.Cluster
import akka.cluster.UniqueAddress
/**
* VersionVector module with helper classes and methods.
*/
object VersionVector {
val empty: VersionVector = new VersionVector(TreeMap.empty[UniqueAddress, Long])
def apply(): VersionVector = empty
/**
* Java API
*/
def create(): VersionVector = empty
sealed trait Ordering
case object After extends Ordering
case object Before extends Ordering
case object Same extends Ordering
case object Concurrent extends Ordering
/**
* Marker to ensure that we do a full order comparison instead of bailing out early.
*/
private case object FullOrder extends Ordering
/**
* Java API: The `VersionVector.After` instance
*/
def AfterInstance = After
/**
* Java API: The `VersionVector.Before` instance
*/
def BeforeInstance = Before
/**
* Java API: The `VersionVector.Same` instance
*/
def SameInstance = Same
/**
* Java API: The `VersionVector.Concurrent` instance
*/
def ConcurrentInstance = Concurrent
private object Timestamp {
final val Zero = 0L
final val EndMarker = Long.MinValue
val counter = new AtomicLong(1L)
}
/**
* Marker to signal that we have reached the end of a version vector.
*/
private val cmpEndMarker = (null, Timestamp.EndMarker)
}
/**
* Representation of a Vector-based clock (counting clock), inspired by Lamport logical clocks.
* {{{
* Reference:
* 1) Leslie Lamport (1978). "Time, clocks, and the ordering of events in a distributed system". Communications of the ACM 21 (7): 558-565.
* 2) Friedemann Mattern (1988). "Virtual Time and Global States of Distributed Systems". Workshop on Parallel and Distributed Algorithms: pp. 215-226
* }}}
*
* Based on code from `akka.cluster.VectorClock`.
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@SerialVersionUID(1L)
final case class VersionVector private[akka] (
private[akka] val versions: TreeMap[UniqueAddress, Long])
extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning {
type T = VersionVector
import VersionVector._
/**
* Increment the version for the node passed as argument. Returns a new VersionVector.
*/
def +(node: Cluster): VersionVector = increment(node)
/**
* INTERNAL API
* Increment the version for the node passed as argument. Returns a new VersionVector.
*/
private[akka] def +(node: UniqueAddress): VersionVector = increment(node)
/**
* Increment the version for the node passed as argument. Returns a new VersionVector.
*/
def increment(node: Cluster): VersionVector = increment(node.selfUniqueAddress)
/**
* INTERNAL API
* Increment the version for the node passed as argument. Returns a new VersionVector.
*/
private[akka] def increment(node: UniqueAddress): VersionVector =
copy(versions = versions.updated(node, Timestamp.counter.getAndIncrement()))
/**
* Returns true if <code>this</code> and <code>that</code> are concurrent else false.
*/
def <>(that: VersionVector): Boolean = compareOnlyTo(that, Concurrent) eq Concurrent
/**
* Returns true if <code>this</code> is before <code>that</code> else false.
*/
def <(that: VersionVector): Boolean = compareOnlyTo(that, Before) eq Before
/**
* Returns true if <code>this</code> is after <code>that</code> else false.
*/
def >(that: VersionVector): Boolean = compareOnlyTo(that, After) eq After
/**
* Returns true if this VersionVector has the same history as the 'that' VersionVector else false.
*/
def ==(that: VersionVector): Boolean = compareOnlyTo(that, Same) eq Same
/**
* Version vector comparison according to the semantics described by compareTo, with the ability to bail
* out early if the we can't reach the Ordering that we are looking for.
*
* The ordering always starts with Same and can then go to Same, Before or After
* If we're on After we can only go to After or Concurrent
* If we're on Before we can only go to Before or Concurrent
* If we go to Concurrent we exit the loop immediately
*
* If you send in the ordering FullOrder, you will get a full comparison.
*/
private final def compareOnlyTo(that: VersionVector, order: Ordering): Ordering = {
def nextOrElse[A](iter: Iterator[A], default: A): A = if (iter.hasNext) iter.next() else default
def compare(i1: Iterator[(UniqueAddress, Long)], i2: Iterator[(UniqueAddress, Long)], requestedOrder: Ordering): Ordering = {
@tailrec
def compareNext(nt1: (UniqueAddress, Long), nt2: (UniqueAddress, Long), currentOrder: Ordering): Ordering =
if ((requestedOrder ne FullOrder) && (currentOrder ne Same) && (currentOrder ne requestedOrder)) currentOrder
else if ((nt1 eq cmpEndMarker) && (nt2 eq cmpEndMarker)) currentOrder
// i1 is empty but i2 is not, so i1 can only be Before
else if (nt1 eq cmpEndMarker) { if (currentOrder eq After) Concurrent else Before }
// i2 is empty but i1 is not, so i1 can only be After
else if (nt2 eq cmpEndMarker) { if (currentOrder eq Before) Concurrent else After }
else {
// compare the nodes
val nc = nt1._1 compareTo nt2._1
if (nc == 0) {
// both nodes exist compare the timestamps
// same timestamp so just continue with the next nodes
if (nt1._2 == nt2._2) compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), currentOrder)
else if (nt1._2 < nt2._2) {
// t1 is less than t2, so i1 can only be Before
if (currentOrder eq After) Concurrent
else compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), Before)
} else {
// t2 is less than t1, so i1 can only be After
if (currentOrder eq Before) Concurrent
else compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), After)
}
} else if (nc < 0) {
// this node only exists in i1 so i1 can only be After
if (currentOrder eq Before) Concurrent
else compareNext(nextOrElse(i1, cmpEndMarker), nt2, After)
} else {
// this node only exists in i2 so i1 can only be Before
if (currentOrder eq After) Concurrent
else compareNext(nt1, nextOrElse(i2, cmpEndMarker), Before)
}
}
compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), Same)
}
if ((this eq that) || (this.versions eq that.versions)) Same
else compare(this.versions.iterator, that.versions.iterator, if (order eq Concurrent) FullOrder else order)
}
/**
* Compare two version vectors. The outcome will be one of the following:
* <p/>
* {{{
* 1. Version 1 is SAME (==) as Version 2 iff for all i c1(i) == c2(i)
* 2. Version 1 is BEFORE (<) Version 2 iff for all i c1(i) <= c2(i) and there exist a j such that c1(j) < c2(j)
* 3. Version 1 is AFTER (>) Version 2 iff for all i c1(i) >= c2(i) and there exist a j such that c1(j) > c2(j).
* 4. Version 1 is CONCURRENT (<>) to Version 2 otherwise.
* }}}
*/
def compareTo(that: VersionVector): Ordering = {
compareOnlyTo(that, FullOrder)
}
/**
* Merges this VersionVector with another VersionVector. E.g. merges its versioned history.
*/
def merge(that: VersionVector): VersionVector = {
var mergedVersions = that.versions
for ((node, time) versions) {
val mergedVersionsCurrentTime = mergedVersions.getOrElse(node, Timestamp.Zero)
if (time > mergedVersionsCurrentTime)
mergedVersions = mergedVersions.updated(node, time)
}
VersionVector(mergedVersions)
}
override def needPruningFrom(removedNode: UniqueAddress): Boolean =
versions.contains(removedNode)
override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): VersionVector =
copy(versions = versions - removedNode) + collapseInto
override def pruningCleanup(removedNode: UniqueAddress): VersionVector = copy(versions = versions - removedNode)
override def toString = versions.map { case ((n, t)) n + " -> " + t }.mkString("VersionVector(", ", ", ")")
}

View file

@ -0,0 +1,404 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata.protobuf
import java.{ lang jl }
import java.util.ArrayList
import java.util.Collections
import java.util.Comparator
import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.collection.breakOut
import akka.actor.ExtendedActorSystem
import akka.cluster.ddata._
import akka.cluster.ddata.Replicator._
import akka.cluster.ddata.Replicator.Internal._
import akka.cluster.ddata.protobuf.msg.{ ReplicatedDataMessages rd }
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages dm }
import akka.serialization.SerializerWithStringManifest
import akka.serialization.BaseSerializer
import com.google.protobuf.ByteString
import akka.util.ByteString.UTF_8
/**
* Protobuf serializer of ReplicatedData.
*/
class ReplicatedDataSerializer(val system: ExtendedActorSystem)
extends SerializerWithStringManifest with SerializationSupport with BaseSerializer {
private val DeletedDataManifest = "A"
private val GSetManifest = "B"
private val GSetKeyManifest = "b"
private val ORSetManifest = "C"
private val ORSetKeyManifest = "c"
private val FlagManifest = "D"
private val FlagKeyManifest = "d"
private val LWWRegisterManifest = "E"
private val LWWRegisterKeyManifest = "e"
private val GCounterManifest = "F"
private val GCounterKeyManifest = "f"
private val PNCounterManifest = "G"
private val PNCounterKeyManifest = "g"
private val ORMapManifest = "H"
private val ORMapKeyManifest = "h"
private val LWWMapManifest = "I"
private val LWWMapKeyManifest = "i"
private val PNCounterMapManifest = "J"
private val PNCounterMapKeyManifest = "j"
private val VersionVectorManifest = "L"
private val fromBinaryMap = collection.immutable.HashMap[String, Array[Byte] AnyRef](
GSetManifest -> gsetFromBinary,
ORSetManifest -> orsetFromBinary,
FlagManifest -> flagFromBinary,
LWWRegisterManifest -> lwwRegisterFromBinary,
GCounterManifest -> gcounterFromBinary,
PNCounterManifest -> pncounterFromBinary,
ORMapManifest -> ormapFromBinary,
LWWMapManifest -> lwwmapFromBinary,
PNCounterMapManifest -> pncountermapFromBinary,
DeletedDataManifest -> (_ DeletedData),
VersionVectorManifest -> versionVectorFromBinary,
GSetKeyManifest -> (bytes GSetKey(keyIdFromBinary(bytes))),
ORSetKeyManifest -> (bytes ORSetKey(keyIdFromBinary(bytes))),
FlagKeyManifest -> (bytes FlagKey(keyIdFromBinary(bytes))),
LWWRegisterKeyManifest -> (bytes LWWRegisterKey(keyIdFromBinary(bytes))),
GCounterKeyManifest -> (bytes GCounterKey(keyIdFromBinary(bytes))),
PNCounterKeyManifest -> (bytes PNCounterKey(keyIdFromBinary(bytes))),
ORMapKeyManifest -> (bytes ORMapKey(keyIdFromBinary(bytes))),
LWWMapKeyManifest -> (bytes LWWMapKey(keyIdFromBinary(bytes))),
PNCounterMapKeyManifest -> (bytes PNCounterMapKey(keyIdFromBinary(bytes))))
override def manifest(obj: AnyRef): String = obj match {
case _: ORSet[_] ORSetManifest
case _: GSet[_] GSetManifest
case _: GCounter GCounterManifest
case _: PNCounter PNCounterManifest
case _: Flag FlagManifest
case _: LWWRegister[_] LWWRegisterManifest
case _: ORMap[_] ORMapManifest
case _: LWWMap[_] LWWMapManifest
case _: PNCounterMap PNCounterMapManifest
case DeletedData DeletedDataManifest
case _: VersionVector VersionVectorManifest
case _: ORSetKey[_] ORSetKeyManifest
case _: GSetKey[_] GSetKeyManifest
case _: GCounterKey GCounterKeyManifest
case _: PNCounterKey PNCounterKeyManifest
case _: FlagKey FlagKeyManifest
case _: LWWRegisterKey[_] LWWRegisterKeyManifest
case _: ORMapKey[_] ORMapKeyManifest
case _: LWWMapKey[_] LWWMapKeyManifest
case _: PNCounterMapKey PNCounterMapKeyManifest
case _
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
}
def toBinary(obj: AnyRef): Array[Byte] = obj match {
case m: ORSet[_] compress(orsetToProto(m))
case m: GSet[_] gsetToProto(m).toByteArray
case m: GCounter gcounterToProto(m).toByteArray
case m: PNCounter pncounterToProto(m).toByteArray
case m: Flag flagToProto(m).toByteArray
case m: LWWRegister[_] lwwRegisterToProto(m).toByteArray
case m: ORMap[_] compress(ormapToProto(m))
case m: LWWMap[_] compress(lwwmapToProto(m))
case m: PNCounterMap compress(pncountermapToProto(m))
case DeletedData dm.Empty.getDefaultInstance.toByteArray
case m: VersionVector versionVectorToProto(m).toByteArray
case Key(id) keyIdToBinary(id)
case _
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
}
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
fromBinaryMap.get(manifest) match {
case Some(f) f(bytes)
case None throw new IllegalArgumentException(
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
}
def gsetToProto(gset: GSet[_]): rd.GSet = {
val b = rd.GSet.newBuilder()
// using java collections and sorting for performance (avoid conversions)
val stringElements = new ArrayList[String]
val intElements = new ArrayList[Integer]
val longElements = new ArrayList[jl.Long]
val otherElements = new ArrayList[dm.OtherMessage]
gset.elements.foreach {
case s: String stringElements.add(s)
case i: Int intElements.add(i)
case l: Long longElements.add(l)
case other otherElements.add(otherMessageToProto(other))
}
if (!stringElements.isEmpty) {
Collections.sort(stringElements)
b.addAllStringElements(stringElements)
}
if (!intElements.isEmpty) {
Collections.sort(intElements)
b.addAllIntElements(intElements)
}
if (!longElements.isEmpty) {
Collections.sort(longElements)
b.addAllLongElements(longElements)
}
if (!otherElements.isEmpty) {
Collections.sort(otherElements, OtherMessageComparator)
b.addAllOtherElements(otherElements)
}
b.build()
}
def gsetFromBinary(bytes: Array[Byte]): GSet[_] =
gsetFromProto(rd.GSet.parseFrom(bytes))
def gsetFromProto(gset: rd.GSet): GSet[Any] =
GSet(gset.getStringElementsList.iterator.asScala.toSet ++
gset.getIntElementsList.iterator.asScala ++
gset.getLongElementsList.iterator.asScala ++
gset.getOtherElementsList.iterator.asScala.map(otherMessageFromProto))
def orsetToProto(orset: ORSet[_]): rd.ORSet =
orsetToProtoImpl(orset.asInstanceOf[ORSet[Any]])
private def orsetToProtoImpl(orset: ORSet[Any]): rd.ORSet = {
val b = rd.ORSet.newBuilder().setVvector(versionVectorToProto(orset.vvector))
// using java collections and sorting for performance (avoid conversions)
val stringElements = new ArrayList[String]
val intElements = new ArrayList[Integer]
val longElements = new ArrayList[jl.Long]
val otherElements = new ArrayList[dm.OtherMessage]
var otherElementsMap = Map.empty[dm.OtherMessage, Any]
orset.elementsMap.keysIterator.foreach {
case s: String stringElements.add(s)
case i: Int intElements.add(i)
case l: Long longElements.add(l)
case other
val enclosedMsg = otherMessageToProto(other)
otherElements.add(enclosedMsg)
// need the mapping back to the `other` when adding dots
otherElementsMap = otherElementsMap.updated(enclosedMsg, other)
}
def addDots(elements: ArrayList[_]): Unit = {
// add corresponding dots in same order
val iter = elements.iterator
while (iter.hasNext) {
val element = iter.next() match {
case enclosedMsg: dm.OtherMessage otherElementsMap(enclosedMsg)
case e e
}
b.addDots(versionVectorToProto(orset.elementsMap(element)))
}
}
if (!stringElements.isEmpty) {
Collections.sort(stringElements)
b.addAllStringElements(stringElements)
addDots(stringElements)
}
if (!intElements.isEmpty) {
Collections.sort(intElements)
b.addAllIntElements(intElements)
addDots(intElements)
}
if (!longElements.isEmpty) {
Collections.sort(longElements)
b.addAllLongElements(longElements)
addDots(longElements)
}
if (!otherElements.isEmpty) {
Collections.sort(otherElements, OtherMessageComparator)
b.addAllOtherElements(otherElements)
addDots(otherElements)
}
b.build()
}
def orsetFromBinary(bytes: Array[Byte]): ORSet[Any] =
orsetFromProto(rd.ORSet.parseFrom(decompress(bytes)))
def orsetFromProto(orset: rd.ORSet): ORSet[Any] = {
val elements: Iterator[Any] =
(orset.getStringElementsList.iterator.asScala ++
orset.getIntElementsList.iterator.asScala ++
orset.getLongElementsList.iterator.asScala ++
orset.getOtherElementsList.iterator.asScala.map(otherMessageFromProto))
val dots = orset.getDotsList.asScala.map(versionVectorFromProto).iterator
val elementsMap = elements.zip(dots).toMap
new ORSet(elementsMap, vvector = versionVectorFromProto(orset.getVvector))
}
def flagToProto(flag: Flag): rd.Flag =
rd.Flag.newBuilder().setEnabled(flag.enabled).build()
def flagFromBinary(bytes: Array[Byte]): Flag =
flagFromProto(rd.Flag.parseFrom(bytes))
def flagFromProto(flag: rd.Flag): Flag =
Flag(flag.getEnabled)
def lwwRegisterToProto(lwwRegister: LWWRegister[_]): rd.LWWRegister =
rd.LWWRegister.newBuilder().
setTimestamp(lwwRegister.timestamp).
setNode(uniqueAddressToProto(lwwRegister.node)).
setState(otherMessageToProto(lwwRegister.value)).
build()
def lwwRegisterFromBinary(bytes: Array[Byte]): LWWRegister[Any] =
lwwRegisterFromProto(rd.LWWRegister.parseFrom(bytes))
def lwwRegisterFromProto(lwwRegister: rd.LWWRegister): LWWRegister[Any] =
new LWWRegister(
uniqueAddressFromProto(lwwRegister.getNode),
otherMessageFromProto(lwwRegister.getState),
lwwRegister.getTimestamp)
def gcounterToProto(gcounter: GCounter): rd.GCounter = {
val b = rd.GCounter.newBuilder()
gcounter.state.toVector.sortBy { case (address, _) address }.foreach {
case (address, value) b.addEntries(rd.GCounter.Entry.newBuilder().
setNode(uniqueAddressToProto(address)).setValue(ByteString.copyFrom(value.toByteArray)))
}
b.build()
}
def gcounterFromBinary(bytes: Array[Byte]): GCounter =
gcounterFromProto(rd.GCounter.parseFrom(bytes))
def gcounterFromProto(gcounter: rd.GCounter): GCounter = {
new GCounter(state = gcounter.getEntriesList.asScala.map(entry
uniqueAddressFromProto(entry.getNode) -> BigInt(entry.getValue.toByteArray))(breakOut))
}
def pncounterToProto(pncounter: PNCounter): rd.PNCounter =
rd.PNCounter.newBuilder().
setIncrements(gcounterToProto(pncounter.increments)).
setDecrements(gcounterToProto(pncounter.decrements)).
build()
def pncounterFromBinary(bytes: Array[Byte]): PNCounter =
pncounterFromProto(rd.PNCounter.parseFrom(bytes))
def pncounterFromProto(pncounter: rd.PNCounter): PNCounter = {
new PNCounter(
increments = gcounterFromProto(pncounter.getIncrements),
decrements = gcounterFromProto(pncounter.getDecrements))
}
def versionVectorToProto(versionVector: VersionVector): rd.VersionVector = {
val b = rd.VersionVector.newBuilder()
versionVector.versions.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 = {
VersionVector(versions = versionVector.getEntriesList.asScala.map(entry
uniqueAddressFromProto(entry.getNode) -> entry.getVersion)(breakOut))
}
def ormapToProto(ormap: ORMap[_]): rd.ORMap = {
val b = rd.ORMap.newBuilder().setKeys(orsetToProto(ormap.keys))
ormap.entries.toVector.sortBy { case (key, _) key }.foreach {
case (key, value) b.addEntries(rd.ORMap.Entry.newBuilder().
setKey(key).setValue(otherMessageToProto(value)))
}
b.build()
}
def ormapFromBinary(bytes: Array[Byte]): ORMap[ReplicatedData] =
ormapFromProto(rd.ORMap.parseFrom(decompress(bytes)))
def ormapFromProto(ormap: rd.ORMap): ORMap[ReplicatedData] = {
val entries = ormap.getEntriesList.asScala.map(entry
entry.getKey -> otherMessageFromProto(entry.getValue).asInstanceOf[ReplicatedData]).toMap
new ORMap(
keys = orsetFromProto(ormap.getKeys).asInstanceOf[ORSet[String]],
entries)
}
def lwwmapToProto(lwwmap: LWWMap[_]): rd.LWWMap = {
val b = rd.LWWMap.newBuilder().setKeys(orsetToProto(lwwmap.underlying.keys))
lwwmap.underlying.entries.toVector.sortBy { case (key, _) key }.foreach {
case (key, value) b.addEntries(rd.LWWMap.Entry.newBuilder().
setKey(key).setValue(lwwRegisterToProto(value)))
}
b.build()
}
def lwwmapFromBinary(bytes: Array[Byte]): LWWMap[Any] =
lwwmapFromProto(rd.LWWMap.parseFrom(decompress(bytes)))
def lwwmapFromProto(lwwmap: rd.LWWMap): LWWMap[Any] = {
val entries = lwwmap.getEntriesList.asScala.map(entry
entry.getKey -> lwwRegisterFromProto(entry.getValue)).toMap
new LWWMap(new ORMap(
keys = orsetFromProto(lwwmap.getKeys).asInstanceOf[ORSet[String]],
entries))
}
def pncountermapToProto(pncountermap: PNCounterMap): rd.PNCounterMap = {
val b = rd.PNCounterMap.newBuilder().setKeys(orsetToProto(pncountermap.underlying.keys))
pncountermap.underlying.entries.toVector.sortBy { case (key, _) key }.foreach {
case (key, value: PNCounter) b.addEntries(rd.PNCounterMap.Entry.newBuilder().
setKey(key).setValue(pncounterToProto(value)))
}
b.build()
}
def pncountermapFromBinary(bytes: Array[Byte]): PNCounterMap =
pncountermapFromProto(rd.PNCounterMap.parseFrom(decompress(bytes)))
def pncountermapFromProto(pncountermap: rd.PNCounterMap): PNCounterMap = {
val entries = pncountermap.getEntriesList.asScala.map(entry
entry.getKey -> pncounterFromProto(entry.getValue)).toMap
new PNCounterMap(new ORMap(
keys = orsetFromProto(pncountermap.getKeys).asInstanceOf[ORSet[String]],
entries))
}
def keyIdToBinary(id: String): Array[Byte] =
id.getBytes(UTF_8)
def keyIdFromBinary(bytes: Array[Byte]): String =
new String(bytes, UTF_8)
}
object OtherMessageComparator extends Comparator[dm.OtherMessage] {
override def compare(a: dm.OtherMessage, b: dm.OtherMessage): Int = {
val aByteString = a.getEnclosedMessage
val bByteString = b.getEnclosedMessage
val aSize = aByteString.size
val bSize = bByteString.size
if (aSize == bSize) {
val aIter = aByteString.iterator
val bIter = bByteString.iterator
@tailrec def findDiff(): Int = {
if (aIter.hasNext) {
val aByte = aIter.nextByte()
val bByte = bIter.nextByte()
if (aByte < bByte) -1
else if (aByte > bByte) 1
else findDiff()
} else 0
}
findDiff()
} else if (aSize < bSize) -1
else 1
}
}

View file

@ -0,0 +1,326 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata.protobuf
import java.util.concurrent.TimeUnit
import scala.collection.JavaConverters._
import scala.collection.breakOut
import scala.concurrent.duration.Duration
import akka.actor.ExtendedActorSystem
import akka.cluster.Member
import akka.cluster.UniqueAddress
import akka.cluster.ddata.PruningState
import akka.cluster.ddata.ReplicatedData
import akka.cluster.ddata.Replicator._
import akka.cluster.ddata.Replicator.Internal._
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages dm }
import akka.serialization.Serialization
import akka.serialization.SerializerWithStringManifest
import akka.serialization.BaseSerializer
import akka.util.{ ByteString AkkaByteString }
import com.google.protobuf.ByteString
import akka.cluster.ddata.Key.KeyR
/**
* Protobuf serializer of ReplicatorMessage messages.
*/
class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
extends SerializerWithStringManifest with SerializationSupport with BaseSerializer {
val GetManifest = "A"
val GetSuccessManifest = "B"
val NotFoundManifest = "C"
val GetFailureManifest = "D"
val SubscribeManifest = "E"
val UnsubscribeManifest = "F"
val ChangedManifest = "G"
val DataEnvelopeManifest = "H"
val WriteManifest = "I"
val WriteAckManifest = "J"
val ReadManifest = "K"
val ReadResultManifest = "L"
val StatusManifest = "M"
val GossipManifest = "N"
private val fromBinaryMap = collection.immutable.HashMap[String, Array[Byte] AnyRef](
GetManifest -> getFromBinary,
GetSuccessManifest -> getSuccessFromBinary,
NotFoundManifest -> notFoundFromBinary,
GetFailureManifest -> getFailureFromBinary,
SubscribeManifest -> subscribeFromBinary,
UnsubscribeManifest -> unsubscribeFromBinary,
ChangedManifest -> changedFromBinary,
DataEnvelopeManifest -> dataEnvelopeFromBinary,
WriteManifest -> writeFromBinary,
WriteAckManifest -> (_ WriteAck),
ReadManifest -> readFromBinary,
ReadResultManifest -> readResultFromBinary,
StatusManifest -> statusFromBinary,
GossipManifest -> gossipFromBinary)
override def manifest(obj: AnyRef): String = obj match {
case _: DataEnvelope DataEnvelopeManifest
case _: Write WriteManifest
case WriteAck WriteAckManifest
case _: Read ReadManifest
case _: ReadResult ReadResultManifest
case _: Status StatusManifest
case _: Get[_] GetManifest
case _: GetSuccess[_] GetSuccessManifest
case _: Changed[_] ChangedManifest
case _: NotFound[_] NotFoundManifest
case _: GetFailure[_] GetFailureManifest
case _: Subscribe[_] SubscribeManifest
case _: Unsubscribe[_] UnsubscribeManifest
case _: Gossip GossipManifest
case _
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
}
def toBinary(obj: AnyRef): Array[Byte] = obj match {
case m: DataEnvelope dataEnvelopeToProto(m).toByteArray
case m: Write writeToProto(m).toByteArray
case WriteAck dm.Empty.getDefaultInstance.toByteArray
case m: Read readToProto(m).toByteArray
case m: ReadResult readResultToProto(m).toByteArray
case m: Status statusToProto(m).toByteArray
case m: Get[_] getToProto(m).toByteArray
case m: GetSuccess[_] getSuccessToProto(m).toByteArray
case m: Changed[_] changedToProto(m).toByteArray
case m: NotFound[_] notFoundToProto(m).toByteArray
case m: GetFailure[_] getFailureToProto(m).toByteArray
case m: Subscribe[_] subscribeToProto(m).toByteArray
case m: Unsubscribe[_] unsubscribeToProto(m).toByteArray
case m: Gossip compress(gossipToProto(m))
case _
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
}
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
fromBinaryMap.get(manifest) match {
case Some(f) f(bytes)
case None throw new IllegalArgumentException(
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
}
private def statusToProto(status: Status): dm.Status = {
val b = dm.Status.newBuilder()
b.setChunk(status.chunk).setTotChunks(status.totChunks)
val entries = status.digests.foreach {
case (key, digest)
b.addEntries(dm.Status.Entry.newBuilder().
setKey(key).
setDigest(ByteString.copyFrom(digest.toArray)))
}
b.build()
}
private def statusFromBinary(bytes: Array[Byte]): Status = {
val status = dm.Status.parseFrom(bytes)
Status(status.getEntriesList.asScala.map(e
e.getKey -> AkkaByteString(e.getDigest.toByteArray()))(breakOut),
status.getChunk, status.getTotChunks)
}
private def gossipToProto(gossip: Gossip): dm.Gossip = {
val b = dm.Gossip.newBuilder().setSendBack(gossip.sendBack)
val entries = gossip.updatedData.foreach {
case (key, data)
b.addEntries(dm.Gossip.Entry.newBuilder().
setKey(key).
setEnvelope(dataEnvelopeToProto(data)))
}
b.build()
}
private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
val gossip = dm.Gossip.parseFrom(decompress(bytes))
Gossip(gossip.getEntriesList.asScala.map(e
e.getKey -> dataEnvelopeFromProto(e.getEnvelope))(breakOut),
sendBack = gossip.getSendBack)
}
private def getToProto(get: Get[_]): dm.Get = {
val consistencyValue = get.consistency match {
case ReadLocal 1
case ReadFrom(n, _) n
case _: ReadMajority 0
case _: ReadAll -1
}
val b = dm.Get.newBuilder().
setKey(otherMessageToProto(get.key)).
setConsistency(consistencyValue).
setTimeout(get.consistency.timeout.toMillis.toInt)
get.request.foreach(o b.setRequest(otherMessageToProto(o)))
b.build()
}
private def getFromBinary(bytes: Array[Byte]): Get[_] = {
val get = dm.Get.parseFrom(bytes)
val key = otherMessageFromProto(get.getKey).asInstanceOf[KeyR]
val request = if (get.hasRequest()) Some(otherMessageFromProto(get.getRequest)) else None
val timeout = Duration(get.getTimeout, TimeUnit.MILLISECONDS)
val consistency = get.getConsistency match {
case 0 ReadMajority(timeout)
case -1 ReadAll(timeout)
case 1 ReadLocal
case n ReadFrom(n, timeout)
}
Get(key, consistency, request)
}
private def getSuccessToProto(getSuccess: GetSuccess[_]): dm.GetSuccess = {
val b = dm.GetSuccess.newBuilder().
setKey(otherMessageToProto(getSuccess.key)).
setData(otherMessageToProto(getSuccess.dataValue))
getSuccess.request.foreach(o b.setRequest(otherMessageToProto(o)))
b.build()
}
private def getSuccessFromBinary(bytes: Array[Byte]): GetSuccess[_] = {
val getSuccess = dm.GetSuccess.parseFrom(bytes)
val key = otherMessageFromProto(getSuccess.getKey).asInstanceOf[KeyR]
val request = if (getSuccess.hasRequest()) Some(otherMessageFromProto(getSuccess.getRequest)) else None
val data = otherMessageFromProto(getSuccess.getData).asInstanceOf[ReplicatedData]
GetSuccess(key, request)(data)
}
private def notFoundToProto(notFound: NotFound[_]): dm.NotFound = {
val b = dm.NotFound.newBuilder().setKey(otherMessageToProto(notFound.key))
notFound.request.foreach(o b.setRequest(otherMessageToProto(o)))
b.build()
}
private def notFoundFromBinary(bytes: Array[Byte]): NotFound[_] = {
val notFound = dm.NotFound.parseFrom(bytes)
val request = if (notFound.hasRequest()) Some(otherMessageFromProto(notFound.getRequest)) else None
val key = otherMessageFromProto(notFound.getKey).asInstanceOf[KeyR]
NotFound(key, request)
}
private def getFailureToProto(getFailure: GetFailure[_]): dm.GetFailure = {
val b = dm.GetFailure.newBuilder().setKey(otherMessageToProto(getFailure.key))
getFailure.request.foreach(o b.setRequest(otherMessageToProto(o)))
b.build()
}
private def getFailureFromBinary(bytes: Array[Byte]): GetFailure[_] = {
val getFailure = dm.GetFailure.parseFrom(bytes)
val request = if (getFailure.hasRequest()) Some(otherMessageFromProto(getFailure.getRequest)) else None
val key = otherMessageFromProto(getFailure.getKey).asInstanceOf[KeyR]
GetFailure(key, request)
}
private def subscribeToProto(subscribe: Subscribe[_]): dm.Subscribe =
dm.Subscribe.newBuilder().
setKey(otherMessageToProto(subscribe.key)).
setRef(Serialization.serializedActorPath(subscribe.subscriber)).
build()
private def subscribeFromBinary(bytes: Array[Byte]): Subscribe[_] = {
val subscribe = dm.Subscribe.parseFrom(bytes)
val key = otherMessageFromProto(subscribe.getKey).asInstanceOf[KeyR]
Subscribe(key, resolveActorRef(subscribe.getRef))
}
private def unsubscribeToProto(unsubscribe: Unsubscribe[_]): dm.Unsubscribe =
dm.Unsubscribe.newBuilder().
setKey(otherMessageToProto(unsubscribe.key)).
setRef(Serialization.serializedActorPath(unsubscribe.subscriber)).
build()
private def unsubscribeFromBinary(bytes: Array[Byte]): Unsubscribe[_] = {
val unsubscribe = dm.Unsubscribe.parseFrom(bytes)
val key = otherMessageFromProto(unsubscribe.getKey).asInstanceOf[KeyR]
Unsubscribe(key, resolveActorRef(unsubscribe.getRef))
}
private def changedToProto(changed: Changed[_]): dm.Changed =
dm.Changed.newBuilder().
setKey(otherMessageToProto(changed.key)).
setData(otherMessageToProto(changed.dataValue)).
build()
private def changedFromBinary(bytes: Array[Byte]): Changed[_] = {
val changed = dm.Changed.parseFrom(bytes)
val data = otherMessageFromProto(changed.getData).asInstanceOf[ReplicatedData]
val key = otherMessageFromProto(changed.getKey).asInstanceOf[KeyR]
Changed(key)(data)
}
private def dataEnvelopeToProto(dataEnvelope: DataEnvelope): dm.DataEnvelope = {
val dataEnvelopeBuilder = dm.DataEnvelope.newBuilder().
setData(otherMessageToProto(dataEnvelope.data))
dataEnvelope.pruning.foreach {
case (removedAddress, state)
val b = dm.DataEnvelope.PruningEntry.newBuilder().
setRemovedAddress(uniqueAddressToProto(removedAddress)).
setOwnerAddress(uniqueAddressToProto(state.owner))
state.phase match {
case PruningState.PruningInitialized(seen)
seen.toVector.sorted(Member.addressOrdering).map(addressToProto).foreach { a b.addSeen(a) }
b.setPerformed(false)
case PruningState.PruningPerformed
b.setPerformed(true)
}
dataEnvelopeBuilder.addPruning(b)
}
dataEnvelopeBuilder.build()
}
private def dataEnvelopeFromBinary(bytes: Array[Byte]): DataEnvelope =
dataEnvelopeFromProto(dm.DataEnvelope.parseFrom(bytes))
private def dataEnvelopeFromProto(dataEnvelope: dm.DataEnvelope): DataEnvelope = {
val pruning: Map[UniqueAddress, PruningState] =
dataEnvelope.getPruningList.asScala.map { pruningEntry
val phase =
if (pruningEntry.getPerformed) PruningState.PruningPerformed
else PruningState.PruningInitialized(pruningEntry.getSeenList.asScala.map(addressFromProto)(breakOut))
val state = PruningState(uniqueAddressFromProto(pruningEntry.getOwnerAddress), phase)
val removed = uniqueAddressFromProto(pruningEntry.getRemovedAddress)
removed -> state
}(breakOut)
val data = otherMessageFromProto(dataEnvelope.getData).asInstanceOf[ReplicatedData]
DataEnvelope(data, pruning)
}
private def writeToProto(write: Write): dm.Write =
dm.Write.newBuilder().
setKey(write.key).
setEnvelope(dataEnvelopeToProto(write.envelope)).
build()
private def writeFromBinary(bytes: Array[Byte]): Write = {
val write = dm.Write.parseFrom(bytes)
Write(write.getKey, dataEnvelopeFromProto(write.getEnvelope))
}
private def readToProto(read: Read): dm.Read =
dm.Read.newBuilder().setKey(read.key).build()
private def readFromBinary(bytes: Array[Byte]): Read =
Read(dm.Read.parseFrom(bytes).getKey)
private def readResultToProto(readResult: ReadResult): dm.ReadResult = {
val b = dm.ReadResult.newBuilder()
readResult.envelope match {
case Some(d) b.setEnvelope(dataEnvelopeToProto(d))
case None
}
b.build()
}
private def readResultFromBinary(bytes: Array[Byte]): ReadResult = {
val readResult = dm.ReadResult.parseFrom(bytes)
val envelope =
if (readResult.hasEnvelope) Some(dataEnvelopeFromProto(readResult.getEnvelope))
else None
ReadResult(envelope)
}
}

View file

@ -0,0 +1,144 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata.protobuf
import java.io.ByteArrayInputStream
import java.io.ByteArrayOutputStream
import java.util.zip.GZIPInputStream
import java.util.zip.GZIPOutputStream
import scala.annotation.tailrec
import akka.actor.ActorRef
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.cluster.UniqueAddress
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages dm }
import akka.serialization.JSerializer
import akka.serialization.Serialization
import akka.serialization.SerializationExtension
import com.google.protobuf.ByteString
import com.google.protobuf.MessageLite
import akka.serialization.SerializerWithStringManifest
/**
* Some useful serialization helper methods.
*/
trait SerializationSupport {
private final val BufferSize = 1024 * 4
def system: ExtendedActorSystem
@volatile
private var ser: Serialization = _
def serialization: Serialization = {
if (ser == null) ser = SerializationExtension(system)
ser
}
@volatile
private var protocol: String = _
def addressProtocol: String = {
if (protocol == null) protocol = system.provider.getDefaultAddress.protocol
protocol
}
@volatile
private var transportInfo: Serialization.Information = _
def transportInformation: Serialization.Information = {
if (transportInfo == null) {
val address = system.provider.getDefaultAddress
transportInfo = Serialization.Information(address, system)
}
transportInfo
}
def compress(msg: MessageLite): Array[Byte] = {
val bos = new ByteArrayOutputStream(BufferSize)
val zip = new GZIPOutputStream(bos)
msg.writeTo(zip)
zip.close()
bos.toByteArray
}
def decompress(bytes: Array[Byte]): Array[Byte] = {
val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
val out = new ByteArrayOutputStream()
val buffer = new Array[Byte](BufferSize)
@tailrec def readChunk(): Unit = in.read(buffer) match {
case -1 ()
case n
out.write(buffer, 0, n)
readChunk()
}
readChunk()
out.toByteArray
}
def addressToProto(address: Address): dm.Address.Builder = address match {
case Address(_, _, Some(host), Some(port))
dm.Address.newBuilder().setHostname(host).setPort(port)
case _ throw new IllegalArgumentException(s"Address [${address}] could not be serialized: host or port missing.")
}
def addressFromProto(address: dm.Address): Address =
Address(addressProtocol, system.name, address.getHostname, address.getPort)
def uniqueAddressToProto(uniqueAddress: UniqueAddress): dm.UniqueAddress.Builder =
dm.UniqueAddress.newBuilder().setAddress(addressToProto(uniqueAddress.address)).setUid(uniqueAddress.uid)
def uniqueAddressFromProto(uniqueAddress: dm.UniqueAddress): UniqueAddress =
UniqueAddress(addressFromProto(uniqueAddress.getAddress), uniqueAddress.getUid)
def resolveActorRef(path: String): ActorRef =
system.provider.resolveActorRef(path)
def otherMessageToProto(msg: Any): dm.OtherMessage = {
def buildOther(): dm.OtherMessage = {
val m = msg.asInstanceOf[AnyRef]
val msgSerializer = serialization.findSerializerFor(m)
val builder = dm.OtherMessage.newBuilder().
setEnclosedMessage(ByteString.copyFrom(msgSerializer.toBinary(m)))
.setSerializerId(msgSerializer.identifier)
msgSerializer match {
case ser2: SerializerWithStringManifest
val manifest = ser2.manifest(m)
if (manifest != "")
builder.setMessageManifest(ByteString.copyFromUtf8(manifest))
case _
if (msgSerializer.includeManifest)
builder.setMessageManifest(ByteString.copyFromUtf8(m.getClass.getName))
}
builder.build()
}
// Serialize actor references with full address information (defaultAddress).
// When sending remote messages currentTransportInformation is already set,
// but when serializing for digests it must be set here.
if (Serialization.currentTransportInformation.value == null)
Serialization.currentTransportInformation.withValue(transportInformation) { buildOther() }
else
buildOther()
}
def otherMessageFromBinary(bytes: Array[Byte]): AnyRef =
otherMessageFromProto(dm.OtherMessage.parseFrom(bytes))
def otherMessageFromProto(other: dm.OtherMessage): AnyRef = {
val manifest = if (other.hasMessageManifest) other.getMessageManifest.toStringUtf8 else ""
serialization.deserialize(
other.getEnclosedMessage.toByteArray,
other.getSerializerId,
manifest).get
}
}
/**
* Java API
*/
abstract class AbstractSerializationSupport extends JSerializer with SerializationSupport

View file

@ -0,0 +1,283 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
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 JepsenInspiredInsertSpec extends MultiNodeConfig {
val controller = role("controller")
val n1 = role("n1")
val n2 = role("n2")
val n3 = role("n3")
val n4 = role("n4")
val n5 = role("n5")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters = off
akka.log-dead-letters-during-shutdown = off
akka.remote.log-remote-lifecycle-events = ERROR
akka.testconductor.barrier-timeout = 60 s
"""))
testTransport(on = true)
}
class JepsenInspiredInsertSpecMultiJvmNode1 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode2 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode3 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode4 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode5 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpecMultiJvmNode6 extends JepsenInspiredInsertSpec
class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) with STMultiNodeSpec with ImplicitSender {
import JepsenInspiredInsertSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val replicator = DistributedData(system).replicator
val nodes = roles.drop(1) // controller not part of active nodes
val nodeCount = nodes.size
val timeout = 3.seconds.dilated
val delayMillis = 0
val totalCount = 200
// val delayMillis = 20
// val totalCount = 2000
val expectedData = (0 until totalCount).toSet
val data: Map[RoleName, Seq[Int]] = {
val nodeIndex = nodes.zipWithIndex.map { case (n, i) i -> n }.toMap
(0 until totalCount).groupBy(i nodeIndex(i % nodeCount))
}
lazy val myData: Seq[Int] = data(myself)
def sleepDelay(): Unit =
if (delayMillis != 0) {
val rndDelay = ThreadLocalRandom.current().nextInt(delayMillis)
if (rndDelay != 0) Thread.sleep(delayMillis)
}
def sleepBeforePartition(): Unit = {
if (delayMillis != 0)
Thread.sleep(delayMillis * totalCount / nodeCount / 10)
}
def sleepDuringPartition(): Unit =
Thread.sleep(math.max(5000, delayMillis * totalCount / nodeCount / 2))
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Insert from 5 nodes" must {
"setup cluster" in {
runOn(nodes: _*) {
nodes.foreach { join(_, n1) }
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(nodes.size))
}
}
}
runOn(controller) {
nodes.foreach { n enterBarrier(n.name + "-joined") }
}
enterBarrier("after-setup")
}
}
"replicate values when all nodes connected" in {
val key = ORSetKey[Int]("A")
runOn(nodes: _*) {
val writeProbe = TestProbe()
val writeAcks = myData.map { i
sleepDelay()
replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ + i), writeProbe.ref)
writeProbe.receiveOne(3.seconds)
}
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] success }
val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure[_] fail }
successWriteAcks.map(_.request.get).toSet should be(myData.toSet)
successWriteAcks.size should be(myData.size)
failureWriteAcks should be(Nil)
(successWriteAcks.size + failureWriteAcks.size) should be(myData.size)
// eventually all nodes will have the data
within(15.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.elements should be(expectedData)
}
}
}
enterBarrier("after-test-1")
}
"write/read to majority when all nodes connected" in {
val key = ORSetKey[Int]("B")
val readMajority = ReadMajority(timeout)
val writeMajority = WriteMajority(timeout)
runOn(nodes: _*) {
val writeProbe = TestProbe()
val writeAcks = myData.map { i
sleepDelay()
replicator.tell(Update(key, ORSet(), writeMajority, Some(i))(_ + i), writeProbe.ref)
writeProbe.receiveOne(timeout + 1.second)
}
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] success }
val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure[_] fail }
successWriteAcks.map(_.request.get).toSet should be(myData.toSet)
successWriteAcks.size should be(myData.size)
failureWriteAcks should be(Nil)
(successWriteAcks.size + failureWriteAcks.size) should be(myData.size)
enterBarrier("data-written-2")
// read from majority of nodes, which is enough to retrieve all data
val readProbe = TestProbe()
replicator.tell(Get(key, readMajority), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
val survivors = result.elements.size
result.elements should be(expectedData)
}
runOn(controller) {
enterBarrier("data-written-2")
}
enterBarrier("after-test-2")
}
"replicate values after partition" in {
val key = ORSetKey[Int]("C")
runOn(controller) {
sleepBeforePartition()
for (a List(n1, n4, n5); b List(n2, n3))
testConductor.blackhole(a, b, Direction.Both).await
sleepDuringPartition()
for (a List(n1, n4, n5); b List(n2, n3))
testConductor.passThrough(a, b, Direction.Both).await
enterBarrier("partition-healed-3")
}
runOn(nodes: _*) {
val writeProbe = TestProbe()
val writeAcks = myData.map { i
sleepDelay()
replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ + i), writeProbe.ref)
writeProbe.receiveOne(3.seconds)
}
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] success }
val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure[_] fail }
successWriteAcks.map(_.request.get).toSet should be(myData.toSet)
successWriteAcks.size should be(myData.size)
failureWriteAcks should be(Nil)
(successWriteAcks.size + failureWriteAcks.size) should be(myData.size)
enterBarrier("partition-healed-3")
// eventually all nodes will have the data
within(15.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.elements should be(expectedData)
}
}
}
enterBarrier("after-test-3")
}
"write to majority during 3+2 partition and read from majority after partition" in {
val key = ORSetKey[Int]("D")
val readMajority = ReadMajority(timeout)
val writeMajority = WriteMajority(timeout)
runOn(controller) {
sleepBeforePartition()
for (a List(n1, n4, n5); b List(n2, n3))
testConductor.blackhole(a, b, Direction.Both).await
sleepDuringPartition()
for (a List(n1, n4, n5); b List(n2, n3))
testConductor.passThrough(a, b, Direction.Both).await
enterBarrier("partition-healed-4")
}
runOn(nodes: _*) {
val writeProbe = TestProbe()
val writeAcks = myData.map { i
sleepDelay()
replicator.tell(Update(key, ORSet(), writeMajority, Some(i))(_ + i), writeProbe.ref)
writeProbe.receiveOne(timeout + 1.second)
}
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] success }
val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure[_] fail }
runOn(n1, n4, n5) {
successWriteAcks.map(_.request.get).toSet should be(myData.toSet)
successWriteAcks.size should be(myData.size)
failureWriteAcks should be(Nil)
}
runOn(n2, n3) {
// without delays all could teoretically have been written before the blackhole
if (delayMillis != 0)
failureWriteAcks should not be (Nil)
}
(successWriteAcks.size + failureWriteAcks.size) should be(myData.size)
enterBarrier("partition-healed-4")
// on the 2 node side, read from majority of nodes is enough to read all writes
runOn(n2, n3) {
val readProbe = TestProbe()
replicator.tell(Get(key, readMajority), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
val survivors = result.elements.size
result.elements should be(expectedData)
}
// but on the 3 node side, read from majority doesn't mean that we are guaranteed to see
// the writes from the other side, yet
// eventually all nodes will have the data
within(15.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.elements should be(expectedData)
}
}
}
enterBarrier("after-test-4")
}
}

View file

@ -0,0 +1,233 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Deploy
import akka.actor.Props
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
object PerformanceSpec extends MultiNodeConfig {
val n1 = role("n1")
val n2 = role("n2")
val n3 = role("n3")
val n4 = role("n4")
val n5 = role("n5")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = ERROR
akka.stdout-loglevel = ERROR
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters = off
akka.log-dead-letters-during-shutdown = off
akka.remote.log-remote-lifecycle-events = ERROR
akka.remote.log-frame-size-exceeding=1000b
akka.testconductor.barrier-timeout = 60 s
akka.cluster.distributed-data.gossip-interval = 1 s
akka.actor.serialize-messages = off
"""))
def countDownProps(latch: TestLatch): Props = Props(new CountDown(latch)).withDeploy(Deploy.local)
class CountDown(latch: TestLatch) extends Actor {
def receive = {
case _
latch.countDown()
if (latch.isOpen)
context.stop(self)
}
}
}
class PerformanceSpecMultiJvmNode1 extends PerformanceSpec
class PerformanceSpecMultiJvmNode2 extends PerformanceSpec
class PerformanceSpecMultiJvmNode3 extends PerformanceSpec
class PerformanceSpecMultiJvmNode4 extends PerformanceSpec
class PerformanceSpecMultiJvmNode5 extends PerformanceSpec
class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpec with ImplicitSender {
import PerformanceSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val replicator = DistributedData(system).replicator
val timeout = 3.seconds.dilated
val factor = 1 // use 3 here for serious tuning
val repeatCount = 3 // use at least 10 here for serious tuning
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
def repeat(description: String, keys: Iterable[ORSetKey[Int]], n: Int, expectedAfterReplication: Option[Set[Int]] = None)(
block: (ORSetKey[Int], Int, ActorRef) Unit, afterEachKey: ORSetKey[Int] Unit = _ ()): Unit = {
keys.foreach { key
val startTime = System.nanoTime()
runOn(n1) {
val latch = TestLatch(n)
val replyTo = system.actorOf(countDownProps(latch))
var i = 0
while (i < n) {
block(key, i, replyTo)
i += 1
}
Await.ready(latch, 5.seconds + (1.second * factor))
}
expectedAfterReplication.foreach { expected
enterBarrier("repeat-" + key + "-before-awaitReplicated")
awaitReplicated(key, expected)
enterBarrier("repeat-" + key + "-after-awaitReplicated")
}
runOn(n1) {
val endTime = System.nanoTime()
val durationMs = (endTime - startTime).nanos.toMillis
val tps = (n * 1000.0 / durationMs).toInt
println(s"## $n $description took $durationMs ms, $tps TPS")
}
afterEachKey(key)
enterBarrier("repeat-" + key + "-done")
}
}
def awaitReplicated(keys: Iterable[ORSetKey[Int]], expectedData: Set[Int]): Unit =
keys.foreach { key awaitReplicated(key, expectedData) }
def awaitReplicated(key: ORSetKey[Int], expectedData: Set[Int]): Unit = {
within(20.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.elements should be(expectedData)
}
}
}
"Performance" must {
"setup cluster" in {
roles.foreach { join(_, n1) }
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
}
enterBarrier("after-setup")
}
"be great for ORSet Update WriteLocal" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("A" + n))
val n = 1000 * factor
val expectedData = (0 until n).toSet
repeat("ORSet Update WriteLocal", keys, n)({ (key, i, replyTo)
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + i), replyTo)
}, key awaitReplicated(key, expectedData))
enterBarrier("after-1")
}
"be blazingly fast for ORSet Get ReadLocal" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("A" + n))
repeat("Get ReadLocal", keys, 100000 * factor) { (key, i, replyTo)
replicator.tell(Get(key, ReadLocal), replyTo)
}
enterBarrier("after-2")
}
"be good for ORSet Update WriteLocal and gossip replication" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("B" + n))
val n = 200 * factor
val expected = Some((0 until n).toSet)
repeat("ORSet Update WriteLocal + gossip", keys, n, expected) { (key, i, replyTo)
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + i), replyTo)
}
enterBarrier("after-3")
}
"be good for ORSet Update WriteLocal and gossip of existing keys" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("B" + n))
val n = 200 * factor
val expected = Some((0 until n).toSet ++ (0 until n).map(-_).toSet)
repeat("ORSet Update WriteLocal existing + gossip", keys, n, expected) { (key, i, replyTo)
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + (-i)), replyTo)
}
enterBarrier("after-4")
}
"be good for ORSet Update WriteTwo and gossip replication" in {
val keys = (1 to repeatCount).map(n ORSetKey[Int]("C" + n))
val n = 200 * factor
val expected = Some((0 until n).toSet)
val writeTwo = WriteTo(2, timeout)
repeat("ORSet Update WriteTwo + gossip", keys, n, expected) { (key, i, replyTo)
replicator.tell(Update(key, ORSet(), writeTwo)(_ + i), replyTo)
}
enterBarrier("after-5")
}
"be awesome for GCounter Update WriteLocal" in {
val startTime = System.nanoTime()
val n = 1000 * factor
val key = GCounterKey("D")
runOn(n1, n2, n3) {
val latch = TestLatch(n)
val replyTo = system.actorOf(countDownProps(latch))
for (_ 0 until n)
replicator.tell(Update(key, GCounter(), WriteLocal)(_ + 1), replyTo)
Await.ready(latch, 5.seconds + (1.second * factor))
enterBarrier("update-done-6")
runOn(n1) {
val endTime = System.nanoTime()
val durationMs = (endTime - startTime).nanos.toMillis
val tps = (3 * n * 1000.0 / durationMs).toInt
println(s"## ${3 * n} GCounter Update took $durationMs ms, $tps TPS")
}
}
runOn(n4, n5) {
enterBarrier("update-done-6")
}
within(20.seconds) {
awaitAssert {
val readProbe = TestProbe()
replicator.tell(Get(key, ReadLocal), readProbe.ref)
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) g.get(key) }
result.value should be(3 * n)
}
}
enterBarrier("replication-done-6")
runOn(n1) {
val endTime = System.nanoTime()
val durationMs = (endTime - startTime).nanos.toMillis
val tps = (n * 1000.0 / durationMs).toInt
println(s"## $n GCounter Update + gossip took $durationMs ms, $tps TPS")
}
enterBarrier("after-6")
}
}
}

View file

@ -0,0 +1,234 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.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 ReplicatorChaosSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
val fourth = role("fourth")
val fifth = role("fifth")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.cluster.roles = ["backend"]
akka.log-dead-letters-during-shutdown = off
"""))
testTransport(on = true)
}
class ReplicatorChaosSpecMultiJvmNode1 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode2 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode3 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode4 extends ReplicatorChaosSpec
class ReplicatorChaosSpecMultiJvmNode5 extends ReplicatorChaosSpec
class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatorChaosSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val replicator = system.actorOf(Replicator.props(
ReplicatorSettings(system).withRole("backend").withGossipInterval(1.second)), "replicator")
val timeout = 3.seconds.dilated
val KeyA = GCounterKey("A")
val KeyB = PNCounterKey("B")
val KeyC = GCounterKey("C")
val KeyD = GCounterKey("D")
val KeyE = GSetKey[String]("E")
val KeyF = ORSetKey[String]("F")
val KeyX = GCounterKey("X")
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: GCounter c.value
case c: PNCounter c.value
case c: GSet[_] c.elements
case c: ORSet[_] c.elements
}
}
value should be(expected)
}
}
def assertDeleted(key: Key[ReplicatedData]): Unit =
within(5.seconds) {
awaitAssert {
replicator ! Get(key, ReadLocal)
expectMsg(DataDeleted(key))
}
}
"Replicator in chaotic cluster" must {
"replicate data in initial phase" in {
join(first, first)
join(second, first)
join(third, first)
join(fourth, first)
join(fifth, first)
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(5))
}
}
runOn(first) {
(0 until 5).foreach { i
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
replicator ! Update(KeyB, PNCounter(), WriteLocal)(_ - 1)
replicator ! Update(KeyC, GCounter(), WriteAll(timeout))(_ + 1)
}
receiveN(15).map(_.getClass).toSet should be(Set(classOf[UpdateSuccess[_]]))
}
runOn(second) {
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 20)
replicator ! Update(KeyB, PNCounter(), WriteTo(2, timeout))(_ + 20)
replicator ! Update(KeyC, GCounter(), WriteAll(timeout))(_ + 20)
receiveN(3).toSet should be(Set(UpdateSuccess(KeyA, None),
UpdateSuccess(KeyB, None), UpdateSuccess(KeyC, None)))
replicator ! Update(KeyE, GSet(), WriteLocal)(_ + "e1" + "e2")
expectMsg(UpdateSuccess(KeyE, None))
replicator ! Update(KeyF, ORSet(), WriteLocal)(_ + "e1" + "e2")
expectMsg(UpdateSuccess(KeyF, None))
}
runOn(fourth) {
replicator ! Update(KeyD, GCounter(), WriteLocal)(_ + 40)
expectMsg(UpdateSuccess(KeyD, None))
replicator ! Update(KeyE, GSet(), WriteLocal)(_ + "e2" + "e3")
expectMsg(UpdateSuccess(KeyE, None))
replicator ! Update(KeyF, ORSet(), WriteLocal)(_ + "e2" + "e3")
expectMsg(UpdateSuccess(KeyF, None))
}
runOn(fifth) {
replicator ! Update(KeyX, GCounter(), WriteTo(2, timeout))(_ + 50)
expectMsg(UpdateSuccess(KeyX, None))
replicator ! Delete(KeyX, WriteLocal)
expectMsg(DeleteSuccess(KeyX))
}
enterBarrier("initial-updates-done")
assertValue(KeyA, 25)
assertValue(KeyB, 15)
assertValue(KeyC, 25)
assertValue(KeyD, 40)
assertValue(KeyE, Set("e1", "e2", "e3"))
assertValue(KeyF, Set("e1", "e2", "e3"))
assertDeleted(KeyX)
enterBarrier("after-1")
}
"be available during network split" in {
val side1 = Seq(first, second)
val side2 = Seq(third, fourth, fifth)
runOn(first) {
for (a side1; b side2)
testConductor.blackhole(a, b, Direction.Both).await
}
enterBarrier("split")
runOn(first) {
replicator ! Update(KeyA, GCounter(), WriteTo(2, timeout))(_ + 1)
expectMsg(UpdateSuccess(KeyA, None))
}
runOn(third) {
replicator ! Update(KeyA, GCounter(), WriteTo(2, timeout))(_ + 2)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Update(KeyE, GSet(), WriteTo(2, timeout))(_ + "e4")
expectMsg(UpdateSuccess(KeyE, None))
replicator ! Update(KeyF, ORSet(), WriteTo(2, timeout))(_ - "e2")
expectMsg(UpdateSuccess(KeyF, None))
}
runOn(fourth) {
replicator ! Update(KeyD, GCounter(), WriteTo(2, timeout))(_ + 1)
expectMsg(UpdateSuccess(KeyD, None))
}
enterBarrier("update-during-split")
runOn(side1: _*) {
assertValue(KeyA, 26)
assertValue(KeyB, 15)
assertValue(KeyD, 40)
assertValue(KeyE, Set("e1", "e2", "e3"))
assertValue(KeyF, Set("e1", "e2", "e3"))
}
runOn(side2: _*) {
assertValue(KeyA, 27)
assertValue(KeyB, 15)
assertValue(KeyD, 41)
assertValue(KeyE, Set("e1", "e2", "e3", "e4"))
assertValue(KeyF, Set("e1", "e3"))
}
enterBarrier("update-during-split-verified")
runOn(first) {
testConductor.exit(fourth, 0).await
}
enterBarrier("after-2")
}
"converge after partition" in {
val side1 = Seq(first, second)
val side2 = Seq(third, fifth) // fourth was shutdown
runOn(first) {
for (a side1; b side2)
testConductor.passThrough(a, b, Direction.Both).await
}
enterBarrier("split-repaired")
assertValue(KeyA, 28)
assertValue(KeyB, 15)
assertValue(KeyC, 25)
assertValue(KeyD, 41)
assertValue(KeyE, Set("e1", "e2", "e3", "e4"))
assertValue(KeyF, Set("e1", "e3"))
assertDeleted(KeyX)
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,197 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.duration._
import akka.cluster.Cluster
import akka.cluster.ClusterEvent.InitialStateAsEvents
import akka.cluster.ClusterEvent.MemberUp
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
object ReplicatorPruningSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
class ReplicatorPruningSpecMultiJvmNode1 extends ReplicatorPruningSpec
class ReplicatorPruningSpecMultiJvmNode2 extends ReplicatorPruningSpec
class ReplicatorPruningSpecMultiJvmNode3 extends ReplicatorPruningSpec
class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatorPruningSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val maxPruningDissemination = 3.seconds
val replicator = system.actorOf(Replicator.props(
ReplicatorSettings(system).withGossipInterval(1.second)
.withPruning(pruningInterval = 1.second, maxPruningDissemination)), "replicator")
val timeout = 2.seconds.dilated
val KeyA = GCounterKey("A")
val KeyB = ORSetKey[String]("B")
val KeyC = PNCounterMapKey("C")
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Pruning of CRDT" must {
"move data from removed node" in {
join(first, first)
join(second, first)
join(third, first)
within(5.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(3))
}
}
// we need the UniqueAddress
val memberProbe = TestProbe()
cluster.subscribe(memberProbe.ref, initialStateMode = InitialStateAsEvents, classOf[MemberUp])
val thirdUniqueAddress = {
val member = memberProbe.fishForMessage(3.seconds) {
case MemberUp(m) if m.address == node(third).address true
case _ false
}.asInstanceOf[MemberUp].member
member.uniqueAddress
}
replicator ! Update(KeyA, GCounter(), WriteAll(timeout))(_ + 3)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Update(KeyB, ORSet(), WriteAll(timeout))(_ + "a" + "b" + "c")
expectMsg(UpdateSuccess(KeyB, None))
replicator ! Update(KeyC, PNCounterMap(), WriteAll(timeout))(_ increment "x" increment "y")
expectMsg(UpdateSuccess(KeyC, None))
enterBarrier("updates-done")
replicator ! Get(KeyA, ReadLocal)
val oldCounter = expectMsgType[GetSuccess[GCounter]].dataValue
oldCounter.value should be(9)
replicator ! Get(KeyB, ReadLocal)
val oldSet = expectMsgType[GetSuccess[ORSet[String]]].dataValue
oldSet.elements should be(Set("a", "b", "c"))
replicator ! Get(KeyC, ReadLocal)
val oldMap = expectMsgType[GetSuccess[PNCounterMap]].dataValue
oldMap.get("x") should be(Some(3))
oldMap.get("y") should be(Some(3))
enterBarrier("get-old")
runOn(first) {
cluster.leave(node(third).address)
}
runOn(first, second) {
within(15.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(2))
}
}
}
enterBarrier("third-removed")
runOn(first, second) {
within(15.seconds) {
awaitAssert {
replicator ! Get(KeyA, ReadLocal)
expectMsgPF() {
case g @ GetSuccess(KeyA, _)
g.get(KeyA).value should be(9)
g.get(KeyA).needPruningFrom(thirdUniqueAddress) should be(false)
}
}
}
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyB, ReadLocal)
expectMsgPF() {
case g @ GetSuccess(KeyB, _)
g.get(KeyB).elements should be(Set("a", "b", "c"))
g.get(KeyB).needPruningFrom(thirdUniqueAddress) should be(false)
}
}
}
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyC, ReadLocal)
expectMsgPF() {
case g @ GetSuccess(KeyC, _)
g.get(KeyC).entries should be(Map("x" -> 3L, "y" -> 3L))
g.get(KeyC).needPruningFrom(thirdUniqueAddress) should be(false)
}
}
}
}
enterBarrier("pruning-done")
// on one of the nodes the data has been updated by the pruning,
// client can update anyway
def updateAfterPruning(expectedValue: Int): Unit = {
replicator ! Update(KeyA, GCounter(), WriteAll(timeout), None)(_ + 1)
expectMsgPF() {
case UpdateSuccess(KeyA, _)
replicator ! Get(KeyA, ReadLocal)
val retrieved = expectMsgType[GetSuccess[GCounter]].dataValue
retrieved.value should be(expectedValue)
}
}
runOn(first) {
updateAfterPruning(expectedValue = 10)
}
enterBarrier("update-first-after-pruning")
runOn(second) {
updateAfterPruning(expectedValue = 11)
}
enterBarrier("update-second-after-pruning")
// after pruning performed and maxDissemination it is tombstoned
// and we should still not be able to update with data from removed node
expectNoMsg(maxPruningDissemination + 3.seconds)
runOn(first) {
updateAfterPruning(expectedValue = 12)
}
enterBarrier("update-first-after-tombstone")
runOn(second) {
updateAfterPruning(expectedValue = 13)
}
enterBarrier("update-second-after-tombstone")
enterBarrier("after-1")
}
}
}

View file

@ -0,0 +1,503 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.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 ReplicatorSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
testTransport(on = true)
}
class ReplicatorSpecMultiJvmNode1 extends ReplicatorSpec
class ReplicatorSpecMultiJvmNode2 extends ReplicatorSpec
class ReplicatorSpecMultiJvmNode3 extends ReplicatorSpec
class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatorSpec._
import Replicator._
override def initialParticipants = roles.size
implicit val cluster = Cluster(system)
val replicator = system.actorOf(Replicator.props(
ReplicatorSettings(system).withGossipInterval(1.second).withMaxDeltaElements(10)), "replicator")
val timeout = 2.seconds.dilated
val writeTwo = WriteTo(2, timeout)
val writeMajority = WriteMajority(timeout)
val writeAll = WriteAll(timeout)
val readTwo = ReadFrom(2, timeout)
val readAll = ReadAll(timeout)
val readMajority = ReadMajority(timeout)
val KeyA = GCounterKey("A")
val KeyB = GCounterKey("B")
val KeyC = GCounterKey("C")
val KeyD = GCounterKey("D")
val KeyE = GCounterKey("E")
val KeyE2 = GCounterKey("E2")
val KeyF = GCounterKey("F")
val KeyG = ORSetKey[String]("G")
val KeyX = GCounterKey("X")
val KeyY = GCounterKey("Y")
val KeyZ = GCounterKey("Z")
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Cluster CRDT" must {
"work in single node cluster" in {
join(first, first)
runOn(first) {
within(5.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(1))
}
}
val changedProbe = TestProbe()
replicator ! Subscribe(KeyA, changedProbe.ref)
replicator ! Subscribe(KeyX, changedProbe.ref)
replicator ! Get(KeyA, ReadLocal)
expectMsg(NotFound(KeyA, None))
val c3 = GCounter() + 3
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 3)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Get(KeyA, ReadLocal)
expectMsg(GetSuccess(KeyA, None)(c3)).dataValue should be(c3)
changedProbe.expectMsg(Changed(KeyA)(c3)).dataValue should be(c3)
val changedProbe2 = TestProbe()
replicator ! Subscribe(KeyA, changedProbe2.ref)
changedProbe2.expectMsg(Changed(KeyA)(c3)).dataValue should be(c3)
val c4 = c3 + 1
// too strong consistency level
replicator ! Update(KeyA, GCounter(), writeTwo)(_ + 1)
expectMsg(UpdateTimeout(KeyA, None))
replicator ! Get(KeyA, ReadLocal)
expectMsg(GetSuccess(KeyA, None)(c4)).dataValue should be(c4)
changedProbe.expectMsg(Changed(KeyA)(c4)).dataValue should be(c4)
val c5 = c4 + 1
// too strong consistency level
replicator ! Update(KeyA, GCounter(), writeMajority)(_ + 1)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Get(KeyA, readMajority)
expectMsg(GetSuccess(KeyA, None)(c5)).dataValue should be(c5)
changedProbe.expectMsg(Changed(KeyA)(c5)).dataValue should be(c5)
val c6 = c5 + 1
replicator ! Update(KeyA, GCounter(), writeAll)(_ + 1)
expectMsg(UpdateSuccess(KeyA, None))
replicator ! Get(KeyA, readAll)
expectMsg(GetSuccess(KeyA, None)(c6)).dataValue should be(c6)
changedProbe.expectMsg(Changed(KeyA)(c6)).dataValue should be(c6)
val c9 = GCounter() + 9
replicator ! Update(KeyX, GCounter(), WriteLocal)(_ + 9)
expectMsg(UpdateSuccess(KeyX, None))
changedProbe.expectMsg(Changed(KeyX)(c9)).dataValue should be(c9)
replicator ! Delete(KeyX, WriteLocal)
expectMsg(DeleteSuccess(KeyX))
changedProbe.expectMsg(DataDeleted(KeyX))
replicator ! Get(KeyX, ReadLocal)
expectMsg(DataDeleted(KeyX))
replicator ! Get(KeyX, readAll)
expectMsg(DataDeleted(KeyX))
replicator ! Update(KeyX, GCounter(), WriteLocal)(_ + 1)
expectMsg(DataDeleted(KeyX))
replicator ! Delete(KeyX, WriteLocal)
expectMsg(DataDeleted(KeyX))
replicator ! GetKeyIds
expectMsg(GetKeyIdsResult(Set("A")))
}
enterBarrier("after-1")
}
}
"reply with ModifyFailure if exception is thrown by modify function" in {
val e = new RuntimeException("errr")
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ throw e)
expectMsgType[ModifyFailure[_]].cause should be(e)
}
"replicate values to new node" in {
join(second, first)
runOn(first, second) {
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(2))
}
}
}
enterBarrier("2-nodes")
runOn(second) {
val changedProbe = TestProbe()
replicator ! Subscribe(KeyA, changedProbe.ref)
// "A" should be replicated via gossip to the new node
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyA, ReadLocal)
val c = expectMsgPF() { case g @ GetSuccess(KeyA, _) g.get(KeyA) }
c.value should be(6)
}
}
val c = changedProbe.expectMsgPF() { case c @ Changed(KeyA) c.get(KeyA) }
c.value should be(6)
}
enterBarrier("after-2")
}
"work in 2 node cluster" in {
runOn(first, second) {
// start with 20 on both nodes
replicator ! Update(KeyB, GCounter(), WriteLocal)(_ + 20)
expectMsg(UpdateSuccess(KeyB, None))
// add 1 on both nodes using WriteTwo
replicator ! Update(KeyB, GCounter(), writeTwo)(_ + 1)
expectMsg(UpdateSuccess(KeyB, None))
// the total, after replication should be 42
awaitAssert {
replicator ! Get(KeyB, readTwo)
val c = expectMsgPF() { case g @ GetSuccess(KeyB, _) g.get(KeyB) }
c.value should be(42)
}
}
enterBarrier("update-42")
runOn(first, second) {
// add 1 on both nodes using WriteAll
replicator ! Update(KeyB, GCounter(), writeAll)(_ + 1)
expectMsg(UpdateSuccess(KeyB, None))
// the total, after replication should be 44
awaitAssert {
replicator ! Get(KeyB, readAll)
val c = expectMsgPF() { case g @ GetSuccess(KeyB, _) g.get(KeyB) }
c.value should be(44)
}
}
enterBarrier("update-44")
runOn(first, second) {
// add 1 on both nodes using WriteMajority
replicator ! Update(KeyB, GCounter(), writeMajority)(_ + 1)
expectMsg(UpdateSuccess(KeyB, None))
// the total, after replication should be 46
awaitAssert {
replicator ! Get(KeyB, readMajority)
val c = expectMsgPF() { case g @ GetSuccess(KeyB, _) g.get(KeyB) }
c.value should be(46)
}
}
enterBarrier("after-3")
}
"be replicated after succesful update" in {
val changedProbe = TestProbe()
runOn(first, second) {
replicator ! Subscribe(KeyC, changedProbe.ref)
}
runOn(first) {
replicator ! Update(KeyC, GCounter(), writeTwo)(_ + 30)
expectMsg(UpdateSuccess(KeyC, None))
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(30)
replicator ! Update(KeyY, GCounter(), writeTwo)(_ + 30)
expectMsg(UpdateSuccess(KeyY, None))
replicator ! Update(KeyZ, GCounter(), writeMajority)(_ + 30)
expectMsg(UpdateSuccess(KeyZ, None))
}
enterBarrier("update-c30")
runOn(second) {
replicator ! Get(KeyC, ReadLocal)
val c30 = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c30.value should be(30)
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(30)
// replicate with gossip after WriteLocal
replicator ! Update(KeyC, GCounter(), WriteLocal)(_ + 1)
expectMsg(UpdateSuccess(KeyC, None))
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(31)
replicator ! Delete(KeyY, WriteLocal)
expectMsg(DeleteSuccess(KeyY))
replicator ! Get(KeyZ, readMajority)
expectMsgPF() { case g @ GetSuccess(KeyZ, _) g.get(KeyZ).value } should be(30)
}
enterBarrier("update-c31")
runOn(first) {
// KeyC and deleted KeyY should be replicated via gossip to the other node
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyC, ReadLocal)
val c = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c.value should be(31)
replicator ! Get(KeyY, ReadLocal)
expectMsg(DataDeleted(KeyY))
}
}
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(31)
}
enterBarrier("verified-c31")
// and also for concurrent updates
runOn(first, second) {
replicator ! Get(KeyC, ReadLocal)
val c31 = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c31.value should be(31)
val c32 = c31 + 1
replicator ! Update(KeyC, GCounter(), WriteLocal)(_ + 1)
expectMsg(UpdateSuccess(KeyC, None))
within(5.seconds) {
awaitAssert {
replicator ! Get(KeyC, ReadLocal)
val c = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c.value should be(33)
}
}
}
enterBarrier("after-4")
}
"converge after partition" in {
runOn(first) {
replicator ! Update(KeyD, GCounter(), writeTwo)(_ + 40)
expectMsg(UpdateSuccess(KeyD, None))
testConductor.blackhole(first, second, Direction.Both).await
}
enterBarrier("blackhole-first-second")
runOn(first, second) {
replicator ! Get(KeyD, ReadLocal)
val c40 = expectMsgPF() { case g @ GetSuccess(KeyD, _) g.get(KeyD) }
c40.value should be(40)
replicator ! Update(KeyD, GCounter() + 1, writeTwo)(_ + 1)
expectMsg(UpdateTimeout(KeyD, None))
replicator ! Update(KeyD, GCounter(), writeTwo)(_ + 1)
expectMsg(UpdateTimeout(KeyD, None))
}
runOn(first) {
for (n 1 to 30) {
val KeyDn = GCounterKey("D" + n)
replicator ! Update(KeyDn, GCounter(), WriteLocal)(_ + n)
expectMsg(UpdateSuccess(KeyDn, None))
}
}
enterBarrier("updates-during-partion")
runOn(first) {
testConductor.passThrough(first, second, Direction.Both).await
}
enterBarrier("passThrough-first-second")
runOn(first, second) {
replicator ! Get(KeyD, readTwo)
val c44 = expectMsgPF() { case g @ GetSuccess(KeyD, _) g.get(KeyD) }
c44.value should be(44)
within(10.seconds) {
awaitAssert {
for (n 1 to 30) {
val KeyDn = GCounterKey("D" + n)
replicator ! Get(KeyDn, ReadLocal)
expectMsgPF() { case g @ GetSuccess(KeyDn, _) g.get(KeyDn) }.value should be(n)
}
}
}
}
enterBarrier("after-5")
}
"support majority quorum write and read with 3 nodes with 1 unreachable" in {
join(third, first)
runOn(first, second, third) {
within(10.seconds) {
awaitAssert {
replicator ! GetReplicaCount
expectMsg(ReplicaCount(3))
}
}
}
enterBarrier("3-nodes")
runOn(first, second, third) {
replicator ! Update(KeyE, GCounter(), writeMajority)(_ + 50)
expectMsg(UpdateSuccess(KeyE, None))
}
enterBarrier("write-inital-majority")
runOn(first, second, third) {
replicator ! Get(KeyE, readMajority)
val c150 = expectMsgPF() { case g @ GetSuccess(KeyE, _) g.get(KeyE) }
c150.value should be(150)
}
enterBarrier("read-inital-majority")
runOn(first) {
testConductor.blackhole(first, third, Direction.Both).await
testConductor.blackhole(second, third, Direction.Both).await
}
enterBarrier("blackhole-third")
runOn(second) {
replicator ! Update(KeyE, GCounter(), WriteLocal)(_ + 1)
expectMsg(UpdateSuccess(KeyE, None))
}
enterBarrier("local-update-from-second")
runOn(first) {
// ReadMajority should retrive the previous update from second, before applying the modification
val probe1 = TestProbe()
val probe2 = TestProbe()
replicator.tell(Get(KeyE, readMajority), probe2.ref)
probe2.expectMsgType[GetSuccess[_]]
replicator.tell(Update(KeyE, GCounter(), writeMajority, None) { data
probe1.ref ! data.value
data + 1
}, probe2.ref)
// verify read your own writes, without waiting for the UpdateSuccess reply
// note that the order of the replies are not defined, and therefore we use separate probes
val probe3 = TestProbe()
replicator.tell(Get(KeyE, readMajority), probe3.ref)
probe1.expectMsg(151)
probe2.expectMsg(UpdateSuccess(KeyE, None))
val c152 = probe3.expectMsgPF() { case g @ GetSuccess(KeyE, _) g.get(KeyE) }
c152.value should be(152)
}
enterBarrier("majority-update-from-first")
runOn(second) {
val probe1 = TestProbe()
replicator.tell(Get(KeyE, readMajority), probe1.ref)
probe1.expectMsgType[GetSuccess[_]]
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(153))(_ + 1), probe1.ref)
// verify read your own writes, without waiting for the UpdateSuccess reply
// note that the order of the replies are not defined, and therefore we use separate probes
val probe2 = TestProbe()
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(154))(_ + 1), probe2.ref)
val probe3 = TestProbe()
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(155))(_ + 1), probe3.ref)
val probe5 = TestProbe()
replicator.tell(Get(KeyE, readMajority), probe5.ref)
probe1.expectMsg(UpdateSuccess(KeyE, Some(153)))
probe2.expectMsg(UpdateSuccess(KeyE, Some(154)))
probe3.expectMsg(UpdateSuccess(KeyE, Some(155)))
val c155 = probe5.expectMsgPF() { case g @ GetSuccess(KeyE, _) g.get(KeyE) }
c155.value should be(155)
}
enterBarrier("majority-update-from-second")
runOn(first, second) {
replicator ! Get(KeyE2, readAll, Some(998))
expectMsg(GetFailure(KeyE2, Some(998)))
replicator ! Get(KeyE2, ReadLocal)
expectMsg(NotFound(KeyE2, None))
}
enterBarrier("read-all-fail-update")
runOn(first) {
testConductor.passThrough(first, third, Direction.Both).await
testConductor.passThrough(second, third, Direction.Both).await
}
enterBarrier("passThrough-third")
runOn(third) {
replicator ! Get(KeyE, readMajority)
val c155 = expectMsgPF() { case g @ GetSuccess(KeyE, _) g.get(KeyE) }
c155.value should be(155)
}
enterBarrier("after-6")
}
"converge after many concurrent updates" in within(10.seconds) {
runOn(first, second, third) {
var c = GCounter()
for (i 0 until 100) {
c += 1
replicator ! Update(KeyF, GCounter(), writeTwo)(_ + 1)
}
val results = receiveN(100)
results.map(_.getClass).toSet should be(Set(classOf[UpdateSuccess[_]]))
}
enterBarrier("100-updates-done")
runOn(first, second, third) {
replicator ! Get(KeyF, readTwo)
val c = expectMsgPF() { case g @ GetSuccess(KeyF, _) g.get(KeyF) }
c.value should be(3 * 100)
}
enterBarrier("after-7")
}
"read-repair happens before GetSuccess" in {
runOn(first) {
replicator ! Update(KeyG, ORSet(), writeTwo)(_ + "a" + "b")
expectMsgType[UpdateSuccess[_]]
}
enterBarrier("a-b-added-to-G")
runOn(second) {
replicator ! Get(KeyG, readAll)
expectMsgPF() { case g @ GetSuccess(KeyG, _) g.get(KeyG).elements } should be(Set("a", "b"))
replicator ! Get(KeyG, ReadLocal)
expectMsgPF() { case g @ GetSuccess(KeyG, _) g.get(KeyG).elements } should be(Set("a", "b"))
}
enterBarrier("after-8")
}
}

View file

@ -0,0 +1,20 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.remote.testkit.MultiNodeSpecCallbacks
import org.scalatest.{ BeforeAndAfterAll, WordSpecLike }
import org.scalatest.Matchers
/**
* Hooks up MultiNodeSpec with ScalaTest
*/
trait STMultiNodeSpec extends MultiNodeSpecCallbacks
with WordSpecLike with Matchers with BeforeAndAfterAll {
override def beforeAll() = multiNodeSpecBeforeAll()
override def afterAll() = multiNodeSpecAfterAll()
}

View file

@ -0,0 +1,187 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.LWWMap
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.LWWMapKey
object ReplicatedCacheSpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
object ReplicatedCache {
import akka.cluster.ddata.Replicator._
def props: Props = Props[ReplicatedCache]
private final case class Request(key: String, replyTo: ActorRef)
final case class PutInCache(key: String, value: Any)
final case class GetFromCache(key: String)
final case class Cached(key: String, value: Option[Any])
final case class Evict(key: String)
}
class ReplicatedCache() extends Actor {
import akka.cluster.ddata.Replicator._
import ReplicatedCache._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
def dataKey(entryKey: String): LWWMapKey[Any] =
LWWMapKey("cache-" + math.abs(entryKey.hashCode) % 100)
def receive = {
case PutInCache(key, value)
replicator ! Update(dataKey(key), LWWMap(), WriteLocal)(_ + (key -> value))
case Evict(key)
replicator ! Update(dataKey(key), LWWMap(), WriteLocal)(_ - key)
case GetFromCache(key)
replicator ! Get(dataKey(key), ReadLocal, Some(Request(key, sender())))
case g @ GetSuccess(LWWMapKey(_), Some(Request(key, replyTo)))
g.dataValue match {
case data: LWWMap[_] data.get(key) match {
case Some(value) replyTo ! Cached(key, Some(value))
case None replyTo ! Cached(key, None)
}
}
case NotFound(_, Some(Request(key, replyTo)))
replyTo ! Cached(key, None)
case _: UpdateResponse[_] // ok
}
}
class ReplicatedCacheSpecMultiJvmNode1 extends ReplicatedCacheSpec
class ReplicatedCacheSpecMultiJvmNode2 extends ReplicatedCacheSpec
class ReplicatedCacheSpecMultiJvmNode3 extends ReplicatedCacheSpec
class ReplicatedCacheSpec extends MultiNodeSpec(ReplicatedCacheSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatedCacheSpec._
import ReplicatedCache._
override def initialParticipants = roles.size
val cluster = Cluster(system)
val replicatedCache = system.actorOf(ReplicatedCache.props)
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated cache" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"replicate cached entry" in within(10.seconds) {
runOn(node1) {
replicatedCache ! PutInCache("key1", "A")
}
awaitAssert {
val probe = TestProbe()
replicatedCache.tell(GetFromCache("key1"), probe.ref)
probe.expectMsg(Cached("key1", Some("A")))
}
enterBarrier("after-2")
}
"replicate many cached entries" in within(10.seconds) {
runOn(node1) {
for (i 100 to 200)
replicatedCache ! PutInCache("key" + i, i)
}
awaitAssert {
val probe = TestProbe()
for (i 100 to 200) {
replicatedCache.tell(GetFromCache("key" + i), probe.ref)
probe.expectMsg(Cached("key" + i, Some(i)))
}
}
enterBarrier("after-3")
}
"replicate evicted entry" in within(15.seconds) {
runOn(node1) {
replicatedCache ! PutInCache("key2", "B")
}
awaitAssert {
val probe = TestProbe()
replicatedCache.tell(GetFromCache("key2"), probe.ref)
probe.expectMsg(Cached("key2", Some("B")))
}
enterBarrier("key2-replicated")
runOn(node3) {
replicatedCache ! Evict("key2")
}
awaitAssert {
val probe = TestProbe()
replicatedCache.tell(GetFromCache("key2"), probe.ref)
probe.expectMsg(Cached("key2", None))
}
enterBarrier("after-4")
}
"replicate updated cached entry" in within(10.seconds) {
runOn(node2) {
replicatedCache ! PutInCache("key1", "A2")
replicatedCache ! PutInCache("key1", "A3")
}
awaitAssert {
val probe = TestProbe()
replicatedCache.tell(GetFromCache("key1"), probe.ref)
probe.expectMsg(Cached("key1", Some("A3")))
}
enterBarrier("after-5")
}
}
}

View file

@ -0,0 +1,200 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import java.lang.management.ManagementFactory
import java.lang.management.MemoryMXBean
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.Address
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ClusterEvent.{ InitialStateAsEvents, MemberUp, MemberRemoved }
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.LWWMap
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.LWWMapKey
object ReplicatedMetricsSpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
object ReplicatedMetrics {
import akka.cluster.ddata.Replicator._
def props(measureInterval: FiniteDuration, cleanupInterval: FiniteDuration): Props =
Props(new ReplicatedMetrics(measureInterval, cleanupInterval))
def props: Props = props(1.second, 1.minute)
private case object Tick
private case object Cleanup
case class UsedHeap(percentPerNode: Map[String, Double]) {
override def toString =
percentPerNode.toSeq.sortBy(_._1).map {
case (key, value) key + " --> " + value + " %"
}.mkString("\n")
}
def nodeKey(address: Address): String = address.host.get + ":" + address.port.get
}
class ReplicatedMetrics(measureInterval: FiniteDuration, cleanupInterval: FiniteDuration)
extends Actor with ActorLogging {
import akka.cluster.ddata.Replicator._
import ReplicatedMetrics._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
val node = nodeKey(cluster.selfAddress)
val tickTask = context.system.scheduler.schedule(measureInterval, measureInterval,
self, Tick)(context.dispatcher)
val cleanupTask = context.system.scheduler.schedule(cleanupInterval, cleanupInterval,
self, Cleanup)(context.dispatcher)
val memoryMBean: MemoryMXBean = ManagementFactory.getMemoryMXBean
val UsedHeapKey = LWWMapKey[Long]("usedHeap")
val MaxHeapKey = LWWMapKey[Long]("maxHeap")
replicator ! Subscribe(UsedHeapKey, self)
replicator ! Subscribe(MaxHeapKey, self)
cluster.subscribe(self, InitialStateAsEvents, classOf[MemberUp], classOf[MemberRemoved])
override def postStop(): Unit = {
tickTask.cancel()
cluster.unsubscribe(self)
super.postStop()
}
var maxHeap = Map.empty[String, Long]
var nodesInCluster = Set.empty[String]
def receive = {
case Tick
val heap = memoryMBean.getHeapMemoryUsage
val used = heap.getUsed
val max = heap.getMax
replicator ! Update(UsedHeapKey, LWWMap.empty[Long], WriteLocal)(_ + (node -> used))
replicator ! Update(MaxHeapKey, LWWMap.empty[Long], WriteLocal) { data
data.get(node) match {
case Some(`max`) data // unchanged
case _ data + (node -> max)
}
}
case c @ Changed(MaxHeapKey)
maxHeap = c.get(MaxHeapKey).entries
case c @ Changed(UsedHeapKey)
val usedHeapPercent = UsedHeap(c.get(UsedHeapKey).entries.collect {
case (key, value) if maxHeap.contains(key)
(key -> (value.toDouble / maxHeap(key)) * 100.0)
})
log.debug("Node {} observed:\n{}", node, usedHeapPercent)
context.system.eventStream.publish(usedHeapPercent)
case _: UpdateResponse[_] // ok
case MemberUp(m)
nodesInCluster += nodeKey(m.address)
case MemberRemoved(m, _)
nodesInCluster -= nodeKey(m.address)
case Cleanup
def cleanupRemoved(data: LWWMap[Long]): LWWMap[Long] =
(data.entries.keySet -- nodesInCluster).foldLeft(data) { case (d, key) d - key }
replicator ! Update(UsedHeapKey, LWWMap.empty[Long], WriteLocal)(cleanupRemoved)
replicator ! Update(MaxHeapKey, LWWMap.empty[Long], WriteLocal)(cleanupRemoved)
}
}
class ReplicatedMetricsSpecMultiJvmNode1 extends ReplicatedMetricsSpec
class ReplicatedMetricsSpecMultiJvmNode2 extends ReplicatedMetricsSpec
class ReplicatedMetricsSpecMultiJvmNode3 extends ReplicatedMetricsSpec
class ReplicatedMetricsSpec extends MultiNodeSpec(ReplicatedMetricsSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatedMetricsSpec._
import ReplicatedMetrics._
override def initialParticipants = roles.size
val cluster = Cluster(system)
val replicatedMetrics = system.actorOf(ReplicatedMetrics.props(1.second, 3.seconds))
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated metrics" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"replicate metrics" in within(10.seconds) {
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[UsedHeap])
awaitAssert {
probe.expectMsgType[UsedHeap].percentPerNode.size should be(3)
}
probe.expectMsgType[UsedHeap].percentPerNode.size should be(3)
probe.expectMsgType[UsedHeap].percentPerNode.size should be(3)
enterBarrier("after-2")
}
"cleanup removed node" in within(15.seconds) {
val node3Address = node(node3).address
runOn(node1) {
cluster.leave(node3Address)
}
runOn(node1, node2) {
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[UsedHeap])
awaitAssert {
probe.expectMsgType[UsedHeap].percentPerNode.size should be(2)
}
probe.expectMsgType[UsedHeap].percentPerNode should not contain (
nodeKey(node3Address))
}
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,267 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.actor.Terminated
import akka.cluster.Cluster
import akka.cluster.ClusterEvent
import akka.cluster.ClusterEvent.LeaderChanged
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.GSet
import akka.cluster.ddata.ORSet
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.GSetKey
import akka.cluster.ddata.ORSetKey
import akka.cluster.ddata.Key
object ReplicatedServiceRegistrySpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
class Service extends Actor {
def receive = {
case s: String sender() ! self.path.name + ": " + s
}
}
}
object ReplicatedServiceRegistry {
import akka.cluster.ddata.Replicator._
val props: Props = Props[ReplicatedServiceRegistry]
/**
* Register a `service` with a `name`. Several services
* can be registered with the same `name`.
* It will be removed when it is terminated.
*/
final case class Register(name: String, service: ActorRef)
/**
* Lookup services registered for a `name`. [[Bindings]] will
* be sent to `sender()`.
*/
final case class Lookup(name: String)
/**
* Reply for [[Lookup]]
*/
final case class Bindings(name: String, services: Set[ActorRef])
/**
* Published to `System.eventStream` when services are changed.
*/
final case class BindingChanged(name: String, services: Set[ActorRef])
final case class ServiceKey(serviceName: String) extends Key[ORSet[ActorRef]](serviceName)
private val AllServicesKey = GSetKey[ServiceKey]("service-keys")
}
class ReplicatedServiceRegistry() extends Actor with ActorLogging {
import akka.cluster.ddata.Replicator._
import ReplicatedServiceRegistry._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
var keys = Set.empty[ServiceKey]
var services = Map.empty[String, Set[ActorRef]]
var leader = false
def serviceKey(serviceName: String): ServiceKey =
ServiceKey("service:" + serviceName)
override def preStart(): Unit = {
replicator ! Subscribe(AllServicesKey, self)
cluster.subscribe(self, ClusterEvent.InitialStateAsEvents, classOf[ClusterEvent.LeaderChanged])
}
override def postStop(): Unit = {
cluster.unsubscribe(self)
}
def receive = {
case Register(name, service)
val dKey = serviceKey(name)
// store the service names in a separate GSet to be able to
// get notifications of new names
if (!keys(dKey))
replicator ! Update(AllServicesKey, GSet(), WriteLocal)(_ + dKey)
// add the service
replicator ! Update(dKey, ORSet(), WriteLocal)(_ + service)
case Lookup(key)
sender() ! Bindings(key, services.getOrElse(key, Set.empty))
case c @ Changed(AllServicesKey)
val newKeys = c.get(AllServicesKey).elements
log.debug("Services changed, added: {}, all: {}", (newKeys -- keys), newKeys)
(newKeys -- keys).foreach { dKey
// subscribe to get notifications of when services with this name are added or removed
replicator ! Subscribe(dKey, self)
}
keys = newKeys
case c @ Changed(ServiceKey(serviceName))
val name = serviceName.split(":").tail.mkString
val newServices = c.get(serviceKey(name)).elements
log.debug("Services changed for name [{}]: {}", name, newServices)
services = services.updated(name, newServices)
context.system.eventStream.publish(BindingChanged(name, newServices))
if (leader)
newServices.foreach(context.watch) // watch is idempotent
case LeaderChanged(node)
// Let one node (the leader) be responsible for removal of terminated services
// to avoid redundant work and too many death watch notifications.
// It is not critical to only do it from one node.
val wasLeader = leader
leader = node.exists(_ == cluster.selfAddress)
// when used with many (> 500) services you must increase the system message buffer
// `akka.remote.system-message-buffer-size`
if (!wasLeader && leader)
for (refs services.valuesIterator; ref refs)
context.watch(ref)
else if (wasLeader && !leader)
for (refs services.valuesIterator; ref refs)
context.unwatch(ref)
case Terminated(ref)
val names = services.collect { case (name, refs) if refs.contains(ref) name }
names.foreach { name
log.debug("Service with name [{}] terminated: {}", name, ref)
replicator ! Update(serviceKey(name), ORSet(), WriteLocal)(_ - ref)
}
case _: UpdateResponse[_] // ok
}
}
class ReplicatedServiceRegistrySpecMultiJvmNode1 extends ReplicatedServiceRegistrySpec
class ReplicatedServiceRegistrySpecMultiJvmNode2 extends ReplicatedServiceRegistrySpec
class ReplicatedServiceRegistrySpecMultiJvmNode3 extends ReplicatedServiceRegistrySpec
class ReplicatedServiceRegistrySpec extends MultiNodeSpec(ReplicatedServiceRegistrySpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatedServiceRegistrySpec._
import ReplicatedServiceRegistry._
override def initialParticipants = roles.size
val cluster = Cluster(system)
val registry = system.actorOf(ReplicatedServiceRegistry.props)
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated service registry" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"replicate service entry" in within(10.seconds) {
runOn(node1) {
val a1 = system.actorOf(Props[Service], name = "a1")
registry ! Register("a", a1)
}
awaitAssert {
val probe = TestProbe()
registry.tell(Lookup("a"), probe.ref)
probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1"))
}
enterBarrier("after-2")
}
"replicate updated service entry, and publish to even bus" in {
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[BindingChanged])
runOn(node2) {
val a2 = system.actorOf(Props[Service], name = "a2")
registry ! Register("a", a2)
}
probe.within(10.seconds) {
probe.expectMsgType[BindingChanged].services.map(_.path.name) should be(Set("a1", "a2"))
registry.tell(Lookup("a"), probe.ref)
probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1", "a2"))
}
enterBarrier("after-4")
}
"remove terminated service" in {
val probe = TestProbe()
system.eventStream.subscribe(probe.ref, classOf[BindingChanged])
runOn(node2) {
registry.tell(Lookup("a"), probe.ref)
val a2 = probe.expectMsgType[Bindings].services.find(_.path.name == "a2").get
a2 ! PoisonPill
}
probe.within(10.seconds) {
probe.expectMsgType[BindingChanged].services.map(_.path.name) should be(Set("a1"))
registry.tell(Lookup("a"), probe.ref)
probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1"))
}
enterBarrier("after-5")
}
"replicate many service entries" in within(10.seconds) {
for (i 100 until 200) {
val service = system.actorOf(Props[Service], name = myself.name + "_" + i)
registry ! Register("a" + i, service)
}
awaitAssert {
val probe = TestProbe()
for (i 100 until 200) {
registry.tell(Lookup("a" + i), probe.ref)
probe.expectMsgType[Bindings].services.map(_.path.name) should be(roles.map(_.name + "_" + i).toSet)
}
}
enterBarrier("after-6")
}
}
}

View file

@ -0,0 +1,214 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.LWWMap
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.LWWMapKey
object ReplicatedShoppingCartSpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
object ShoppingCart {
import akka.cluster.ddata.Replicator._
def props(userId: String): Props = Props(new ShoppingCart(userId))
case object GetCart
final case class AddItem(item: LineItem)
final case class RemoveItem(productId: String)
final case class Cart(items: Set[LineItem])
final case class LineItem(productId: String, title: String, quantity: Int)
//#read-write-majority
private val timeout = 3.seconds
private val readMajority = ReadMajority(timeout)
private val writeMajority = WriteMajority(timeout)
//#read-write-majority
}
class ShoppingCart(userId: String) extends Actor {
import ShoppingCart._
import akka.cluster.ddata.Replicator._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
val DataKey = LWWMapKey[LineItem]("cart-" + userId)
def receive = receiveGetCart
.orElse[Any, Unit](receiveAddItem)
.orElse[Any, Unit](receiveRemoveItem)
.orElse[Any, Unit](receiveOther)
//#get-cart
def receiveGetCart: Receive = {
case GetCart
replicator ! Get(DataKey, readMajority, Some(sender()))
case g @ GetSuccess(DataKey, Some(replyTo: ActorRef))
val data = g.get(DataKey)
val cart = Cart(data.entries.values.toSet)
replyTo ! cart
case NotFound(DataKey, Some(replyTo: ActorRef))
replyTo ! Cart(Set.empty)
case GetFailure(DataKey, Some(replyTo: ActorRef))
// ReadMajority failure, try again with local read
replicator ! Get(DataKey, ReadLocal, Some(replyTo))
}
//#get-cart
//#add-item
def receiveAddItem: Receive = {
case cmd @ AddItem(item)
val update = Update(DataKey, LWWMap.empty[LineItem], writeMajority, Some(cmd)) {
cart updateCart(cart, item)
}
replicator ! update
case GetFailure(DataKey, Some(AddItem(item)))
// ReadMajority of Update failed, fall back to best effort local value
replicator ! Update(DataKey, LWWMap.empty[LineItem], writeMajority, None) {
cart updateCart(cart, item)
}
}
//#add-item
//#remove-item
def receiveRemoveItem: Receive = {
case cmd @ RemoveItem(productId)
// Try to fetch latest from a majority of nodes first, since ORMap
// remove must have seen the item to be able to remove it.
replicator ! Get(DataKey, readMajority, Some(cmd))
case GetSuccess(DataKey, Some(RemoveItem(productId)))
replicator ! Update(DataKey, LWWMap(), writeMajority, None) {
_ - productId
}
case GetFailure(DataKey, Some(RemoveItem(productId)))
// ReadMajority failed, fall back to best effort local value
replicator ! Update(DataKey, LWWMap(), writeMajority, None) {
_ - productId
}
case NotFound(DataKey, Some(RemoveItem(productId)))
// nothing to remove
}
//#remove-item
def receiveOther: Receive = {
case _: UpdateSuccess[_] | _: UpdateTimeout[_]
// UpdateTimeout, will eventually be replicated
case e: UpdateFailure[_] throw new IllegalStateException("Unexpected failure: " + e)
}
def updateCart(data: LWWMap[LineItem], item: LineItem): LWWMap[LineItem] =
data.get(item.productId) match {
case Some(LineItem(_, _, existingQuantity))
data + (item.productId -> item.copy(quantity = existingQuantity + item.quantity))
case None data + (item.productId -> item)
}
}
class ReplicatedShoppingCartSpecMultiJvmNode1 extends ReplicatedShoppingCartSpec
class ReplicatedShoppingCartSpecMultiJvmNode2 extends ReplicatedShoppingCartSpec
class ReplicatedShoppingCartSpecMultiJvmNode3 extends ReplicatedShoppingCartSpec
class ReplicatedShoppingCartSpec extends MultiNodeSpec(ReplicatedShoppingCartSpec) with STMultiNodeSpec with ImplicitSender {
import ReplicatedShoppingCartSpec._
import ShoppingCart._
override def initialParticipants = roles.size
val cluster = Cluster(system)
val shoppingCart = system.actorOf(ShoppingCart.props("user-1"))
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated shopping cart" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"handle updates directly after start" in within(15.seconds) {
runOn(node2) {
shoppingCart ! ShoppingCart.AddItem(LineItem("1", "Apples", quantity = 2))
shoppingCart ! ShoppingCart.AddItem(LineItem("2", "Oranges", quantity = 3))
}
enterBarrier("updates-done")
awaitAssert {
shoppingCart ! ShoppingCart.GetCart
val cart = expectMsgType[Cart]
cart.items should be(Set(LineItem("1", "Apples", quantity = 2), LineItem("2", "Oranges", quantity = 3)))
}
enterBarrier("after-2")
}
"handle updates from different nodes" in within(5.seconds) {
runOn(node2) {
shoppingCart ! ShoppingCart.AddItem(LineItem("1", "Apples", quantity = 5))
shoppingCart ! ShoppingCart.RemoveItem("2")
}
runOn(node3) {
shoppingCart ! ShoppingCart.AddItem(LineItem("3", "Bananas", quantity = 4))
}
enterBarrier("updates-done")
awaitAssert {
shoppingCart ! ShoppingCart.GetCart
val cart = expectMsgType[Cart]
cart.items should be(Set(LineItem("1", "Apples", quantity = 7), LineItem("3", "Bananas", quantity = 4)))
}
enterBarrier("after-3")
}
}
}

View file

@ -0,0 +1,184 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package sample.datareplication
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.Flag
import akka.cluster.ddata.PNCounterMap
import akka.cluster.ddata.Replicator.GetReplicaCount
import akka.cluster.ddata.Replicator.ReplicaCount
import akka.cluster.ddata.STMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.FlagKey
import akka.cluster.ddata.PNCounterMapKey
object VotingContestSpec extends MultiNodeConfig {
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
"""))
}
object VotingService {
case object Open
case object OpenAck
case object Close
case object CloseAck
final case class Vote(participant: String)
case object GetVotes
final case class Votes(result: Map[String, BigInt], open: Boolean)
private final case class GetVotesReq(replyTo: ActorRef)
}
class VotingService extends Actor {
import akka.cluster.ddata.Replicator._
import VotingService._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
val OpenedKey = FlagKey("contestOpened")
val ClosedKey = FlagKey("contestClosed")
val CountersKey = PNCounterMapKey("contestCounters")
replicator ! Subscribe(OpenedKey, self)
def receive = {
case Open
replicator ! Update(OpenedKey, Flag(), WriteAll(5.seconds))(_.switchOn)
becomeOpen()
case c @ Changed(OpenedKey) if c.get(OpenedKey).enabled
becomeOpen()
case GetVotes
sender() ! Votes(Map.empty, open = false)
}
def becomeOpen(): Unit = {
replicator ! Unsubscribe(OpenedKey, self)
replicator ! Subscribe(ClosedKey, self)
context.become(open orElse getVotes(open = true))
}
def open: Receive = {
case v @ Vote(participant)
val update = Update(CountersKey, PNCounterMap(), WriteLocal, request = Some(v)) {
_.increment(participant, 1)
}
replicator ! update
case _: UpdateSuccess[_]
case Close
replicator ! Update(ClosedKey, Flag(), WriteAll(5.seconds))(_.switchOn)
context.become(getVotes(open = false))
case c @ Changed(ClosedKey) if c.get(ClosedKey).enabled
context.become(getVotes(open = false))
}
def getVotes(open: Boolean): Receive = {
case GetVotes
replicator ! Get(CountersKey, ReadAll(3.seconds), Some(GetVotesReq(sender())))
case g @ GetSuccess(CountersKey, Some(GetVotesReq(replyTo)))
val data = g.get(CountersKey)
replyTo ! Votes(data.entries, open)
case NotFound(CountersKey, Some(GetVotesReq(replyTo)))
replyTo ! Votes(Map.empty, open)
case _: GetFailure[_]
case _: UpdateSuccess[_]
}
}
class VotingContestSpecMultiJvmNode1 extends VotingContestSpec
class VotingContestSpecMultiJvmNode2 extends VotingContestSpec
class VotingContestSpecMultiJvmNode3 extends VotingContestSpec
class VotingContestSpec extends MultiNodeSpec(VotingContestSpec) with STMultiNodeSpec with ImplicitSender {
import VotingContestSpec._
override def initialParticipants = roles.size
val cluster = Cluster(system)
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster join node(to).address
}
enterBarrier(from.name + "-joined")
}
"Demo of a replicated voting" must {
"join cluster" in within(10.seconds) {
join(node1, node1)
join(node2, node1)
join(node3, node1)
awaitAssert {
DistributedData(system).replicator ! GetReplicaCount
expectMsg(ReplicaCount(roles.size))
}
enterBarrier("after-1")
}
"count votes correctly" in within(15.seconds) {
import VotingService._
val votingService = system.actorOf(Props[VotingService], "votingService")
val N = 1000
runOn(node1) {
votingService ! Open
for (n 1 to N) {
votingService ! Vote("#" + ((n % 20) + 1))
}
}
runOn(node2, node3) {
// wait for it to open
val p = TestProbe()
awaitAssert {
votingService.tell(GetVotes, p.ref)
p.expectMsgPF(3.seconds) { case Votes(_, true) true }
}
for (n 1 to N) {
votingService ! Vote("#" + ((n % 20) + 1))
}
}
enterBarrier("voting-done")
runOn(node3) {
votingService ! Close
}
val expected = (1 to 20).map(n "#" + n -> BigInt(3L * N / 20)).toMap
awaitAssert {
votingService ! GetVotes
expectMsg(3.seconds, Votes(expected, false))
}
enterBarrier("after-2")
}
}
}

View file

@ -0,0 +1,29 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata;
import akka.cluster.UniqueAddress;
public class JavaImplOfReplicatedData extends AbstractReplicatedData implements RemovedNodePruning {
@Override
public JavaImplOfReplicatedData merge(ReplicatedData other) {
return this;
}
@Override
public boolean needPruningFrom(UniqueAddress removedNode) {
return false;
}
@Override
public JavaImplOfReplicatedData prune(UniqueAddress removedNode, UniqueAddress collapseInto) {
return this;
}
@Override
public JavaImplOfReplicatedData pruningCleanup(UniqueAddress removedNode) {
return this;
}
}

View file

@ -0,0 +1,2 @@
akka.actor.serialize-messages = on
akka.actor.serialize-creators = on

View file

@ -0,0 +1,45 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class FlagSpec extends WordSpec with Matchers {
"A Flag" must {
"be able to switch on once" in {
val f1 = Flag()
val f2 = f1.switchOn
val f3 = f2.switchOn
f1.enabled should be(false)
f2.enabled should be(true)
f3.enabled should be(true)
}
"merge by picking true" in {
val f1 = Flag()
val f2 = f1.switchOn
val m1 = f1 merge f2
m1.enabled should be(true)
val m2 = f2 merge f1
m2.enabled should be(true)
}
"have unapply extractor" in {
val f1 = Flag.empty.switchOn
val Flag(value1) = f1
val value2: Boolean = value1
Changed(FlagKey("key"))(f1) match {
case c @ Changed(FlagKey("key"))
val Flag(value3) = c.dataValue
val value4: Boolean = value3
value4 should be(true)
}
}
}
}

View file

@ -0,0 +1,171 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class GCounterSpec extends WordSpec with Matchers {
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
val node3 = UniqueAddress(node1.address.copy(port = Some(2553)), 3)
"A GCounter" must {
"be able to increment each node's record by one" in {
val c1 = GCounter()
val c2 = c1 increment node1
val c3 = c2 increment node1
val c4 = c3 increment node2
val c5 = c4 increment node2
val c6 = c5 increment node2
c6.state(node1) should be(2)
c6.state(node2) should be(3)
}
"be able to increment each node's record by arbitrary delta" in {
val c1 = GCounter()
val c2 = c1 increment (node1, 3)
val c3 = c2 increment (node1, 4)
val c4 = c3 increment (node2, 2)
val c5 = c4 increment (node2, 7)
val c6 = c5 increment node2
c6.state(node1) should be(7)
c6.state(node2) should be(10)
}
"be able to summarize the history to the correct aggregated value" in {
val c1 = GCounter()
val c2 = c1 increment (node1, 3)
val c3 = c2 increment (node1, 4)
val c4 = c3 increment (node2, 2)
val c5 = c4 increment (node2, 7)
val c6 = c5 increment node2
c6.state(node1) should be(7)
c6.state(node2) should be(10)
c6.value should be(17)
}
"be able to have its history correctly merged with another GCounter 1" in {
// counter 1
val c11 = GCounter()
val c12 = c11 increment (node1, 3)
val c13 = c12 increment (node1, 4)
val c14 = c13 increment (node2, 2)
val c15 = c14 increment (node2, 7)
val c16 = c15 increment node2
c16.state(node1) should be(7)
c16.state(node2) should be(10)
c16.value should be(17)
// counter 1
val c21 = GCounter()
val c22 = c21 increment (node1, 2)
val c23 = c22 increment (node1, 2)
val c24 = c23 increment (node2, 3)
val c25 = c24 increment (node2, 2)
val c26 = c25 increment node2
c26.state(node1) should be(4)
c26.state(node2) should be(6)
c26.value should be(10)
// merge both ways
val merged1 = c16 merge c26
merged1.state(node1) should be(7)
merged1.state(node2) should be(10)
merged1.value should be(17)
val merged2 = c26 merge c16
merged2.state(node1) should be(7)
merged2.state(node2) should be(10)
merged2.value should be(17)
}
"be able to have its history correctly merged with another GCounter 2" in {
// counter 1
val c11 = GCounter()
val c12 = c11 increment (node1, 2)
val c13 = c12 increment (node1, 2)
val c14 = c13 increment (node2, 2)
val c15 = c14 increment (node2, 7)
val c16 = c15 increment node2
c16.state(node1) should be(4)
c16.state(node2) should be(10)
c16.value should be(14)
// counter 1
val c21 = GCounter()
val c22 = c21 increment (node1, 3)
val c23 = c22 increment (node1, 4)
val c24 = c23 increment (node2, 3)
val c25 = c24 increment (node2, 2)
val c26 = c25 increment node2
c26.state(node1) should be(7)
c26.state(node2) should be(6)
c26.value should be(13)
// merge both ways
val merged1 = c16 merge c26
merged1.state(node1) should be(7)
merged1.state(node2) should be(10)
merged1.value should be(17)
val merged2 = c26 merge c16
merged2.state(node1) should be(7)
merged2.state(node2) should be(10)
merged2.value should be(17)
}
"have support for pruning" in {
val c1 = GCounter()
val c2 = c1 increment node1
val c3 = c2 increment node2
c2.needPruningFrom(node1) should be(true)
c2.needPruningFrom(node2) should be(false)
c3.needPruningFrom(node1) should be(true)
c3.needPruningFrom(node2) should be(true)
c3.value should be(2)
val c4 = c3.prune(node1, node2)
c4.needPruningFrom(node2) should be(true)
c4.needPruningFrom(node1) should be(false)
c4.value should be(2)
val c5 = (c4 increment node1).pruningCleanup(node1)
c5.needPruningFrom(node1) should be(false)
c4.value should be(2)
}
"have unapply extractor" in {
val c1 = GCounter.empty.increment(node1).increment(node2)
val GCounter(value1) = c1
val value2: BigInt = value1
Changed(GCounterKey("key"))(c1) match {
case c @ Changed(GCounterKey("key"))
val GCounter(value3) = c.dataValue
val value4: BigInt = value3
value4 should be(2L)
}
}
}
}

View file

@ -0,0 +1,119 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class GSetSpec extends WordSpec with Matchers {
val user1 = """{"username":"john","password":"coltrane"}"""
val user2 = """{"username":"sonny","password":"rollins"}"""
val user3 = """{"username":"charlie","password":"parker"}"""
val user4 = """{"username":"charles","password":"mingus"}"""
"A GSet" must {
"be able to add user" in {
val c1 = GSet.empty[String]
val c2 = c1 + user1
val c3 = c2 + user2
val c4 = c3 + user4
val c5 = c4 + user3
c5.elements should contain(user1)
c5.elements should contain(user2)
c5.elements should contain(user3)
c5.elements should contain(user4)
}
"be able to have its user set correctly merged with another GSet with unique user sets" in {
// set 1
val c11 = GSet.empty[String]
val c12 = c11 + user1
val c13 = c12 + user2
c13.elements should contain(user1)
c13.elements should contain(user2)
// set 2
val c21 = GSet.empty[String]
val c22 = c21 + user3
val c23 = c22 + user4
c23.elements should contain(user3)
c23.elements should contain(user4)
// merge both ways
val merged1 = c13 merge c23
merged1.elements should contain(user1)
merged1.elements should contain(user2)
merged1.elements should contain(user3)
merged1.elements should contain(user4)
val merged2 = c23 merge c13
merged2.elements should contain(user1)
merged2.elements should contain(user2)
merged2.elements should contain(user3)
merged2.elements should contain(user4)
}
"be able to have its user set correctly merged with another GSet with overlapping user sets" in {
// set 1
val c10 = GSet.empty[String]
val c11 = c10 + user1
val c12 = c11 + user2
val c13 = c12 + user3
c13.elements should contain(user1)
c13.elements should contain(user2)
c13.elements should contain(user3)
// set 2
val c20 = GSet.empty[String]
val c21 = c20 + user2
val c22 = c21 + user3
val c23 = c22 + user4
c23.elements should contain(user2)
c23.elements should contain(user3)
c23.elements should contain(user4)
// merge both ways
val merged1 = c13 merge c23
merged1.elements should contain(user1)
merged1.elements should contain(user2)
merged1.elements should contain(user3)
merged1.elements should contain(user4)
val merged2 = c23 merge c13
merged2.elements should contain(user1)
merged2.elements should contain(user2)
merged2.elements should contain(user3)
merged2.elements should contain(user4)
}
"have unapply extractor" in {
val s1 = GSet.empty + "a" + "b"
val s2: GSet[String] = s1
val GSet(elements1) = s1
val elements2: Set[String] = elements1
Changed(GSetKey[String]("key"))(s1) match {
case c @ Changed(GSetKey("key"))
val GSet(elements3) = c.dataValue
val elements4: Set[String] = elements3
elements4 should be(Set("a", "b"))
}
}
}
}

View file

@ -0,0 +1,63 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class LWWMapSpec extends WordSpec with Matchers {
import LWWRegister.defaultClock
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
"A LWWMap" must {
"be able to set entries" in {
val m = LWWMap.empty[Int].put(node1, "a", 1, defaultClock[Int]).put(node2, "b", 2, defaultClock[Int])
m.entries should be(Map("a" -> 1, "b" -> 2))
}
"be able to have its entries correctly merged with another LWWMap with other entries" in {
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
// merge both ways
val expected = Map("a" -> 1, "b" -> 2, "c" -> 3)
(m1 merge m2).entries should be(expected)
(m2 merge m1).entries should be(expected)
}
"be able to remove entry" in {
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
val merged1 = m1 merge m2
val m3 = merged1.remove(node1, "b")
(merged1 merge m3).entries should be(Map("a" -> 1, "c" -> 3))
// but if there is a conflicting update the entry is not removed
val m4 = merged1.put(node2, "b", 22, defaultClock[Int])
(m3 merge m4).entries should be(Map("a" -> 1, "b" -> 22, "c" -> 3))
}
"have unapply extractor" in {
val m1 = LWWMap.empty.put(node1, "a", 1L, defaultClock[Long])
val LWWMap(entries1) = m1
val entries2: Map[String, Long] = entries1
Changed(LWWMapKey[Long]("key"))(m1) match {
case c @ Changed(LWWMapKey("key"))
val LWWMap(entries3) = c.dataValue
val entries4: Map[String, Long] = entries3
entries4 should be(Map("a" -> 1L))
}
}
}
}

View file

@ -0,0 +1,92 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class LWWRegisterSpec extends WordSpec with Matchers {
import LWWRegister.defaultClock
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
"A LWWRegister" must {
"use latest of successive assignments" in {
val r = (1 to 100).foldLeft(LWWRegister(node1, 0, defaultClock[Int])) {
case (r, n)
r.value should be(n - 1)
r.withValue(node1, n, defaultClock[Int])
}
r.value should be(100)
}
"merge by picking max timestamp" in {
val clock = new LWWRegister.Clock[String] {
val i = Iterator.from(100)
override def apply(current: Long, value: String): Long = i.next()
}
val r1 = LWWRegister(node1, "A", clock)
r1.timestamp should be(100)
val r2 = r1.withValue(node2, "B", clock)
r2.timestamp should be(101)
val m1 = r1 merge r2
m1.value should be("B")
m1.timestamp should be(101)
val m2 = r2 merge r1
m2.value should be("B")
m2.timestamp should be(101)
}
"merge by picking least address when same timestamp" in {
val clock = new LWWRegister.Clock[String] {
override def apply(current: Long, value: String): Long = 100
}
val r1 = LWWRegister(node1, "A", clock)
val r2 = LWWRegister(node2, "B", clock)
val m1 = r1 merge r2
m1.value should be("A")
val m2 = r2 merge r1
m2.value should be("A")
}
"use monotonically increasing defaultClock" in {
(1 to 100).foldLeft(LWWRegister(node1, 0, defaultClock)) {
case (r, n)
r.value should be(n - 1)
val r2 = r.withValue(node1, n, defaultClock[Int])
r2.timestamp should be > r.timestamp
r2
}
}
"have unapply extractor" in {
val r1 = LWWRegister(node1, "a", defaultClock)
val LWWRegister(value1) = r1
val value2: String = value1
Changed(LWWRegisterKey[String]("key"))(r1) match {
case c @ Changed(LWWRegisterKey("key"))
val LWWRegister(value3) = c.dataValue
val value4: String = value3
value4 should be("a")
}
}
"can be used as first-write-wins-register" in {
import LWWRegister.reverseClock
val r = (1 to 100).foldLeft(LWWRegister(node1, 0, reverseClock[Int])) {
case (r, n)
r.value should be(0)
val newRegister = r.merge(r.withValue(node1, n, reverseClock[Int]))
newRegister should be(r)
newRegister
}
r.value should be(0)
}
}
}

View file

@ -0,0 +1,81 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorSystem
import akka.actor.Props
import akka.actor.Stash
import akka.cluster.Cluster
import akka.testkit.ImplicitSender
import akka.testkit.TestKit
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfterAll
import org.scalatest.Matchers
import org.scalatest.WordSpecLike
object LocalConcurrencySpec {
final case class Add(s: String)
object Updater {
val key = ORSetKey[String]("key")
}
class Updater extends Actor with Stash {
implicit val cluster = Cluster(context.system)
val replicator = DistributedData(context.system).replicator
def receive = {
case s: String
val update = Replicator.Update(Updater.key, ORSet.empty[String], Replicator.WriteLocal)(_ + s)
replicator ! update
}
}
}
class LocalConcurrencySpec(_system: ActorSystem) extends TestKit(_system)
with WordSpecLike with Matchers with BeforeAndAfterAll with ImplicitSender {
import LocalConcurrencySpec._
def this() {
this(ActorSystem("LocalConcurrencySpec",
ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.netty.tcp.port=0
""")))
}
override def afterAll(): Unit = {
shutdown(system)
}
val replicator = DistributedData(system).replicator
"Updates from same node" must {
"be possible to do from two actors" in {
val updater1 = system.actorOf(Props[Updater], "updater1")
val updater2 = system.actorOf(Props[Updater], "updater2")
val numMessages = 100
for (n 1 to numMessages) {
updater1 ! s"a$n"
updater2 ! s"b$n"
}
val expected = ((1 to numMessages).map("a" + _) ++ (1 to numMessages).map("b" + _)).toSet
awaitAssert {
replicator ! Replicator.Get(Updater.key, Replicator.ReadLocal)
val ORSet(elements) = expectMsgType[Replicator.GetSuccess[_]].get(Updater.key)
elements should be(expected)
}
}
}
}

View file

@ -0,0 +1,205 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class ORMapSpec extends WordSpec with Matchers {
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
"A ORMap" must {
"be able to add entries" in {
val m = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B")
val GSet(a) = m.entries("a")
a should be(Set("A"))
val GSet(b) = m.entries("b")
b should be(Set("B"))
val m2 = m.put(node1, "a", GSet() + "C")
val GSet(a2) = m2.entries("a")
a2 should be(Set("C"))
}
"be able to remove entry" in {
val m = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B").remove(node1, "a")
m.entries.keySet should not contain ("a")
m.entries.keySet should contain("b")
}
"be able to add removed" in {
val m = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B").remove(node1, "a")
m.entries.keySet should not contain ("a")
m.entries.keySet should contain("b")
val m2 = m.put(node1, "a", GSet() + "C")
m2.entries.keySet should contain("a")
m2.entries.keySet should contain("b")
}
"be able to have its entries correctly merged with another ORMap with other entries" in {
val m1 = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B")
val m2 = ORMap().put(node2, "c", GSet() + "C")
// merge both ways
val merged1 = m1 merge m2
merged1.entries.keySet should contain("a")
merged1.entries.keySet should contain("b")
merged1.entries.keySet should contain("c")
val merged2 = m2 merge m1
merged2.entries.keySet should contain("a")
merged2.entries.keySet should contain("b")
merged2.entries.keySet should contain("c")
}
"be able to have its entries correctly merged with another ORMap with overlapping entries" in {
val m1 = ORMap().put(node1, "a", GSet() + "A1").put(node1, "b", GSet() + "B1").
remove(node1, "a").put(node1, "d", GSet() + "D1")
val m2 = ORMap().put(node2, "c", GSet() + "C2").put(node2, "a", GSet() + "A2").
put(node2, "b", GSet() + "B2").remove(node2, "b").put(node2, "d", GSet() + "D2")
// merge both ways
val merged1 = m1 merge m2
merged1.entries.keySet should contain("a")
val GSet(a1) = merged1.entries("a")
a1 should be(Set("A2"))
merged1.entries.keySet should contain("b")
val GSet(b1) = merged1.entries("b")
b1 should be(Set("B1"))
merged1.entries.keySet should contain("c")
merged1.entries.keySet should contain("d")
val GSet(d1) = merged1.entries("d")
d1 should be(Set("D1", "D2"))
val merged2 = m2 merge m1
merged2.entries.keySet should contain("a")
val GSet(a2) = merged1.entries("a")
a2 should be(Set("A2"))
merged2.entries.keySet should contain("b")
val GSet(b2) = merged2.entries("b")
b2 should be(Set("B1"))
merged2.entries.keySet should contain("c")
merged2.entries.keySet should contain("d")
val GSet(d2) = merged2.entries("d")
d2 should be(Set("D1", "D2"))
}
"illustrate the danger of using remove+put to replace an entry" in {
val m1 = ORMap.empty.put(node1, "a", GSet.empty + "A").put(node1, "b", GSet.empty + "B")
val m2 = ORMap.empty.put(node2, "c", GSet.empty + "C")
val merged1 = m1 merge m2
val m3 = merged1.remove(node1, "b").put(node1, "b", GSet.empty + "B2")
// same thing if only put is used
// val m3 = merged1.put(node1, "b", GSet() + "B2")
val merged2 = merged1 merge m3
merged2.entries("a").elements should be(Set("A"))
// note that B is included, because GSet("B") is merged with GSet("B2")
merged2.entries("b").elements should be(Set("B", "B2"))
merged2.entries("c").elements should be(Set("C"))
}
"not allow put for ORSet elements type" in {
val m = ORMap().put(node1, "a", ORSet().add(node1, "A"))
intercept[IllegalArgumentException] {
m.put(node1, "a", ORSet().add(node1, "B"))
}
}
"be able to update entry" in {
val m1 = ORMap.empty[ORSet[String]].put(node1, "a", ORSet.empty.add(node1, "A"))
.put(node1, "b", ORSet.empty.add(node1, "B01").add(node1, "B02").add(node1, "B03"))
val m2 = ORMap.empty[ORSet[String]].put(node2, "c", ORSet.empty.add(node2, "C"))
val merged1: ORMap[ORSet[String]] = m1 merge m2
val m3 = merged1.updated(node1, "b", ORSet.empty[String])(_.clear(node1).add(node1, "B2"))
val merged2 = merged1 merge m3
merged2.entries("a").elements should be(Set("A"))
merged2.entries("b").elements should be(Set("B2"))
merged2.entries("c").elements should be(Set("C"))
val m4 = merged1.updated(node2, "b", ORSet.empty[String])(_.add(node2, "B3"))
val merged3 = m3 merge m4
merged3.entries("a").elements should be(Set("A"))
merged3.entries("b").elements should be(Set("B2", "B3"))
merged3.entries("c").elements should be(Set("C"))
}
"be able to update ORSet entry with remove+put" in {
val m1 = ORMap.empty[ORSet[String]].put(node1, "a", ORSet.empty.add(node1, "A01"))
.updated(node1, "a", ORSet.empty[String])(_.add(node1, "A02"))
.updated(node1, "a", ORSet.empty[String])(_.add(node1, "A03"))
.put(node1, "b", ORSet.empty.add(node1, "B01").add(node1, "B02").add(node1, "B03"))
val m2 = ORMap.empty[ORSet[String]].put(node2, "c", ORSet.empty.add(node2, "C"))
val merged1 = m1 merge m2
// note that remove + put work because the new VersionVector version is incremented
// from a global counter
val m3 = merged1.remove(node1, "b").put(node1, "b", ORSet.empty.add(node1, "B2"))
val merged2 = merged1 merge m3
merged2.entries("a").elements should be(Set("A01", "A02", "A03"))
merged2.entries("b").elements should be(Set("B2"))
merged2.entries("c").elements should be(Set("C"))
val m4 = merged1.updated(node2, "b", ORSet.empty[String])(_.add(node2, "B3"))
val merged3 = m3 merge m4
merged3.entries("a").elements should be(Set("A01", "A02", "A03"))
merged3.entries("b").elements should be(Set("B2", "B3"))
merged3.entries("c").elements should be(Set("C"))
}
"be able to update ORSet entry with remove -> merge -> put" in {
val m1 = ORMap.empty.put(node1, "a", ORSet.empty.add(node1, "A"))
.put(node1, "b", ORSet.empty.add(node1, "B01").add(node1, "B02").add(node1, "B03"))
val m2 = ORMap.empty.put(node2, "c", ORSet.empty.add(node2, "C"))
val merged1 = m1 merge m2
val m3 = merged1.remove(node1, "b")
val merged2 = merged1 merge m3
merged2.entries("a").elements should be(Set("A"))
merged2.contains("b") should be(false)
merged2.entries("c").elements should be(Set("C"))
val m4 = merged2.put(node1, "b", ORSet.empty.add(node1, "B2"))
val m5 = merged2.updated(node2, "c", ORSet.empty[String])(_.add(node2, "C2"))
.put(node2, "b", ORSet.empty.add(node2, "B3"))
val merged3 = m5 merge m4
merged3.entries("a").elements should be(Set("A"))
merged3.entries("b").elements should be(Set("B2", "B3"))
merged3.entries("c").elements should be(Set("C", "C2"))
}
"have unapply extractor" in {
val m1 = ORMap.empty.put(node1, "a", Flag(true)).put(node2, "b", Flag(false))
val m2: ORMap[Flag] = m1
val ORMap(entries1) = m1
val entries2: Map[String, Flag] = entries1
Changed(ORMapKey[Flag]("key"))(m1) match {
case c @ Changed(ORMapKey("key"))
val ORMap(entries3) = c.dataValue
val entries4: Map[String, ReplicatedData] = entries3
entries4 should be(Map("a" -> Flag(true), "b" -> Flag(false)))
}
}
}
}

View file

@ -0,0 +1,355 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import scala.collection.immutable.TreeMap
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class ORSetSpec extends WordSpec with Matchers {
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
val nodeA = UniqueAddress(Address("akka.tcp", "Sys", "a", 2552), 1)
val nodeB = UniqueAddress(nodeA.address.copy(host = Some("b")), 2)
val nodeC = UniqueAddress(nodeA.address.copy(host = Some("c")), 3)
val nodeD = UniqueAddress(nodeA.address.copy(host = Some("d")), 4)
val nodeE = UniqueAddress(nodeA.address.copy(host = Some("e")), 5)
val nodeF = UniqueAddress(nodeA.address.copy(host = Some("f")), 6)
val nodeG = UniqueAddress(nodeA.address.copy(host = Some("g")), 7)
val nodeH = UniqueAddress(nodeA.address.copy(host = Some("h")), 8)
val user1 = """{"username":"john","password":"coltrane"}"""
val user2 = """{"username":"sonny","password":"rollins"}"""
val user3 = """{"username":"charlie","password":"parker"}"""
val user4 = """{"username":"charles","password":"mingus"}"""
"A ORSet" must {
"be able to add user" in {
val c1 = ORSet()
val c2 = c1.add(node1, user1)
val c3 = c2.add(node1, user2)
val c4 = c3.add(node1, user4)
val c5 = c4.add(node1, user3)
c5.elements should contain(user1)
c5.elements should contain(user2)
c5.elements should contain(user3)
c5.elements should contain(user4)
}
"be able to remove added user" in {
val c1 = ORSet()
val c2 = c1.add(node1, user1)
val c3 = c2.add(node1, user2)
val c4 = c3.remove(node1, user2)
val c5 = c4.remove(node1, user1)
c5.elements should not contain (user1)
c5.elements should not contain (user2)
}
"be able to add removed" in {
val c1 = ORSet()
val c2 = c1.remove(node1, user1)
val c3 = c2.add(node1, user1)
c3.elements should contain(user1)
val c4 = c3.remove(node1, user1)
c4.elements should not contain (user1)
val c5 = c4.add(node1, user1)
c5.elements should contain(user1)
}
"be able to remove and add several times" in {
val c1 = ORSet()
val c2 = c1.add(node1, user1)
val c3 = c2.add(node1, user2)
val c4 = c3.remove(node1, user1)
c4.elements should not contain (user1)
c4.elements should contain(user2)
val c5 = c4.add(node1, user1)
val c6 = c5.add(node1, user2)
c6.elements should contain(user1)
c6.elements should contain(user2)
val c7 = c6.remove(node1, user1)
val c8 = c7.add(node1, user2)
val c9 = c8.remove(node1, user1)
c9.elements should not contain (user1)
c9.elements should contain(user2)
}
"be able to have its user set correctly merged with another ORSet with unique user sets" in {
// set 1
val c1 = ORSet().add(node1, user1).add(node1, user2)
c1.elements should contain(user1)
c1.elements should contain(user2)
// set 2
val c2 = ORSet().add(node2, user3).add(node2, user4).remove(node2, user3)
c2.elements should not contain (user3)
c2.elements should contain(user4)
// merge both ways
val merged1 = c1 merge c2
merged1.elements should contain(user1)
merged1.elements should contain(user2)
merged1.elements should not contain (user3)
merged1.elements should contain(user4)
val merged2 = c2 merge c1
merged2.elements should contain(user1)
merged2.elements should contain(user2)
merged2.elements should not contain (user3)
merged2.elements should contain(user4)
}
"be able to have its user set correctly merged with another ORSet with overlapping user sets" in {
// set 1
val c1 = ORSet().add(node1, user1).add(node1, user2).add(node1, user3).remove(node1, user1).remove(node1, user3)
c1.elements should not contain (user1)
c1.elements should contain(user2)
c1.elements should not contain (user3)
// set 2
val c2 = ORSet().add(node2, user1).add(node2, user2).add(node2, user3).add(node2, user4).remove(node2, user3)
c2.elements should contain(user1)
c2.elements should contain(user2)
c2.elements should not contain (user3)
c2.elements should contain(user4)
// merge both ways
val merged1 = c1 merge c2
merged1.elements should contain(user1)
merged1.elements should contain(user2)
merged1.elements should not contain (user3)
merged1.elements should contain(user4)
val merged2 = c2 merge c1
merged2.elements should contain(user1)
merged2.elements should contain(user2)
merged2.elements should not contain (user3)
merged2.elements should contain(user4)
}
"be able to have its user set correctly merged for concurrent updates" in {
val c1 = ORSet().add(node1, user1).add(node1, user2).add(node1, user3)
c1.elements should contain(user1)
c1.elements should contain(user2)
c1.elements should contain(user3)
val c2 = c1.add(node2, user1).remove(node2, user2).remove(node2, user3)
c2.elements should contain(user1)
c2.elements should not contain (user2)
c2.elements should not contain (user3)
// merge both ways
val merged1 = c1 merge c2
merged1.elements should contain(user1)
merged1.elements should not contain (user2)
merged1.elements should not contain (user3)
val merged2 = c2 merge c1
merged2.elements should contain(user1)
merged2.elements should not contain (user2)
merged2.elements should not contain (user3)
val c3 = c1.add(node1, user4).remove(node1, user3).add(node1, user2)
// merge both ways
val merged3 = c2 merge c3
merged3.elements should contain(user1)
merged3.elements should contain(user2)
merged3.elements should not contain (user3)
merged3.elements should contain(user4)
val merged4 = c3 merge c2
merged4.elements should contain(user1)
merged4.elements should contain(user2)
merged4.elements should not contain (user3)
merged4.elements should contain(user4)
}
"be able to have its user set correctly merged after remove" in {
val c1 = ORSet().add(node1, user1).add(node1, user2)
val c2 = c1.remove(node2, user2)
// merge both ways
val merged1 = c1 merge c2
merged1.elements should contain(user1)
merged1.elements should not contain (user2)
val merged2 = c2 merge c1
merged2.elements should contain(user1)
merged2.elements should not contain (user2)
val c3 = c1.add(node1, user3)
// merge both ways
val merged3 = c3 merge c2
merged3.elements should contain(user1)
merged3.elements should not contain (user2)
merged3.elements should contain(user3)
val merged4 = c2 merge c3
merged4.elements should contain(user1)
merged4.elements should not contain (user2)
merged4.elements should contain(user3)
}
}
"ORSet unit test" must {
"verify subtractDots" in {
val dot = new VersionVector(TreeMap(nodeA -> 3, nodeB -> 2, nodeD -> 14, nodeG -> 22))
val vvector = new VersionVector(TreeMap(nodeA -> 4, nodeB -> 1, nodeC -> 1, nodeD -> 14, nodeE -> 5, nodeF -> 2))
val expected = new VersionVector(TreeMap(nodeB -> 2, nodeG -> 22))
ORSet.subtractDots(dot, vvector) should be(expected)
}
"verify mergeCommonKeys" in {
val commonKeys: Set[String] = Set("K1", "K2")
val thisDot1 = new VersionVector(TreeMap(nodeA -> 3, nodeD -> 7))
val thisDot2 = new VersionVector(TreeMap(nodeB -> 5, nodeC -> 2))
val thisVvector = new VersionVector(TreeMap(nodeA -> 3, nodeB -> 5, nodeC -> 2, nodeD -> 7))
val thisSet = new ORSet(
elementsMap = Map("K1" -> thisDot1, "K2" -> thisDot2),
vvector = thisVvector)
val thatDot1 = new VersionVector(TreeMap(nodeA -> 3))
val thatDot2 = new VersionVector(TreeMap(nodeB -> 6))
val thatVvector = new VersionVector(TreeMap(nodeA -> 3, nodeB -> 6, nodeC -> 1, nodeD -> 8))
val thatSet = new ORSet(
elementsMap = Map("K1" -> thatDot1, "K2" -> thatDot2),
vvector = thatVvector)
val expectedDots = Map(
"K1" -> new VersionVector(TreeMap(nodeA -> 3)),
"K2" -> new VersionVector(TreeMap(nodeB -> 6, nodeC -> 2)))
ORSet.mergeCommonKeys(commonKeys, thisSet, thatSet) should be(expectedDots)
}
"verify mergeDisjointKeys" in {
val keys: Set[Any] = Set("K3", "K4", "K5")
val elements: Map[Any, VersionVector] = Map(
"K3" -> new VersionVector(TreeMap(nodeA -> 4)),
"K4" -> new VersionVector(TreeMap(nodeA -> 3, nodeD -> 8)),
"K5" -> new VersionVector(TreeMap(nodeA -> 2)))
val vvector = new VersionVector(TreeMap(nodeA -> 3, nodeD -> 7))
val acc: Map[Any, VersionVector] = Map("K1" -> new VersionVector(TreeMap(nodeA -> 3)))
val expectedDots = acc ++ Map(
"K3" -> new VersionVector(TreeMap(nodeA -> 4)),
"K4" -> new VersionVector(TreeMap(nodeD -> 8))) // "a" -> 3 removed, optimized to include only those unseen
ORSet.mergeDisjointKeys(keys, elements, vvector, acc) should be(expectedDots)
}
"verify disjoint merge" in {
val a1 = ORSet().add(node1, "bar")
val b1 = ORSet().add(node2, "baz")
val c = a1.merge(b1)
val a2 = a1.remove(node1, "bar")
val d = a2.merge(c)
d.elements should be(Set("baz"))
}
"verify removed after merge" in {
// Add Z at node1 replica
val a = ORSet().add(node1, "Z")
// Replicate it to some node3, i.e. it has dot 'Z'->{node1 -> 1}
val c = a
// Remove Z at node1 replica
val a2 = a.remove(node1, "Z")
// Add Z at node2, a new replica
val b = ORSet().add(node2, "Z")
// Replicate b to node1, so now node1 has a Z, the one with a Dot of
// {node2 -> 1} and version vector of [{node1 -> 1}, {node2 -> 1}]
val a3 = b.merge(a2)
a3.elements should be(Set("Z"))
// Remove the 'Z' at node2 replica
val b2 = b.remove(node2, "Z")
// Both node3 (c) and node1 (a3) have a 'Z', but when they merge, there should be
// no 'Z' as node3 (c)'s has been removed by node1 and node1 (a3)'s has been removed by
// node2
c.elements should be(Set("Z"))
a3.elements should be(Set("Z"))
b2.elements should be(Set())
a3.merge(c).merge(b2).elements should be(Set.empty)
a3.merge(b2).merge(c).elements should be(Set.empty)
c.merge(b2).merge(a3).elements should be(Set.empty)
c.merge(a3).merge(b2).elements should be(Set.empty)
b2.merge(c).merge(a3).elements should be(Set.empty)
b2.merge(a3).merge(c).elements should be(Set.empty)
}
"verify removed after merge 2" in {
val a = ORSet().add(node1, "Z")
val b = ORSet().add(node2, "Z")
// replicate node3
val c = a
val a2 = a.remove(node1, "Z")
// replicate b to node1, now node1 has node2's 'Z'
val a3 = a2.merge(b)
a3.elements should be(Set("Z"))
// Remove node2's 'Z'
val b2 = b.remove(node2, "Z")
// Replicate c to node2, now node2 has node1's old 'Z'
val b3 = b2.merge(c)
b3.elements should be(Set("Z"))
// Merge everytyhing
a3.merge(c).merge(b3).elements should be(Set.empty)
a3.merge(b3).merge(c).elements should be(Set.empty)
c.merge(b3).merge(a3).elements should be(Set.empty)
c.merge(a3).merge(b3).elements should be(Set.empty)
b3.merge(c).merge(a3).elements should be(Set.empty)
b3.merge(a3).merge(c).elements should be(Set.empty)
}
"have unapply extractor" in {
val s1 = ORSet.empty.add(node1, "a").add(node2, "b")
val s2: ORSet[String] = s1
val ORSet(elements1) = s1 // `unapply[A](s: ORSet[A])` is used here
val elements2: Set[String] = elements1
Changed(ORSetKey[String]("key"))(s1) match {
case c @ Changed(ORSetKey("key"))
val x: ORSet[String] = c.dataValue
val ORSet(elements3) = c.dataValue
val elements4: Set[String] = elements3
elements4 should be(Set("a", "b"))
}
val msg: Any = Changed(ORSetKey[String]("key"))(s1)
msg match {
case c @ Changed(ORSetKey("key"))
val ORSet(elements3) = c.dataValue // `unapply(a: ReplicatedData)` is used here
// if `unapply(a: ReplicatedData)` isn't defined the next line doesn't compile:
// type mismatch; found : scala.collection.immutable.Set[A] where type A required: Set[Any] Note: A <: Any,
// but trait Set is invariant in type A. You may wish to investigate a wildcard type such as _ <: Any. (SLS 3.2.10)
val elements4: Set[Any] = elements3
elements4 should be(Set("a", "b"))
}
}
}
}

View file

@ -0,0 +1,62 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class PNCounterMapSpec extends WordSpec with Matchers {
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
"A PNCounterMap" must {
"be able to increment and decrement entries" in {
val m = PNCounterMap().increment(node1, "a", 2).increment(node1, "b", 3).decrement(node2, "a", 1)
m.entries should be(Map("a" -> 1, "b" -> 3))
}
"be able to have its entries correctly merged with another ORMap with other entries" in {
val m1 = PNCounterMap().increment(node1, "a", 1).increment(node1, "b", 3).increment(node1, "c", 2)
val m2 = PNCounterMap().increment(node2, "c", 5)
// merge both ways
val expected = Map("a" -> 1, "b" -> 3, "c" -> 7)
(m1 merge m2).entries should be(expected)
(m2 merge m1).entries should be(expected)
}
"be able to remove entry" in {
val m1 = PNCounterMap().increment(node1, "a", 1).increment(node1, "b", 3).increment(node1, "c", 2)
val m2 = PNCounterMap().increment(node2, "c", 5)
val merged1 = m1 merge m2
val m3 = merged1.remove(node1, "b")
(merged1 merge m3).entries should be(Map("a" -> 1, "c" -> 7))
// but if there is a conflicting update the entry is not removed
val m4 = merged1.increment(node2, "b", 10)
(m3 merge m4).entries should be(Map("a" -> 1, "b" -> 13, "c" -> 7))
}
"have unapply extractor" in {
val m1 = PNCounterMap.empty.increment(node1, "a", 1).increment(node2, "b", 2)
val PNCounterMap(entries1) = m1
val entries2: Map[String, BigInt] = entries1
Changed(PNCounterMapKey("key"))(m1) match {
case c @ Changed(PNCounterMapKey("key"))
val PNCounterMap(entries3) = c.dataValue
val entries4: Map[String, BigInt] = entries3
entries4 should be(Map("a" -> 1L, "b" -> 2L))
}
}
}
}

View file

@ -0,0 +1,172 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.cluster.ddata.Replicator.Changed
import org.scalatest.Matchers
import org.scalatest.WordSpec
class PNCounterSpec extends WordSpec with Matchers {
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
"A PNCounter" must {
"be able to increment each node's record by one" in {
val c1 = PNCounter()
val c2 = c1 increment node1
val c3 = c2 increment node1
val c4 = c3 increment node2
val c5 = c4 increment node2
val c6 = c5 increment node2
c6.increments.state(node1) should be(2)
c6.increments.state(node2) should be(3)
}
"be able to decrement each node's record by one" in {
val c1 = PNCounter()
val c2 = c1 decrement node1
val c3 = c2 decrement node1
val c4 = c3 decrement node2
val c5 = c4 decrement node2
val c6 = c5 decrement node2
c6.decrements.state(node1) should be(2)
c6.decrements.state(node2) should be(3)
}
"be able to increment each node's record by arbitrary delta" in {
val c1 = PNCounter()
val c2 = c1 increment (node1, 3)
val c3 = c2 increment (node1, 4)
val c4 = c3 increment (node2, 2)
val c5 = c4 increment (node2, 7)
val c6 = c5 increment node2
c6.increments.state(node1) should be(7)
c6.increments.state(node2) should be(10)
}
"be able to decrement each node's record by arbitrary delta" in {
val c1 = PNCounter()
val c2 = c1 decrement (node1, 3)
val c3 = c2 decrement (node1, 4)
val c4 = c3 decrement (node2, 2)
val c5 = c4 decrement (node2, 7)
val c6 = c5 decrement node2
c6.decrements.state(node1) should be(7)
c6.decrements.state(node2) should be(10)
}
"be able to increment and decrement each node's record by arbitrary delta" in {
val c1 = PNCounter()
val c2 = c1 increment (node1, 3)
val c3 = c2 decrement (node1, 2)
val c4 = c3 increment (node2, 5)
val c5 = c4 decrement (node2, 2)
val c6 = c5 increment node2
c6.increments.value should be(9)
c6.decrements.value should be(4)
}
"be able to summarize the history to the correct aggregated value of increments and decrements" in {
val c1 = PNCounter()
val c2 = c1 increment (node1, 3)
val c3 = c2 decrement (node1, 2)
val c4 = c3 increment (node2, 5)
val c5 = c4 decrement (node2, 2)
val c6 = c5 increment node2
c6.increments.value should be(9)
c6.decrements.value should be(4)
c6.value should be(5)
}
"be able to have its history correctly merged with another GCounter" in {
// counter 1
val c11 = PNCounter()
val c12 = c11 increment (node1, 3)
val c13 = c12 decrement (node1, 2)
val c14 = c13 increment (node2, 5)
val c15 = c14 decrement (node2, 2)
val c16 = c15 increment node2
c16.increments.value should be(9)
c16.decrements.value should be(4)
c16.value should be(5)
// counter 1
val c21 = PNCounter()
val c22 = c21 increment (node1, 2)
val c23 = c22 decrement (node1, 3)
val c24 = c23 increment (node2, 3)
val c25 = c24 decrement (node2, 2)
val c26 = c25 increment node2
c26.increments.value should be(6)
c26.decrements.value should be(5)
c26.value should be(1)
// merge both ways
val merged1 = c16 merge c26
merged1.increments.value should be(9)
merged1.decrements.value should be(5)
merged1.value should be(4)
val merged2 = c26 merge c16
merged2.increments.value should be(9)
merged2.decrements.value should be(5)
merged2.value should be(4)
}
"have support for pruning" in {
val c1 = PNCounter()
val c2 = c1 increment node1
val c3 = c2 decrement node2
c2.needPruningFrom(node1) should be(true)
c2.needPruningFrom(node2) should be(false)
c3.needPruningFrom(node1) should be(true)
c3.needPruningFrom(node2) should be(true)
val c4 = c3.prune(node1, node2)
c4.needPruningFrom(node2) should be(true)
c4.needPruningFrom(node1) should be(false)
val c5 = (c4 increment node1).pruningCleanup(node1)
c5.needPruningFrom(node1) should be(false)
}
"have unapply extractor" in {
val c1 = PNCounter.empty.increment(node1).increment(node1).decrement(node2)
val PNCounter(value1) = c1
val value2: BigInt = value1
Changed(PNCounterKey("key"))(c1) match {
case c @ Changed(PNCounterKey("key"))
val PNCounter(value3) = c.dataValue
val value4: BigInt = value3
value4 should be(1L)
}
}
}
}

View file

@ -0,0 +1,46 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.Address
import akka.cluster.UniqueAddress
import org.scalatest.Matchers
import org.scalatest.WordSpec
class PruningStateSpec extends WordSpec with Matchers {
import PruningState._
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
val node3 = UniqueAddress(node1.address.copy(port = Some(2553)), 3)
val node4 = UniqueAddress(node1.address.copy(port = Some(2554)), 4)
"Pruning state" must {
"merge phase correctly" in {
val p1 = PruningState(node1, PruningInitialized(Set.empty))
val p2 = PruningState(node1, PruningPerformed)
p1.merge(p2).phase should be(PruningPerformed)
p2.merge(p1).phase should be(PruningPerformed)
}
"merge owner correctly" in {
val p1 = PruningState(node1, PruningInitialized(Set.empty))
val p2 = PruningState(node2, PruningInitialized(Set.empty))
val expected = PruningState(node1, PruningInitialized(Set.empty))
p1.merge(p2) should be(expected)
p2.merge(p1) should be(expected)
}
"merge seen correctly" in {
val p1 = PruningState(node1, PruningInitialized(Set(node2.address)))
val p2 = PruningState(node1, PruningInitialized(Set(node4.address)))
val expected = PruningState(node1, PruningInitialized(Set(node2.address, node4.address)))
p1.merge(p2) should be(expected)
p2.merge(p1) should be(expected)
}
}
}

View file

@ -0,0 +1,249 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata
import akka.actor.ActorSystem
import akka.actor.Address
import akka.cluster.UniqueAddress
import akka.testkit.TestKit
import org.scalatest.BeforeAndAfterAll
import org.scalatest.Matchers
import org.scalatest.WordSpecLike
class VersionVectorSpec extends TestKit(ActorSystem("VersionVectorSpec"))
with WordSpecLike with Matchers with BeforeAndAfterAll {
val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1)
val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2)
val node3 = UniqueAddress(node1.address.copy(port = Some(2553)), 3)
val node4 = UniqueAddress(node1.address.copy(port = Some(2554)), 4)
override def afterAll {
shutdown()
}
"A VersionVector" must {
"have zero versions when created" in {
val vv = VersionVector()
vv.versions should be(Map())
}
"not happen before itself" in {
val vv1 = VersionVector()
val vv2 = VersionVector()
vv1 <> vv2 should be(false)
}
"pass misc comparison test 1" in {
val vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node1
val vv3_1 = vv2_1 + node2
val vv4_1 = vv3_1 + node1
val vv1_2 = VersionVector()
val vv2_2 = vv1_2 + node1
val vv3_2 = vv2_2 + node2
val vv4_2 = vv3_2 + node1
vv4_1 <> vv4_2 should be(false)
}
"pass misc comparison test 2" in {
val vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node1
val vv3_1 = vv2_1 + node2
val vv4_1 = vv3_1 + node1
val vv1_2 = VersionVector()
val vv2_2 = vv1_2 + node1
val vv3_2 = vv2_2 + node2
val vv4_2 = vv3_2 + node1
val vv5_2 = vv4_2 + node3
vv4_1 < vv5_2 should be(true)
}
"pass misc comparison test 3" in {
var vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node1
val vv1_2 = VersionVector()
val vv2_2 = vv1_2 + node2
vv2_1 <> vv2_2 should be(true)
}
"pass misc comparison test 4" in {
val vv1_3 = VersionVector()
val vv2_3 = vv1_3 + node1
val vv3_3 = vv2_3 + node2
val vv4_3 = vv3_3 + node1
val vv1_4 = VersionVector()
val vv2_4 = vv1_4 + node1
val vv3_4 = vv2_4 + node1
val vv4_4 = vv3_4 + node3
vv4_3 <> vv4_4 should be(true)
}
"pass misc comparison test 5" in {
val vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node2
val vv3_1 = vv2_1 + node2
val vv1_2 = VersionVector()
val vv2_2 = vv1_2 + node1
val vv3_2 = vv2_2 + node2
val vv4_2 = vv3_2 + node2
val vv5_2 = vv4_2 + node3
vv3_1 < vv5_2 should be(true)
vv5_2 > vv3_1 should be(true)
}
"pass misc comparison test 6" in {
val vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node1
val vv3_1 = vv2_1 + node2
val vv1_2 = VersionVector()
val vv2_2 = vv1_2 + node1
val vv3_2 = vv2_2 + node1
vv3_1 <> vv3_2 should be(true)
vv3_2 <> vv3_1 should be(true)
}
"pass misc comparison test 7" in {
val vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node1
val vv3_1 = vv2_1 + node2
val vv4_1 = vv3_1 + node2
val vv5_1 = vv4_1 + node3
val vv1_2 = vv4_1
val vv2_2 = vv1_2 + node2
val vv3_2 = vv2_2 + node2
vv5_1 <> vv3_2 should be(true)
vv3_2 <> vv5_1 should be(true)
}
"pass misc comparison test 8" in {
val vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node1
val vv3_1 = vv2_1 + node3
val vv1_2 = vv3_1 + node2
val vv4_1 = vv3_1 + node3
vv4_1 <> vv1_2 should be(true)
vv1_2 <> vv4_1 should be(true)
}
"correctly merge two version vectors" in {
val vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node1
val vv3_1 = vv2_1 + node2
val vv4_1 = vv3_1 + node2
val vv5_1 = vv4_1 + node3
val vv1_2 = vv4_1
val vv2_2 = vv1_2 + node2
val vv3_2 = vv2_2 + node2
val merged1 = vv3_2 merge vv5_1
merged1.versions.size should be(3)
merged1.versions.contains(node1) should be(true)
merged1.versions.contains(node2) should be(true)
merged1.versions.contains(node3) should be(true)
val merged2 = vv5_1 merge vv3_2
merged2.versions.size should be(3)
merged2.versions.contains(node1) should be(true)
merged2.versions.contains(node2) should be(true)
merged2.versions.contains(node3) should be(true)
vv3_2 < merged1 should be(true)
vv5_1 < merged1 should be(true)
vv3_2 < merged2 should be(true)
vv5_1 < merged2 should be(true)
merged1 == merged2 should be(true)
}
"correctly merge two disjoint version vectors" in {
val vv1_1 = VersionVector()
val vv2_1 = vv1_1 + node1
val vv3_1 = vv2_1 + node2
val vv4_1 = vv3_1 + node2
val vv5_1 = vv4_1 + node3
val vv1_2 = VersionVector()
val vv2_2 = vv1_2 + node4
val vv3_2 = vv2_2 + node4
val merged1 = vv3_2 merge vv5_1
merged1.versions.size should be(4)
merged1.versions.contains(node1) should be(true)
merged1.versions.contains(node2) should be(true)
merged1.versions.contains(node3) should be(true)
merged1.versions.contains(node4) should be(true)
val merged2 = vv5_1 merge vv3_2
merged2.versions.size should be(4)
merged2.versions.contains(node1) should be(true)
merged2.versions.contains(node2) should be(true)
merged2.versions.contains(node3) should be(true)
merged2.versions.contains(node4) should be(true)
vv3_2 < merged1 should be(true)
vv5_1 < merged1 should be(true)
vv3_2 < merged2 should be(true)
vv5_1 < merged2 should be(true)
merged1 == merged2 should be(true)
}
"pass blank version vector incrementing" in {
val v1 = VersionVector()
val v2 = VersionVector()
val vv1 = v1 + node1
val vv2 = v2 + node2
(vv1 > v1) should be(true)
(vv2 > v2) should be(true)
(vv1 > v2) should be(true)
(vv2 > v1) should be(true)
(vv2 > vv1) should be(false)
(vv1 > vv2) should be(false)
}
"pass merging behavior" in {
val a = VersionVector()
val b = VersionVector()
val a1 = a + node1
val b1 = b + node2
var a2 = a1 + node1
var c = a2.merge(b1)
var c1 = c + node3
(c1 > a2) should be(true)
(c1 > b1) should be(true)
}
}
}

View file

@ -0,0 +1,173 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata.protobuf
import scala.concurrent.duration._
import org.scalatest.BeforeAndAfterAll
import org.scalatest.Matchers
import org.scalatest.WordSpecLike
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.cluster.ddata.Flag
import akka.cluster.ddata.GCounter
import akka.cluster.ddata.GSet
import akka.cluster.ddata.LWWMap
import akka.cluster.ddata.LWWRegister
import akka.cluster.ddata.ORMap
import akka.cluster.ddata.ORSet
import akka.cluster.ddata.PNCounter
import akka.cluster.ddata.PNCounterMap
import akka.cluster.ddata.Replicator._
import akka.cluster.ddata.Replicator.Internal._
import akka.cluster.ddata.VersionVector
import akka.testkit.TestKit
import akka.cluster.UniqueAddress
import com.typesafe.config.ConfigFactory
class ReplicatedDataSerializerSpec extends TestKit(ActorSystem("ReplicatedDataSerializerSpec",
ConfigFactory.parseString("""
akka.actor.provider=akka.cluster.ClusterActorRefProvider
akka.remote.netty.tcp.port=0
"""))) with WordSpecLike with Matchers with BeforeAndAfterAll {
val serializer = new ReplicatedDataSerializer(system.asInstanceOf[ExtendedActorSystem])
val address1 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4711), 1)
val address2 = UniqueAddress(Address("akka.tcp", system.name, "other.host.org", 4711), 2)
val address3 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4712), 3)
override def afterAll {
shutdown()
}
def checkSerialization(obj: AnyRef): Unit = {
val blob = serializer.toBinary(obj)
val ref = serializer.fromBinary(blob, serializer.manifest(obj))
ref should be(obj)
}
def checkSameContent(a: AnyRef, b: AnyRef): Unit = {
a should be(b)
val blobA = serializer.toBinary(a)
val blobB = serializer.toBinary(b)
blobA.toSeq should be(blobB.toSeq)
}
"ReplicatedDataSerializer" must {
"serialize GSet" in {
checkSerialization(GSet())
checkSerialization(GSet() + "a")
checkSerialization(GSet() + "a" + "b")
checkSerialization(GSet() + 1 + 2 + 3)
checkSerialization(GSet() + address1 + address2)
checkSerialization(GSet() + 1L + "2" + 3 + address1)
checkSameContent(GSet() + "a" + "b", GSet() + "a" + "b")
checkSameContent(GSet() + "a" + "b", GSet() + "b" + "a")
checkSameContent(GSet() + address1 + address2 + address3, GSet() + address2 + address1 + address3)
checkSameContent(GSet() + address1 + address2 + address3, GSet() + address3 + address2 + address1)
}
"serialize ORSet" in {
checkSerialization(ORSet())
checkSerialization(ORSet().add(address1, "a"))
checkSerialization(ORSet().add(address1, "a").add(address2, "a"))
checkSerialization(ORSet().add(address1, "a").remove(address2, "a"))
checkSerialization(ORSet().add(address1, "a").add(address2, "b").remove(address1, "a"))
checkSerialization(ORSet().add(address1, 1).add(address2, 2))
checkSerialization(ORSet().add(address1, 1L).add(address2, 2L))
checkSerialization(ORSet().add(address1, "a").add(address2, 2).add(address3, 3L).add(address3, address3))
val s1 = ORSet().add(address1, "a").add(address2, "b")
val s2 = ORSet().add(address2, "b").add(address1, "a")
checkSameContent(s1.merge(s2), s2.merge(s1))
val s3 = ORSet().add(address1, "a").add(address2, 17).remove(address3, 17)
val s4 = ORSet().add(address2, 17).remove(address3, 17).add(address1, "a")
checkSameContent(s3.merge(s4), s4.merge(s3))
}
"serialize Flag" in {
checkSerialization(Flag())
checkSerialization(Flag().switchOn)
}
"serialize LWWRegister" in {
checkSerialization(LWWRegister(address1, "value1", LWWRegister.defaultClock))
checkSerialization(LWWRegister(address1, "value2", LWWRegister.defaultClock[String])
.withValue(address2, "value3", LWWRegister.defaultClock[String]))
}
"serialize GCounter" in {
checkSerialization(GCounter())
checkSerialization(GCounter().increment(address1, 3))
checkSerialization(GCounter().increment(address1, 2).increment(address2, 5))
checkSameContent(
GCounter().increment(address1, 2).increment(address2, 5),
GCounter().increment(address2, 5).increment(address1, 1).increment(address1, 1))
checkSameContent(
GCounter().increment(address1, 2).increment(address3, 5),
GCounter().increment(address3, 5).increment(address1, 2))
}
"serialize PNCounter" in {
checkSerialization(PNCounter())
checkSerialization(PNCounter().increment(address1, 3))
checkSerialization(PNCounter().increment(address1, 3).decrement(address1, 1))
checkSerialization(PNCounter().increment(address1, 2).increment(address2, 5))
checkSerialization(PNCounter().increment(address1, 2).increment(address2, 5).decrement(address1, 1))
checkSameContent(
PNCounter().increment(address1, 2).increment(address2, 5),
PNCounter().increment(address2, 5).increment(address1, 1).increment(address1, 1))
checkSameContent(
PNCounter().increment(address1, 2).increment(address3, 5),
PNCounter().increment(address3, 5).increment(address1, 2))
checkSameContent(
PNCounter().increment(address1, 2).decrement(address1, 1).increment(address3, 5),
PNCounter().increment(address3, 5).increment(address1, 2).decrement(address1, 1))
}
"serialize ORMap" in {
checkSerialization(ORMap())
checkSerialization(ORMap().put(address1, "a", GSet() + "A"))
checkSerialization(ORMap().put(address1, "a", GSet() + "A").put(address2, "b", GSet() + "B"))
}
"serialize LWWMap" in {
checkSerialization(LWWMap())
checkSerialization(LWWMap().put(address1, "a", "value1", LWWRegister.defaultClock[Any]))
checkSerialization(LWWMap().put(address1, "a", "value1", LWWRegister.defaultClock[Any])
.put(address2, "b", 17, LWWRegister.defaultClock[Any]))
}
"serialize PNCounterMap" in {
checkSerialization(PNCounterMap())
checkSerialization(PNCounterMap().increment(address1, "a", 3))
checkSerialization(PNCounterMap().increment(address1, "a", 3).decrement(address2, "a", 2).
increment(address2, "b", 5))
}
"serialize DeletedData" in {
checkSerialization(DeletedData)
}
"serialize VersionVector" in {
checkSerialization(VersionVector())
checkSerialization(VersionVector().increment(address1))
checkSerialization(VersionVector().increment(address1).increment(address2))
val v1 = VersionVector().increment(address1).increment(address1)
val v2 = VersionVector().increment(address2)
checkSameContent(v1.merge(v2), v2.merge(v1))
}
}
}

View file

@ -0,0 +1,81 @@
/**
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata.protobuf
import scala.concurrent.duration._
import org.scalatest.BeforeAndAfterAll
import org.scalatest.Matchers
import org.scalatest.WordSpecLike
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.actor.Props
import akka.cluster.ddata.GSet
import akka.cluster.ddata.GSetKey
import akka.cluster.ddata.PruningState
import akka.cluster.ddata.PruningState.PruningInitialized
import akka.cluster.ddata.PruningState.PruningPerformed
import akka.cluster.ddata.Replicator._
import akka.cluster.ddata.Replicator.Internal._
import akka.testkit.TestKit
import akka.util.ByteString
import akka.cluster.UniqueAddress
import com.typesafe.config.ConfigFactory
class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem("ReplicatorMessageSerializerSpec",
ConfigFactory.parseString("""
akka.actor.provider=akka.cluster.ClusterActorRefProvider
akka.remote.netty.tcp.port=0
"""))) with WordSpecLike with Matchers with BeforeAndAfterAll {
val serializer = new ReplicatorMessageSerializer(system.asInstanceOf[ExtendedActorSystem])
val address1 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4711), 1)
val address2 = UniqueAddress(Address("akka.tcp", system.name, "other.host.org", 4711), 2)
val address3 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4712), 3)
val keyA = GSetKey[String]("A")
override def afterAll {
shutdown()
}
def checkSerialization(obj: AnyRef): Unit = {
val blob = serializer.toBinary(obj)
val ref = serializer.fromBinary(blob, serializer.manifest(obj))
ref should be(obj)
}
"ReplicatorMessageSerializer" must {
"serialize Replicator messages" in {
val ref1 = system.actorOf(Props.empty, "ref1")
val data1 = GSet.empty[String] + "a"
checkSerialization(Get(keyA, ReadLocal))
checkSerialization(Get(keyA, ReadMajority(2.seconds), Some("x")))
checkSerialization(GetSuccess(keyA, None)(data1))
checkSerialization(GetSuccess(keyA, Some("x"))(data1))
checkSerialization(NotFound(keyA, Some("x")))
checkSerialization(GetFailure(keyA, Some("x")))
checkSerialization(Subscribe(keyA, ref1))
checkSerialization(Unsubscribe(keyA, ref1))
checkSerialization(Changed(keyA)(data1))
checkSerialization(DataEnvelope(data1))
checkSerialization(DataEnvelope(data1, pruning = Map(
address1 -> PruningState(address2, PruningPerformed),
address3 -> PruningState(address2, PruningInitialized(Set(address1.address))))))
checkSerialization(Write("A", DataEnvelope(data1)))
checkSerialization(WriteAck)
checkSerialization(Read("A"))
checkSerialization(ReadResult(Some(DataEnvelope(data1))))
checkSerialization(ReadResult(None))
checkSerialization(Status(Map("A" -> ByteString.fromString("a"),
"B" -> ByteString.fromString("b")), chunk = 3, totChunks = 10))
checkSerialization(Gossip(Map("A" -> DataEnvelope(data1),
"B" -> DataEnvelope(GSet() + "b" + "c")), sendBack = true))
}
}
}

View file

@ -0,0 +1,98 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata.sample
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorSystem
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.ORSet
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.Replicator
import akka.cluster.ddata.ORSetKey
object DataBot {
def main(args: Array[String]): Unit = {
if (args.isEmpty)
startup(Seq("2551", "2552", "0"))
else
startup(args)
}
def startup(ports: Seq[String]): Unit = {
ports.foreach { port
// Override the configuration of the port
val config = ConfigFactory.parseString("akka.remote.netty.tcp.port=" + port).
withFallback(ConfigFactory.load(
ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote {
netty.tcp {
hostname = "127.0.0.1"
port = 0
}
}
akka.cluster {
seed-nodes = [
"akka.tcp://ClusterSystem@127.0.0.1:2551",
"akka.tcp://ClusterSystem@127.0.0.1:2552"]
auto-down-unreachable-after = 10s
}
""")))
// Create an Akka system
val system = ActorSystem("ClusterSystem", config)
// Create an actor that handles cluster domain events
system.actorOf(Props[DataBot], name = "dataBot")
}
}
private case object Tick
}
class DataBot extends Actor with ActorLogging {
import DataBot._
import Replicator._
val replicator = DistributedData(context.system).replicator
implicit val node = Cluster(context.system)
import context.dispatcher
val tickTask = context.system.scheduler.schedule(5.seconds, 5.seconds, self, Tick)
val DataKey = ORSetKey[String]("key")
replicator ! Subscribe(DataKey, self)
def receive = {
case Tick
val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString
if (ThreadLocalRandom.current().nextBoolean()) {
// add
log.info("Adding: {}", s)
replicator ! Update(DataKey, ORSet.empty[String], WriteLocal)(_ + s)
} else {
// remove
log.info("Removing: {}", s)
replicator ! Update(DataKey, ORSet.empty[String], WriteLocal)(_ - s)
}
case _: UpdateResponse[_] // ignore
case c @ Changed(DataKey)
log.info("Current elements: {}", c.get(DataKey).elements)
}
override def postStop(): Unit = tickTask.cancel()
}

View file

@ -0,0 +1,137 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.ddata.sample
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorSystem
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.ORSet
import com.typesafe.config.ConfigFactory
import akka.cluster.ddata.Replicator
import akka.cluster.ddata.ORSetKey
object LotsOfDataBot {
def main(args: Array[String]): Unit = {
if (args.isEmpty)
startup(Seq("2551", "2552", "0"))
else
startup(args)
}
def startup(ports: Seq[String]): Unit = {
ports.foreach { port
// Override the configuration of the port
val config = ConfigFactory.parseString("akka.remote.netty.tcp.port=" + port).
withFallback(ConfigFactory.load(
ConfigFactory.parseString("""
passive = off
max-entries = 100000
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote {
netty.tcp {
hostname = "127.0.0.1"
port = 0
}
}
akka.cluster {
seed-nodes = [
"akka.tcp://ClusterSystem@127.0.0.1:2551",
"akka.tcp://ClusterSystem@127.0.0.1:2552"]
auto-down-unreachable-after = 10s
}
akka.cluster.distributed-data.use-offheap-memory = off
akka.remote.log-frame-size-exceeding = 10000b
""")))
// Create an Akka system
val system = ActorSystem("ClusterSystem", config)
// Create an actor that handles cluster domain events
system.actorOf(Props[LotsOfDataBot], name = "dataBot")
}
}
private case object Tick
}
class LotsOfDataBot extends Actor with ActorLogging {
import LotsOfDataBot._
import Replicator._
val replicator = DistributedData(context.system).replicator
implicit val cluster = Cluster(context.system)
import context.dispatcher
val isPassive = context.system.settings.config.getBoolean("passive")
var tickTask =
if (isPassive)
context.system.scheduler.schedule(1.seconds, 1.seconds, self, Tick)
else
context.system.scheduler.schedule(20.millis, 20.millis, self, Tick)
val startTime = System.nanoTime()
var count = 1L
val maxEntries = context.system.settings.config.getInt("max-entries")
def receive = if (isPassive) passive else active
def active: Receive = {
case Tick
val loop = if (count >= maxEntries) 1 else 100
for (_ 1 to loop) {
count += 1
if (count % 10000 == 0)
log.info("Reached {} entries", count)
if (count == maxEntries) {
log.info("Reached {} entries", count)
tickTask.cancel()
tickTask = context.system.scheduler.schedule(1.seconds, 1.seconds, self, Tick)
}
val key = ORSetKey[String]((count % maxEntries).toString)
if (count <= 100)
replicator ! Subscribe(key, self)
val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString
if (count <= maxEntries || ThreadLocalRandom.current().nextBoolean()) {
// add
replicator ! Update(key, ORSet(), WriteLocal)(_ + s)
} else {
// remove
replicator ! Update(key, ORSet(), WriteLocal)(_ - s)
}
}
case _: UpdateResponse[_] // ignore
case c @ Changed(ORSetKey(id))
val ORSet(elements) = c.dataValue
log.info("Current elements: {} -> {}", id, elements)
}
def passive: Receive = {
case Tick
if (!tickTask.isCancelled)
replicator ! GetKeyIds
case GetKeyIdsResult(keys)
if (keys.size >= maxEntries) {
tickTask.cancel()
val duration = (System.nanoTime() - startTime).nanos.toMillis
log.info("It took {} ms to replicate {} entries", duration, keys.size)
}
case c @ Changed(ORSetKey(id))
val ORSet(elements) = c.dataValue
log.info("Current elements: {} -> {}", id, elements)
}
override def postStop(): Unit = tickTask.cancel()
}

View file

@ -484,3 +484,35 @@ akka-testkit
.. literalinclude:: ../../../akka-testkit/src/main/resources/reference.conf
:language: none
.. _config-cluster-metrics:
akka-cluster-metrics
~~~~~~~~~~~~--------
.. literalinclude:: ../../../akka-cluster-metrics/src/main/resources/reference.conf
:language: none
.. _config-cluster-tools:
akka-cluster-tools
~~~~~~~~~~~~------
.. literalinclude:: ../../../akka-cluster-tools/src/main/resources/reference.conf
:language: none
.. _config-cluster-sharding:
akka-cluster-sharding
~~~~~~~~~~~~---------
.. literalinclude:: ../../../akka-cluster-sharding/src/main/resources/reference.conf
:language: none
.. _config-distributed-data:
akka-distributed-data
~~~~~~~~~~~~---------
.. literalinclude:: ../../../akka-distributed-data/src/main/resources/reference.conf
:language: none

View file

@ -0,0 +1,95 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.ddata;
//#data-bot
import static java.util.concurrent.TimeUnit.SECONDS;
import scala.concurrent.duration.Duration;
import scala.concurrent.forkjoin.ThreadLocalRandom;
import akka.actor.AbstractActor;
import akka.actor.ActorRef;
import akka.actor.Cancellable;
import akka.cluster.Cluster;
import akka.cluster.ddata.DistributedData;
import akka.cluster.ddata.Key;
import akka.cluster.ddata.ORSet;
import akka.cluster.ddata.ORSetKey;
import akka.cluster.ddata.Replicator;
import akka.cluster.ddata.Replicator.Changed;
import akka.cluster.ddata.Replicator.Subscribe;
import akka.cluster.ddata.Replicator.Update;
import akka.cluster.ddata.Replicator.UpdateResponse;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.japi.pf.ReceiveBuilder;
public class DataBot extends AbstractActor {
private static final String TICK = "tick";
private final LoggingAdapter log = Logging.getLogger(context().system(), this);
private final ActorRef replicator =
DistributedData.get(context().system()).replicator();
private final Cluster node = Cluster.get(context().system());
private final Cancellable tickTask = context().system().scheduler().schedule(
Duration.create(5, SECONDS), Duration.create(5, SECONDS), self(), TICK,
context().dispatcher(), self());
private final Key<ORSet<String>> dataKey = ORSetKey.create("key");
public DataBot() {
receive(ReceiveBuilder.
match(String.class, a -> a.equals(TICK), a -> {
String s = String.valueOf((char) ThreadLocalRandom.current().nextInt(97, 123));
if (ThreadLocalRandom.current().nextBoolean()) {
// add
log.info("Adding: {}", s);
Update<ORSet<String>> update = new Update<>(
dataKey,
ORSet.create(),
Replicator.writeLocal(),
curr -> curr.add(node, s));
replicator.tell(update, self());
} else {
// remove
log.info("Removing: {}", s);
Update<ORSet<String>> update = new Update<>(
dataKey,
ORSet.create(),
Replicator.writeLocal(),
curr -> curr.remove(node, s));
replicator.tell(update, self());
}
}).
match(UpdateResponse.class, r -> {
// ignore
}).
match(Changed.class, c -> c.key().equals(dataKey), c -> {
@SuppressWarnings("unchecked")
Changed<ORSet<String>> c2 = c;
ORSet<String> data = c2.dataValue();
log.info("Current elements: {}", data.getElements());
}).
matchAny(o -> log.info("received unknown message")).build()
);
}
@Override
public void preStart() {
Subscribe<ORSet<String>> subscribe = new Subscribe<>(dataKey, self());
replicator.tell(subscribe, ActorRef.noSender());
}
@Override
public void postStop(){
tickTask.cancel();
}
}
//#data-bot

View file

@ -0,0 +1,411 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.ddata;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.junit.Assert.assertEquals;
import com.typesafe.config.ConfigFactory;
import java.math.BigInteger;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.PartialFunction;
import scala.concurrent.duration.Duration;
import scala.runtime.BoxedUnit;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.cluster.Cluster;
import akka.cluster.ddata.DistributedData;
import akka.cluster.ddata.Flag;
import akka.cluster.ddata.FlagKey;
import akka.cluster.ddata.GSet;
import akka.cluster.ddata.GSetKey;
import akka.cluster.ddata.Key;
import akka.cluster.ddata.LWWRegister;
import akka.cluster.ddata.ORSet;
import akka.cluster.ddata.ORSetKey;
import akka.cluster.ddata.PNCounter;
import akka.cluster.ddata.PNCounterKey;
import akka.cluster.ddata.PNCounterMap;
import akka.cluster.ddata.Replicator;
import akka.cluster.ddata.Replicator.Changed;
import akka.cluster.ddata.Replicator.Delete;
import akka.cluster.ddata.Replicator.GetFailure;
import akka.cluster.ddata.Replicator.GetSuccess;
import akka.cluster.ddata.Replicator.NotFound;
import akka.cluster.ddata.Replicator.ReadAll;
import akka.cluster.ddata.Replicator.ReadConsistency;
import akka.cluster.ddata.Replicator.ReadFrom;
import akka.cluster.ddata.Replicator.ReadMajority;
import akka.cluster.ddata.Replicator.Subscribe;
import akka.cluster.ddata.Replicator.UpdateSuccess;
import akka.cluster.ddata.Replicator.UpdateTimeout;
import akka.cluster.ddata.Replicator.WriteAll;
import akka.cluster.ddata.Replicator.WriteConsistency;
import akka.cluster.ddata.Replicator.WriteMajority;
import akka.cluster.ddata.Replicator.WriteTo;
import akka.japi.pf.ReceiveBuilder;
import akka.testkit.JavaTestKit;
public class DistributedDataDocTest {
static ActorSystem system;
void receive(PartialFunction<Object, BoxedUnit> pf) {
}
JavaTestKit probe = new JavaTestKit(system);
ActorRef self() {
return probe.getRef();
}
ActorRef sender() {
return probe.getRef();
}
@BeforeClass
public static void setup() {
system = ActorSystem.create("DistributedDataDocTest",
ConfigFactory.parseString(DistributedDataDocSpec.config()));
}
@AfterClass
public static void tearDown() {
JavaTestKit.shutdownActorSystem(system);
system = null;
}
@Test
public void demonstrateUpdate() {
probe = new JavaTestKit(system);
//#update
final Cluster node = Cluster.get(system);
final ActorRef replicator = DistributedData.get(system).replicator();
final Key<PNCounter> counter1Key = PNCounterKey.create("counter1");
final Key<GSet<String>> set1Key = GSetKey.create("set1");
final Key<ORSet<String>> set2Key = ORSetKey.create("set2");
final Key<Flag> activeFlagKey = FlagKey.create("active");
replicator.tell(new Replicator.Update<PNCounter>(counter1Key, PNCounter.create(),
Replicator.writeLocal(), curr -> curr.increment(node, 1)), self());
final WriteConsistency writeTo3 = new WriteTo(3, Duration.create(1, SECONDS));
replicator.tell(new Replicator.Update<GSet<String>>(set1Key, GSet.create(),
writeTo3, curr -> curr.add("hello")), self());
final WriteConsistency writeMajority =
new WriteMajority(Duration.create(5, SECONDS));
replicator.tell(new Replicator.Update<ORSet<String>>(set2Key, ORSet.create(),
writeMajority, curr -> curr.add(node, "hello")), self());
final WriteConsistency writeAll = new WriteAll(Duration.create(5, SECONDS));
replicator.tell(new Replicator.Update<Flag>(activeFlagKey, Flag.create(),
writeAll, curr -> curr.switchOn()), self());
//#update
probe.expectMsgClass(UpdateSuccess.class);
//#update-response1
receive(ReceiveBuilder.
match(UpdateSuccess.class, a -> a.key().equals(counter1Key), a -> {
// ok
}).build());
//#update-response1
//#update-response2
receive(ReceiveBuilder.
match(UpdateSuccess.class, a -> a.key().equals(set1Key), a -> {
// ok
}).
match(UpdateTimeout.class, a -> a.key().equals(set1Key), a -> {
// write to 3 nodes failed within 1.second
}).build());
//#update-response2
}
@Test
public void demonstrateUpdateWithRequestContext() {
probe = new JavaTestKit(system);
//#update-request-context
final Cluster node = Cluster.get(system);
final ActorRef replicator = DistributedData.get(system).replicator();
final WriteConsistency writeTwo = new WriteTo(2, Duration.create(3, SECONDS));
final Key<PNCounter> counter1Key = PNCounterKey.create("counter1");
receive(ReceiveBuilder.
match(String.class, a -> a.equals("increment"), a -> {
// incoming command to increase the counter
Optional<Object> reqContext = Optional.of(sender());
Replicator.Update<PNCounter> upd = new Replicator.Update<PNCounter>(counter1Key,
PNCounter.create(), writeTwo, reqContext, curr -> curr.increment(node, 1));
replicator.tell(upd, self());
}).
match(UpdateSuccess.class, a -> a.key().equals(counter1Key), a -> {
ActorRef replyTo = (ActorRef) a.getRequest().get();
replyTo.tell("ack", self());
}).
match(UpdateTimeout.class, a -> a.key().equals(counter1Key), a -> {
ActorRef replyTo = (ActorRef) a.getRequest().get();
replyTo.tell("nack", self());
}).build());
//#update-request-context
}
@SuppressWarnings({ "unused", "unchecked" })
@Test
public void demonstrateGet() {
probe = new JavaTestKit(system);
//#get
final ActorRef replicator = DistributedData.get(system).replicator();
final Key<PNCounter> counter1Key = PNCounterKey.create("counter1");
final Key<GSet<String>> set1Key = GSetKey.create("set1");
final Key<ORSet<String>> set2Key = ORSetKey.create("set2");
final Key<Flag> activeFlagKey = FlagKey.create("active");
replicator.tell(new Replicator.Get<PNCounter>(counter1Key,
Replicator.readLocal()), self());
final ReadConsistency readFrom3 = new ReadFrom(3, Duration.create(1, SECONDS));
replicator.tell(new Replicator.Get<GSet<String>>(set1Key,
readFrom3), self());
final ReadConsistency readMajority = new ReadMajority(Duration.create(5, SECONDS));
replicator.tell(new Replicator.Get<ORSet<String>>(set2Key,
readMajority), self());
final ReadConsistency readAll = new ReadAll(Duration.create(5, SECONDS));
replicator.tell(new Replicator.Get<Flag>(activeFlagKey,
readAll), self());
//#get
//#get-response1
receive(ReceiveBuilder.
match(GetSuccess.class, a -> a.key().equals(counter1Key), a -> {
GetSuccess<PNCounter> g = a;
BigInteger value = g.dataValue().getValue();
}).
match(NotFound.class, a -> a.key().equals(counter1Key), a -> {
// key counter1 does not exist
}).build());
//#get-response1
//#get-response2
receive(ReceiveBuilder.
match(GetSuccess.class, a -> a.key().equals(set1Key), a -> {
GetSuccess<GSet<String>> g = a;
Set<String> value = g.dataValue().getElements();
}).
match(GetFailure.class, a -> a.key().equals(set1Key), a -> {
// read from 3 nodes failed within 1.second
}).
match(NotFound.class, a -> a.key().equals(set1Key), a -> {
// key set1 does not exist
}).build());
//#get-response2
}
@SuppressWarnings("unchecked")
@Test
public void demonstrateGetWithRequestContext() {
probe = new JavaTestKit(system);
//#get-request-context
final ActorRef replicator = DistributedData.get(system).replicator();
final ReadConsistency readTwo = new ReadFrom(2, Duration.create(3, SECONDS));
final Key<PNCounter> counter1Key = PNCounterKey.create("counter1");
receive(ReceiveBuilder.
match(String.class, a -> a.equals("get-count"), a -> {
// incoming request to retrieve current value of the counter
Optional<Object> reqContext = Optional.of(sender());
replicator.tell(new Replicator.Get<PNCounter>(counter1Key,
readTwo), self());
}).
match(GetSuccess.class, a -> a.key().equals(counter1Key), a -> {
ActorRef replyTo = (ActorRef) a.getRequest().get();
GetSuccess<PNCounter> g = a;
long value = g.dataValue().getValue().longValue();
replyTo.tell(value, self());
}).
match(GetFailure.class, a -> a.key().equals(counter1Key), a -> {
ActorRef replyTo = (ActorRef) a.getRequest().get();
replyTo.tell(-1L, self());
}).
match(NotFound.class, a -> a.key().equals(counter1Key), a -> {
ActorRef replyTo = (ActorRef) a.getRequest().get();
replyTo.tell(0L, self());
}).build());
//#get-request-context
}
@SuppressWarnings("unchecked")
abstract class MyActor {
//#subscribe
final ActorRef replicator = DistributedData.get(system).replicator();
final Key<PNCounter> counter1Key = PNCounterKey.create("counter1");
BigInteger currentValue = BigInteger.valueOf(0);
public MyActor() {
receive(ReceiveBuilder.
match(Changed.class, a -> a.key().equals(counter1Key), a -> {
Changed<PNCounter> g = a;
currentValue = g.dataValue().getValue();
}).
match(String.class, a -> a.equals("get-count"), a -> {
// incoming request to retrieve current value of the counter
sender().tell(currentValue, sender());
}).build());
}
public void preStart() {
// subscribe to changes of the Counter1Key value
replicator.tell(new Subscribe<PNCounter>(counter1Key, self()), ActorRef.noSender());
}
//#subscribe
}
@Test
public void demonstrateDelete() {
probe = new JavaTestKit(system);
//#delete
final ActorRef replicator = DistributedData.get(system).replicator();
final Key<PNCounter> counter1Key = PNCounterKey.create("counter1");
final Key<ORSet<String>> set2Key = ORSetKey.create("set2");
replicator.tell(new Delete<PNCounter>(counter1Key,
Replicator.writeLocal()), self());
final WriteConsistency writeMajority =
new WriteMajority(Duration.create(5, SECONDS));
replicator.tell(new Delete<PNCounter>(counter1Key,
writeMajority), self());
//#delete
}
public void demonstratePNCounter() {
//#pncounter
final Cluster node = Cluster.get(system);
final PNCounter c0 = PNCounter.create();
final PNCounter c1 = c0.increment(node, 1);
final PNCounter c2 = c1.increment(node, 7);
final PNCounter c3 = c2.decrement(node, 2);
System.out.println(c3.value()); // 6
//#pncounter
}
public void demonstratePNCounterMap() {
//#pncountermap
final Cluster node = Cluster.get(system);
final PNCounterMap m0 = PNCounterMap.create();
final PNCounterMap m1 = m0.increment(node, "a", 7);
final PNCounterMap m2 = m1.decrement(node, "a", 2);
final PNCounterMap m3 = m2.increment(node, "b", 1);
System.out.println(m3.get("a")); // 5
System.out.println(m3.getEntries());
//#pncountermap
}
public void demonstrateGSet() {
//#gset
final GSet<String> s0 = GSet.create();
final GSet<String> s1 = s0.add("a");
final GSet<String> s2 = s1.add("b").add("c");
if (s2.contains("a"))
System.out.println(s2.getElements()); // a, b, c
//#gset
}
public void demonstrateORSet() {
//#orset
final Cluster node = Cluster.get(system);
final ORSet<String> s0 = ORSet.create();
final ORSet<String> s1 = s0.add(node, "a");
final ORSet<String> s2 = s1.add(node, "b");
final ORSet<String> s3 = s2.remove(node, "a");
System.out.println(s3.getElements()); // b
//#orset
}
public void demonstrateFlag() {
//#flag
final Flag f0 = Flag.create();
final Flag f1 = f0.switchOn();
System.out.println(f1.enabled());
//#flag
}
@Test
public void demonstrateLWWRegister() {
//#lwwregister
final Cluster node = Cluster.get(system);
final LWWRegister<String> r1 = LWWRegister.create(node, "Hello");
final LWWRegister<String> r2 = r1.withValue(node, "Hi");
System.out.println(r1.value() + " by " + r1.updatedBy() + " at " + r1.timestamp());
//#lwwregister
assertEquals("Hi", r2.value());
}
static
//#lwwregister-custom-clock
class Record {
public final int version;
public final String name;
public final String address;
public Record(int version, String name, String address) {
this.version = version;
this.name = name;
this.address = address;
}
}
//#lwwregister-custom-clock
public void demonstrateLWWRegisterWithCustomClock() {
//#lwwregister-custom-clock
final Cluster node = Cluster.get(system);
final LWWRegister.Clock<Record> recordClock = new LWWRegister.Clock<Record>() {
@Override
public long apply(long currentTimestamp, Record value) {
return value.version;
}
};
final Record record1 = new Record(1, "Alice", "Union Square");
final LWWRegister<Record> r1 = LWWRegister.create(node, record1);
final Record record2 = new Record(2, "Alice", "Madison Square");
final LWWRegister<Record> r2 = LWWRegister.create(node, record2);
final LWWRegister<Record> r3 = r1.merge(r2);
System.out.println(r3.value());
//#lwwregister-custom-clock
assertEquals("Madison Square", r3.value().address);
}
}

View file

@ -0,0 +1,516 @@
.. _distributed_data_java:
##################
Distributed Data
##################
*Akka Distributed Data* is useful when you need to share data between nodes in an
Akka Cluster. The data is accessed with an actor providing a key-value store like API.
The keys are unique identifiers with type information of the data values. The values
are *Conflict Free Replicated Data Types* (CRDTs).
All data entries are spread to all nodes, or nodes with a certain role, in the cluster
via direct replication and gossip based dissemination. You have fine grained control
of the consistency level for reads and writes.
The nature CRDTs makes it possible to perform updates from any node without coordination.
Concurrent updates from different nodes will automatically be resolved by the monotonic
merge function, which all data types must provide. The state changes always converge.
Several useful data types for counters, sets, maps and registers are provided and
you can also implement your own custom data types.
It is eventually consistent and geared toward providing high read and write availability
(partition tolerance), with low latency. Note that in an eventually consistent system a read may return an
out-of-date value.
Using the Replicator
====================
The ``akka.cluster.ddata.Replicator`` actor provides the API for interacting with the data.
The ``Replicator`` actor must be started on each node in the cluster, or group of nodes tagged
with a specific role. It communicates with other ``Replicator`` instances with the same path
(without address) that are running on other nodes . For convenience it can be used with the
``akka.cluster.ddata.DistributedData`` extension.
Below is an example of an actor that schedules tick messages to itself and for each tick
adds or removes elements from a ``ORSet`` (observed-remove set). It also subscribes to
changes of this.
.. includecode:: code/docs/ddata/DataBot.java#data-bot
.. _replicator_update_java:
Update
------
To modify and replicate a data value you send a ``Replicator.Update`` message to the the local
``Replicator``.
The current data value for the ``key`` of the ``Update`` is passed as parameter to the ``modify``
function of the ``Update``. The function is supposed to return the new value of the data, which
will then be replicated according to the given consistency level.
The ``modify`` function is called by the ``Replicator`` actor and must therefore be a pure
function that only uses the data parameter and stable fields from enclosing scope. It must
for example not access ``sender()`` reference of an enclosing actor.
``Update`` is intended to only be sent from an actor running in same local ``ActorSystem`` as
* the `Replicator`, because the `modify` function is typically not serializable.
You supply a write consistency level which has the following meaning:
* ``writeLocal`` the value will immediately only be written to the local replica,
and later disseminated with gossip
* ``writeTo(n)`` the value will immediately be written to at least ``n`` replicas,
including the local replica
* ``writeMajority`` the value will immediately be written to a majority of replicas, i.e.
at least **N/2 + 1** replicas, where N is the number of nodes in the cluster
(or cluster role group)
* ``writeAll`` the value will immediately be written to all nodes in the cluster
(or all nodes in the cluster role group)
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#update
As reply of the ``Update`` a ``Replicator.UpdateSuccess`` is sent to the sender of the
``Update`` if the value was successfully replicated according to the supplied consistency
level within the supplied timeout. Otherwise a ``Replicator.UpdateFailure`` subclass is
sent back. Note that a ``Replicator.UpdateTimeout`` reply does not mean that the update completely failed
or was rolled back. It may still have been replicated to some nodes, and will eventually
be replicated to all nodes with the gossip protocol.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#update-response1
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#update-response2
You will always see your own writes. For example if you send two ``Update`` messages
changing the value of the same ``key``, the ``modify`` function of the second message will
see the change that was performed by the first ``Update`` message.
In the ``Update`` message you can pass an optional request context, which the ``Replicator``
does not care about, but is included in the reply messages. This is a convenient
way to pass contextual information (e.g. original sender) without having to use ``ask``
or maintain local correlation data structures.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#update-request-context
.. _replicator_get_java:
Get
---
To retrieve the current value of a data you send ``Replicator.Get`` message to the
``Replicator``. You supply a consistency level which has the following meaning:
* ``readLocal`` the value will only be read from the local replica
* ``readFrom(n)`` the value will be read and merged from ``n`` replicas,
including the local replica
* ``readMajority`` the value will be read and merged from a majority of replicas, i.e.
at least **N/2 + 1** replicas, where N is the number of nodes in the cluster
(or cluster role group)
* ``readAll`` the value will be read and merged from all nodes in the cluster
(or all nodes in the cluster role group)
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#get
As reply of the ``Get`` a ``Replicator.GetSuccess`` is sent to the sender of the
``Get`` if the value was successfully retrieved according to the supplied consistency
level within the supplied timeout. Otherwise a ``Replicator.GetFailure`` is sent.
If the key does not exist the reply will be ``Replicator.NotFound``.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#get-response1
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#get-response2
You will always read your own writes. For example if you send a ``Update`` message
followed by a ``Get`` of the same ``key`` the ``Get`` will retrieve the change that was
performed by the preceding ``Update`` message. However, the order of the reply messages are
not defined, i.e. in the previous example you may receive the ``GetSuccess`` before
the ``UpdateSuccess``.
In the ``Get`` message you can pass an optional request context in the same way as for the
``Update`` message, described above. For example the original sender can be passed and replied
to after receiving and transforming ``GetSuccess``.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#get-request-context
Consistency
-----------
The consistency level that is supplied in the :ref:`replicator_update_java` and :ref:`replicator_get_java`
specifies per request how many replicas that must respond successfully to a write and read request.
For low latency reads you use ``ReadLocal`` with the risk of retrieving stale data, i.e. updates
from other nodes might not be visible yet.
When using ``writeLocal`` the update is only written to the local replica and then disseminated
in the background with the gossip protocol, which can take few seconds to spread to all nodes.
``writeAll`` and ``readAll`` is the strongest consistency level, but also the slowest and with
lowest availability. For example, it is enough that one node is unavailable for a ``Get`` request
and you will not receive the value.
If consistency is important, you can ensure that a read always reflects the most recent
write by using the following formula::
(nodes_written + nodes_read) > N
where N is the total number of nodes in the cluster, or the number of nodes with the role that is
used for the ``Replicator``.
For example, in a 7 node cluster this these consistency properties are achieved by writing to 4 nodes
and reading from 4 nodes, or writing to 5 nodes and reading from 3 nodes.
By combining ``writeMajority`` and ``readMajority`` levels a read always reflects the most recent write.
The ``Replicator`` writes and reads to a majority of replicas, i.e. **N / 2 + 1**. For example,
in a 5 node cluster it writes to 3 nodes and reads from 3 nodes. In a 6 node cluster it writes
to 4 nodes and reads from 4 nodes.
Here is an example of using ``writeMajority`` and ``readMajority``:
**FIXME convert this example to Java**
.. includecode:: ../../../akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala#read-write-majority
.. includecode:: ../../../akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala#get-cart
.. includecode:: ../../../akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala#add-item
In some rare cases, when performing an ``Update`` it is needed to first try to fetch latest data from
other nodes. That can be done by first sending a ``Get`` with ``ReadMajority`` and then continue with
the ``Update`` when the ``GetSuccess``, ``GetFailure`` or ``NotFound`` reply is received. This might be
needed when you need to base a decision on latest information or when removing entries from ``ORSet``
or ``ORMap``. If an entry is added to an ``ORSet`` or ``ORMap`` from one node and removed from another
node the entry will only be removed if the added entry is visible on the node where the removal is
performed (hence the name observed-removed set).
The following example illustrates how to do that:
**FIXME convert this example to Java**
.. includecode:: ../../../akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala#remove-item
.. warning::
*Caveat:* Even if you use ``writeMajority`` and ``readMajority`` there is small risk that you may
read stale data if the cluster membership has changed between the ``Update`` and the ``Get``.
For example, in cluster of 5 nodes when you ``Update`` and that change is written to 3 nodes:
n1, n2, n3. Then 2 more nodes are added and a ``Get`` request is reading from 4 nodes, which
happens to be n4, n5, n6, n7, i.e. the value on n1, n2, n3 is not seen in the response of the
``Get`` request.
Subscribe
---------
You may also register interest in change notifications by sending ``Replicator.Subscribe``
message to the ``Replicator``. It will send ``Replicator.Changed`` messages to the registered
subscriber when the data for the subscribed key is updated. Subscribers will be notified
periodically with the configured ``notify-subscribers-interval``, and it is also possible to
send an explicit ``Replicator.FlushChanges`` message to the ``Replicator`` to notify the subscribers
immediately.
The subscriber is automatically removed if the subscriber is terminated. A subscriber can
also be deregistered with the ``Replicator.Unsubscribe`` message.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#subscribe
Delete
------
A data entry can be deleted by sending a ``Replicator.Delete`` message to the local
local ``Replicator``. As reply of the ``Delete`` a ``Replicator.DeleteSuccess`` is sent to
the sender of the ``Delete`` if the value was successfully deleted according to the supplied
consistency level within the supplied timeout. Otherwise a ``Replicator.ReplicationDeleteFailure``
is sent. Note that ``ReplicationDeleteFailure`` does not mean that the delete completely failed or
was rolled back. It may still have been replicated to some nodes, and may eventually be replicated
to all nodes.
A deleted key cannot be reused again, but it is still recommended to delete unused
data entries because that reduces the replication overhead when new nodes join the cluster.
Subsequent ``Delete``, ``Update`` and ``Get`` requests will be replied with ``Replicator.DataDeleted``.
Subscribers will receive ``Replicator.DataDeleted``.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#delete
Data Types
==========
The data types must be convergent (stateful) CRDTs and implement the ``ReplicatedData`` trait,
i.e. they provide a monotonic merge function and the state changes always converge.
You can use your own custom ``ReplicatedData`` types, and several types are provided
by this package, such as:
* Counters: ``GCounter``, ``PNCounter``
* Sets: ``GSet``, ``ORSet``
* Maps: ``ORMap``, ``LWWMap``, ``PNCounterMap``
* Registers: ``LWWRegister``, ``Flag``
Counters
--------
``GCounter`` is a "grow only counter". It only supports increments, no decrements.
It works in a similar way as a vector clock. It keeps track of one counter per node and the total
value is the sum of these counters. The ``merge`` is implemented by taking the maximum count for
each node.
If you need both increments and decrements you can use the ``PNCounter`` (positive/negative counter).
It is tracking the increments (P) separate from the decrements (N). Both P and N are represented
as two internal ``GCounter``. Merge is handled by merging the internal P and N counters.
The value of the counter is the value of the P counter minus the value of the N counter.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#pncounter
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
values they are guaranteed to be replicated together as one unit, which is sometimes necessary for
related data.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#pncountermap
Sets
----
If you only need to add elements to a set and not remove elements the ``GSet`` (grow-only set) is
the data type to use. The elements can be any type of values that can be serialized.
Merge is simply the union of the two sets.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#gset
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
removed, the add will win. You cannot remove an element that you have not seen.
The ``ORSet`` has a version vector that is incremented when an element is added to the set.
The version for the node that added the element is also tracked for each element in a so
called "birth dot". The version vector and the dots are used by the ``merge`` function to
track causality of the operations and resolve concurrent updates.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#orset
Maps
----
``ORMap`` (observed-remove map) is a map with ``String`` keys and the values are ``ReplicatedData``
types themselves. It supports add, remove and delete any number of times for a map entry.
If an entry is concurrently added and removed, the add will win. You cannot remove an entry that
you have not seen. This is the same semantics as for the ``ORSet``.
If an entry is concurrently updated to different values the values will be merged, hence the
requirement that the values must be ``ReplicatedData`` types.
It is rather inconvenient to use the ``ORMap`` directly since it does not expose specific types
of the values. The ``ORMap`` is intended as a low level tool for building more specific maps,
such as the following specialized maps.
``PNCounterMap`` (positive negative counter map) is a map of named counters. It is a specialized
``ORMap`` with ``PNCounter`` values.
``LWWMap`` (last writer wins map) is a specialized ``ORMap`` with ``LWWRegister`` (last writer wins register)
values.
Note that ``LWWRegister`` and therefore ``LWWMap`` relies on synchronized clocks and should only be used
when the choice of value is not important for concurrent updates occurring within the clock skew.
Instead of using timestamps based on ``System.currentTimeMillis()`` time it is possible to
use a timestamp value based on something else, for example an increasing version number
from a database record that is used for optimistic concurrency control.
When a data entry is changed the full state of that entry is replicated to other nodes, i.e.
when you update an map the whole map is replicated. Therefore, instead of using one ``ORMap``
with 1000 elements it is more efficient to split that up in 10 top level ``ORMap`` entries
with 100 elements each. Top level entries are replicated individually, which has the
trade-off that different entries may not be replicated at the same time and you may see
inconsistencies between related entries. Separate top level entries cannot be updated atomically
together.
Flags and Registers
-------------------
``Flag`` is a data type for a boolean value that is initialized to ``false`` and can be switched
to ``true``. Thereafter it cannot be changed. ``true`` wins over ``false`` in merge.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#flag
``LWWRegister`` (last writer wins register) can hold any (serializable) value.
Merge of a ``LWWRegister`` takes the the register with highest timestamp. Note that this
relies on synchronized clocks. `LWWRegister` should only be used when the choice of
value is not important for concurrent updates occurring within the clock skew.
Merge takes the register updated by the node with lowest address (``UniqueAddress`` is ordered)
if the timestamps are exactly the same.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#lwwregister
Instead of using timestamps based on ``System.currentTimeMillis()`` time it is possible to
use a timestamp value based on something else, for example an increasing version number
from a database record that is used for optimistic concurrency control.
.. includecode:: code/docs/ddata/DistributedDataDocTest.java#lwwregister-custom-clock
For first-write-wins semantics you can use the ``LWWRegister#reverseClock`` instead of the
``LWWRegister#defaultClock``.
Custom Data Type
----------------
You can rather easily implement your own data types. The only requirement is that it implements
the ``merge`` function of the ``AbstractReplicatedData`` class.
A nice property of stateful CRDTs is that they typically compose nicely, i.e. you can combine several
smaller data types to build richer data structures. For example, the ``PNCounter`` is composed of
two internal ``GCounter`` instances to keep track of increments and decrements separately.
Here is s simple implementation of a custom ``TwoPhaseSet`` that is using two internal ``GSet`` types
to keep track of addition and removals. A ``TwoPhaseSet`` is a set where an element may be added and
removed, but never added again thereafter.
**FIXME convert this example to Java**
.. includecode:: ../scala/code/docs/ddata/TwoPhaseSet.scala#twophaseset
Data types should be immutable, i.e. "modifying" methods should return a new instance.
Serialization
^^^^^^^^^^^^^
The data types must be serializable with an :ref:`Akka Serializer <serialization-java>`.
It is highly recommended that you implement efficient serialization with Protobuf or similar
for your custom data types. The built in data types are marked with ``ReplicatedDataSerialization``
and serialized with ``akka.cluster.ddata.protobuf.ReplicatedDataSerializer``.
Serialization of the data types are used in remote messages and also for creating message
digests (SHA-1) to detect changes. Therefore it is important that the serialization is efficient
and produce the same bytes for the same content. For example sets and maps should be sorted
deterministically in the serialization.
This is a protobuf representation of the above ``TwoPhaseSet``:
.. includecode:: ../../src/main/protobuf/TwoPhaseSetMessages.proto#twophaseset
The serializer for the ``TwoPhaseSet``:
**FIXME convert this example to Java**
.. includecode:: ../scala/code/docs/ddata/protobuf/TwoPhaseSetSerializer.scala#serializer
Note that the elements of the sets are sorted so the SHA-1 digests are the same
for the same elements.
You register the serializer in configuration:
.. includecode:: ../scala/code/docs/ddata/DistributedDataDocSpec.scala#serializer-config
Using compression can sometimes be a good idea to reduce the data size. Gzip compression is
provided by the ``akka.cluster.ddata.protobuf.SerializationSupport`` trait:
**FIXME convert this example to Java**
.. includecode:: ../scala/code/docs/ddata/protobuf/TwoPhaseSetSerializer.scala#compression
The two embedded ``GSet`` can be serialized as illustrated above, but in general when composing
new data types from the existing built in types it is better to make use of the existing
serializer for those types. This can be done by declaring those as bytes fields in protobuf:
.. includecode:: ../../src/main/protobuf/TwoPhaseSetMessages.proto#twophaseset2
and use the methods ``otherMessageToProto`` and ``otherMessageFromBinary`` that are provided
by the ``SerializationSupport`` trait to serialize and deserialize the ``GSet`` instances. This
works with any type that has a registered Akka serializer. This is how such an serializer would
look like for the ``TwoPhaseSet``:
**FIXME convert this example to Java**
.. includecode:: ../scala/code/docs/ddata/protobuf/TwoPhaseSetSerializer2.scala#serializer
CRDT Garbage
------------
One thing that can be problematic with CRDTs is that some data types accumulate history (garbage).
For example a ``GCounter`` keeps track of one counter per node. If a ``GCounter`` has been updated
from one node it will associate the identifier of that node forever. That can become a problem
for long running systems with many cluster nodes being added and removed. To solve this problem
the ``Replicator`` performs pruning of data associated with nodes that have been removed from the
cluster. Data types that need pruning have to implement the ``RemovedNodePruning`` trait.
Samples
=======
**FIXME convert these sampes to Java and activator template**
* `Replicated Cache <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala>`_
* `Replicated Metrics <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala>`_
* `Replicated Service Registry <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedServiceRegistrySpec.scala>`_
* `VotingService <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/VotingContestSpec.scala>`_
* `ShoppingCart <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala>`_
Limitations
===========
There are some limitations that you should be aware of.
CRDTs cannot be used for all types of problems, and eventual consistency does not fit
all domains. Sometimes you need strong consistency.
It is not intended for *Big Data*. The number of top level entries should not exceed 100000.
When a new node is added to the cluster all these entries are transferred (gossiped) to the
new node. The entries are split up in chunks and all existing nodes collaborate in the gossip,
but it will take a while (tens of seconds) to transfer all entries and this means that you
cannot have too many top level entries. The current recommended limit is 100000. We will
be able to improve this if needed, but the design is still not intended for billions of entries.
All data is held in memory, which is another reason why it is not intended for *Big Data*.
When a data entry is changed the full state of that entry is replicated to other nodes. For example,
if you add one element to a Set with 100 existing elements, all 101 elements are transferred to
other nodes. This means that you cannot have too large data entries, because then the remote message
size will be too large. We might be able to make this more efficient by implementing
`Efficient State-based CRDTs by Delta-Mutation <http://gsd.di.uminho.pt/members/cbm/ps/delta-crdt-draft16may2014.pdf>`_.
The data is only kept in memory. It is redundant since it is replicated to other nodes
in the cluster, but if you stop all nodes the data is lost, unless you have saved it
elsewhere. Making the data durable is a possible future feature, but even if we implement that
it is not intended to be a full featured database.
Learn More about CRDTs
======================
* `The Final Causal Frontier <http://www.ustream.tv/recorded/61448875>`_
talk by Sean Cribbs
* `Eventually Consistent Data Structures <https://vimeo.com/43903960>`_
talk by Sean Cribbs
* `Strong Eventual Consistency and Conflict-free Replicated Data Types <http://research.microsoft.com/apps/video/default.aspx?id=153540&r=1>`_
talk by Mark Shapiro
* `A comprehensive study of Convergent and Commutative Replicated Data Types <http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf>`_
paper by Mark Shapiro et. al.
Dependencies
------------
To use Distributed Data you must add the following dependency in your project.
sbt::
"com.typesafe.akka" %% "akka-distributed-data" % "@version@" @crossString@
maven::
<dependency>
<groupId>com.typesafe.akka</groupId>
<artifactId>akka-distributed-data_@binVersion@</artifactId>
<version>@version@</version>
</dependency>
Configuration
=============
The ``DistributedData`` extension can be configured with the following properties:
.. includecode:: ../../../akka-distributed-data/src/main/resources/reference.conf#distributed-data

View file

@ -11,6 +11,7 @@ Networking
../scala/cluster-client
../scala/cluster-sharding
cluster-metrics
distributed-data
remoting
serialization
io

View file

@ -0,0 +1,381 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.ddata
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.cluster.Cluster
import akka.cluster.ddata._
import akka.cluster.ddata.Replicator._
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.testkit.TestProbe
import akka.actor.ActorRef
import akka.serialization.SerializationExtension
object DistributedDataDocSpec {
val config =
"""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.netty.tcp.port = 0
#//#serializer-config
akka.actor {
serializers {
two-phase-set = "docs.ddata.protobuf.TwoPhaseSetSerializer"
}
serialization-bindings {
"docs.ddata.TwoPhaseSet" = two-phase-set
}
}
#//#serializer-config
"""
//#data-bot
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.cluster.Cluster
import akka.cluster.ddata.DistributedData
import akka.cluster.ddata.ORSet
import akka.cluster.ddata.ORSetKey
import akka.cluster.ddata.Replicator
import akka.cluster.ddata.Replicator._
object DataBot {
private case object Tick
}
class DataBot extends Actor with ActorLogging {
import DataBot._
val replicator = DistributedData(context.system).replicator
implicit val node = Cluster(context.system)
import context.dispatcher
val tickTask = context.system.scheduler.schedule(5.seconds, 5.seconds, self, Tick)
val DataKey = ORSetKey[String]("key")
replicator ! Subscribe(DataKey, self)
def receive = {
case Tick =>
val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString
if (ThreadLocalRandom.current().nextBoolean()) {
// add
log.info("Adding: {}", s)
replicator ! Update(DataKey, ORSet.empty[String], WriteLocal)(_ + s)
} else {
// remove
log.info("Removing: {}", s)
replicator ! Update(DataKey, ORSet.empty[String], WriteLocal)(_ - s)
}
case _: UpdateResponse[_] => // ignore
case c @ Changed(DataKey) =>
val data = c.get(DataKey)
log.info("Current elements: {}", data.elements)
}
override def postStop(): Unit = tickTask.cancel()
}
//#data-bot
}
class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
import Replicator._
import DistributedDataDocSpec._
"demonstrate update" in {
val probe = TestProbe()
implicit val self = probe.ref
//#update
implicit val node = Cluster(system)
val replicator = DistributedData(system).replicator
val Counter1Key = PNCounterKey("counter1")
val Set1Key = GSetKey[String]("set1")
val Set2Key = ORSetKey[String]("set2")
val ActiveFlagKey = FlagKey("active")
replicator ! Update(Counter1Key, PNCounter(), WriteLocal)(_ + 1)
val writeTo3 = WriteTo(n = 3, timeout = 1.second)
replicator ! Update(Set1Key, GSet.empty[String], writeTo3)(_ + "hello")
val writeMajority = WriteMajority(timeout = 5.seconds)
replicator ! Update(Set2Key, ORSet.empty[String], writeMajority)(_ + "hello")
val writeAll = WriteAll(timeout = 5.seconds)
replicator ! Update(ActiveFlagKey, Flag.empty, writeAll)(_.switchOn)
//#update
probe.expectMsgType[UpdateResponse[_]] match {
//#update-response1
case UpdateSuccess(Counter1Key, req) => // ok
//#update-response1
case unexpected => fail("Unexpected response: " + unexpected)
}
probe.expectMsgType[UpdateResponse[_]] match {
//#update-response2
case UpdateSuccess(Set1Key, req) => // ok
case UpdateTimeout(Set1Key, req) =>
// write to 3 nodes failed within 1.second
//#update-response2
case UpdateSuccess(Set2Key, None) =>
case unexpected => fail("Unexpected response: " + unexpected)
}
}
"demonstrate update with request context" in {
import Actor.Receive
val probe = TestProbe()
implicit val self = probe.ref
def sender() = self
//#update-request-context
implicit val node = Cluster(system)
val replicator = DistributedData(system).replicator
val writeTwo = WriteTo(n = 2, timeout = 3.second)
val Counter1Key = PNCounterKey("counter1")
def receive: Receive = {
case "increment" =>
// incoming command to increase the counter
val upd = Update(Counter1Key, PNCounter(), writeTwo, request = Some(sender()))(_ + 1)
replicator ! upd
case UpdateSuccess(Counter1Key, Some(replyTo: ActorRef)) =>
replyTo ! "ack"
case UpdateTimeout(Counter1Key, Some(replyTo: ActorRef)) =>
replyTo ! "nack"
}
//#update-request-context
}
"demonstrate get" in {
val probe = TestProbe()
implicit val self = probe.ref
//#get
val replicator = DistributedData(system).replicator
val Counter1Key = PNCounterKey("counter1")
val Set1Key = GSetKey[String]("set1")
val Set2Key = ORSetKey[String]("set2")
val ActiveFlagKey = FlagKey("active")
replicator ! Get(Counter1Key, ReadLocal)
val readFrom3 = ReadFrom(n = 3, timeout = 1.second)
replicator ! Get(Set1Key, readFrom3)
val readMajority = ReadMajority(timeout = 5.seconds)
replicator ! Get(Set2Key, readMajority)
val readAll = ReadAll(timeout = 5.seconds)
replicator ! Get(ActiveFlagKey, readAll)
//#get
probe.expectMsgType[GetResponse[_]] match {
//#get-response1
case g @ GetSuccess(Counter1Key, req) =>
val value = g.get(Counter1Key).value
case NotFound(Counter1Key, req) => // key counter1 does not exist
//#get-response1
case unexpected => fail("Unexpected response: " + unexpected)
}
probe.expectMsgType[GetResponse[_]] match {
//#get-response2
case g @ GetSuccess(Set1Key, req) =>
val elements = g.get(Set1Key).elements
case GetFailure(Set1Key, req) =>
// read from 3 nodes failed within 1.second
case NotFound(Set1Key, req) => // key set1 does not exist
//#get-response2
case g @ GetSuccess(Set2Key, None) =>
val elements = g.get(Set2Key).elements
case unexpected => fail("Unexpected response: " + unexpected)
}
}
"demonstrate get with request context" in {
import Actor.Receive
val probe = TestProbe()
implicit val self = probe.ref
def sender() = self
//#get-request-context
implicit val node = Cluster(system)
val replicator = DistributedData(system).replicator
val readTwo = ReadFrom(n = 2, timeout = 3.second)
val Counter1Key = PNCounterKey("counter1")
def receive: Receive = {
case "get-count" =>
// incoming request to retrieve current value of the counter
replicator ! Get(Counter1Key, readTwo, request = Some(sender()))
case g @ GetSuccess(Counter1Key, Some(replyTo: ActorRef)) =>
val value = g.get(Counter1Key).value.longValue
replyTo ! value
case GetFailure(Counter1Key, Some(replyTo: ActorRef)) =>
replyTo ! -1L
case NotFound(Counter1Key, Some(replyTo: ActorRef)) =>
replyTo ! 0L
}
//#get-request-context
}
"demonstrate subscribe" in {
import Actor.Receive
val probe = TestProbe()
implicit val self = probe.ref
def sender() = self
//#subscribe
val replicator = DistributedData(system).replicator
val Counter1Key = PNCounterKey("counter1")
// subscribe to changes of the Counter1Key value
replicator ! Subscribe(Counter1Key, self)
var currentValue = BigInt(0)
def receive: Receive = {
case c @ Changed(Counter1Key) =>
currentValue = c.get(Counter1Key).value
case "get-count" =>
// incoming request to retrieve current value of the counter
sender() ! currentValue
}
//#subscribe
}
"demonstrate delete" in {
val probe = TestProbe()
implicit val self = probe.ref
//#delete
val replicator = DistributedData(system).replicator
val Counter1Key = PNCounterKey("counter1")
val Set2Key = ORSetKey[String]("set2")
replicator ! Delete(Counter1Key, WriteLocal)
val writeMajority = WriteMajority(timeout = 5.seconds)
replicator ! Delete(Set2Key, writeMajority)
//#delete
}
"demonstrate PNCounter" in {
def println(o: Any): Unit = ()
//#pncounter
implicit val node = Cluster(system)
val c0 = PNCounter.empty
val c1 = c0 + 1
val c2 = c1 + 7
val c3: PNCounter = c2 - 2
println(c3.value) // 6
//#pncounter
}
"demonstrate PNCounterMap" in {
def println(o: Any): Unit = ()
//#pncountermap
implicit val node = Cluster(system)
val m0 = PNCounterMap.empty
val m1 = m0.increment("a", 7)
val m2 = m1.decrement("a", 2)
val m3 = m2.increment("b", 1)
println(m3.get("a")) // 5
m3.entries.foreach { case (key, value) => println(s"$key -> $value") }
//#pncountermap
}
"demonstrate GSet" in {
def println(o: Any): Unit = ()
//#gset
val s0 = GSet.empty[String]
val s1 = s0 + "a"
val s2 = s1 + "b" + "c"
if (s2.contains("a"))
println(s2.elements) // a, b, c
//#gset
}
"demonstrate ORSet" in {
def println(o: Any): Unit = ()
//#orset
implicit val node = Cluster(system)
val s0 = ORSet.empty[String]
val s1 = s0 + "a"
val s2 = s1 + "b"
val s3 = s2 - "a"
println(s3.elements) // b
//#orset
}
"demonstrate Flag" in {
def println(o: Any): Unit = ()
//#flag
val f0 = Flag.empty
val f1 = f0.switchOn
println(f1.enabled)
//#flag
}
"demonstrate LWWRegister" in {
def println(o: Any): Unit = ()
//#lwwregister
implicit val node = Cluster(system)
val r1 = LWWRegister("Hello")
val r2 = r1.withValue("Hi")
println(s"${r1.value} by ${r1.updatedBy} at ${r1.timestamp}")
//#lwwregister
r2.value should be("Hi")
}
"demonstrate LWWRegister with custom clock" in {
def println(o: Any): Unit = ()
//#lwwregister-custom-clock
case class Record(version: Int, name: String, address: String)
implicit val node = Cluster(system)
implicit val recordClock = new LWWRegister.Clock[Record] {
override def apply(currentTimestamp: Long, value: Record): Long =
value.version
}
val record1 = Record(version = 1, "Alice", "Union Square")
val r1 = LWWRegister(record1)
val record2 = Record(version = 2, "Alice", "Madison Square")
val r2 = LWWRegister(record2)
val r3 = r1.merge(r2)
println(r3.value)
//#lwwregister-custom-clock
r3.value.address should be("Madison Square")
}
"test TwoPhaseSetSerializer" in {
val s1 = TwoPhaseSet().add("a").add("b").add("c").remove("b")
s1.elements should be(Set("a", "c"))
val serializer = SerializationExtension(system).findSerializerFor(s1)
val blob = serializer.toBinary(s1)
val s2 = serializer.fromBinary(blob, None)
s1 should be(s1)
}
}

View file

@ -0,0 +1,29 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.ddata
import akka.cluster.ddata.ReplicatedData
import akka.cluster.ddata.GSet
//#twophaseset
case class TwoPhaseSet(
adds: GSet[String] = GSet.empty,
removals: GSet[String] = GSet.empty)
extends ReplicatedData {
type T = TwoPhaseSet
def add(element: String): TwoPhaseSet =
copy(adds = adds.add(element))
def remove(element: String): TwoPhaseSet =
copy(removals = removals.add(element))
def elements: Set[String] = adds.elements -- removals.elements
override def merge(that: TwoPhaseSet): TwoPhaseSet =
copy(
adds = GSet(this.adds.elements ++ that.adds.elements),
removals = GSet(this.removals.elements ++ that.removals.elements))
}
//#twophaseset

View file

@ -0,0 +1,75 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.ddata.protobuf
//#serializer
import java.util.ArrayList
import java.util.Collections
import scala.collection.JavaConverters._
import akka.actor.ExtendedActorSystem
import akka.cluster.ddata.GSet
import akka.cluster.ddata.protobuf.SerializationSupport
import akka.serialization.Serializer
import docs.ddata.TwoPhaseSet
import docs.ddata.protobuf.msg.TwoPhaseSetMessages
class TwoPhaseSetSerializer(val system: ExtendedActorSystem)
extends Serializer with SerializationSupport {
override def includeManifest: Boolean = false
override def identifier = 99999
override def toBinary(obj: AnyRef): Array[Byte] = obj match {
case m: TwoPhaseSet twoPhaseSetToProto(m).toByteArray
case _ throw new IllegalArgumentException(
s"Can't serialize object of type ${obj.getClass}")
}
override def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
twoPhaseSetFromBinary(bytes)
}
def twoPhaseSetToProto(twoPhaseSet: TwoPhaseSet): TwoPhaseSetMessages.TwoPhaseSet = {
val b = TwoPhaseSetMessages.TwoPhaseSet.newBuilder()
// using java collections and sorting for performance (avoid conversions)
val adds = new ArrayList[String]
twoPhaseSet.adds.elements.foreach(adds.add)
if (!adds.isEmpty) {
Collections.sort(adds)
b.addAllAdds(adds)
}
val removals = new ArrayList[String]
twoPhaseSet.removals.elements.foreach(removals.add)
if (!removals.isEmpty) {
Collections.sort(removals)
b.addAllRemovals(removals)
}
b.build()
}
def twoPhaseSetFromBinary(bytes: Array[Byte]): TwoPhaseSet = {
val msg = TwoPhaseSetMessages.TwoPhaseSet.parseFrom(bytes)
TwoPhaseSet(
adds = GSet(msg.getAddsList.iterator.asScala.toSet),
removals = GSet(msg.getRemovalsList.iterator.asScala.toSet))
}
}
//#serializer
class TwoPhaseSetSerializerWithCompression(system: ExtendedActorSystem)
extends TwoPhaseSetSerializer(system) {
//#compression
override def toBinary(obj: AnyRef): Array[Byte] = obj match {
case m: TwoPhaseSet compress(twoPhaseSetToProto(m))
case _ throw new IllegalArgumentException(
s"Can't serialize object of type ${obj.getClass}")
}
override def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
twoPhaseSetFromBinary(decompress(bytes))
}
//#compression
}

View file

@ -0,0 +1,59 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.ddata.protobuf
//#serializer
import scala.collection.JavaConverters._
import akka.actor.ExtendedActorSystem
import akka.cluster.ddata.GSet
import akka.cluster.ddata.protobuf.ReplicatedDataSerializer
import akka.cluster.ddata.protobuf.SerializationSupport
import akka.serialization.Serializer
import docs.ddata.TwoPhaseSet
import docs.ddata.protobuf.msg.TwoPhaseSetMessages
class TwoPhaseSetSerializer2(val system: ExtendedActorSystem)
extends Serializer with SerializationSupport {
override def includeManifest: Boolean = false
override def identifier = 99999
val replicatedDataSerializer = new ReplicatedDataSerializer(system)
override def toBinary(obj: AnyRef): Array[Byte] = obj match {
case m: TwoPhaseSet twoPhaseSetToProto(m).toByteArray
case _ throw new IllegalArgumentException(
s"Can't serialize object of type ${obj.getClass}")
}
override def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
twoPhaseSetFromBinary(bytes)
}
def twoPhaseSetToProto(twoPhaseSet: TwoPhaseSet): TwoPhaseSetMessages.TwoPhaseSet2 = {
val b = TwoPhaseSetMessages.TwoPhaseSet2.newBuilder()
if (!twoPhaseSet.adds.isEmpty)
b.setAdds(otherMessageToProto(twoPhaseSet.adds).toByteString())
if (!twoPhaseSet.removals.isEmpty)
b.setRemovals(otherMessageToProto(twoPhaseSet.removals).toByteString())
b.build()
}
def twoPhaseSetFromBinary(bytes: Array[Byte]): TwoPhaseSet = {
val msg = TwoPhaseSetMessages.TwoPhaseSet2.parseFrom(bytes)
val adds =
if (msg.hasAdds)
otherMessageFromBinary(msg.getAdds.toByteArray).asInstanceOf[GSet[String]]
else
GSet.empty[String]
val removals =
if (msg.hasRemovals)
otherMessageFromBinary(msg.getRemovals.toByteArray).asInstanceOf[GSet[String]]
else
GSet.empty[String]
TwoPhaseSet(adds, removals)
}
}
//#serializer

View file

@ -0,0 +1,504 @@
.. _distributed_data_scala:
##################
Distributed Data
##################
*Akka Distributed Data* is useful when you need to share data between nodes in an
Akka Cluster. The data is accessed with an actor providing a key-value store like API.
The keys are unique identifiers with type information of the data values. The values
are *Conflict Free Replicated Data Types* (CRDTs).
All data entries are spread to all nodes, or nodes with a certain role, in the cluster
via direct replication and gossip based dissemination. You have fine grained control
of the consistency level for reads and writes.
The nature CRDTs makes it possible to perform updates from any node without coordination.
Concurrent updates from different nodes will automatically be resolved by the monotonic
merge function, which all data types must provide. The state changes always converge.
Several useful data types for counters, sets, maps and registers are provided and
you can also implement your own custom data types.
It is eventually consistent and geared toward providing high read and write availability
(partition tolerance), with low latency. Note that in an eventually consistent system a read may return an
out-of-date value.
Using the Replicator
====================
The ``akka.cluster.ddata.Replicator`` actor provides the API for interacting with the data.
The ``Replicator`` actor must be started on each node in the cluster, or group of nodes tagged
with a specific role. It communicates with other ``Replicator`` instances with the same path
(without address) that are running on other nodes . For convenience it can be used with the
``akka.cluster.ddata.DistributedData`` extension.
Below is an example of an actor that schedules tick messages to itself and for each tick
adds or removes elements from a ``ORSet`` (observed-remove set). It also subscribes to
changes of this.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#data-bot
.. _replicator_update_scala:
Update
------
To modify and replicate a data value you send a ``Replicator.Update`` message to the the local
``Replicator``.
The current data value for the ``key`` of the ``Update`` is passed as parameter to the ``modify``
function of the ``Update``. The function is supposed to return the new value of the data, which
will then be replicated according to the given consistency level.
The ``modify`` function is called by the ``Replicator`` actor and must therefore be a pure
function that only uses the data parameter and stable fields from enclosing scope. It must
for example not access ``sender()`` reference of an enclosing actor.
``Update`` is intended to only be sent from an actor running in same local ``ActorSystem`` as
* the `Replicator`, because the `modify` function is typically not serializable.
You supply a write consistency level which has the following meaning:
* ``WriteLocal`` the value will immediately only be written to the local replica,
and later disseminated with gossip
* ``WriteTo(n)`` the value will immediately be written to at least ``n`` replicas,
including the local replica
* ``WriteMajority`` the value will immediately be written to a majority of replicas, i.e.
at least **N/2 + 1** replicas, where N is the number of nodes in the cluster
(or cluster role group)
* ``WriteAll`` the value will immediately be written to all nodes in the cluster
(or all nodes in the cluster role group)
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#update
As reply of the ``Update`` a ``Replicator.UpdateSuccess`` is sent to the sender of the
``Update`` if the value was successfully replicated according to the supplied consistency
level within the supplied timeout. Otherwise a ``Replicator.UpdateFailure`` subclass is
sent back. Note that a ``Replicator.UpdateTimeout`` reply does not mean that the update completely failed
or was rolled back. It may still have been replicated to some nodes, and will eventually
be replicated to all nodes with the gossip protocol.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#update-response1
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#update-response2
You will always see your own writes. For example if you send two ``Update`` messages
changing the value of the same ``key``, the ``modify`` function of the second message will
see the change that was performed by the first ``Update`` message.
In the ``Update`` message you can pass an optional request context, which the ``Replicator``
does not care about, but is included in the reply messages. This is a convenient
way to pass contextual information (e.g. original sender) without having to use ``ask``
or maintain local correlation data structures.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#update-request-context
.. _replicator_get_scala:
Get
---
To retrieve the current value of a data you send ``Replicator.Get`` message to the
``Replicator``. You supply a consistency level which has the following meaning:
* ``ReadLocal`` the value will only be read from the local replica
* ``ReadFrom(n)`` the value will be read and merged from ``n`` replicas,
including the local replica
* ``ReadMajority`` the value will be read and merged from a majority of replicas, i.e.
at least **N/2 + 1** replicas, where N is the number of nodes in the cluster
(or cluster role group)
* ``ReadAll`` the value will be read and merged from all nodes in the cluster
(or all nodes in the cluster role group)
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#get
As reply of the ``Get`` a ``Replicator.GetSuccess`` is sent to the sender of the
``Get`` if the value was successfully retrieved according to the supplied consistency
level within the supplied timeout. Otherwise a ``Replicator.GetFailure`` is sent.
If the key does not exist the reply will be ``Replicator.NotFound``.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#get-response1
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#get-response2
You will always read your own writes. For example if you send a ``Update`` message
followed by a ``Get`` of the same ``key`` the ``Get`` will retrieve the change that was
performed by the preceding ``Update`` message. However, the order of the reply messages are
not defined, i.e. in the previous example you may receive the ``GetSuccess`` before
the ``UpdateSuccess``.
In the ``Get`` message you can pass an optional request context in the same way as for the
``Update`` message, described above. For example the original sender can be passed and replied
to after receiving and transforming ``GetSuccess``.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#get-request-context
Consistency
-----------
The consistency level that is supplied in the :ref:`replicator_update_scala` and :ref:`replicator_get_scala`
specifies per request how many replicas that must respond successfully to a write and read request.
For low latency reads you use ``ReadLocal`` with the risk of retrieving stale data, i.e. updates
from other nodes might not be visible yet.
When using ``WriteLocal`` the update is only written to the local replica and then disseminated
in the background with the gossip protocol, which can take few seconds to spread to all nodes.
``WriteAll`` and ``ReadAll`` is the strongest consistency level, but also the slowest and with
lowest availability. For example, it is enough that one node is unavailable for a ``Get`` request
and you will not receive the value.
If consistency is important, you can ensure that a read always reflects the most recent
write by using the following formula::
(nodes_written + nodes_read) > N
where N is the total number of nodes in the cluster, or the number of nodes with the role that is
used for the ``Replicator``.
For example, in a 7 node cluster this these consistency properties are achieved by writing to 4 nodes
and reading from 4 nodes, or writing to 5 nodes and reading from 3 nodes.
By combining ``WriteMajority`` and ``ReadMajority`` levels a read always reflects the most recent write.
The ``Replicator`` writes and reads to a majority of replicas, i.e. **N / 2 + 1**. For example,
in a 5 node cluster it writes to 3 nodes and reads from 3 nodes. In a 6 node cluster it writes
to 4 nodes and reads from 4 nodes.
Here is an example of using ``WriteMajority`` and ``ReadMajority``:
.. includecode:: ../../../akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala#read-write-majority
.. includecode:: ../../../akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala#get-cart
.. includecode:: ../../../akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala#add-item
In some rare cases, when performing an ``Update`` it is needed to first try to fetch latest data from
other nodes. That can be done by first sending a ``Get`` with ``ReadMajority`` and then continue with
the ``Update`` when the ``GetSuccess``, ``GetFailure`` or ``NotFound`` reply is received. This might be
needed when you need to base a decision on latest information or when removing entries from ``ORSet``
or ``ORMap``. If an entry is added to an ``ORSet`` or ``ORMap`` from one node and removed from another
node the entry will only be removed if the added entry is visible on the node where the removal is
performed (hence the name observed-removed set).
The following example illustrates how to do that:
.. includecode:: ../../../akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala#remove-item
.. warning::
*Caveat:* Even if you use ``WriteMajority`` and ``ReadMajority`` there is small risk that you may
read stale data if the cluster membership has changed between the ``Update`` and the ``Get``.
For example, in cluster of 5 nodes when you ``Update`` and that change is written to 3 nodes:
n1, n2, n3. Then 2 more nodes are added and a ``Get`` request is reading from 4 nodes, which
happens to be n4, n5, n6, n7, i.e. the value on n1, n2, n3 is not seen in the response of the
``Get`` request.
Subscribe
---------
You may also register interest in change notifications by sending ``Replicator.Subscribe``
message to the ``Replicator``. It will send ``Replicator.Changed`` messages to the registered
subscriber when the data for the subscribed key is updated. Subscribers will be notified
periodically with the configured ``notify-subscribers-interval``, and it is also possible to
send an explicit ``Replicator.FlushChanges`` message to the ``Replicator`` to notify the subscribers
immediately.
The subscriber is automatically removed if the subscriber is terminated. A subscriber can
also be deregistered with the ``Replicator.Unsubscribe`` message.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#subscribe
Delete
------
A data entry can be deleted by sending a ``Replicator.Delete`` message to the local
local ``Replicator``. As reply of the ``Delete`` a ``Replicator.DeleteSuccess`` is sent to
the sender of the ``Delete`` if the value was successfully deleted according to the supplied
consistency level within the supplied timeout. Otherwise a ``Replicator.ReplicationDeleteFailure``
is sent. Note that ``ReplicationDeleteFailure`` does not mean that the delete completely failed or
was rolled back. It may still have been replicated to some nodes, and may eventually be replicated
to all nodes.
A deleted key cannot be reused again, but it is still recommended to delete unused
data entries because that reduces the replication overhead when new nodes join the cluster.
Subsequent ``Delete``, ``Update`` and ``Get`` requests will be replied with ``Replicator.DataDeleted``.
Subscribers will receive ``Replicator.DataDeleted``.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#delete
Data Types
==========
The data types must be convergent (stateful) CRDTs and implement the ``ReplicatedData`` trait,
i.e. they provide a monotonic merge function and the state changes always converge.
You can use your own custom ``ReplicatedData`` types, and several types are provided
by this package, such as:
* Counters: ``GCounter``, ``PNCounter``
* Sets: ``GSet``, ``ORSet``
* Maps: ``ORMap``, ``LWWMap``, ``PNCounterMap``
* Registers: ``LWWRegister``, ``Flag``
Counters
--------
``GCounter`` is a "grow only counter". It only supports increments, no decrements.
It works in a similar way as a vector clock. It keeps track of one counter per node and the total
value is the sum of these counters. The ``merge`` is implemented by taking the maximum count for
each node.
If you need both increments and decrements you can use the ``PNCounter`` (positive/negative counter).
It is tracking the increments (P) separate from the decrements (N). Both P and N are represented
as two internal ``GCounter``. Merge is handled by merging the internal P and N counters.
The value of the counter is the value of the P counter minus the value of the N counter.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#pncounter
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
values they are guaranteed to be replicated together as one unit, which is sometimes necessary for
related data.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#pncountermap
Sets
----
If you only need to add elements to a set and not remove elements the ``GSet`` (grow-only set) is
the data type to use. The elements can be any type of values that can be serialized.
Merge is simply the union of the two sets.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#gset
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
removed, the add will win. You cannot remove an element that you have not seen.
The ``ORSet`` has a version vector that is incremented when an element is added to the set.
The version for the node that added the element is also tracked for each element in a so
called "birth dot". The version vector and the dots are used by the ``merge`` function to
track causality of the operations and resolve concurrent updates.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#orset
Maps
----
``ORMap`` (observed-remove map) is a map with ``String`` keys and the values are ``ReplicatedData``
types themselves. It supports add, remove and delete any number of times for a map entry.
If an entry is concurrently added and removed, the add will win. You cannot remove an entry that
you have not seen. This is the same semantics as for the ``ORSet``.
If an entry is concurrently updated to different values the values will be merged, hence the
requirement that the values must be ``ReplicatedData`` types.
It is rather inconvenient to use the ``ORMap`` directly since it does not expose specific types
of the values. The ``ORMap`` is intended as a low level tool for building more specific maps,
such as the following specialized maps.
``PNCounterMap`` (positive negative counter map) is a map of named counters. It is a specialized
``ORMap`` with ``PNCounter`` values.
``LWWMap`` (last writer wins map) is a specialized ``ORMap`` with ``LWWRegister`` (last writer wins register)
values.
Note that ``LWWRegister`` and therefore ``LWWMap`` relies on synchronized clocks and should only be used
when the choice of value is not important for concurrent updates occurring within the clock skew.
Instead of using timestamps based on ``System.currentTimeMillis()`` time it is possible to
use a timestamp value based on something else, for example an increasing version number
from a database record that is used for optimistic concurrency control.
When a data entry is changed the full state of that entry is replicated to other nodes, i.e.
when you update an map the whole map is replicated. Therefore, instead of using one ``ORMap``
with 1000 elements it is more efficient to split that up in 10 top level ``ORMap`` entries
with 100 elements each. Top level entries are replicated individually, which has the
trade-off that different entries may not be replicated at the same time and you may see
inconsistencies between related entries. Separate top level entries cannot be updated atomically
together.
Flags and Registers
-------------------
``Flag`` is a data type for a boolean value that is initialized to ``false`` and can be switched
to ``true``. Thereafter it cannot be changed. ``true`` wins over ``false`` in merge.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#flag
``LWWRegister`` (last writer wins register) can hold any (serializable) value.
Merge of a ``LWWRegister`` takes the the register with highest timestamp. Note that this
relies on synchronized clocks. `LWWRegister` should only be used when the choice of
value is not important for concurrent updates occurring within the clock skew.
Merge takes the register updated by the node with lowest address (``UniqueAddress`` is ordered)
if the timestamps are exactly the same.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#lwwregister
Instead of using timestamps based on ``System.currentTimeMillis()`` time it is possible to
use a timestamp value based on something else, for example an increasing version number
from a database record that is used for optimistic concurrency control.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#lwwregister-custom-clock
For first-write-wins semantics you can use the ``LWWRegister#reverseClock`` instead of the
``LWWRegister#defaultClock``.
Custom Data Type
----------------
You can rather easily implement your own data types. The only requirement is that it implements
the ``merge`` function of the ``ReplicatedData`` trait.
A nice property of stateful CRDTs is that they typically compose nicely, i.e. you can combine several
smaller data types to build richer data structures. For example, the ``PNCounter`` is composed of
two internal ``GCounter`` instances to keep track of increments and decrements separately.
Here is s simple implementation of a custom ``TwoPhaseSet`` that is using two internal ``GSet`` types
to keep track of addition and removals. A ``TwoPhaseSet`` is a set where an element may be added and
removed, but never added again thereafter.
.. includecode:: code/docs/ddata/TwoPhaseSet.scala#twophaseset
Data types should be immutable, i.e. "modifying" methods should return a new instance.
Serialization
^^^^^^^^^^^^^
The data types must be serializable with an :ref:`Akka Serializer <serialization-scala>`.
It is highly recommended that you implement efficient serialization with Protobuf or similar
for your custom data types. The built in data types are marked with ``ReplicatedDataSerialization``
and serialized with ``akka.cluster.ddata.protobuf.ReplicatedDataSerializer``.
Serialization of the data types are used in remote messages and also for creating message
digests (SHA-1) to detect changes. Therefore it is important that the serialization is efficient
and produce the same bytes for the same content. For example sets and maps should be sorted
deterministically in the serialization.
This is a protobuf representation of the above ``TwoPhaseSet``:
.. includecode:: ../../src/main/protobuf/TwoPhaseSetMessages.proto#twophaseset
The serializer for the ``TwoPhaseSet``:
.. includecode:: code/docs/ddata/protobuf/TwoPhaseSetSerializer.scala#serializer
Note that the elements of the sets are sorted so the SHA-1 digests are the same
for the same elements.
You register the serializer in configuration:
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#serializer-config
Using compression can sometimes be a good idea to reduce the data size. Gzip compression is
provided by the ``akka.cluster.ddata.protobuf.SerializationSupport`` trait:
.. includecode:: code/docs/ddata/protobuf/TwoPhaseSetSerializer.scala#compression
The two embedded ``GSet`` can be serialized as illustrated above, but in general when composing
new data types from the existing built in types it is better to make use of the existing
serializer for those types. This can be done by declaring those as bytes fields in protobuf:
.. includecode:: ../../src/main/protobuf/TwoPhaseSetMessages.proto#twophaseset2
and use the methods ``otherMessageToProto`` and ``otherMessageFromBinary`` that are provided
by the ``SerializationSupport`` trait to serialize and deserialize the ``GSet`` instances. This
works with any type that has a registered Akka serializer. This is how such an serializer would
look like for the ``TwoPhaseSet``:
.. includecode:: code/docs/ddata/protobuf/TwoPhaseSetSerializer2.scala#serializer
CRDT Garbage
------------
One thing that can be problematic with CRDTs is that some data types accumulate history (garbage).
For example a ``GCounter`` keeps track of one counter per node. If a ``GCounter`` has been updated
from one node it will associate the identifier of that node forever. That can become a problem
for long running systems with many cluster nodes being added and removed. To solve this problem
the ``Replicator`` performs pruning of data associated with nodes that have been removed from the
cluster. Data types that need pruning have to implement the ``RemovedNodePruning`` trait.
Samples
=======
**FIXME convert these samples to activator template**
* `Replicated Cache <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala>`_
* `Replicated Metrics <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala>`_
* `Replicated Service Registry <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedServiceRegistrySpec.scala>`_
* `VotingService <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/VotingContestSpec.scala>`_
* `ShoppingCart <@github@/akka-distributed-data/src/multi-jvm/scala/sample/distributeddata/ReplicatedShoppingCartSpec.scala>`_
Limitations
===========
There are some limitations that you should be aware of.
CRDTs cannot be used for all types of problems, and eventual consistency does not fit
all domains. Sometimes you need strong consistency.
It is not intended for *Big Data*. The number of top level entries should not exceed 100000.
When a new node is added to the cluster all these entries are transferred (gossiped) to the
new node. The entries are split up in chunks and all existing nodes collaborate in the gossip,
but it will take a while (tens of seconds) to transfer all entries and this means that you
cannot have too many top level entries. The current recommended limit is 100000. We will
be able to improve this if needed, but the design is still not intended for billions of entries.
All data is held in memory, which is another reason why it is not intended for *Big Data*.
When a data entry is changed the full state of that entry is replicated to other nodes. For example,
if you add one element to a Set with 100 existing elements, all 101 elements are transferred to
other nodes. This means that you cannot have too large data entries, because then the remote message
size will be too large. We might be able to make this more efficient by implementing
`Efficient State-based CRDTs by Delta-Mutation <http://gsd.di.uminho.pt/members/cbm/ps/delta-crdt-draft16may2014.pdf>`_.
The data is only kept in memory. It is redundant since it is replicated to other nodes
in the cluster, but if you stop all nodes the data is lost, unless you have saved it
elsewhere. Making the data durable is a possible future feature, but even if we implement that
it is not intended to be a full featured database.
Learn More about CRDTs
======================
* `The Final Causal Frontier <http://www.ustream.tv/recorded/61448875>`_
talk by Sean Cribbs
* `Eventually Consistent Data Structures <https://vimeo.com/43903960>`_
talk by Sean Cribbs
* `Strong Eventual Consistency and Conflict-free Replicated Data Types <http://research.microsoft.com/apps/video/default.aspx?id=153540&r=1>`_
talk by Mark Shapiro
* `A comprehensive study of Convergent and Commutative Replicated Data Types <http://hal.upmc.fr/file/index/docid/555588/filename/techreport.pdf>`_
paper by Mark Shapiro et. al.
Dependencies
------------
To use Distributed Data you must add the following dependency in your project.
sbt::
"com.typesafe.akka" %% "akka-distributed-data" % "@version@" @crossString@
maven::
<dependency>
<groupId>com.typesafe.akka</groupId>
<artifactId>akka-distributed-data_@binVersion@</artifactId>
<version>@version@</version>
</dependency>
Configuration
=============
The ``DistributedData`` extension can be configured with the following properties:
.. includecode:: ../../../akka-distributed-data/src/main/resources/reference.conf#distributed-data

View file

@ -11,6 +11,7 @@ Networking
cluster-client
cluster-sharding
cluster-metrics
distributed-data
remoting
serialization
io

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,24 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.ddata;
//#twophaseset
option java_package = "docs.ddata.protobuf.msg";
option optimize_for = SPEED;
message TwoPhaseSet {
repeated string adds = 1;
repeated string removals = 2;
}
//#twophaseset
//#twophaseset2
message TwoPhaseSet2 {
optional bytes adds = 1;
optional bytes removals = 2;
}
//#twophaseset2

View file

@ -51,7 +51,8 @@ object AkkaBuild extends Build {
archivesPathFinder.get.map(file => (file -> ("akka/" + file.getName)))
}
),
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, clusterMetrics, clusterTools, clusterSharding,
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel,
cluster, clusterMetrics, clusterTools, clusterSharding, distributedData,
slf4j, agent, persistence, persistenceTck, kernel, osgi, docs, contrib, samples, multiNodeTestkit, benchJmh, typed)
)
@ -60,7 +61,8 @@ object AkkaBuild extends Build {
base = file("akka-scala-nightly"),
// remove dependencies that we have to build ourselves (Scala STM)
// samples don't work with dbuild right now
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, clusterMetrics, clusterTools, clusterSharding,
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel,
cluster, clusterMetrics, clusterTools, clusterSharding, distributedData,
slf4j, persistence, persistenceTck, kernel, osgi, contrib, multiNodeTestkit, benchJmh, typed)
).disablePlugins(ValidatePullRequest)
@ -135,6 +137,12 @@ object AkkaBuild extends Build {
dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm",
persistence % "compile;test->provided", clusterTools)
) configs (MultiJvm)
lazy val distributedData = Project(
id = "akka-distributed-data",
base = file("akka-distributed-data"),
dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm")
) configs (MultiJvm)
lazy val slf4j = Project(
id = "akka-slf4j",
@ -184,7 +192,7 @@ object AkkaBuild extends Build {
dependencies = Seq(actor, testkit % "test->test",
remote % "compile;test->test", cluster, clusterMetrics, slf4j, agent, camel, osgi,
persistence % "compile;provided->provided;test->test", persistenceTck,
typed % "compile;test->test")
typed % "compile;test->test", distributedData)
)
lazy val contrib = Project(

View file

@ -94,6 +94,8 @@ object Dependencies {
val clusterSharding = l ++= Seq(Test.junit, Test.scalatest.value, Test.commonsIo)
val clusterMetrics = l ++= Seq(Provided.sigarLoader, Test.slf4jJul, Test.slf4jLog4j, Test.logback, Test.mockito)
val distributedData = l ++= Seq(Test.junit, Test.scalatest.value)
val slf4j = l ++= Seq(slf4jApi, Test.logback)

View file

@ -34,6 +34,8 @@ object OSGi {
val clusterSharding = exports(Seq("akka.cluster.sharding.*"), imports = Seq(protobufImport()))
val clusterMetrics = exports(Seq("akka.cluster.metrics.*"), imports = Seq(protobufImport(),kamonImport(),sigarImport()))
val distributedData = exports(Seq("akka.cluster.ddata.*"), imports = Seq(protobufImport()))
val contrib = exports(Seq("akka.contrib.*"))