+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:
parent
bf28260cd0
commit
cbe5dd2cf5
69 changed files with 40036 additions and 3 deletions
14
akka-distributed-data/build.sbt
Normal file
14
akka-distributed-data/build.sbt
Normal 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)
|
||||
File diff suppressed because it is too large
Load diff
File diff suppressed because it is too large
Load diff
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
118
akka-distributed-data/src/main/protobuf/ReplicatorMessages.proto
Normal file
118
akka-distributed-data/src/main/protobuf/ReplicatorMessages.proto
Normal 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;
|
||||
}
|
||||
|
||||
|
||||
61
akka-distributed-data/src/main/resources/reference.conf
Normal file
61
akka-distributed-data/src/main/resources/reference.conf
Normal 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
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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 <= 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 -> 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 -> 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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -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
|
|
@ -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(", ", ", ")")
|
||||
}
|
||||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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()
|
||||
}
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
2
akka-distributed-data/src/test/resources/reference.conf
Normal file
2
akka-distributed-data/src/test/resources/reference.conf
Normal file
|
|
@ -0,0 +1,2 @@
|
|||
akka.actor.serialize-messages = on
|
||||
akka.actor.serialize-creators = on
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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"))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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)))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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"))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -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))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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()
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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()
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
95
akka-docs/rst/java/code/docs/ddata/DataBot.java
Normal file
95
akka-docs/rst/java/code/docs/ddata/DataBot.java
Normal 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
|
||||
411
akka-docs/rst/java/code/docs/ddata/DistributedDataDocTest.java
Normal file
411
akka-docs/rst/java/code/docs/ddata/DistributedDataDocTest.java
Normal 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);
|
||||
}
|
||||
|
||||
}
|
||||
516
akka-docs/rst/java/distributed-data.rst
Normal file
516
akka-docs/rst/java/distributed-data.rst
Normal 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
|
||||
|
||||
|
|
@ -11,6 +11,7 @@ Networking
|
|||
../scala/cluster-client
|
||||
../scala/cluster-sharding
|
||||
cluster-metrics
|
||||
distributed-data
|
||||
remoting
|
||||
serialization
|
||||
io
|
||||
|
|
|
|||
381
akka-docs/rst/scala/code/docs/ddata/DistributedDataDocSpec.scala
Normal file
381
akka-docs/rst/scala/code/docs/ddata/DistributedDataDocSpec.scala
Normal 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)
|
||||
}
|
||||
|
||||
}
|
||||
29
akka-docs/rst/scala/code/docs/ddata/TwoPhaseSet.scala
Normal file
29
akka-docs/rst/scala/code/docs/ddata/TwoPhaseSet.scala
Normal 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
|
||||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
@ -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
|
||||
504
akka-docs/rst/scala/distributed-data.rst
Normal file
504
akka-docs/rst/scala/distributed-data.rst
Normal 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
|
||||
|
||||
|
|
@ -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
24
akka-docs/src/main/protobuf/TwoPhaseSetMessages.proto
Normal file
24
akka-docs/src/main/protobuf/TwoPhaseSetMessages.proto
Normal 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
|
||||
|
||||
|
||||
|
||||
|
|
@ -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(
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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.*"))
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue