├── project ├── build.properties ├── plugins.sbt └── Protobuf.scala ├── src ├── multi-jvm │ └── scala │ │ ├── akka │ │ └── contrib │ │ │ └── datareplication │ │ │ ├── STMultiNodeSpec.scala │ │ │ ├── ReplicatorPruningSpec.scala │ │ │ ├── ReplicatorChaosSpec.scala │ │ │ ├── PerformanceSpec.scala │ │ │ └── JepsenInspiredInsertSpec.scala │ │ └── sample │ │ └── datareplication │ │ ├── VotingContestSpec.scala │ │ ├── ReplicatedCacheSpec.scala │ │ ├── ReplicatedMetricsSpec.scala │ │ ├── ReplicatedShoppingCartSpec.scala │ │ └── ReplicatedServiceRegistrySpec.scala ├── test │ ├── java │ │ └── akka │ │ │ └── contrib │ │ │ └── datareplication │ │ │ └── JavaImplOfReplicatedData.java │ └── scala │ │ └── akka │ │ └── contrib │ │ └── datareplication │ │ ├── FlagSpec.scala │ │ ├── PruningStateSpec.scala │ │ ├── LWWMapSpec.scala │ │ ├── PNCounterMapSpec.scala │ │ ├── LocalConcurrencySpec.scala │ │ ├── LWWRegisterSpec.scala │ │ ├── sample │ │ ├── DataBot.scala │ │ └── LotsOfDataBot.scala │ │ ├── protobuf │ │ ├── ReplicatorMessageSerializerSpec.scala │ │ └── ReplicatedDataSerializerSpec.scala │ │ ├── GSetSpec.scala │ │ ├── ORMultiMapSpec.scala │ │ ├── GCounterSpec.scala │ │ ├── PNCounterSpec.scala │ │ ├── VectorClockSpec.scala │ │ └── ORMapSpec.scala └── main │ ├── scala │ └── akka │ │ └── contrib │ │ └── datareplication │ │ ├── Flag.scala │ │ ├── GSet.scala │ │ ├── PruningState.scala │ │ ├── DataReplication.scala │ │ ├── ReplicatedData.scala │ │ ├── GCounter.scala │ │ ├── PNCounterMap.scala │ │ ├── PNCounter.scala │ │ ├── LWWMap.scala │ │ ├── protobuf │ │ └── SerializationSupport.scala │ │ ├── LWWRegister.scala │ │ ├── ORMultiMap.scala │ │ ├── ORMap.scala │ │ └── VectorClock.scala │ ├── resources │ └── reference.conf │ └── protobuf │ ├── ReplicatorMessages.proto │ └── ReplicatedDataMessages.proto ├── .gitignore ├── LICENSE └── README.md /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=0.13.5 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | 2 | resolvers += Classpaths.typesafeResolver 3 | 4 | addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.3.8") 5 | 6 | resolvers += Resolver.url( 7 | "bintray-sbt-plugin-releases", 8 | url("http://dl.bintray.com/content/sbt/sbt-plugin-releases"))( 9 | Resolver.ivyStylePatterns) 10 | 11 | addSbtPlugin("me.lessis" % "bintray-sbt" % "0.1.1") 12 | 13 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/akka/contrib/datareplication/STMultiNodeSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import org.scalatest.{ BeforeAndAfterAll, WordSpecLike } 7 | import org.scalatest.Matchers 8 | import akka.remote.testkit.MultiNodeSpecCallbacks 9 | 10 | /** 11 | * Hooks up MultiNodeSpec with ScalaTest 12 | */ 13 | trait STMultiNodeSpec extends MultiNodeSpecCallbacks 14 | with WordSpecLike with Matchers with BeforeAndAfterAll { 15 | 16 | override def beforeAll() = multiNodeSpecBeforeAll() 17 | 18 | override def afterAll() = multiNodeSpecAfterAll() 19 | } 20 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *# 2 | *.iml 3 | *.ipr 4 | *.iws 5 | *.pyc 6 | *.tm.epoch 7 | *.vim 8 | */project/boot 9 | */project/build/target 10 | */project/project.target.config-classes 11 | *-shim.sbt 12 | *~ 13 | .#* 14 | .*.swp 15 | .DS_Store 16 | .cache 17 | .cache 18 | .classpath 19 | .codefellow 20 | .ensime* 21 | .eprj 22 | .history 23 | .idea 24 | .manager 25 | .multi-jvm 26 | .project 27 | .scala_dependencies 28 | .scalastyle 29 | .settings 30 | .tags 31 | .tags_sorted_by_file 32 | .target 33 | .worksheet 34 | Makefile 35 | TAGS 36 | lib_managed 37 | logs 38 | project/boot/* 39 | project/plugins/project 40 | src_managed 41 | target 42 | tm*.lck 43 | tm*.log 44 | tm.out 45 | worker*.log 46 | /bin 47 | .tmpBin 48 | 49 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | This software is licensed under the Apache 2 license, quoted below. 2 | 3 | Copyright 2014 Typesafe Inc. [http://www.typesafe.com] 4 | 5 | Licensed under the Apache License, Version 2.0 (the "License"); you may not 6 | use this file except in compliance with the License. You may obtain a copy of 7 | the License at 8 | 9 | [http://www.apache.org/licenses/LICENSE-2.0] 10 | 11 | Unless required by applicable law or agreed to in writing, software 12 | distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 13 | WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 14 | License for the specific language governing permissions and limitations under 15 | the License. 16 | 17 | -------------------------------------------------------------------------------- /src/test/java/akka/contrib/datareplication/JavaImplOfReplicatedData.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication; 5 | 6 | import akka.cluster.UniqueAddress; 7 | 8 | public class JavaImplOfReplicatedData extends AbstractReplicatedData implements RemovedNodePruning { 9 | 10 | @Override 11 | public JavaImplOfReplicatedData merge(ReplicatedData other) { 12 | return this; 13 | } 14 | 15 | @Override 16 | public boolean needPruningFrom(UniqueAddress removedNode) { 17 | return false; 18 | } 19 | 20 | @Override 21 | public JavaImplOfReplicatedData prune(UniqueAddress removedNode, UniqueAddress collapseInto) { 22 | return this; 23 | } 24 | 25 | @Override 26 | public JavaImplOfReplicatedData pruningCleanup(UniqueAddress removedNode) { 27 | return this; 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/Flag.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | object Flag { 7 | /** 8 | * `Flag` that is initialized to `false`. 9 | */ 10 | val empty = new Flag(false) 11 | def apply(): Flag = empty 12 | /** 13 | * Java API: `Flag` that is initialized to `false`. 14 | */ 15 | def create(): Flag = empty 16 | 17 | // unapply from case class 18 | } 19 | 20 | /** 21 | * Implements a boolean flag CRDT that is initialized to `false` and 22 | * can be switched to `true`. `true` wins over `false` in merge. 23 | * 24 | * This class is immutable, i.e. "modifying" methods return a new instance. 25 | */ 26 | @SerialVersionUID(1L) 27 | final case class Flag(enabled: Boolean) extends ReplicatedData with ReplicatedDataSerialization { 28 | 29 | type T = Flag 30 | 31 | def switchOn: Flag = 32 | if (enabled) this 33 | else Flag(true) 34 | 35 | override def merge(that: Flag): Flag = 36 | if (that.enabled) that 37 | else this 38 | } 39 | 40 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/FlagSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.contrib.datareplication.Replicator.Changed 10 | 11 | class FlagSpec extends WordSpec with Matchers { 12 | 13 | "A Flag" must { 14 | 15 | "be able to switch on once" in { 16 | val f1 = Flag() 17 | val f2 = f1.switchOn 18 | val f3 = f2.switchOn 19 | f1.enabled should be(false) 20 | f2.enabled should be(true) 21 | f3.enabled should be(true) 22 | } 23 | 24 | "merge by picking true" in { 25 | val f1 = Flag() 26 | val f2 = f1.switchOn 27 | val m1 = f1 merge f2 28 | m1.enabled should be(true) 29 | val m2 = f2 merge f1 30 | m2.enabled should be(true) 31 | } 32 | 33 | "have unapply extractor" in { 34 | val f1 = Flag.empty.switchOn 35 | val Flag(value1) = f1 36 | val value2: Boolean = value1 37 | Changed("key", f1) match { 38 | case Changed("key", Flag(value3)) => 39 | val value4: Boolean = value3 40 | value4 should be(true) 41 | } 42 | } 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/GSet.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | object GSet { 7 | private val _empty: GSet[Any] = new GSet(Set.empty) 8 | def empty[A]: GSet[A] = _empty.asInstanceOf[GSet[A]] 9 | def apply(): GSet[Any] = _empty 10 | /** 11 | * Java API 12 | */ 13 | def create[A](): GSet[A] = empty[A] 14 | 15 | // unapply from case class 16 | } 17 | 18 | /** 19 | * Implements a 'Add Set' CRDT, also called a 'G-Set'. You can't 20 | * remove an element of a G-Set. 21 | * 22 | * A G-Set doesn't accumulate any garbage apart from the elements themselves. 23 | * 24 | * This class is immutable, i.e. "modifying" methods return a new instance. 25 | */ 26 | @SerialVersionUID(1L) 27 | final case class GSet[A](elements: Set[A]) extends ReplicatedData with ReplicatedDataSerialization { 28 | 29 | type T = GSet[A] 30 | 31 | /** 32 | * Java API 33 | */ 34 | def getElements(): java.util.Set[A] = { 35 | import scala.collection.JavaConverters._ 36 | elements.asJava 37 | } 38 | 39 | def contains(a: A): Boolean = elements(a) 40 | 41 | /** 42 | * Adds an element to the set 43 | */ 44 | def +(element: A): GSet[A] = add(element) 45 | 46 | /** 47 | * Adds an element to the set 48 | */ 49 | def add(element: A): GSet[A] = copy(elements + element) 50 | 51 | override def merge(that: GSet[A]): GSet[A] = copy(elements ++ that.elements) 52 | } 53 | 54 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/PruningState.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.actor.Address 7 | import akka.cluster.Member 8 | import akka.cluster.UniqueAddress 9 | 10 | /** 11 | * INTERNAL API 12 | */ 13 | private[akka] object PruningState { 14 | sealed trait PruningPhase 15 | final case class PruningInitialized(seen: Set[Address]) extends PruningPhase 16 | case object PruningPerformed extends PruningPhase 17 | } 18 | 19 | /** 20 | * INTERNAL API 21 | */ 22 | private[akka] final case class PruningState(owner: UniqueAddress, phase: PruningState.PruningPhase) { 23 | import PruningState._ 24 | 25 | def merge(that: PruningState): PruningState = 26 | (this.phase, that.phase) match { 27 | case (PruningPerformed, _) ⇒ this 28 | case (_, PruningPerformed) ⇒ that 29 | case (PruningInitialized(thisSeen), PruningInitialized(thatSeen)) ⇒ 30 | if (this.owner == that.owner) 31 | copy(phase = PruningInitialized(thisSeen ++ thatSeen)) 32 | else if (Member.addressOrdering.compare(this.owner.address, that.owner.address) > 0) 33 | that 34 | else 35 | this 36 | } 37 | 38 | def addSeen(node: Address): PruningState = phase match { 39 | case PruningInitialized(seen) ⇒ 40 | if (seen(node) || owner.address == node) this 41 | else copy(phase = PruningInitialized(seen + node)) 42 | case _ ⇒ this 43 | } 44 | } 45 | 46 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/PruningStateSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.actor.Address 10 | import akka.cluster.UniqueAddress 11 | 12 | class PruningStateSpec extends WordSpec with Matchers { 13 | import PruningState._ 14 | 15 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 16 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 17 | val node3 = UniqueAddress(node1.address.copy(port = Some(2553)), 3) 18 | val node4 = UniqueAddress(node1.address.copy(port = Some(2554)), 4) 19 | 20 | "Pruning state" must { 21 | 22 | "merge phase correctly" in { 23 | val p1 = PruningState(node1, PruningInitialized(Set.empty)) 24 | val p2 = PruningState(node1, PruningPerformed) 25 | p1.merge(p2).phase should be(PruningPerformed) 26 | p2.merge(p1).phase should be(PruningPerformed) 27 | } 28 | 29 | "merge owner correctly" in { 30 | val p1 = PruningState(node1, PruningInitialized(Set.empty)) 31 | val p2 = PruningState(node2, PruningInitialized(Set.empty)) 32 | val expected = PruningState(node1, PruningInitialized(Set.empty)) 33 | p1.merge(p2) should be(expected) 34 | p2.merge(p1) should be(expected) 35 | } 36 | 37 | "merge seen correctly" in { 38 | val p1 = PruningState(node1, PruningInitialized(Set(node2.address))) 39 | val p2 = PruningState(node1, PruningInitialized(Set(node4.address))) 40 | val expected = PruningState(node1, PruningInitialized(Set(node2.address, node4.address))) 41 | p1.merge(p2) should be(expected) 42 | p2.merge(p1) should be(expected) 43 | } 44 | 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/DataReplication.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.actor.ExtendedActorSystem 7 | import akka.actor.Extension 8 | import akka.actor.ExtensionIdProvider 9 | import akka.actor.ExtensionId 10 | import akka.actor.ActorSystem 11 | import java.util.concurrent.TimeUnit.MILLISECONDS 12 | import akka.actor.ActorRef 13 | import akka.cluster.Cluster 14 | import scala.concurrent.duration._ 15 | 16 | object DataReplication extends ExtensionId[DataReplication] with ExtensionIdProvider { 17 | override def get(system: ActorSystem): DataReplication = super.get(system) 18 | 19 | override def lookup = DataReplication 20 | 21 | override def createExtension(system: ExtendedActorSystem): DataReplication = 22 | new DataReplication(system) 23 | } 24 | 25 | /** 26 | * Akka extension for convenient configuration and use of the 27 | * [[Replicator]]. Configuration settings are defined in the 28 | * `akka.contrib.datareplication` section, see `reference.conf`. 29 | */ 30 | class DataReplication(system: ExtendedActorSystem) extends Extension { 31 | 32 | private val config = system.settings.config.getConfig("akka.contrib.data-replication") 33 | private val settings = ReplicatorSettings(config) 34 | 35 | /** 36 | * Returns true if this member is not tagged with the role configured for the 37 | * replicas. 38 | */ 39 | def isTerminated: Boolean = Cluster(system).isTerminated || !settings.role.forall(Cluster(system).selfRoles.contains) 40 | 41 | /** 42 | * `ActorRef` of the [[Replicator]] . 43 | */ 44 | val replicator: ActorRef = 45 | if (isTerminated) { 46 | system.log.warning("Replicator points to dead letters: Make sure the cluster node is not terminated and has the proper role!") 47 | system.deadLetters 48 | } else { 49 | val name = config.getString("name") 50 | system.actorOf(Replicator.props(settings), name) 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/resources/reference.conf: -------------------------------------------------------------------------------- 1 | ############################################### 2 | # Akka Data Replication Reference Config File # 3 | ############################################### 4 | 5 | # This is the reference config file that contains all the default settings. 6 | # Make your edits/overrides in your application.conf. 7 | 8 | 9 | # Settings for the DataReplication extension 10 | akka.contrib.data-replication { 11 | # Actor name of the Replicator actor, /user/replicator 12 | name = replicator 13 | # Replicas are running on members tagged with this role. 14 | # All members are used if undefined or empty. 15 | role = "" 16 | # How often the Replicator should send out gossip information 17 | gossip-interval = 2 s 18 | # How often the subscribers will be notified of changes, if any 19 | notify-subscribers-interval = 500 ms 20 | # Maximum number of entries to transfer in one gossip message when synchronizing the replicas. 21 | # Next chunk will be transferred in next round of gossip. 22 | max-delta-elements = 1000 23 | # How often the Replicator checks for pruning of data associated with 24 | # removed cluster nodes. 25 | pruning-interval = 30 s 26 | # How long time it takes (worst case) to spread the data to all other replica nodes. 27 | # This is used when initiating and completing the pruning process of data associated 28 | # with removed cluster nodes. The time measurement is stopped when any replica is 29 | # unreachable, so it should be configured to worst case in a healthy cluster. 30 | max-pruning-dissemination = 60 s 31 | } 32 | 33 | # Protobuf serializer for cluster DataReplication messages 34 | akka.actor { 35 | serializers { 36 | akka-data-replication = "akka.contrib.datareplication.protobuf.ReplicatorMessageSerializer" 37 | akka-replicated-data = "akka.contrib.datareplication.protobuf.ReplicatedDataSerializer" 38 | } 39 | serialization-bindings { 40 | "akka.contrib.datareplication.Replicator$ReplicatorMessage" = akka-data-replication 41 | "akka.contrib.datareplication.ReplicatedDataSerialization" = akka-replicated-data 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/LWWMapSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.actor.Address 10 | import akka.cluster.UniqueAddress 11 | import akka.contrib.datareplication.Replicator.Changed 12 | 13 | class LWWMapSpec extends WordSpec with Matchers { 14 | import LWWRegister.defaultClock 15 | 16 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 17 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 18 | 19 | "A LWWMap" must { 20 | 21 | "be able to set entries" in { 22 | val m = LWWMap().put(node1, "a", 1, defaultClock).put(node2, "b", 2, defaultClock) 23 | m.entries should be(Map("a" -> 1, "b" -> 2)) 24 | } 25 | 26 | "be able to have its entries correctly merged with another LWWMap with other entries" in { 27 | val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock).put(node1, "b", 2, defaultClock) 28 | val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock) 29 | 30 | // merge both ways 31 | val expected = Map("a" -> 1, "b" -> 2, "c" -> 3) 32 | (m1 merge m2).entries should be(expected) 33 | (m2 merge m1).entries should be(expected) 34 | } 35 | 36 | "be able to remove entry" in { 37 | val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock).put(node1, "b", 2, defaultClock) 38 | val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock) 39 | 40 | val merged1 = m1 merge m2 41 | 42 | val m3 = merged1.remove(node1, "b") 43 | (merged1 merge m3).entries should be(Map("a" -> 1, "c" -> 3)) 44 | 45 | // but if there is a conflicting update the entry is not removed 46 | val m4 = merged1.put(node2, "b", 22, defaultClock) 47 | (m3 merge m4).entries should be(Map("a" -> 1, "b" -> 22, "c" -> 3)) 48 | } 49 | 50 | "have unapply extractor" in { 51 | val m1 = LWWMap.empty.put(node1, "a", 1L, defaultClock) 52 | val LWWMap(entries1) = m1 53 | val entries2: Map[String, Long] = entries1 54 | Changed("key", m1) match { 55 | case Changed("key", LWWMap(entries3)) => 56 | val entries4: Map[String, Any] = entries3 57 | entries4 should be(Map("a" -> 1L)) 58 | } 59 | } 60 | 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/PNCounterMapSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.actor.Address 10 | import akka.cluster.UniqueAddress 11 | import akka.contrib.datareplication.Replicator.Changed 12 | 13 | class PNCounterMapSpec extends WordSpec with Matchers { 14 | 15 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 16 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 17 | 18 | "A PNCounterMap" must { 19 | 20 | "be able to increment and decrement entries" in { 21 | val m = PNCounterMap().increment(node1, "a", 2).increment(node1, "b", 3).decrement(node2, "a", 1) 22 | m.entries should be(Map("a" -> 1, "b" -> 3)) 23 | } 24 | 25 | "be able to have its entries correctly merged with another ORMap with other entries" in { 26 | val m1 = PNCounterMap().increment(node1, "a", 1).increment(node1, "b", 3).increment(node1, "c", 2) 27 | val m2 = PNCounterMap().increment(node2, "c", 5) 28 | 29 | // merge both ways 30 | val expected = Map("a" -> 1, "b" -> 3, "c" -> 7) 31 | (m1 merge m2).entries should be(expected) 32 | (m2 merge m1).entries should be(expected) 33 | } 34 | 35 | "be able to remove entry" in { 36 | val m1 = PNCounterMap().increment(node1, "a", 1).increment(node1, "b", 3).increment(node1, "c", 2) 37 | val m2 = PNCounterMap().increment(node2, "c", 5) 38 | 39 | val merged1 = m1 merge m2 40 | 41 | val m3 = merged1.remove(node1, "b") 42 | (merged1 merge m3).entries should be(Map("a" -> 1, "c" -> 7)) 43 | 44 | // but if there is a conflicting update the entry is not removed 45 | val m4 = merged1.increment(node2, "b", 10) 46 | (m3 merge m4).entries should be(Map("a" -> 1, "b" -> 13, "c" -> 7)) 47 | } 48 | 49 | "have unapply extractor" in { 50 | val m1 = PNCounterMap.empty.increment(node1, "a", 1).increment(node2, "b", 2) 51 | val PNCounterMap(entries1) = m1 52 | val entries2: Map[String, Long] = entries1 53 | Changed("key", m1) match { 54 | case Changed("key", PNCounterMap(entries3)) => 55 | val entries4: Map[String, Long] = entries3 56 | entries4 should be(Map("a" -> 1L, "b" -> 2L)) 57 | } 58 | } 59 | 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/LocalConcurrencySpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import scala.concurrent.duration._ 8 | import org.scalatest.BeforeAndAfterAll 9 | import org.scalatest.Matchers 10 | import org.scalatest.WordSpecLike 11 | import com.typesafe.config.ConfigFactory 12 | import akka.actor.Actor 13 | import akka.actor.ActorSystem 14 | import akka.actor.Props 15 | import akka.actor.Stash 16 | import akka.cluster.Cluster 17 | import akka.testkit.ImplicitSender 18 | import akka.testkit.TestKit 19 | 20 | object LocalConcurrencySpec { 21 | 22 | final case class Add(s: String) 23 | 24 | class Updater extends Actor with Stash { 25 | implicit val cluster = Cluster(context.system) 26 | val replicator = DataReplication(context.system).replicator 27 | val key = "key" 28 | 29 | def receive = { 30 | case s: String => 31 | val update = Replicator.Update(key, ORSet.empty[String], Replicator.WriteLocal)(_ + s) 32 | replicator ! update 33 | } 34 | } 35 | } 36 | 37 | class LocalConcurrencySpec(_system: ActorSystem) extends TestKit(_system) 38 | with WordSpecLike with Matchers with BeforeAndAfterAll with ImplicitSender { 39 | import LocalConcurrencySpec._ 40 | 41 | def this() { 42 | this(ActorSystem("LocalConcurrencySpec", 43 | ConfigFactory.parseString(""" 44 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 45 | """))) 46 | } 47 | 48 | override def afterAll(): Unit = { 49 | system.shutdown() 50 | system.awaitTermination(10.seconds) 51 | } 52 | 53 | val replicator = DataReplication(system).replicator 54 | 55 | "Updates from same node" must { 56 | 57 | "be possible to do from two actors" in { 58 | val updater1 = system.actorOf(Props[Updater], "updater1") 59 | val updater2 = system.actorOf(Props[Updater], "updater2") 60 | 61 | val numMessages = 100 62 | for (n <- 1 to numMessages) { 63 | updater1 ! s"a$n" 64 | updater2 ! s"b$n" 65 | } 66 | 67 | val expected = ((1 to numMessages).map("a" + _) ++ (1 to numMessages).map("b" + _)).toSet 68 | awaitAssert { 69 | replicator ! Replicator.Get("key", Replicator.ReadLocal) 70 | val ORSet(elements) = expectMsgType[Replicator.GetSuccess].data 71 | elements should be(expected) 72 | } 73 | 74 | } 75 | 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/protobuf/ReplicatorMessages.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication; 5 | 6 | option java_package = "akka.contrib.datareplication.protobuf.msg"; 7 | option optimize_for = SPEED; 8 | 9 | message Get { 10 | required string key = 1; 11 | required int32 consistency = 2; 12 | required int32 timeout = 3; 13 | optional OtherMessage request = 4; 14 | } 15 | 16 | message GetSuccess { 17 | required string key = 1; 18 | required OtherMessage data = 2; 19 | optional OtherMessage request = 4; 20 | } 21 | 22 | message NotFound { 23 | required string key = 1; 24 | optional OtherMessage request = 2; 25 | } 26 | 27 | message GetFailure { 28 | required string key = 1; 29 | optional OtherMessage request = 2; 30 | } 31 | 32 | message Subscribe { 33 | required string key = 1; 34 | required string ref = 2; 35 | } 36 | 37 | message Unsubscribe { 38 | required string key = 1; 39 | required string ref = 2; 40 | } 41 | 42 | message Changed { 43 | required string key = 1; 44 | required OtherMessage data = 2; 45 | } 46 | 47 | message Write { 48 | required string key = 1; 49 | required DataEnvelope envelope = 2; 50 | } 51 | 52 | // message WriteAck, via Empty 53 | 54 | message Empty { 55 | } 56 | 57 | message Read { 58 | required string key = 1; 59 | } 60 | 61 | message ReadResult { 62 | optional DataEnvelope envelope = 1; 63 | } 64 | 65 | message DataEnvelope { 66 | message PruningEntry { 67 | required UniqueAddress removedAddress = 1; 68 | required UniqueAddress ownerAddress = 2; 69 | required bool performed = 3; 70 | repeated Address seen = 4; 71 | } 72 | 73 | required OtherMessage data = 1; 74 | repeated PruningEntry pruning = 2; 75 | } 76 | 77 | message Status { 78 | message Entry { 79 | required string key = 1; 80 | required bytes digest = 2; 81 | } 82 | 83 | required uint32 chunk = 1; 84 | required uint32 totChunks = 2; 85 | repeated Entry entries = 3; 86 | } 87 | 88 | message Gossip { 89 | message Entry { 90 | required string key = 1; 91 | required DataEnvelope envelope = 2; 92 | } 93 | 94 | required bool sendBack = 1; 95 | repeated Entry entries = 2; 96 | } 97 | 98 | message UniqueAddress { 99 | required Address address = 1; 100 | required uint32 uid = 2; 101 | } 102 | 103 | message Address { 104 | required string hostname = 1; 105 | required uint32 port = 2; 106 | } 107 | 108 | message OtherMessage { 109 | required bytes enclosedMessage = 1; 110 | required int32 serializerId = 2; 111 | optional bytes messageManifest = 4; 112 | } 113 | 114 | message StringGSet { 115 | repeated string elements = 1; 116 | } 117 | 118 | 119 | -------------------------------------------------------------------------------- /src/main/protobuf/ReplicatedDataMessages.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication; 5 | 6 | option java_package = "akka.contrib.datareplication.protobuf.msg"; 7 | option optimize_for = SPEED; 8 | import "ReplicatorMessages.proto"; 9 | 10 | message GSet { 11 | repeated string stringElements = 1; 12 | repeated int32 intElements = 2; 13 | repeated int64 longElements = 3; 14 | repeated OtherMessage otherElements = 4; 15 | } 16 | 17 | message ORSet { 18 | message StringEntry { 19 | required string element = 1; 20 | required VectorClock dot = 2; 21 | } 22 | message IntEntry { 23 | required int32 element = 1; 24 | required VectorClock dot = 2; 25 | } 26 | message LongEntry { 27 | required int64 element = 1; 28 | required VectorClock dot = 2; 29 | } 30 | message OtherEntry { 31 | required OtherMessage element = 1; 32 | required VectorClock dot = 2; 33 | } 34 | 35 | required VectorClock vclock = 1; 36 | repeated StringEntry stringElements = 2; 37 | repeated IntEntry intElements = 3; 38 | repeated LongEntry longElements = 4; 39 | repeated OtherEntry otherElements = 5; 40 | } 41 | 42 | message Flag { 43 | required bool enabled = 1; 44 | } 45 | 46 | message LWWRegister { 47 | required int64 timestamp = 1; 48 | required UniqueAddress node = 2; 49 | required OtherMessage state = 3; 50 | } 51 | 52 | message GCounter { 53 | message Entry { 54 | required UniqueAddress node = 1; 55 | required int64 value = 2; 56 | } 57 | 58 | repeated Entry entries = 1; 59 | } 60 | 61 | message PNCounter { 62 | required GCounter increments = 1; 63 | required GCounter decrements = 2; 64 | } 65 | 66 | message VectorClock { 67 | message Entry { 68 | required UniqueAddress node = 1; 69 | required int64 clock = 2; 70 | } 71 | repeated Entry entries = 1; 72 | } 73 | 74 | message ORMap { 75 | message Entry { 76 | required string key = 1; 77 | required OtherMessage value = 2; 78 | } 79 | 80 | required ORSet keys = 1; 81 | repeated Entry entries = 2; 82 | } 83 | 84 | message LWWMap { 85 | message Entry { 86 | required string key = 1; 87 | required LWWRegister value = 2; 88 | } 89 | 90 | required ORSet keys = 1; 91 | repeated Entry entries = 2; 92 | } 93 | 94 | message PNCounterMap { 95 | message Entry { 96 | required string key = 1; 97 | required PNCounter value = 2; 98 | } 99 | 100 | required ORSet keys = 1; 101 | repeated Entry entries = 2; 102 | } 103 | 104 | message ORMultiMap { 105 | message Entry { 106 | required string key = 1; 107 | required ORSet value = 2; 108 | } 109 | 110 | required ORSet keys = 1; 111 | repeated Entry entries = 2; 112 | } 113 | 114 | 115 | 116 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/ReplicatedData.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.cluster.UniqueAddress 7 | 8 | /** 9 | * Interface for implementing a state based convergent 10 | * replicated data type (CvRDT). 11 | * 12 | * ReplicatedData types must be serializable with an Akka 13 | * Serializer. It is higly recommended to implement a serializer with 14 | * Protobuf or similar. The built in data types are marked with 15 | * [[ReplicatedDataSerialization]] and serialized with 16 | * [[akka.contrib.datareplication.protobuf.ReplicatedDataSerializer]]. 17 | * 18 | * Serialization of the data types are used in remote messages and also 19 | * for creating message digests (SHA-1) to detect changes. Therefore it is 20 | * important that the serialization produce the same bytes for the same content. 21 | * For example sets and maps should be sorted deterministically in the serialization. 22 | * 23 | * ReplicatedData types should be immutable, i.e. "modifying" methods should return 24 | * a new instance. 25 | */ 26 | trait ReplicatedData { 27 | type T <: ReplicatedData 28 | 29 | /** 30 | * Monotonic merge function. 31 | */ 32 | def merge(that: T): T 33 | 34 | } 35 | 36 | /** 37 | * Java API: Interface for implementing a [[ReplicatedData]] in 38 | * Java. 39 | */ 40 | abstract class AbstractReplicatedData extends ReplicatedData { 41 | // it is not possible to use a more strict type, because it is erased somehow, and 42 | // the implementation is anyway required to implement 43 | // merge(that: ReplicatedData): ReplicatedData 44 | type T = AbstractReplicatedData 45 | 46 | } 47 | 48 | /** 49 | * [[ReplicatedData]] that has support for pruning of data 50 | * belonging to a specific node may implement this interface. 51 | * When a node is removed from the cluster these methods will be 52 | * used by the [[Replicator]] to collapse data from the removed node 53 | * into some other node in the cluster. 54 | */ 55 | trait RemovedNodePruning { this: ReplicatedData ⇒ 56 | 57 | /** 58 | * Does it have any state changes from a specific node, 59 | * which has been removed from the cluster. 60 | */ 61 | def needPruningFrom(removedNode: UniqueAddress): Boolean 62 | 63 | /** 64 | * When the `removed` node has been removed from the cluster the state 65 | * changes from that node will be pruned by collapsing the data entries 66 | * to another node. 67 | */ 68 | def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): T 69 | 70 | /** 71 | * Remove data entries from a node that has been removed from the cluster 72 | * and already been pruned. 73 | */ 74 | def pruningCleanup(removedNode: UniqueAddress): T 75 | } 76 | 77 | /** 78 | * Marker trait for `ReplicatedData` serialized by 79 | * [[akka.contrib.datareplication.protobuf.ReplicatedDataSerializer]]. 80 | */ 81 | trait ReplicatedDataSerialization extends Serializable 82 | 83 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/LWWRegisterSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.actor.Address 10 | import akka.cluster.UniqueAddress 11 | import akka.contrib.datareplication.Replicator.Changed 12 | 13 | class LWWRegisterSpec extends WordSpec with Matchers { 14 | import LWWRegister.defaultClock 15 | 16 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 17 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 18 | 19 | "A LWWRegister" must { 20 | "use latest of successive assignments" in { 21 | val r = (1 to 100).foldLeft(LWWRegister(node1, 0, defaultClock)) { 22 | case (r, n) ⇒ 23 | r.value should be(n - 1) 24 | r.withValue(node1, n, defaultClock) 25 | } 26 | r.value should be(100) 27 | } 28 | 29 | "merge by picking max timestamp" in { 30 | val clock = new LWWRegister.Clock { 31 | val i = Iterator.from(100) 32 | override def nextTimestamp(current: Long): Long = i.next() 33 | } 34 | val r1 = LWWRegister(node1, "A", clock) 35 | r1.timestamp should be(100) 36 | val r2 = r1.withValue(node2, "B", clock) 37 | r2.timestamp should be(101) 38 | val m1 = r1 merge r2 39 | m1.value should be("B") 40 | m1.timestamp should be(101) 41 | val m2 = r2 merge r1 42 | m2.value should be("B") 43 | m2.timestamp should be(101) 44 | } 45 | 46 | "merge by picking least address when same timestamp" in { 47 | val clock = new LWWRegister.Clock { 48 | override def nextTimestamp(current: Long): Long = 100 49 | } 50 | val r1 = LWWRegister(node1, "A", clock) 51 | val r2 = LWWRegister(node2, "B", clock) 52 | val m1 = r1 merge r2 53 | m1.value should be("A") 54 | val m2 = r2 merge r1 55 | m2.value should be("A") 56 | } 57 | 58 | "use monotonically increasing defaultClock" in { 59 | (1 to 100).foldLeft(LWWRegister(node1, 0, defaultClock)) { 60 | case (r, n) ⇒ 61 | r.value should be(n - 1) 62 | val r2 = r.withValue(node1, n, defaultClock) 63 | r2.timestamp should be > r.timestamp 64 | r2 65 | } 66 | } 67 | 68 | "have unapply extractor" in { 69 | val r1 = LWWRegister(node1, "a", defaultClock) 70 | val LWWRegister(value1) = r1 71 | val value2: String = value1 72 | Changed("key", r1) match { 73 | case Changed("key", LWWRegister(value3)) => 74 | val value4: Any = value3 75 | value4 should be("a") 76 | } 77 | } 78 | 79 | "can be used as first-write-wins-register" in { 80 | import LWWRegister.reverseClock 81 | val r = (1 to 100).foldLeft(LWWRegister(node1, 0, reverseClock)) { 82 | case (r, n) ⇒ 83 | r.value should be(0) 84 | val newRegister = r.merge(r.withValue(node1, n, reverseClock)) 85 | newRegister should be(r) 86 | newRegister 87 | } 88 | r.value should be(0) 89 | } 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/sample/DataBot.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2015 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication.sample 5 | 6 | import scala.concurrent.duration._ 7 | import akka.actor.ActorLogging 8 | import akka.contrib.datareplication.protobuf.msg.ReplicatorMessages.GetSuccess 9 | import akka.contrib.datareplication.DataReplication 10 | import akka.cluster.Cluster 11 | import akka.contrib.datareplication.Replicator 12 | import akka.actor.Actor 13 | import akka.contrib.datareplication.ORSet 14 | import scala.concurrent.forkjoin.ThreadLocalRandom 15 | import akka.actor.ActorSystem 16 | import com.typesafe.config.ConfigFactory 17 | import akka.actor.Props 18 | 19 | object DataBot { 20 | 21 | def main(args: Array[String]): Unit = { 22 | if (args.isEmpty) 23 | startup(Seq("2551", "2552", "0")) 24 | else 25 | startup(args) 26 | } 27 | 28 | def startup(ports: Seq[String]): Unit = { 29 | ports.foreach { port => 30 | // Override the configuration of the port 31 | val config = ConfigFactory.parseString("akka.remote.netty.tcp.port=" + port). 32 | withFallback(ConfigFactory.load( 33 | ConfigFactory.parseString(""" 34 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 35 | akka.remote { 36 | netty.tcp { 37 | hostname = "127.0.0.1" 38 | port = 0 39 | } 40 | } 41 | 42 | akka.cluster { 43 | seed-nodes = [ 44 | "akka.tcp://ClusterSystem@127.0.0.1:2551", 45 | "akka.tcp://ClusterSystem@127.0.0.1:2552"] 46 | 47 | auto-down-unreachable-after = 10s 48 | } 49 | """))) 50 | 51 | // Create an Akka system 52 | val system = ActorSystem("ClusterSystem", config) 53 | // Create an actor that handles cluster domain events 54 | system.actorOf(Props[DataBot], name = "dataBot") 55 | } 56 | } 57 | 58 | private case object Tick 59 | 60 | } 61 | 62 | // This sample is used in the README.md (remember to copy when it is changed) 63 | class DataBot extends Actor with ActorLogging { 64 | import DataBot._ 65 | import Replicator._ 66 | 67 | val replicator = DataReplication(context.system).replicator 68 | implicit val cluster = Cluster(context.system) 69 | 70 | import context.dispatcher 71 | val tickTask = context.system.scheduler.schedule(5.seconds, 5.seconds, self, Tick) 72 | 73 | replicator ! Subscribe("key", self) 74 | 75 | def receive = { 76 | case Tick => 77 | val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString 78 | if (ThreadLocalRandom.current().nextBoolean()) { 79 | // add 80 | log.info("Adding: {}", s) 81 | replicator ! Update("key", ORSet(), WriteLocal)(_ + s) 82 | } else { 83 | // remove 84 | log.info("Removing: {}", s) 85 | replicator ! Update("key", ORSet(), WriteLocal)(_ - s) 86 | } 87 | 88 | case _: UpdateResponse => // ignore 89 | 90 | case Changed("key", ORSet(elements)) => 91 | log.info("Current elements: {}", elements) 92 | } 93 | 94 | override def postStop(): Unit = tickTask.cancel() 95 | 96 | } 97 | 98 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/protobuf/ReplicatorMessageSerializerSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication.protobuf 5 | 6 | import scala.concurrent.duration._ 7 | import org.scalatest.BeforeAndAfterAll 8 | import org.scalatest.Matchers 9 | import org.scalatest.WordSpecLike 10 | import akka.actor.ActorSystem 11 | import akka.actor.Address 12 | import akka.actor.ExtendedActorSystem 13 | import akka.actor.Props 14 | import akka.contrib.datareplication.GSet 15 | import akka.contrib.datareplication.PruningState 16 | import akka.contrib.datareplication.PruningState.PruningInitialized 17 | import akka.contrib.datareplication.PruningState.PruningPerformed 18 | import akka.contrib.datareplication.Replicator._ 19 | import akka.contrib.datareplication.Replicator.Internal._ 20 | import akka.testkit.TestKit 21 | import akka.util.ByteString 22 | import akka.cluster.UniqueAddress 23 | import com.typesafe.config.ConfigFactory 24 | 25 | class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem("ReplicatorMessageSerializerSpec", 26 | ConfigFactory.parseString("akka.actor.provider=akka.cluster.ClusterActorRefProvider"))) 27 | with WordSpecLike with Matchers with BeforeAndAfterAll { 28 | 29 | val serializer = new ReplicatorMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) 30 | 31 | val address1 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4711), 1) 32 | val address2 = UniqueAddress(Address("akka.tcp", system.name, "other.host.org", 4711), 2) 33 | val address3 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4712), 3) 34 | 35 | override def afterAll { 36 | shutdown() 37 | } 38 | 39 | def checkSerialization(obj: AnyRef): Unit = { 40 | val blob = serializer.toBinary(obj) 41 | val ref = serializer.fromBinary(blob, obj.getClass) 42 | ref should be(obj) 43 | } 44 | 45 | "ReplicatorMessageSerializer" must { 46 | 47 | "serialize Replicator messages" in { 48 | val ref1 = system.actorOf(Props.empty, "ref1") 49 | val data1 = GSet() + "a" 50 | 51 | checkSerialization(Get("A", ReadLocal)) 52 | checkSerialization(Get("A", ReadQuorum(2.seconds), Some("x"))) 53 | checkSerialization(GetSuccess("A", data1, None)) 54 | checkSerialization(GetSuccess("A", data1, Some("x"))) 55 | checkSerialization(NotFound("A", Some("x"))) 56 | checkSerialization(GetFailure("A", Some("x"))) 57 | checkSerialization(Subscribe("A", ref1)) 58 | checkSerialization(Unsubscribe("A", ref1)) 59 | checkSerialization(Changed("A", data1)) 60 | checkSerialization(DataEnvelope(data1)) 61 | checkSerialization(DataEnvelope(data1, pruning = Map( 62 | address1 -> PruningState(address2, PruningPerformed), 63 | address3 -> PruningState(address2, PruningInitialized(Set(address1.address)))))) 64 | checkSerialization(Write("A", DataEnvelope(data1))) 65 | checkSerialization(WriteAck) 66 | checkSerialization(Read("A")) 67 | checkSerialization(ReadResult(Some(DataEnvelope(data1)))) 68 | checkSerialization(ReadResult(None)) 69 | checkSerialization(Status(Map("A" -> ByteString.fromString("a"), 70 | "B" -> ByteString.fromString("b")), chunk = 3, totChunks = 10)) 71 | checkSerialization(Gossip(Map("A" -> DataEnvelope(data1), 72 | "B" -> DataEnvelope(GSet() + "b" + "c")), sendBack = true)) 73 | } 74 | 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /project/Protobuf.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package generate.protobuf 6 | 7 | import sbt._ 8 | import Process._ 9 | import Keys._ 10 | 11 | import java.io.File 12 | 13 | object Protobuf { 14 | val paths = SettingKey[Seq[File]]("protobuf-paths", "The paths that contain *.proto files.") 15 | val outputPaths = SettingKey[Seq[File]]("protobuf-output-paths", "The paths where to save the generated *.java files.") 16 | val protoc = SettingKey[String]("protobuf-protoc", "The path and name of the protoc executable.") 17 | val protocVersion = SettingKey[String]("protobuf-protoc-version", "The version of the protoc executable.") 18 | val generate = TaskKey[Unit]("protobuf-generate", "Compile the protobuf sources and do all processing.") 19 | 20 | lazy val settings: Seq[Setting[_]] = Seq( 21 | paths <<= (sourceDirectory in Compile, sourceDirectory in Test) { (a, b) => Seq(a, b).map(_ / "protobuf") }, 22 | outputPaths <<= (sourceDirectory in Compile, sourceDirectory in Test) { (a, b) => Seq(a, b).map(_ / "java") }, 23 | protoc := "protoc", 24 | protocVersion := "2.5.0", 25 | generate <<= generateSourceTask 26 | ) 27 | 28 | private def callProtoc[T](protoc: String, args: Seq[String], log: Logger, thunk: (ProcessBuilder, Logger) => T): T = 29 | try { 30 | val proc = Process(protoc, args) 31 | thunk(proc, log) 32 | } catch { case e: Exception => 33 | throw new RuntimeException("error while executing '%s' with args: %s" format(protoc, args.mkString(" ")), e) 34 | } 35 | 36 | private def checkProtocVersion(protoc: String, protocVersion: String, log: Logger): Unit = { 37 | val res = callProtoc(protoc, Seq("--version"), log, { (p, l) => p !! l }) 38 | val version = res.split(" ").last.trim 39 | if (version != protocVersion) { 40 | sys.error("Wrong protoc version! Expected %s but got %s" format (protocVersion, version)) 41 | } 42 | } 43 | 44 | private def generate(protoc: String, srcDir: File, targetDir: File, log: Logger): Unit = { 45 | val protoFiles = (srcDir ** "*.proto").get 46 | if (srcDir.exists) 47 | if (protoFiles.isEmpty) 48 | log.info("Skipping empty source directory %s" format srcDir) 49 | else { 50 | targetDir.mkdirs() 51 | 52 | log.info("Generating %d protobuf files from %s to %s".format(protoFiles.size, srcDir, targetDir)) 53 | protoFiles.foreach { proto => log.info("Compiling %s" format proto) } 54 | 55 | val exitCode = callProtoc(protoc, Seq("-I" + srcDir.absolutePath, "--java_out=%s" format targetDir.absolutePath) ++ 56 | protoFiles.map(_.absolutePath), log, { (p, l) => p ! l }) 57 | if (exitCode != 0) 58 | sys.error("protoc returned exit code: %d" format exitCode) 59 | } 60 | } 61 | 62 | private def generateSourceTask: Project.Initialize[Task[Unit]] = (streams, paths, outputPaths, protoc, protocVersion).map { 63 | (out, sourceDirs, targetDirs, protoc, protocVersion) => 64 | if (sourceDirs.size != targetDirs.size) 65 | sys.error("Unbalanced number of paths and destination paths!\nPaths: %s\nDestination Paths: %s" format 66 | (sourceDirs, targetDirs)) 67 | 68 | if (sourceDirs.exists { _.exists }) { 69 | checkProtocVersion(protoc, protocVersion, out.log) 70 | 71 | (sourceDirs zip targetDirs).map { 72 | case (sourceDir, targetDir) => 73 | generate(protoc, sourceDir, targetDir, out.log) 74 | } 75 | } 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/GCounter.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.cluster.Cluster 7 | import akka.cluster.UniqueAddress 8 | 9 | object GCounter { 10 | val empty: GCounter = new GCounter 11 | def apply(): GCounter = empty 12 | /** 13 | * Java API 14 | */ 15 | def create(): GCounter = empty 16 | 17 | /** 18 | * Extract the [[GCounter#value]]. 19 | */ 20 | def unapply(c: GCounter): Option[Long] = Some(c.value) 21 | } 22 | 23 | /** 24 | * Implements a 'Growing Counter' CRDT, also called a 'G-Counter'. 25 | * 26 | * A G-Counter is a increment-only counter (inspired by vector clocks) in 27 | * which only increment and merge are possible. Incrementing the counter 28 | * adds 1 to the count for the current node. Divergent histories are 29 | * resolved by taking the maximum count for each node (like a vector 30 | * clock merge). The value of the counter is the sum of all node counts. 31 | * 32 | * This class is immutable, i.e. "modifying" methods return a new instance. 33 | */ 34 | @SerialVersionUID(1L) 35 | final class GCounter private[akka] ( 36 | private[akka] val state: Map[UniqueAddress, Long] = Map.empty) 37 | extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning { 38 | 39 | type T = GCounter 40 | 41 | /** 42 | * Current total value of the counter. 43 | */ 44 | def value: Long = state.values.sum 45 | 46 | /** 47 | * Increment the counter with the delta specified. 48 | * The delta must be zero or positive. 49 | */ 50 | def +(delta: Long)(implicit node: Cluster): GCounter = increment(node, delta) 51 | 52 | /** 53 | * Increment the counter with the delta specified. 54 | * The delta must be zero or positive. 55 | */ 56 | def increment(node: Cluster, delta: Long = 1): GCounter = 57 | increment(node.selfUniqueAddress, delta) 58 | 59 | /** 60 | * INTERNAL API 61 | */ 62 | private[akka] def increment(key: UniqueAddress): GCounter = increment(key, 1) 63 | 64 | /** 65 | * INTERNAL API 66 | */ 67 | private[akka] def increment(key: UniqueAddress, delta: Long): GCounter = { 68 | require(delta >= 0, "Can't decrement a GCounter") 69 | if (delta == 0) this 70 | else state.get(key) match { 71 | case Some(v) ⇒ 72 | val tot = v + delta 73 | require(tot >= 0, "Number overflow") 74 | new GCounter(state + (key -> tot)) 75 | case None ⇒ new GCounter(state + (key -> delta)) 76 | } 77 | } 78 | 79 | override def merge(that: GCounter): GCounter = { 80 | var merged = that.state 81 | for ((key, thisValue) ← state) { 82 | val thatValue = merged.getOrElse(key, 0L) 83 | if (thisValue > thatValue) 84 | merged = merged.updated(key, thisValue) 85 | } 86 | new GCounter(merged) 87 | } 88 | 89 | override def needPruningFrom(removedNode: UniqueAddress): Boolean = 90 | state.contains(removedNode) 91 | 92 | override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): GCounter = 93 | state.get(removedNode) match { 94 | case Some(value) ⇒ new GCounter(state - removedNode).increment(collapseInto, value) 95 | case None ⇒ this 96 | } 97 | 98 | override def pruningCleanup(removedNode: UniqueAddress): GCounter = 99 | new GCounter(state - removedNode) 100 | 101 | // this class cannot be a `case class` because we need different `unapply` 102 | 103 | override def toString: String = s"GCounter($value)" 104 | 105 | override def equals(o: Any): Boolean = o match { 106 | case other: GCounter => state == other.state 107 | case _ => false 108 | } 109 | 110 | override def hashCode: Int = state.hashCode 111 | 112 | } 113 | 114 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/GSetSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.contrib.datareplication.Replicator.Changed 10 | 11 | class GSetSpec extends WordSpec with Matchers { 12 | 13 | val user1 = """{"username":"john","password":"coltrane"}""" 14 | val user2 = """{"username":"sonny","password":"rollins"}""" 15 | val user3 = """{"username":"charlie","password":"parker"}""" 16 | val user4 = """{"username":"charles","password":"mingus"}""" 17 | 18 | "A GSet" must { 19 | 20 | "be able to add user" in { 21 | val c1 = GSet.empty[String] 22 | 23 | val c2 = c1 + user1 24 | val c3 = c2 + user2 25 | 26 | val c4 = c3 + user4 27 | val c5 = c4 + user3 28 | 29 | c5.elements should contain(user1) 30 | c5.elements should contain(user2) 31 | c5.elements should contain(user3) 32 | c5.elements should contain(user4) 33 | } 34 | 35 | "be able to have its user set correctly merged with another GSet with unique user sets" in { 36 | // set 1 37 | val c11 = GSet.empty[String] 38 | 39 | val c12 = c11 + user1 40 | val c13 = c12 + user2 41 | 42 | c13.elements should contain(user1) 43 | c13.elements should contain(user2) 44 | 45 | // set 2 46 | val c21 = GSet.empty[String] 47 | 48 | val c22 = c21 + user3 49 | val c23 = c22 + user4 50 | 51 | c23.elements should contain(user3) 52 | c23.elements should contain(user4) 53 | 54 | // merge both ways 55 | val merged1 = c13 merge c23 56 | merged1.elements should contain(user1) 57 | merged1.elements should contain(user2) 58 | merged1.elements should contain(user3) 59 | merged1.elements should contain(user4) 60 | 61 | val merged2 = c23 merge c13 62 | merged2.elements should contain(user1) 63 | merged2.elements should contain(user2) 64 | merged2.elements should contain(user3) 65 | merged2.elements should contain(user4) 66 | } 67 | 68 | "be able to have its user set correctly merged with another GSet with overlapping user sets" in { 69 | // set 1 70 | val c10 = GSet.empty[String] 71 | 72 | val c11 = c10 + user1 73 | val c12 = c11 + user2 74 | val c13 = c12 + user3 75 | 76 | c13.elements should contain(user1) 77 | c13.elements should contain(user2) 78 | c13.elements should contain(user3) 79 | 80 | // set 2 81 | val c20 = GSet.empty[String] 82 | 83 | val c21 = c20 + user2 84 | val c22 = c21 + user3 85 | val c23 = c22 + user4 86 | 87 | c23.elements should contain(user2) 88 | c23.elements should contain(user3) 89 | c23.elements should contain(user4) 90 | 91 | // merge both ways 92 | val merged1 = c13 merge c23 93 | merged1.elements should contain(user1) 94 | merged1.elements should contain(user2) 95 | merged1.elements should contain(user3) 96 | merged1.elements should contain(user4) 97 | 98 | val merged2 = c23 merge c13 99 | merged2.elements should contain(user1) 100 | merged2.elements should contain(user2) 101 | merged2.elements should contain(user3) 102 | merged2.elements should contain(user4) 103 | } 104 | 105 | "have unapply extractor" in { 106 | val s1 = GSet.empty + "a" + "b" 107 | val s2: GSet[String] = s1 108 | val GSet(elements1) = s1 109 | val elements2: Set[String] = elements1 110 | Changed("key", s1) match { 111 | case Changed("key", GSet(elements3)) => 112 | val elements4: Set[Any] = elements3 113 | elements4 should be(Set("a", "b")) 114 | } 115 | } 116 | 117 | } 118 | } 119 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/ORMultiMapSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.actor.Address 7 | import akka.cluster.UniqueAddress 8 | import org.scalatest.{ Matchers, WordSpec } 9 | import akka.contrib.datareplication.Replicator.Changed 10 | 11 | class ORMultiMapSpec extends WordSpec with Matchers { 12 | 13 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 14 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 15 | 16 | "A ORMultiMap" must { 17 | 18 | "be able to add entries" in { 19 | val m = ORMultiMap().addBinding(node1, "a", "A").addBinding(node1, "b", "B") 20 | m.entries should be(Map("a" -> Set("A"), "b" -> Set("B"))) 21 | 22 | val m2 = m.addBinding(node1, "a", "C") 23 | m2.entries should be(Map("a" -> Set("A", "C"), "b" -> Set("B"))) 24 | } 25 | 26 | "be able to remove entry" in { 27 | val m = ORMultiMap().addBinding(node1, "a", "A").addBinding(node1, "b", "B").removeBinding(node1, "a", "A") 28 | m.entries should be(Map("b" -> Set("B"))) 29 | } 30 | 31 | "be able to replace an entry" in { 32 | val m = ORMultiMap().addBinding(node1, "a", "A").replaceBinding(node1, "a", "A", "B") 33 | m.entries should be(Map("a" -> Set("B"))) 34 | } 35 | 36 | "be able to have its entries correctly merged with another ORMultiMap with other entries" in { 37 | val m1 = ORMultiMap().addBinding(node1, "a", "A").addBinding(node1, "b", "B") 38 | val m2 = ORMultiMap().addBinding(node2, "c", "C") 39 | 40 | // merge both ways 41 | 42 | val expectedMerge = Map( 43 | "a" -> Set("A"), 44 | "b" -> Set("B"), 45 | "c" -> Set("C")) 46 | 47 | val merged1 = m1 merge m2 48 | merged1.entries should be(expectedMerge) 49 | 50 | val merged2 = m2 merge m1 51 | merged2.entries should be(expectedMerge) 52 | } 53 | 54 | "be able to have its entries correctly merged with another ORMultiMap with overlapping entries" in { 55 | val m1 = ORMultiMap() 56 | .addBinding(node1, "a", "A1") 57 | .addBinding(node1, "b", "B1") 58 | .removeBinding(node1, "a", "A1") 59 | .addBinding(node1, "d", "D1") 60 | val m2 = ORMultiMap() 61 | .addBinding(node2, "c", "C2") 62 | .addBinding(node2, "a", "A2") 63 | .addBinding(node2, "b", "B2") 64 | .removeBinding(node2, "b", "B2") 65 | .addBinding(node2, "d", "D2") 66 | 67 | // merge both ways 68 | 69 | val expectedMerged = Map( 70 | "a" -> Set("A2"), 71 | "b" -> Set("B1"), 72 | "c" -> Set("C2"), 73 | "d" -> Set("D1", "D2")) 74 | 75 | val merged1 = m1 merge m2 76 | merged1.entries should be(expectedMerged) 77 | 78 | val merged2 = m2 merge m1 79 | merged2.entries should be(expectedMerged) 80 | } 81 | } 82 | 83 | "be able to get all bindings for an entry and then reduce them upon putting them back" in { 84 | val m = ORMultiMap().addBinding(node1, "a", "A1").addBinding(node1, "a", "A2").addBinding(node1, "b", "B1") 85 | val Some(a) = m.get("a") 86 | 87 | a should be(Set("A1", "A2")) 88 | 89 | val m2 = m.put(node1, "a", a - "A1") 90 | 91 | val expectedMerged = Map( 92 | "a" -> Set("A2"), 93 | "b" -> Set("B1")) 94 | 95 | m2.entries should be(expectedMerged) 96 | } 97 | 98 | "return the value for an existing key and the default for a non-existing one when using getOrElse" in { 99 | val m = ORMultiMap().addBinding(node1, "a", "A") 100 | m.getOrElse("a", Set("B")) shouldBe Set("A") 101 | m.getOrElse("b", Set("B")) shouldBe Set("B") 102 | } 103 | 104 | "remove all bindings for a given key" in { 105 | val m = ORMultiMap().addBinding(node1, "a", "A1").addBinding(node1, "a", "A2").addBinding(node1, "b", "B1") 106 | val m2 = m.remove(node1, "a") 107 | m2.entries should be(Map("b" -> Set("B1"))) 108 | } 109 | 110 | "have unapply extractor" in { 111 | val m1 = ORMultiMap.empty.put(node1, "a", Set(1L, 2L)).put(node2, "b", Set(3L)) 112 | val m2: ORMultiMap[Long] = m1 113 | val ORMultiMap(entries1) = m1 114 | val entries2: Map[String, Set[Long]] = entries1 115 | Changed("key", m1) match { 116 | case Changed("key", ORMultiMap(entries3)) => 117 | val entries4: Map[String, Set[Any]] = entries3 118 | entries4 should be(Map("a" -> Set(1L, 2L), "b" -> Set(3L))) 119 | } 120 | } 121 | } 122 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/PNCounterMap.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.cluster.Cluster 7 | import akka.cluster.UniqueAddress 8 | 9 | object PNCounterMap { 10 | val empty: PNCounterMap = new PNCounterMap(ORMap.empty) 11 | def apply(): PNCounterMap = empty 12 | /** 13 | * Java API 14 | */ 15 | def create(): PNCounterMap = empty 16 | 17 | /** 18 | * Extract the [[PNCounterMap#entries]]. 19 | */ 20 | def unapply(m: PNCounterMap): Option[Map[String, Long]] = Some(m.entries) 21 | } 22 | 23 | /** 24 | * Map of named counters. Specialized [[ORMap]] with [[PNCounter]] values. 25 | * 26 | * This class is immutable, i.e. "modifying" methods return a new instance. 27 | */ 28 | @SerialVersionUID(1L) 29 | final class PNCounterMap private[akka] ( 30 | private[akka] val underlying: ORMap[PNCounter]) 31 | extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning { 32 | 33 | type T = PNCounterMap 34 | 35 | def entries: Map[String, Long] = underlying.entries.map { case (k, c) ⇒ k -> c.value } 36 | 37 | def get(key: String): Option[Long] = underlying.get(key).map(_.value) 38 | 39 | /** 40 | * Increment the counter with the delta specified. 41 | * If the delta is negative then it will decrement instead of increment. 42 | */ 43 | def increment(key: String, delta: Long = 1)(implicit node: Cluster): PNCounterMap = 44 | increment(node, key, delta) 45 | 46 | /** 47 | * Increment the counter with the delta specified. 48 | * If the delta is negative then it will decrement instead of increment. 49 | */ 50 | def increment(node: Cluster, key: String, delta: Long): PNCounterMap = 51 | increment(node.selfUniqueAddress, key, delta) 52 | 53 | /** 54 | * INTERNAL API 55 | */ 56 | private[akka] def increment(node: UniqueAddress, key: String, delta: Long): PNCounterMap = 57 | new PNCounterMap(underlying.updated(node, key, PNCounter())(_.increment(node, delta))) 58 | 59 | /** 60 | * Decrement the counter with the delta specified. 61 | * If the delta is negative then it will increment instead of decrement. 62 | */ 63 | def decrement(key: String, delta: Long = 1)(implicit node: Cluster): PNCounterMap = 64 | decrement(node, key, delta) 65 | 66 | /** 67 | * Decrement the counter with the delta specified. 68 | * If the delta is negative then it will increment instead of decrement. 69 | */ 70 | def decrement(node: Cluster, key: String, delta: Long): PNCounterMap = 71 | decrement(node.selfUniqueAddress, key, delta) 72 | 73 | /** 74 | * INTERNAL API 75 | */ 76 | private[akka] def decrement(node: UniqueAddress, key: String, delta: Long): PNCounterMap = { 77 | new PNCounterMap(underlying.updated(node, key, PNCounter())(_.decrement(node, delta))) 78 | } 79 | 80 | /** 81 | * Removes an entry from the map. 82 | * Note that if there is a conflicting update on another node the entry will 83 | * not be removed after merge. 84 | */ 85 | def -(key: String)(implicit node: Cluster): PNCounterMap = remove(node, key) 86 | 87 | /** 88 | * Removes an entry from the map. 89 | * Note that if there is a conflicting update on another node the entry will 90 | * not be removed after merge. 91 | */ 92 | def remove(node: Cluster, key: String): PNCounterMap = 93 | remove(node.selfUniqueAddress, key) 94 | 95 | /** 96 | * INTERNAL API 97 | */ 98 | private[akka] def remove(node: UniqueAddress, key: String): PNCounterMap = 99 | new PNCounterMap(underlying.remove(node, key)) 100 | 101 | override def merge(that: PNCounterMap): PNCounterMap = 102 | new PNCounterMap(underlying.merge(that.underlying)) 103 | 104 | override def needPruningFrom(removedNode: UniqueAddress): Boolean = 105 | underlying.needPruningFrom(removedNode) 106 | 107 | override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): PNCounterMap = 108 | new PNCounterMap(underlying.prune(removedNode, collapseInto)) 109 | 110 | override def pruningCleanup(removedNode: UniqueAddress): PNCounterMap = 111 | new PNCounterMap(underlying.pruningCleanup(removedNode)) 112 | 113 | // this class cannot be a `case class` because we need different `unapply` 114 | 115 | override def toString: String = s"ORMulti$entries" 116 | 117 | override def equals(o: Any): Boolean = o match { 118 | case other: PNCounterMap => underlying == other.underlying 119 | case _ => false 120 | } 121 | 122 | override def hashCode: Int = underlying.hashCode 123 | } 124 | 125 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/sample/LotsOfDataBot.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2015 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication.sample 5 | 6 | import scala.concurrent.duration._ 7 | import scala.concurrent.forkjoin.ThreadLocalRandom 8 | import akka.actor.Actor 9 | import akka.actor.ActorLogging 10 | import akka.actor.ActorSystem 11 | import akka.actor.Props 12 | import akka.cluster.Cluster 13 | import akka.contrib.datareplication.DataReplication 14 | import akka.contrib.datareplication.ORSet 15 | import akka.contrib.datareplication.Replicator 16 | import com.typesafe.config.ConfigFactory 17 | 18 | object LotsOfDataBot { 19 | 20 | def main(args: Array[String]): Unit = { 21 | if (args.isEmpty) 22 | startup(Seq("2551", "2552", "0")) 23 | else 24 | startup(args) 25 | } 26 | 27 | def startup(ports: Seq[String]): Unit = { 28 | ports.foreach { port => 29 | // Override the configuration of the port 30 | val config = ConfigFactory.parseString("akka.remote.netty.tcp.port=" + port). 31 | withFallback(ConfigFactory.load( 32 | ConfigFactory.parseString(""" 33 | passive = off 34 | max-entries = 100000 35 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 36 | akka.remote { 37 | netty.tcp { 38 | hostname = "127.0.0.1" 39 | port = 0 40 | } 41 | } 42 | 43 | akka.cluster { 44 | seed-nodes = [ 45 | "akka.tcp://ClusterSystem@127.0.0.1:2551", 46 | "akka.tcp://ClusterSystem@127.0.0.1:2552"] 47 | 48 | auto-down-unreachable-after = 10s 49 | } 50 | akka.contrib.data-replication.use-offheap-memory = off 51 | akka.remote.log-frame-size-exceeding = 10000b 52 | """))) 53 | 54 | // Create an Akka system 55 | val system = ActorSystem("ClusterSystem", config) 56 | // Create an actor that handles cluster domain events 57 | system.actorOf(Props[LotsOfDataBot], name = "dataBot") 58 | } 59 | } 60 | 61 | private case object Tick 62 | 63 | } 64 | 65 | class LotsOfDataBot extends Actor with ActorLogging { 66 | import LotsOfDataBot._ 67 | import Replicator._ 68 | 69 | val replicator = DataReplication(context.system).replicator 70 | implicit val cluster = Cluster(context.system) 71 | 72 | import context.dispatcher 73 | val isPassive = context.system.settings.config.getBoolean("passive") 74 | var tickTask = 75 | if (isPassive) 76 | context.system.scheduler.schedule(1.seconds, 1.seconds, self, Tick) 77 | else 78 | context.system.scheduler.schedule(20.millis, 20.millis, self, Tick) 79 | 80 | val startTime = System.nanoTime() 81 | var count = 1L 82 | val maxEntries = context.system.settings.config.getInt("max-entries") 83 | 84 | def receive = if (isPassive) passive else active 85 | 86 | def active: Receive = { 87 | case Tick => 88 | val loop = if (count >= maxEntries) 1 else 100 89 | for (_ <- 1 to loop) { 90 | count += 1 91 | if (count % 10000 == 0) 92 | log.info("Reached {} entries", count) 93 | if (count == maxEntries) { 94 | log.info("Reached {} entries", count) 95 | tickTask.cancel() 96 | tickTask = context.system.scheduler.schedule(1.seconds, 1.seconds, self, Tick) 97 | } 98 | val key = (count % maxEntries).toString 99 | if (count <= 100) 100 | replicator ! Subscribe(key, self) 101 | val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString 102 | if (count <= maxEntries || ThreadLocalRandom.current().nextBoolean()) { 103 | // add 104 | replicator ! Update(key, ORSet(), WriteLocal)(_ + s) 105 | } else { 106 | // remove 107 | replicator ! Update(key, ORSet(), WriteLocal)(_ - s) 108 | } 109 | } 110 | 111 | case _: UpdateResponse => // ignore 112 | 113 | case Changed(key, ORSet(elements)) => 114 | log.info("Current elements: {} -> {}", key, elements) 115 | } 116 | 117 | def passive: Receive = { 118 | case Tick => 119 | if (!tickTask.isCancelled) 120 | replicator ! GetKeys 121 | case GetKeysResult(keys) => 122 | if (keys.size >= maxEntries) { 123 | tickTask.cancel() 124 | val duration = (System.nanoTime() - startTime).nanos.toMillis 125 | log.info("It took {} ms to replicate {} entries", duration, keys.size) 126 | } 127 | case Changed(key, ORSet(elements)) => 128 | log.info("Current elements: {} -> {}", key, elements) 129 | } 130 | 131 | override def postStop(): Unit = tickTask.cancel() 132 | 133 | } 134 | 135 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/PNCounter.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.cluster.Cluster 7 | import akka.cluster.UniqueAddress 8 | import akka.util.HashCode 9 | 10 | object PNCounter { 11 | val empty: PNCounter = new PNCounter(GCounter.empty, GCounter.empty) 12 | def apply(): PNCounter = empty 13 | /** 14 | * Java API 15 | */ 16 | def create(): PNCounter = empty 17 | 18 | /** 19 | * Extract the [[GCounter#value]]. 20 | */ 21 | def unapply(c: PNCounter): Option[Long] = Some(c.value) 22 | } 23 | 24 | /** 25 | * Implements a 'Increment/Decrement Counter' CRDT, also called a 'PN-Counter'. 26 | * 27 | * PN-Counters allow the counter to be incremented by tracking the 28 | * increments (P) separate from the decrements (N). Both P and N are represented 29 | * as two internal [[GCounter]]s. Merge is handled by merging the internal P and N 30 | * counters. The value of the counter is the value of the P counter minus 31 | * the value of the N counter. 32 | * 33 | * This class is immutable, i.e. "modifying" methods return a new instance. 34 | */ 35 | @SerialVersionUID(1L) 36 | final class PNCounter private[akka] ( 37 | private[akka] val increments: GCounter, private[akka] val decrements: GCounter) 38 | extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning { 39 | 40 | type T = PNCounter 41 | 42 | /** 43 | * Current total value of the counter. 44 | */ 45 | def value: Long = increments.value - decrements.value 46 | 47 | /** 48 | * Increment the counter with the delta specified. 49 | * If the delta is negative then it will decrement instead of increment. 50 | */ 51 | def +(delta: Long)(implicit node: Cluster): PNCounter = increment(node, delta) 52 | 53 | /** 54 | * Increment the counter with the delta specified. 55 | * If the delta is negative then it will decrement instead of increment. 56 | */ 57 | def increment(node: Cluster, delta: Long = 1): PNCounter = 58 | increment(node.selfUniqueAddress, delta) 59 | 60 | /** 61 | * Decrement the counter with the delta specified. 62 | * If the delta is negative then it will increment instead of decrement. 63 | */ 64 | def -(delta: Long)(implicit node: Cluster): PNCounter = decrement(node, delta) 65 | 66 | /** 67 | * Decrement the counter with the delta specified. 68 | * If the delta is negative then it will increment instead of decrement. 69 | */ 70 | def decrement(node: Cluster, delta: Long = 1): PNCounter = 71 | decrement(node.selfUniqueAddress, delta) 72 | 73 | private[akka] def increment(key: UniqueAddress, delta: Long): PNCounter = change(key, delta) 74 | private[akka] def increment(key: UniqueAddress): PNCounter = increment(key, 1) 75 | private[akka] def decrement(key: UniqueAddress, delta: Long): PNCounter = change(key, -delta) 76 | private[akka] def decrement(key: UniqueAddress): PNCounter = decrement(key, 1) 77 | 78 | private[akka] def change(key: UniqueAddress, delta: Long): PNCounter = 79 | if (delta > 0) copy(increments = increments.increment(key, delta)) 80 | else if (delta < 0) copy(decrements = decrements.increment(key, -delta)) 81 | else this 82 | 83 | override def merge(that: PNCounter): PNCounter = 84 | copy(increments = that.increments.merge(this.increments), 85 | decrements = that.decrements.merge(this.decrements)) 86 | 87 | override def needPruningFrom(removedNode: UniqueAddress): Boolean = 88 | increments.needPruningFrom(removedNode) || decrements.needPruningFrom(removedNode) 89 | 90 | override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): PNCounter = 91 | copy(increments = increments.prune(removedNode, collapseInto), 92 | decrements = decrements.prune(removedNode, collapseInto)) 93 | 94 | override def pruningCleanup(removedNode: UniqueAddress): PNCounter = 95 | copy(increments = increments.pruningCleanup(removedNode), 96 | decrements = decrements.pruningCleanup(removedNode)) 97 | 98 | private def copy(increments: GCounter = this.increments, decrements: GCounter = this.decrements): PNCounter = 99 | new PNCounter(increments, decrements) 100 | 101 | // this class cannot be a `case class` because we need different `unapply` 102 | 103 | override def toString: String = s"PNCounter($value)" 104 | 105 | override def equals(o: Any): Boolean = o match { 106 | case other: PNCounter => 107 | increments == other.increments && decrements == other.decrements 108 | case _ => false 109 | } 110 | 111 | override def hashCode: Int = { 112 | var result = HashCode.SEED 113 | result = HashCode.hash(result, increments) 114 | result = HashCode.hash(result, decrements) 115 | result 116 | } 117 | 118 | } 119 | 120 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/LWWMap.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.cluster.Cluster 7 | import akka.cluster.UniqueAddress 8 | 9 | object LWWMap { 10 | private val _empty: LWWMap[Any] = new LWWMap(ORMap.empty) 11 | def empty[A]: LWWMap[A] = _empty.asInstanceOf[LWWMap[A]] 12 | def apply(): LWWMap[Any] = _empty 13 | /** 14 | * Java API 15 | */ 16 | def create[A](): LWWMap[A] = empty 17 | 18 | /** 19 | * Extract the [[LWWMap#entries]]. 20 | */ 21 | def unapply[A](m: LWWMap[A]): Option[Map[String, A]] = Some(m.entries) 22 | } 23 | 24 | /** 25 | * Specialized [[ORMap]] with [[LWWRegister]] values. 26 | * 27 | * `LWWRegister` relies on synchronized clocks and should only be used when the choice of 28 | * value is not important for concurrent updates occurring within the clock skew. 29 | * 30 | * Instead of using timestamps based on `System.currentTimeMillis()` time it is possible to 31 | * use a timestamp value based on something else, for example an increasing version number 32 | * from a database record that is used for optimistic concurrency control. 33 | * 34 | * For first-write-wins semantics you can use the [[LWWRegister#reverseClock]] instead of the 35 | * [[LWWRegister#defaultClock]] 36 | * 37 | * This class is immutable, i.e. "modifying" methods return a new instance. 38 | */ 39 | @SerialVersionUID(1L) 40 | final class LWWMap[A] private[akka] ( 41 | private[akka] val underlying: ORMap[LWWRegister[A]]) 42 | extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning { 43 | import LWWRegister.{ Clock, defaultClock } 44 | 45 | type T = LWWMap[A] 46 | 47 | def entries: Map[String, A] = underlying.entries.map { case (k, r) ⇒ k -> r.value } 48 | 49 | def get(key: String): Option[A] = underlying.get(key).map(_.value) 50 | 51 | /** 52 | * Adds an entry to the map 53 | */ 54 | def +(entry: (String, A))(implicit node: Cluster): LWWMap[A] = { 55 | val (key, value) = entry 56 | put(node, key, value) 57 | } 58 | 59 | /** 60 | * Adds an entry to the map 61 | */ 62 | def put(node: Cluster, key: String, value: A): LWWMap[A] = 63 | put(node, key, value, defaultClock) 64 | 65 | /** 66 | * Adds an entry to the map. 67 | * 68 | * You can provide your `clock` implementation instead of using timestamps based 69 | * on ´System.currentTimeMillis()` time. The timestamp can for example be an 70 | * increasing version number from a database record that is used for optimistic 71 | * concurrency control. 72 | */ 73 | def put(node: Cluster, key: String, value: A, clock: Clock): LWWMap[A] = 74 | put(node.selfUniqueAddress, key, value, clock) 75 | 76 | /** 77 | * INTERNAL API 78 | */ 79 | private[akka] def put(node: UniqueAddress, key: String, value: A, clock: Clock): LWWMap[A] = { 80 | val newRegister = underlying.get(key) match { 81 | case Some(r) ⇒ r.withValue(node, value, clock) 82 | case None ⇒ LWWRegister(node, value, clock) 83 | } 84 | new LWWMap(underlying.put(node, key, newRegister)) 85 | } 86 | 87 | /** 88 | * Removes an entry from the map. 89 | * Note that if there is a conflicting update on another node the entry will 90 | * not be removed after merge. 91 | */ 92 | def -(key: String)(implicit node: Cluster): LWWMap[A] = remove(node, key) 93 | 94 | /** 95 | * Removes an entry from the map. 96 | * Note that if there is a conflicting update on another node the entry will 97 | * not be removed after merge. 98 | */ 99 | def remove(node: Cluster, key: String): LWWMap[A] = 100 | remove(node.selfUniqueAddress, key) 101 | 102 | /** 103 | * INTERNAL API 104 | */ 105 | private[akka] def remove(node: UniqueAddress, key: String): LWWMap[A] = 106 | new LWWMap(underlying.remove(node, key)) 107 | 108 | override def merge(that: LWWMap[A]): LWWMap[A] = 109 | new LWWMap(underlying.merge(that.underlying)) 110 | 111 | override def needPruningFrom(removedNode: UniqueAddress): Boolean = 112 | underlying.needPruningFrom(removedNode) 113 | 114 | override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): LWWMap[A] = 115 | new LWWMap(underlying.prune(removedNode, collapseInto)) 116 | 117 | override def pruningCleanup(removedNode: UniqueAddress): LWWMap[A] = 118 | new LWWMap(underlying.pruningCleanup(removedNode)) 119 | 120 | // this class cannot be a `case class` because we need different `unapply` 121 | 122 | override def toString: String = s"LWW$entries" //e.g. LWWMap(a -> 1, b -> 2) 123 | 124 | override def equals(o: Any): Boolean = o match { 125 | case other: LWWMap[_] => underlying == other.underlying 126 | case _ => false 127 | } 128 | 129 | override def hashCode: Int = underlying.hashCode 130 | } 131 | 132 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/protobuf/SerializationSupport.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication.protobuf 5 | 6 | import java.io.ByteArrayInputStream 7 | import java.io.ByteArrayOutputStream 8 | import java.util.zip.GZIPInputStream 9 | import java.util.zip.GZIPOutputStream 10 | import scala.annotation.tailrec 11 | import com.google.protobuf.ByteString 12 | import com.google.protobuf.MessageLite 13 | import akka.actor.ActorRef 14 | import akka.actor.Address 15 | import akka.actor.ExtendedActorSystem 16 | import akka.contrib.datareplication.protobuf.msg.{ ReplicatorMessages ⇒ dm } 17 | import akka.serialization.SerializationExtension 18 | import akka.serialization.JSerializer 19 | import akka.cluster.UniqueAddress 20 | import akka.serialization.Serialization 21 | 22 | /** 23 | * Some useful serialization helper methods. 24 | */ 25 | trait SerializationSupport { 26 | 27 | private final val BufferSize = 1024 * 4 28 | 29 | def system: ExtendedActorSystem 30 | 31 | @volatile 32 | private var ser: Serialization = _ 33 | def serialization: Serialization = { 34 | if (ser == null) ser = SerializationExtension(system) 35 | ser 36 | } 37 | 38 | @volatile 39 | private var protocol: String = _ 40 | def addressProtocol: String = { 41 | if (protocol == null) protocol = system.provider.getDefaultAddress.protocol 42 | protocol 43 | } 44 | 45 | @volatile 46 | private var transportInfo: Serialization.Information = _ 47 | def transportInformation: Serialization.Information = { 48 | if (transportInfo == null) { 49 | val address = system.provider.getDefaultAddress 50 | transportInfo = Serialization.Information(address, system) 51 | } 52 | transportInfo 53 | } 54 | 55 | def compress(msg: MessageLite): Array[Byte] = { 56 | val bos = new ByteArrayOutputStream(BufferSize) 57 | val zip = new GZIPOutputStream(bos) 58 | msg.writeTo(zip) 59 | zip.close() 60 | bos.toByteArray 61 | } 62 | 63 | def decompress(bytes: Array[Byte]): Array[Byte] = { 64 | val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) 65 | val out = new ByteArrayOutputStream() 66 | val buffer = new Array[Byte](BufferSize) 67 | 68 | @tailrec def readChunk(): Unit = in.read(buffer) match { 69 | case -1 ⇒ () 70 | case n ⇒ 71 | out.write(buffer, 0, n) 72 | readChunk() 73 | } 74 | 75 | readChunk() 76 | out.toByteArray 77 | } 78 | 79 | def addressToProto(address: Address): dm.Address.Builder = address match { 80 | case Address(_, _, Some(host), Some(port)) ⇒ 81 | dm.Address.newBuilder().setHostname(host).setPort(port) 82 | case _ ⇒ throw new IllegalArgumentException(s"Address [${address}] could not be serialized: host or port missing.") 83 | } 84 | 85 | def addressFromProto(address: dm.Address): Address = 86 | Address(addressProtocol, system.name, address.getHostname, address.getPort) 87 | 88 | def uniqueAddressToProto(uniqueAddress: UniqueAddress): dm.UniqueAddress.Builder = 89 | dm.UniqueAddress.newBuilder().setAddress(addressToProto(uniqueAddress.address)).setUid(uniqueAddress.uid) 90 | 91 | def uniqueAddressFromProto(uniqueAddress: dm.UniqueAddress): UniqueAddress = 92 | UniqueAddress(addressFromProto(uniqueAddress.getAddress), uniqueAddress.getUid) 93 | 94 | def resolveActorRef(path: String): ActorRef = 95 | system.provider.resolveActorRef(path) 96 | 97 | def otherMessageToProto(msg: Any): dm.OtherMessage = { 98 | def buildOther(): dm.OtherMessage = { 99 | val m = msg.asInstanceOf[AnyRef] 100 | val msgSerializer = serialization.findSerializerFor(m) 101 | val builder = dm.OtherMessage.newBuilder(). 102 | setEnclosedMessage(ByteString.copyFrom(msgSerializer.toBinary(m))) 103 | .setSerializerId(msgSerializer.identifier) 104 | if (msgSerializer.includeManifest) 105 | builder.setMessageManifest(ByteString.copyFromUtf8(m.getClass.getName)) 106 | builder.build() 107 | } 108 | 109 | // Serialize actor references with full address information (defaultAddress). 110 | // When sending remote messages currentTransportInformation is already set, 111 | // but when serializing for digests it must be set here. 112 | if (Serialization.currentTransportInformation.value == null) 113 | Serialization.currentTransportInformation.withValue(transportInformation) { buildOther() } 114 | else 115 | buildOther() 116 | } 117 | 118 | def otherMessageFromProto(other: dm.OtherMessage): AnyRef = { 119 | serialization.deserialize( 120 | other.getEnclosedMessage.toByteArray, 121 | other.getSerializerId, 122 | if (other.hasMessageManifest) 123 | Some(system.dynamicAccess.getClassFor[AnyRef](other.getMessageManifest.toStringUtf8).get) else None).get 124 | } 125 | 126 | } 127 | 128 | /** 129 | * Java API 130 | */ 131 | abstract class AbstractSerializationSupport extends JSerializer with SerializationSupport 132 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/GCounterSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.actor.Address 10 | import akka.cluster.UniqueAddress 11 | import akka.contrib.datareplication.Replicator.Changed 12 | 13 | class GCounterSpec extends WordSpec with Matchers { 14 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 15 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 16 | 17 | "A GCounter" must { 18 | 19 | "be able to increment each node's record by one" in { 20 | val c1 = GCounter() 21 | 22 | val c2 = c1 increment node1 23 | val c3 = c2 increment node1 24 | 25 | val c4 = c3 increment node2 26 | val c5 = c4 increment node2 27 | val c6 = c5 increment node2 28 | 29 | c6.state(node1) should be(2) 30 | c6.state(node2) should be(3) 31 | } 32 | 33 | "be able to increment each node's record by arbitrary delta" in { 34 | val c1 = GCounter() 35 | 36 | val c2 = c1 increment (node1, 3) 37 | val c3 = c2 increment (node1, 4) 38 | 39 | val c4 = c3 increment (node2, 2) 40 | val c5 = c4 increment (node2, 7) 41 | val c6 = c5 increment node2 42 | 43 | c6.state(node1) should be(7) 44 | c6.state(node2) should be(10) 45 | } 46 | 47 | "be able to summarize the history to the correct aggregated value" in { 48 | val c1 = GCounter() 49 | 50 | val c2 = c1 increment (node1, 3) 51 | val c3 = c2 increment (node1, 4) 52 | 53 | val c4 = c3 increment (node2, 2) 54 | val c5 = c4 increment (node2, 7) 55 | val c6 = c5 increment node2 56 | 57 | c6.state(node1) should be(7) 58 | c6.state(node2) should be(10) 59 | 60 | c6.value should be(17) 61 | } 62 | 63 | "be able to have its history correctly merged with another GCounter 1" in { 64 | // counter 1 65 | val c11 = GCounter() 66 | val c12 = c11 increment (node1, 3) 67 | val c13 = c12 increment (node1, 4) 68 | val c14 = c13 increment (node2, 2) 69 | val c15 = c14 increment (node2, 7) 70 | val c16 = c15 increment node2 71 | 72 | c16.state(node1) should be(7) 73 | c16.state(node2) should be(10) 74 | c16.value should be(17) 75 | 76 | // counter 1 77 | val c21 = GCounter() 78 | val c22 = c21 increment (node1, 2) 79 | val c23 = c22 increment (node1, 2) 80 | val c24 = c23 increment (node2, 3) 81 | val c25 = c24 increment (node2, 2) 82 | val c26 = c25 increment node2 83 | 84 | c26.state(node1) should be(4) 85 | c26.state(node2) should be(6) 86 | c26.value should be(10) 87 | 88 | // merge both ways 89 | val merged1 = c16 merge c26 90 | merged1.state(node1) should be(7) 91 | merged1.state(node2) should be(10) 92 | merged1.value should be(17) 93 | 94 | val merged2 = c26 merge c16 95 | merged2.state(node1) should be(7) 96 | merged2.state(node2) should be(10) 97 | merged2.value should be(17) 98 | } 99 | 100 | "be able to have its history correctly merged with another GCounter 2" in { 101 | // counter 1 102 | val c11 = GCounter() 103 | val c12 = c11 increment (node1, 2) 104 | val c13 = c12 increment (node1, 2) 105 | val c14 = c13 increment (node2, 2) 106 | val c15 = c14 increment (node2, 7) 107 | val c16 = c15 increment node2 108 | 109 | c16.state(node1) should be(4) 110 | c16.state(node2) should be(10) 111 | c16.value should be(14) 112 | 113 | // counter 1 114 | val c21 = GCounter() 115 | val c22 = c21 increment (node1, 3) 116 | val c23 = c22 increment (node1, 4) 117 | val c24 = c23 increment (node2, 3) 118 | val c25 = c24 increment (node2, 2) 119 | val c26 = c25 increment node2 120 | 121 | c26.state(node1) should be(7) 122 | c26.state(node2) should be(6) 123 | c26.value should be(13) 124 | 125 | // merge both ways 126 | val merged1 = c16 merge c26 127 | merged1.state(node1) should be(7) 128 | merged1.state(node2) should be(10) 129 | merged1.value should be(17) 130 | 131 | val merged2 = c26 merge c16 132 | merged2.state(node1) should be(7) 133 | merged2.state(node2) should be(10) 134 | merged2.value should be(17) 135 | } 136 | 137 | "have support for pruning" in { 138 | val c1 = GCounter() 139 | val c2 = c1 increment node1 140 | val c3 = c2 increment node2 141 | c2.needPruningFrom(node1) should be(true) 142 | c2.needPruningFrom(node2) should be(false) 143 | c3.needPruningFrom(node1) should be(true) 144 | c3.needPruningFrom(node2) should be(true) 145 | 146 | val c4 = c3.prune(node1, node2) 147 | c4.needPruningFrom(node2) should be(true) 148 | c4.needPruningFrom(node1) should be(false) 149 | 150 | val c5 = (c4 increment node1).pruningCleanup(node1) 151 | c5.needPruningFrom(node1) should be(false) 152 | } 153 | 154 | "have unapply extractor" in { 155 | val c1 = GCounter.empty.increment(node1).increment(node2) 156 | val GCounter(value1) = c1 157 | val value2: Long = value1 158 | Changed("key", c1) match { 159 | case Changed("key", GCounter(value3)) => 160 | val value4: Long = value3 161 | value4 should be(2L) 162 | } 163 | } 164 | 165 | } 166 | } 167 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/LWWRegister.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.cluster.Cluster 7 | import akka.cluster.UniqueAddress 8 | import akka.util.HashCode 9 | 10 | object LWWRegister { 11 | 12 | abstract class Clock { 13 | /** 14 | * @param currentTimestamp the current `timestamp` value of the `LWWRegister` 15 | */ 16 | def nextTimestamp(currentTimestamp: Long): Long 17 | } 18 | 19 | final case class ClockValue(value: Long) extends Clock { 20 | override def nextTimestamp(currentTimestamp: Long): Long = value 21 | } 22 | 23 | /** 24 | * The default [[Clock]] is using max value of `System.currentTimeMillis()` 25 | * and `currentTimestamp + 1`. 26 | */ 27 | val defaultClock = new Clock { 28 | override def nextTimestamp(currentTimestamp: Long): Long = 29 | math.max(System.currentTimeMillis(), currentTimestamp + 1) 30 | } 31 | 32 | /** 33 | * This [[Clock]] can be used for first-write-wins semantics. It is using min value of 34 | * `-System.currentTimeMillis()` and `currentTimestamp + 1`, i.e. it is counting backwards. 35 | */ 36 | val reverseClock = new Clock { 37 | override def nextTimestamp(currentTimestamp: Long): Long = 38 | math.min(-System.currentTimeMillis(), currentTimestamp - 1) 39 | } 40 | 41 | /** 42 | * INTERNAL API 43 | */ 44 | private[akka] def apply[A](node: UniqueAddress, initialValue: A, clock: Clock): LWWRegister[A] = 45 | new LWWRegister(node, initialValue, clock.nextTimestamp(0L)) 46 | 47 | def apply[A](node: Cluster, initialValue: A, clock: Clock = defaultClock): LWWRegister[A] = 48 | apply(node.selfUniqueAddress, initialValue, clock) 49 | 50 | /** 51 | * Java API 52 | */ 53 | def create[A](node: Cluster, initialValue: A): LWWRegister[A] = 54 | apply(node, initialValue) 55 | 56 | /** 57 | * Java API 58 | */ 59 | def create[A](node: Cluster, initialValue: A, clock: Clock): LWWRegister[A] = 60 | apply(node, initialValue, clock) 61 | 62 | /** 63 | * Extract the [[LWWRegister#value]]. 64 | */ 65 | def unapply[A](c: LWWRegister[A]): Option[A] = Some(c.value) 66 | 67 | } 68 | 69 | /** 70 | * Implements a 'Last Writer Wins Register' CRDT, also called a 'LWW-Register'. 71 | * 72 | * Merge takes the the register with highest timestamp. Note that this 73 | * relies on synchronized clocks. `LWWRegister` should only be used when the choice of 74 | * value is not important for concurrent updates occurring within the clock skew. 75 | * 76 | * Merge takes the register updated by the node with lowest address (`UniqueAddress` is ordered) 77 | * if the timestamps are exactly the same. 78 | * 79 | * Instead of using timestamps based on ´System.currentTimeMillis()` time it is possible to 80 | * use a timestamp value based on something else, for example an increasing version number 81 | * from a database record that is used for optimistic concurrency control. 82 | * 83 | * For first-write-wins semantics you can use the [[LWWRegister#reverseClock]] instead of the 84 | * [[LWWRegister#defaultClock]] 85 | * 86 | * This class is immutable, i.e. "modifying" methods return a new instance. 87 | */ 88 | @SerialVersionUID(1L) 89 | final class LWWRegister[A] private[akka] ( 90 | private[akka] val node: UniqueAddress, 91 | val value: A, 92 | val timestamp: Long) 93 | extends ReplicatedData with ReplicatedDataSerialization { 94 | import LWWRegister.{ Clock, defaultClock } 95 | 96 | type T = LWWRegister[A] 97 | 98 | /** 99 | * Java API 100 | */ 101 | def getValue(): A = value 102 | 103 | /** 104 | * Change the value of the register. 105 | */ 106 | def withValue(node: Cluster, value: A): LWWRegister[A] = 107 | withValue(node, value, defaultClock) 108 | 109 | /** 110 | * Change the value of the register. 111 | * 112 | * You can provide your `clock` implementation instead of using timestamps based 113 | * on ´System.currentTimeMillis()` time. The timestamp can for example be an 114 | * increasing version number from a database record that is used for optimistic 115 | * concurrency control. 116 | */ 117 | def withValue(node: Cluster, value: A, clock: Clock): LWWRegister[A] = 118 | withValue(node.selfUniqueAddress, value, clock) 119 | 120 | /** 121 | * The current `value` was set by this node. 122 | */ 123 | def updatedBy: UniqueAddress = node 124 | 125 | /** 126 | * INTERNAL API 127 | */ 128 | private[akka] def withValue(node: UniqueAddress, value: A, clock: Clock): LWWRegister[A] = 129 | new LWWRegister(node, value, clock.nextTimestamp(timestamp)) 130 | 131 | override def merge(that: LWWRegister[A]): LWWRegister[A] = 132 | if (that.timestamp > this.timestamp) that 133 | else if (that.timestamp < this.timestamp) this 134 | else if (that.node < this.node) that 135 | else this 136 | 137 | // this class cannot be a `case class` because we need different `unapply` 138 | 139 | override def toString: String = s"LWWRegister($value)" 140 | 141 | override def equals(o: Any): Boolean = o match { 142 | case other: LWWRegister[_] => 143 | timestamp == other.timestamp && value == other.value && node == other.node 144 | case _ => false 145 | } 146 | 147 | override def hashCode: Int = { 148 | var result = HashCode.SEED 149 | result = HashCode.hash(result, timestamp) 150 | result = HashCode.hash(result, node) 151 | result = HashCode.hash(result, value) 152 | result 153 | } 154 | 155 | } 156 | 157 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/PNCounterSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.actor.Address 10 | import akka.cluster.UniqueAddress 11 | import akka.contrib.datareplication.Replicator.Changed 12 | 13 | class PNCounterSpec extends WordSpec with Matchers { 14 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 15 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 16 | 17 | "A PNCounter" must { 18 | 19 | "be able to increment each node's record by one" in { 20 | val c1 = PNCounter() 21 | 22 | val c2 = c1 increment node1 23 | val c3 = c2 increment node1 24 | 25 | val c4 = c3 increment node2 26 | val c5 = c4 increment node2 27 | val c6 = c5 increment node2 28 | 29 | c6.increments.state(node1) should be(2) 30 | c6.increments.state(node2) should be(3) 31 | } 32 | 33 | "be able to decrement each node's record by one" in { 34 | val c1 = PNCounter() 35 | 36 | val c2 = c1 decrement node1 37 | val c3 = c2 decrement node1 38 | 39 | val c4 = c3 decrement node2 40 | val c5 = c4 decrement node2 41 | val c6 = c5 decrement node2 42 | 43 | c6.decrements.state(node1) should be(2) 44 | c6.decrements.state(node2) should be(3) 45 | } 46 | 47 | "be able to increment each node's record by arbitrary delta" in { 48 | val c1 = PNCounter() 49 | 50 | val c2 = c1 increment (node1, 3) 51 | val c3 = c2 increment (node1, 4) 52 | 53 | val c4 = c3 increment (node2, 2) 54 | val c5 = c4 increment (node2, 7) 55 | val c6 = c5 increment node2 56 | 57 | c6.increments.state(node1) should be(7) 58 | c6.increments.state(node2) should be(10) 59 | } 60 | 61 | "be able to decrement each node's record by arbitrary delta" in { 62 | val c1 = PNCounter() 63 | 64 | val c2 = c1 decrement (node1, 3) 65 | val c3 = c2 decrement (node1, 4) 66 | 67 | val c4 = c3 decrement (node2, 2) 68 | val c5 = c4 decrement (node2, 7) 69 | val c6 = c5 decrement node2 70 | 71 | c6.decrements.state(node1) should be(7) 72 | c6.decrements.state(node2) should be(10) 73 | } 74 | 75 | "be able to increment and decrement each node's record by arbitrary delta" in { 76 | val c1 = PNCounter() 77 | 78 | val c2 = c1 increment (node1, 3) 79 | val c3 = c2 decrement (node1, 2) 80 | 81 | val c4 = c3 increment (node2, 5) 82 | val c5 = c4 decrement (node2, 2) 83 | val c6 = c5 increment node2 84 | 85 | c6.increments.value should be(9) 86 | c6.decrements.value should be(4) 87 | } 88 | 89 | "be able to summarize the history to the correct aggregated value of increments and decrements" in { 90 | val c1 = PNCounter() 91 | 92 | val c2 = c1 increment (node1, 3) 93 | val c3 = c2 decrement (node1, 2) 94 | 95 | val c4 = c3 increment (node2, 5) 96 | val c5 = c4 decrement (node2, 2) 97 | val c6 = c5 increment node2 98 | 99 | c6.increments.value should be(9) 100 | c6.decrements.value should be(4) 101 | 102 | c6.value should be(5) 103 | } 104 | 105 | "be able to have its history correctly merged with another GCounter" in { 106 | // counter 1 107 | val c11 = PNCounter() 108 | val c12 = c11 increment (node1, 3) 109 | val c13 = c12 decrement (node1, 2) 110 | val c14 = c13 increment (node2, 5) 111 | val c15 = c14 decrement (node2, 2) 112 | val c16 = c15 increment node2 113 | 114 | c16.increments.value should be(9) 115 | c16.decrements.value should be(4) 116 | c16.value should be(5) 117 | 118 | // counter 1 119 | val c21 = PNCounter() 120 | val c22 = c21 increment (node1, 2) 121 | val c23 = c22 decrement (node1, 3) 122 | val c24 = c23 increment (node2, 3) 123 | val c25 = c24 decrement (node2, 2) 124 | val c26 = c25 increment node2 125 | 126 | c26.increments.value should be(6) 127 | c26.decrements.value should be(5) 128 | c26.value should be(1) 129 | 130 | // merge both ways 131 | val merged1 = c16 merge c26 132 | merged1.increments.value should be(9) 133 | merged1.decrements.value should be(5) 134 | merged1.value should be(4) 135 | 136 | val merged2 = c26 merge c16 137 | merged2.increments.value should be(9) 138 | merged2.decrements.value should be(5) 139 | merged2.value should be(4) 140 | } 141 | 142 | "have support for pruning" in { 143 | val c1 = PNCounter() 144 | val c2 = c1 increment node1 145 | val c3 = c2 decrement node2 146 | c2.needPruningFrom(node1) should be(true) 147 | c2.needPruningFrom(node2) should be(false) 148 | c3.needPruningFrom(node1) should be(true) 149 | c3.needPruningFrom(node2) should be(true) 150 | 151 | val c4 = c3.prune(node1, node2) 152 | c4.needPruningFrom(node2) should be(true) 153 | c4.needPruningFrom(node1) should be(false) 154 | 155 | val c5 = (c4 increment node1).pruningCleanup(node1) 156 | c5.needPruningFrom(node1) should be(false) 157 | } 158 | 159 | "have unapply extractor" in { 160 | val c1 = PNCounter.empty.increment(node1).increment(node1).decrement(node2) 161 | val PNCounter(value1) = c1 162 | val value2: Long = value1 163 | Changed("key", c1) match { 164 | case Changed("key", PNCounter(value3)) => 165 | val value4: Long = value3 166 | value4 should be(1L) 167 | } 168 | } 169 | 170 | } 171 | } 172 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/sample/datareplication/VotingContestSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package sample.datareplication 5 | 6 | import scala.concurrent.duration._ 7 | import akka.actor.Actor 8 | import akka.actor.ActorRef 9 | import akka.actor.Props 10 | import akka.cluster.Cluster 11 | import akka.contrib.datareplication.DataReplication 12 | import akka.contrib.datareplication.Flag 13 | import akka.contrib.datareplication.PNCounterMap 14 | import akka.contrib.datareplication.Replicator.GetReplicaCount 15 | import akka.contrib.datareplication.Replicator.ReplicaCount 16 | import akka.contrib.datareplication.STMultiNodeSpec 17 | import akka.remote.testconductor.RoleName 18 | import akka.remote.testkit.MultiNodeConfig 19 | import akka.remote.testkit.MultiNodeSpec 20 | import akka.testkit._ 21 | import com.typesafe.config.ConfigFactory 22 | 23 | object VotingContestSpec extends MultiNodeConfig { 24 | val node1 = role("node-1") 25 | val node2 = role("node-2") 26 | val node3 = role("node-3") 27 | 28 | commonConfig(ConfigFactory.parseString(""" 29 | akka.loglevel = INFO 30 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 31 | akka.log-dead-letters-during-shutdown = off 32 | """)) 33 | 34 | } 35 | 36 | object VotingService { 37 | case object Open 38 | case object OpenAck 39 | case object Close 40 | case object CloseAck 41 | final case class Vote(participant: String) 42 | case object GetVotes 43 | final case class Votes(result: Map[String, Long], open: Boolean) 44 | 45 | private final case class GetVotesReq(replyTo: ActorRef) 46 | } 47 | 48 | class VotingService extends Actor { 49 | import akka.contrib.datareplication.Replicator._ 50 | import VotingService._ 51 | 52 | val replicator = DataReplication(context.system).replicator 53 | implicit val cluster = Cluster(context.system) 54 | val OpenedKey = "contestOpened" 55 | val ClosedKey = "contestClosed" 56 | val CountersKey = "contestCounters" 57 | 58 | replicator ! Subscribe(OpenedKey, self) 59 | 60 | def receive = { 61 | case Open ⇒ 62 | replicator ! Update(OpenedKey, Flag(), WriteAll(5.seconds))(_.switchOn) 63 | becomeOpen() 64 | 65 | case Changed(OpenedKey, flag: Flag) if flag.enabled ⇒ 66 | becomeOpen() 67 | 68 | case GetVotes ⇒ 69 | sender() ! Votes(Map.empty, open = false) 70 | } 71 | 72 | def becomeOpen(): Unit = { 73 | replicator ! Unsubscribe(OpenedKey, self) 74 | replicator ! Subscribe(ClosedKey, self) 75 | context.become(open orElse getVotes(open = true)) 76 | } 77 | 78 | def open: Receive = { 79 | case v @ Vote(participant) ⇒ 80 | val update = Update(CountersKey, PNCounterMap(), WriteLocal, request = Some(v)) { 81 | _.increment(participant, 1) 82 | } 83 | replicator ! update 84 | 85 | case _: UpdateSuccess ⇒ 86 | 87 | case Close ⇒ 88 | replicator ! Update(ClosedKey, Flag(), WriteAll(5.seconds))(_.switchOn) 89 | context.become(getVotes(open = false)) 90 | 91 | case Changed(ClosedKey, flag: Flag) if flag.enabled ⇒ 92 | context.become(getVotes(open = false)) 93 | } 94 | 95 | def getVotes(open: Boolean): Receive = { 96 | case GetVotes ⇒ 97 | replicator ! Get(CountersKey, ReadAll(3.seconds), Some(GetVotesReq(sender()))) 98 | 99 | case GetSuccess(CountersKey, d: PNCounterMap, Some(GetVotesReq(replyTo))) ⇒ 100 | replyTo ! Votes(d.entries, open) 101 | 102 | case NotFound(CountersKey, Some(GetVotesReq(replyTo))) ⇒ 103 | replyTo ! Votes(Map.empty, open) 104 | 105 | case _: GetFailure ⇒ 106 | 107 | case _: UpdateSuccess ⇒ 108 | } 109 | 110 | } 111 | 112 | class VotingContestSpecMultiJvmNode1 extends VotingContestSpec 113 | class VotingContestSpecMultiJvmNode2 extends VotingContestSpec 114 | class VotingContestSpecMultiJvmNode3 extends VotingContestSpec 115 | 116 | class VotingContestSpec extends MultiNodeSpec(VotingContestSpec) with STMultiNodeSpec with ImplicitSender { 117 | import VotingContestSpec._ 118 | import ShoppingCart._ 119 | 120 | override def initialParticipants = roles.size 121 | 122 | val cluster = Cluster(system) 123 | 124 | def join(from: RoleName, to: RoleName): Unit = { 125 | runOn(from) { 126 | cluster join node(to).address 127 | } 128 | enterBarrier(from.name + "-joined") 129 | } 130 | 131 | "Demo of a replicated voting" must { 132 | 133 | "join cluster" in within(10.seconds) { 134 | join(node1, node1) 135 | join(node2, node1) 136 | join(node3, node1) 137 | 138 | awaitAssert { 139 | DataReplication(system).replicator ! GetReplicaCount 140 | expectMsg(ReplicaCount(roles.size)) 141 | } 142 | enterBarrier("after-1") 143 | } 144 | 145 | "count votes correctly" in within(15.seconds) { 146 | import VotingService._ 147 | val votingService = system.actorOf(Props[VotingService], "votingService") 148 | val N = 1000 149 | runOn(node1) { 150 | votingService ! Open 151 | for (n ← 1 to N) { 152 | votingService ! Vote("#" + ((n % 20) + 1)) 153 | } 154 | } 155 | runOn(node2, node3) { 156 | // wait for it to open 157 | val p = TestProbe() 158 | awaitAssert { 159 | votingService.tell(GetVotes, p.ref) 160 | p.expectMsgPF(3.seconds) { case Votes(_, true) ⇒ true } 161 | } 162 | for (n ← 1 to N) { 163 | votingService ! Vote("#" + ((n % 20) + 1)) 164 | } 165 | } 166 | enterBarrier("voting-done") 167 | runOn(node3) { 168 | votingService ! Close 169 | } 170 | 171 | val expected = (1 to 20).map(n ⇒ "#" + n -> (3L * N / 20)).toMap 172 | awaitAssert { 173 | votingService ! GetVotes 174 | expectMsg(3.seconds, Votes(expected, false)) 175 | } 176 | 177 | enterBarrier("after-2") 178 | } 179 | } 180 | 181 | } 182 | 183 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/sample/datareplication/ReplicatedCacheSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package sample.datareplication 5 | 6 | import scala.concurrent.duration._ 7 | import akka.actor.Actor 8 | import akka.actor.ActorRef 9 | import akka.actor.Props 10 | import akka.cluster.Cluster 11 | import akka.contrib.datareplication.DataReplication 12 | import akka.contrib.datareplication.LWWMap 13 | import akka.contrib.datareplication.Replicator.GetReplicaCount 14 | import akka.contrib.datareplication.Replicator.ReplicaCount 15 | import akka.contrib.datareplication.STMultiNodeSpec 16 | import akka.remote.testconductor.RoleName 17 | import akka.remote.testkit.MultiNodeConfig 18 | import akka.remote.testkit.MultiNodeSpec 19 | import akka.testkit._ 20 | import com.typesafe.config.ConfigFactory 21 | 22 | object ReplicatedCacheSpec extends MultiNodeConfig { 23 | val node1 = role("node-1") 24 | val node2 = role("node-2") 25 | val node3 = role("node-3") 26 | 27 | commonConfig(ConfigFactory.parseString(""" 28 | akka.loglevel = INFO 29 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 30 | akka.log-dead-letters-during-shutdown = off 31 | """)) 32 | 33 | } 34 | 35 | object ReplicatedCache { 36 | import akka.contrib.datareplication.Replicator._ 37 | 38 | def props: Props = Props[ReplicatedCache] 39 | 40 | private final case class Request(key: String, replyTo: ActorRef) 41 | 42 | final case class PutInCache(key: String, value: Any) 43 | final case class GetFromCache(key: String) 44 | final case class Cached(key: String, value: Option[Any]) 45 | final case class Evict(key: String) 46 | } 47 | 48 | class ReplicatedCache() extends Actor { 49 | import akka.contrib.datareplication.Replicator._ 50 | import ReplicatedCache._ 51 | 52 | val replicator = DataReplication(context.system).replicator 53 | implicit val cluster = Cluster(context.system) 54 | 55 | def dataKey(entryKey: String): String = 56 | "cache-" + math.abs(entryKey.hashCode) % 100 57 | 58 | def receive = { 59 | case PutInCache(key, value) => 60 | replicator ! Update(dataKey(key), LWWMap(), WriteLocal)(_ + (key -> value)) 61 | case Evict(key) => 62 | replicator ! Update(dataKey(key), LWWMap(), WriteLocal)(_ - key) 63 | case GetFromCache(key) => 64 | replicator ! Get(dataKey(key), ReadLocal, Some(Request(key, sender()))) 65 | case GetSuccess(_, data: LWWMap[Any] @unchecked, Some(Request(key, replyTo))) => 66 | data.get(key) match { 67 | case Some(value) => replyTo ! Cached(key, Some(value)) 68 | case None => replyTo ! Cached(key, None) 69 | } 70 | case NotFound(_, Some(Request(key, replyTo))) => 71 | replyTo ! Cached(key, None) 72 | case _: UpdateResponse => // ok 73 | } 74 | 75 | } 76 | 77 | class ReplicatedCacheSpecMultiJvmNode1 extends ReplicatedCacheSpec 78 | class ReplicatedCacheSpecMultiJvmNode2 extends ReplicatedCacheSpec 79 | class ReplicatedCacheSpecMultiJvmNode3 extends ReplicatedCacheSpec 80 | 81 | class ReplicatedCacheSpec extends MultiNodeSpec(ReplicatedCacheSpec) with STMultiNodeSpec with ImplicitSender { 82 | import ReplicatedCacheSpec._ 83 | import ReplicatedCache._ 84 | 85 | override def initialParticipants = roles.size 86 | 87 | val cluster = Cluster(system) 88 | val replicatedCache = system.actorOf(ReplicatedCache.props) 89 | 90 | def join(from: RoleName, to: RoleName): Unit = { 91 | runOn(from) { 92 | cluster join node(to).address 93 | } 94 | enterBarrier(from.name + "-joined") 95 | } 96 | 97 | "Demo of a replicated cache" must { 98 | "join cluster" in within(10.seconds) { 99 | join(node1, node1) 100 | join(node2, node1) 101 | join(node3, node1) 102 | 103 | awaitAssert { 104 | DataReplication(system).replicator ! GetReplicaCount 105 | expectMsg(ReplicaCount(roles.size)) 106 | } 107 | enterBarrier("after-1") 108 | } 109 | 110 | "replicate cached entry" in within(10.seconds) { 111 | runOn(node1) { 112 | replicatedCache ! PutInCache("key1", "A") 113 | } 114 | 115 | awaitAssert { 116 | val probe = TestProbe() 117 | replicatedCache.tell(GetFromCache("key1"), probe.ref) 118 | probe.expectMsg(Cached("key1", Some("A"))) 119 | } 120 | 121 | enterBarrier("after-2") 122 | } 123 | 124 | "replicate many cached entries" in within(10.seconds) { 125 | runOn(node1) { 126 | for (i <- 100 to 200) 127 | replicatedCache ! PutInCache("key" + i, i) 128 | } 129 | 130 | awaitAssert { 131 | val probe = TestProbe() 132 | for (i <- 100 to 200) { 133 | replicatedCache.tell(GetFromCache("key" + i), probe.ref) 134 | probe.expectMsg(Cached("key" + i, Some(i))) 135 | } 136 | } 137 | 138 | enterBarrier("after-3") 139 | } 140 | 141 | "replicate evicted entry" in within(15.seconds) { 142 | runOn(node1) { 143 | replicatedCache ! PutInCache("key2", "B") 144 | } 145 | 146 | awaitAssert { 147 | val probe = TestProbe() 148 | replicatedCache.tell(GetFromCache("key2"), probe.ref) 149 | probe.expectMsg(Cached("key2", Some("B"))) 150 | } 151 | enterBarrier("key2-replicated") 152 | 153 | runOn(node3) { 154 | replicatedCache ! Evict("key2") 155 | } 156 | 157 | awaitAssert { 158 | val probe = TestProbe() 159 | replicatedCache.tell(GetFromCache("key2"), probe.ref) 160 | probe.expectMsg(Cached("key2", None)) 161 | } 162 | 163 | enterBarrier("after-4") 164 | } 165 | 166 | "replicate updated cached entry" in within(10.seconds) { 167 | runOn(node2) { 168 | replicatedCache ! PutInCache("key1", "A2") 169 | replicatedCache ! PutInCache("key1", "A3") 170 | } 171 | 172 | awaitAssert { 173 | val probe = TestProbe() 174 | replicatedCache.tell(GetFromCache("key1"), probe.ref) 175 | probe.expectMsg(Cached("key1", Some("A3"))) 176 | } 177 | 178 | enterBarrier("after-5") 179 | } 180 | 181 | } 182 | 183 | } 184 | 185 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/akka/contrib/datareplication/ReplicatorPruningSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import scala.concurrent.duration._ 7 | import akka.remote.testconductor.RoleName 8 | import akka.remote.testkit.MultiNodeConfig 9 | import akka.remote.testkit.MultiNodeSpec 10 | import com.typesafe.config.ConfigFactory 11 | import akka.cluster.Cluster 12 | import akka.remote.transport.ThrottlerTransportAdapter.Direction 13 | import akka.testkit._ 14 | import akka.cluster.ClusterEvent.InitialStateAsEvents 15 | import akka.cluster.ClusterEvent.MemberUp 16 | 17 | object ReplicatorPruningSpec extends MultiNodeConfig { 18 | val first = role("first") 19 | val second = role("second") 20 | val third = role("third") 21 | 22 | commonConfig(ConfigFactory.parseString(""" 23 | akka.loglevel = INFO 24 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 25 | akka.log-dead-letters-during-shutdown = off 26 | """)) 27 | 28 | } 29 | 30 | class ReplicatorPruningSpecMultiJvmNode1 extends ReplicatorPruningSpec 31 | class ReplicatorPruningSpecMultiJvmNode2 extends ReplicatorPruningSpec 32 | class ReplicatorPruningSpecMultiJvmNode3 extends ReplicatorPruningSpec 33 | 34 | class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with STMultiNodeSpec with ImplicitSender { 35 | import ReplicatorPruningSpec._ 36 | import Replicator._ 37 | 38 | override def initialParticipants = roles.size 39 | 40 | implicit val cluster = Cluster(system) 41 | val maxPruningDissemination = 3.seconds 42 | val replicator = system.actorOf(Replicator.props( 43 | ReplicatorSettings(role = None, gossipInterval = 1.second, pruningInterval = 1.second, 44 | maxPruningDissemination = maxPruningDissemination)), "replicator") 45 | val timeout = 2.seconds.dilated 46 | 47 | def join(from: RoleName, to: RoleName): Unit = { 48 | runOn(from) { 49 | cluster join node(to).address 50 | } 51 | enterBarrier(from.name + "-joined") 52 | } 53 | 54 | "Pruning of CRDT" must { 55 | 56 | "move data from removed node" in { 57 | join(first, first) 58 | join(second, first) 59 | join(third, first) 60 | 61 | within(5.seconds) { 62 | awaitAssert { 63 | replicator ! GetReplicaCount 64 | expectMsg(ReplicaCount(3)) 65 | } 66 | } 67 | 68 | // we need the UniqueAddress 69 | val memberProbe = TestProbe() 70 | cluster.subscribe(memberProbe.ref, initialStateMode = InitialStateAsEvents, classOf[MemberUp]) 71 | val thirdUniqueAddress = { 72 | val member = memberProbe.fishForMessage(3.seconds) { 73 | case MemberUp(m) if m.address == node(third).address ⇒ true 74 | case _ ⇒ false 75 | }.asInstanceOf[MemberUp].member 76 | member.uniqueAddress 77 | } 78 | 79 | replicator ! Update("A", GCounter(), WriteAll(timeout))(_ + 3) 80 | expectMsg(UpdateSuccess("A", None)) 81 | 82 | replicator ! Update("B", ORSet(), WriteAll(timeout))(_ + "a" + "b" + "c") 83 | expectMsg(UpdateSuccess("B", None)) 84 | 85 | replicator ! Update("C", PNCounterMap(), WriteAll(timeout))(_ increment "x" increment "y") 86 | expectMsg(UpdateSuccess("C", None)) 87 | 88 | enterBarrier("updates-done") 89 | 90 | replicator ! Get("A", ReadLocal) 91 | val oldCounter = expectMsgType[GetSuccess].data.asInstanceOf[GCounter] 92 | oldCounter.value should be(9) 93 | 94 | replicator ! Get("B", ReadLocal) 95 | val oldSet = expectMsgType[GetSuccess].data.asInstanceOf[ORSet[String]] 96 | oldSet.elements should be(Set("a", "b", "c")) 97 | 98 | replicator ! Get("C", ReadLocal) 99 | val oldMap = expectMsgType[GetSuccess].data.asInstanceOf[PNCounterMap] 100 | oldMap.get("x") should be(Some(3)) 101 | oldMap.get("y") should be(Some(3)) 102 | 103 | enterBarrier("get-old") 104 | 105 | runOn(first) { 106 | cluster.leave(node(third).address) 107 | } 108 | 109 | runOn(first, second) { 110 | within(15.seconds) { 111 | awaitAssert { 112 | replicator ! GetReplicaCount 113 | expectMsg(ReplicaCount(2)) 114 | } 115 | } 116 | } 117 | enterBarrier("third-removed") 118 | 119 | runOn(first, second) { 120 | within(15.seconds) { 121 | awaitAssert { 122 | replicator ! Get("A", ReadLocal) 123 | expectMsgPF() { 124 | case GetSuccess(_, c: GCounter, _) ⇒ 125 | c.value should be(9) 126 | c.needPruningFrom(thirdUniqueAddress) should be(false) 127 | } 128 | } 129 | } 130 | within(5.seconds) { 131 | awaitAssert { 132 | replicator ! Get("B", ReadLocal) 133 | expectMsgPF() { 134 | case GetSuccess(_, s: ORSet[String] @unchecked, _) ⇒ 135 | s.elements should be(Set("a", "b", "c")) 136 | s.needPruningFrom(thirdUniqueAddress) should be(false) 137 | } 138 | } 139 | } 140 | within(5.seconds) { 141 | awaitAssert { 142 | replicator ! Get("C", ReadLocal) 143 | expectMsgPF() { 144 | case GetSuccess(_, m: PNCounterMap, _) ⇒ 145 | m.entries should be(Map("x" -> 3L, "y" -> 3L)) 146 | m.needPruningFrom(thirdUniqueAddress) should be(false) 147 | } 148 | } 149 | } 150 | } 151 | enterBarrier("pruning-done") 152 | 153 | // on one of the nodes the data has been updated by the pruning, 154 | // client can update anyway 155 | def updateAfterPruning(expectedValue: Int): Unit = { 156 | replicator ! Update("A", GCounter(), WriteAll(timeout), None)(_ + 1) 157 | expectMsgPF() { 158 | case UpdateSuccess("A", _) ⇒ 159 | replicator ! Get("A", ReadLocal) 160 | val retrieved = expectMsgType[GetSuccess].data.asInstanceOf[GCounter] 161 | retrieved.value should be(expectedValue) 162 | } 163 | } 164 | runOn(first) { 165 | updateAfterPruning(expectedValue = 10) 166 | } 167 | enterBarrier("update-first-after-pruning") 168 | 169 | runOn(second) { 170 | updateAfterPruning(expectedValue = 11) 171 | } 172 | enterBarrier("update-second-after-pruning") 173 | 174 | // after pruning performed and maxDissemination it is tombstoned 175 | // and we should still not be able to update with data from removed node 176 | expectNoMsg(maxPruningDissemination + 3.seconds) 177 | 178 | runOn(first) { 179 | updateAfterPruning(expectedValue = 12) 180 | } 181 | enterBarrier("update-first-after-tombstone") 182 | 183 | runOn(second) { 184 | updateAfterPruning(expectedValue = 13) 185 | } 186 | enterBarrier("update-second-after-tombstone") 187 | 188 | enterBarrier("after-1") 189 | } 190 | } 191 | 192 | } 193 | 194 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | Akka Distributed Data 2 | ===================== 3 | 4 | This library (akka-data-replication) has been included in Akka, in the module 5 | [Distributed Data](http://doc.akka.io/docs/akka/current/scala/distributed-data.html). 6 | 7 | **It will not be maintained in patriknw/akka-data-replication.** All bug fixes and new features 8 | will be done in [akka/akka](https://github.com/akka/akka/). 9 | 10 | Migration Guide 11 | --------------- 12 | 13 | The functionality of akka-distributed-data-experimental 2.4.0 is very similar to akka-data-replication 0.11. 14 | Here is a list of the most important changes: 15 | 16 | * Dependency 17 | `"com.typesafe.akka" % "akka-distributed-data-experimental_2.11" % 2.4.0` 18 | (or later) 19 | * The package name changed to `akka.cluster.ddata` 20 | * The extension was renamed to `DistributedData` 21 | * The keys changed from strings to classes with unique identifiers and type information of the data values, 22 | e.g. `ORSetKey[Int]("set2")` 23 | * The data value was removed from unapply extractor in `GetSuccess` and `Changed` messages. Instead it 24 | is accessed with the `get` method. E.g. `case c @ Changed(DataKey) => val e = c.get(DataKey).elements`. 25 | The reason is to utilize the type information from the typed keys. 26 | * The optional read consistency parameter was removed from the `Update` message. If you need to read from 27 | other replicas before performing the update you have to first send a `Get` message and then continue with 28 | the ``Update`` when the ``GetSuccess`` is received. 29 | * `BigInt` is used in `GCounter` and `PNCounter` instead of `Long` 30 | * Improvements of java api 31 | 32 | Akka Data Replication 33 | ===================== 34 | 35 | This was (see above) an **EARLY PREVIEW** of a library for replication of data in an Akka cluster. 36 | It is a replicated in-memory data store supporting low latency and high availability 37 | requirements. The data must be so called **Conflict Free Replicated Data Types** (CRDTs), 38 | i.e. they provide a monotonic merge function and the state changes always converge. 39 | 40 | For good introduction to CRDTs you should watch the 41 | [Eventually Consistent Data Structures](http://www.google.com/url?q=http%3A%2F%2Fvimeo.com%2F43903960&sa=D&sntz=1&usg=AFQjCNF0yKi4WGCi3bhhdtLvBc33uVia6w) 42 | talk by Sean Cribbs. 43 | 44 | CRDTs can't be used for all types of problems, but when they can they have very nice properties: 45 | 46 | - low latency of both read and writes 47 | - high availability (partition tolerance) 48 | - scalable (no central coordinator) 49 | - strong eventual consistency (eventual consistency without conflicts) 50 | 51 | Built in data types: 52 | 53 | - Counters: `GCounter`, `PNCounter` 54 | - Registers: `LWWRegister`, `Flag` 55 | - Sets: `GSet`, `ORSet` 56 | - Maps: `ORMap`, `LWWMap`, `PNCounterMap`, `ORMultiMap` 57 | 58 | You can use your own custom data types by implementing the `merge` function of the `ReplicatedData` 59 | trait. Note that CRDTs typically compose nicely, i.e. you can use the provided data types to build richer 60 | data structures. 61 | 62 | The `Replicator` actor implements the infrastructure for replication of the data. It uses 63 | direct replication and gossip based dissemination. The `Replicator` actor is started on each node 64 | in the cluster, or group of nodes tagged with a specific role. It communicates with other 65 | `Replicator` instances with the same path (without address) that are running on other nodes. 66 | For convenience it is typically used with the `DataReplication` Akka extension. 67 | 68 | A short example of how to use it: 69 | 70 | ``` scala 71 | class DataBot extends Actor with ActorLogging { 72 | import DataBot._ 73 | import Replicator._ 74 | 75 | val replicator = DataReplication(context.system).replicator 76 | implicit val cluster = Cluster(context.system) 77 | 78 | import context.dispatcher 79 | val tickTask = context.system.scheduler.schedule(5.seconds, 5.seconds, self, Tick) 80 | 81 | replicator ! Subscribe("key", self) 82 | 83 | def receive = { 84 | case Tick => 85 | val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString 86 | if (ThreadLocalRandom.current().nextBoolean()) { 87 | // add 88 | log.info("Adding: {}", s) 89 | replicator ! Update("key", ORSet(), WriteLocal)(_ + s) 90 | } else { 91 | // remove 92 | log.info("Removing: {}", s) 93 | replicator ! Update("key", ORSet(), WriteLocal)(_ - s) 94 | } 95 | 96 | case _: UpdateResponse => // ignore 97 | 98 | case Changed("key", ORSet(elements) => 99 | log.info("Current elements: {}", elements) 100 | } 101 | 102 | override def postStop(): Unit = tickTask.cancel() 103 | 104 | } 105 | ``` 106 | 107 | The full source code for this sample is in 108 | [DataBot.scala](https://github.com/patriknw/akka-data-replication/blob/v0.11/src/test/scala/akka/contrib/datareplication/sample/DataBot.scala). 109 | 110 | More detailed documentation can be found in the 111 | [ScalaDoc](http://dl.bintray.com/patriknw/maven/com/github/patriknw/akka-data-replication_2.11/0.11/akka-data-replication_2.11-0.11-javadoc.jar) 112 | of `Replicator` and linked classes. 113 | 114 | Other examples: 115 | 116 | - [Replicated Cache](https://github.com/patriknw/akka-data-replication/blob/v0.11/src/multi-jvm/scala/sample/datareplication/ReplicatedCacheSpec.scala#L30) 117 | - [Replicated Metrics](https://github.com/patriknw/akka-data-replication/blob/v0.11/src/multi-jvm/scala/sample/datareplication/ReplicatedMetricsSpec.scala#L30) 118 | - [Replicated Service Registry](https://github.com/patriknw/akka-data-replication/blob/v0.11/src/multi-jvm/scala/sample/datareplication/ReplicatedServiceRegistrySpec.scala#L46) 119 | - [VotingService](https://github.com/patriknw/akka-data-replication/blob/v0.11/src/multi-jvm/scala/sample/datareplication/VotingContestSpec.scala#L30) 120 | - [ShoppingCart](https://github.com/patriknw/akka-data-replication/blob/v0.11/src/multi-jvm/scala/sample/datareplication/ReplicatedShoppingCartSpec.scala#L31) 121 | 122 | Dependency 123 | ---------- 124 | 125 | Latest version of `akka-data-replication` is `0.11`. This version depends on Akka 2.3.9 and is 126 | cross-built against Scala 2.10.5 and 2.11.6. 127 | 128 | Add the following lines to your `build.sbt` file: 129 | 130 | resolvers += "patriknw at bintray" at "http://dl.bintray.com/patriknw/maven" 131 | 132 | libraryDependencies += "com.github.patriknw" %% "akka-data-replication" % "0.11" 133 | 134 | More Resources 135 | -------------- 136 | 137 | * [Eventually Consistent Data Structures](http://www.google.com/url?q=http%3A%2F%2Fvimeo.com%2F43903960&sa=D&sntz=1&usg=AFQjCNF0yKi4WGCi3bhhdtLvBc33uVia6w) 138 | talk by Sean Cribbs 139 | * [Strong Eventual Consistency and Conflict-free Replicated Data Types](http://www.google.com/url?q=http%3A%2F%2Fresearch.microsoft.com%2Fapps%2Fvideo%2Fdl.aspx%3Fid%3D153540&sa=D&sntz=1&usg=AFQjCNFiwLpLjF-AQXPUm1Nmoy8hNIfrSQ) 140 | talk by Mark Shapiro 141 | * [A comprehensive study of Convergent and Commutative Replicated Data Types](http://www.google.com/url?q=http%3A%2F%2Fhal.upmc.fr%2Fdocs%2F00%2F55%2F55%2F88%2FPDF%2Ftechreport.pdf&sa=D&sntz=1&usg=AFQjCNEGvFJ9I5m7yKpcAs8hcMP9Y5vy6A) 142 | paper by Mark Shapiro et. al. 143 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/ORMultiMap.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.cluster.{ UniqueAddress, Cluster } 7 | 8 | object ORMultiMap { 9 | 10 | val _empty: ORMultiMap[Any] = new ORMultiMap(ORMap.empty) 11 | /** 12 | * Provides an empty multimap. 13 | */ 14 | def empty[A]: ORMultiMap[A] = _empty.asInstanceOf[ORMultiMap[A]] 15 | def apply(): ORMultiMap[Any] = _empty 16 | 17 | /** 18 | * Java API 19 | */ 20 | def create[A](): ORMultiMap[A] = empty[A] 21 | 22 | /** 23 | * Extract the [[ORMultiMap#elements]]. 24 | */ 25 | def unapply[A](m: ORMultiMap[A]): Option[Map[String, Set[A]]] = Some(m.entries) 26 | 27 | /** 28 | * Extract the [[ORMultiMap#elements]] of an `ORMultiMap`. 29 | */ 30 | def unapply(value: Any): Option[Map[String, Set[Any]]] = value match { 31 | case m: ORMultiMap[Any] @unchecked ⇒ Some(m.entries) 32 | case _ ⇒ None 33 | } 34 | } 35 | 36 | /** 37 | * An immutable multi-map implementation. This class wraps an 38 | * [[ORMap]] with an [[ORSet]] for the map's value. 39 | * 40 | * This class is immutable, i.e. "modifying" methods return a new instance. 41 | */ 42 | @SerialVersionUID(1L) 43 | final class ORMultiMap[A] private[akka] (private[akka] val underlying: ORMap[ORSet[A]]) 44 | extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning { 45 | 46 | override type T = ORMultiMap[A] 47 | 48 | override def merge(that: T): T = 49 | new ORMultiMap(underlying.merge(that.underlying)) 50 | 51 | /** 52 | * Scala API: All entries of a multimap where keys are strings and values are sets. 53 | */ 54 | def entries: Map[String, Set[A]] = 55 | underlying.entries.map { case (k, v) ⇒ k -> v.elements } 56 | 57 | /** 58 | * Java API: All entries of a multimap where keys are strings and values are sets. 59 | */ 60 | def getEntries(): java.util.Map[String, java.util.Set[A]] = { 61 | import scala.collection.JavaConverters._ 62 | val result = new java.util.HashMap[String, java.util.Set[A]] 63 | underlying.entries.foreach { 64 | case (k, v) ⇒ result.put(k, v.elements.asJava) 65 | } 66 | result 67 | } 68 | 69 | /** 70 | * Get the set associated with the key if there is one. 71 | */ 72 | def get(key: String): Option[Set[A]] = 73 | underlying.get(key).map(_.elements) 74 | 75 | /** 76 | * Scala API: Get the set associated with the key if there is one, 77 | * else return the given default. 78 | */ 79 | def getOrElse(key: String, default: ⇒ Set[A]): Set[A] = 80 | get(key).getOrElse(default) 81 | 82 | /** 83 | * Convenience for put. Requires an implicit Cluster. 84 | * @see [[#put]] 85 | */ 86 | def +(entry: (String, Set[A]))(implicit node: Cluster): ORMultiMap[A] = { 87 | val (key, value) = entry 88 | put(node, key, value) 89 | } 90 | 91 | /** 92 | * Associate an entire set with the key while retaining the history of the previous 93 | * replicated data set. 94 | */ 95 | def put(node: Cluster, key: String, value: Set[A]): ORMultiMap[A] = 96 | put(node.selfUniqueAddress, key, value) 97 | 98 | /** 99 | * INTERNAL API 100 | */ 101 | private[akka] def put(node: UniqueAddress, key: String, value: Set[A]): ORMultiMap[A] = { 102 | val newUnderlying = underlying.updated(node, key, ORSet.empty[A]) { existing => 103 | value.foldLeft(existing.clear(node)) { (s, element) => s.add(node, element) } 104 | } 105 | new ORMultiMap(newUnderlying) 106 | } 107 | 108 | /** 109 | * Convenience for remove. Requires an implicit Cluster. 110 | * @see [[#remove]] 111 | */ 112 | def -(key: String)(implicit node: Cluster): ORMultiMap[A] = 113 | remove(node, key) 114 | 115 | /** 116 | * Remove an entire set associated with the key. 117 | */ 118 | def remove(node: Cluster, key: String): ORMultiMap[A] = 119 | remove(node.selfUniqueAddress, key) 120 | 121 | /** 122 | * INTERNAL API 123 | */ 124 | private[akka] def remove(node: UniqueAddress, key: String): ORMultiMap[A] = 125 | new ORMultiMap(underlying.remove(node, key)) 126 | 127 | /** 128 | * Add an element to a set associated with a key. If there is no existing set then one will be initialised. 129 | */ 130 | def addBinding(key: String, element: A)(implicit cluster: Cluster): ORMultiMap[A] = 131 | addBinding(cluster.selfUniqueAddress, key, element) 132 | 133 | /** 134 | * INTERNAL API 135 | */ 136 | private[akka] def addBinding(node: UniqueAddress, key: String, element: A): ORMultiMap[A] = { 137 | val newUnderlying = underlying.updated(node, key, ORSet.empty[A])(_.add(node, element)) 138 | new ORMultiMap(newUnderlying) 139 | } 140 | 141 | /** 142 | * Remove an element of a set associated with a key. If there are no more elements in the set then the 143 | * entire set will be removed. 144 | */ 145 | def removeBinding(key: String, element: A)(implicit cluster: Cluster): ORMultiMap[A] = 146 | removeBinding(cluster.selfUniqueAddress, key, element) 147 | 148 | /** 149 | * INTERNAL API 150 | */ 151 | private[akka] def removeBinding(node: UniqueAddress, key: String, element: A): ORMultiMap[A] = { 152 | val newUnderlying = { 153 | val u = underlying.updated(node, key, ORSet.empty[A])(_.remove(node, element)) 154 | u.get(key) match { 155 | case Some(s) if s.isEmpty => u.remove(node, key) 156 | case _ => u 157 | } 158 | } 159 | new ORMultiMap(newUnderlying) 160 | } 161 | 162 | /** 163 | * Replace an element of a set associated with a key with a new one if it is different. This is useful when an element is removed 164 | * and another one is added within the same Update. The order of addition and removal is important in order 165 | * to retain history for replicated data. 166 | */ 167 | def replaceBinding(key: String, oldElement: A, newElement: A)(implicit cluster: Cluster): ORMultiMap[A] = 168 | replaceBinding(cluster.selfUniqueAddress, key, oldElement, newElement) 169 | 170 | /** 171 | * INTERNAL API 172 | */ 173 | private[akka] def replaceBinding(node: UniqueAddress, key: String, oldElement: A, newElement: A): ORMultiMap[A] = 174 | if (newElement != oldElement) 175 | addBinding(node, key, newElement).removeBinding(node, key, oldElement) 176 | else 177 | this 178 | 179 | override def needPruningFrom(removedNode: UniqueAddress): Boolean = 180 | underlying.needPruningFrom(removedNode) 181 | 182 | override def pruningCleanup(removedNode: UniqueAddress): T = 183 | new ORMultiMap(underlying.pruningCleanup(removedNode)) 184 | 185 | override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): T = 186 | new ORMultiMap(underlying.prune(removedNode, collapseInto)) 187 | 188 | // this class cannot be a `case class` because we need different `unapply` 189 | 190 | override def toString: String = s"ORMulti$entries" 191 | 192 | override def equals(o: Any): Boolean = o match { 193 | case other: ORMultiMap[_] => underlying == other.underlying 194 | case _ => false 195 | } 196 | 197 | override def hashCode: Int = underlying.hashCode 198 | } 199 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/sample/datareplication/ReplicatedMetricsSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package sample.datareplication 5 | 6 | import java.lang.management.ManagementFactory 7 | import java.lang.management.MemoryMXBean 8 | import scala.concurrent.duration._ 9 | import akka.actor.Actor 10 | import akka.actor.ActorLogging 11 | import akka.actor.Address 12 | import akka.actor.Props 13 | import akka.cluster.Cluster 14 | import akka.cluster.ClusterEvent.{ InitialStateAsEvents, MemberUp, MemberRemoved } 15 | import akka.contrib.datareplication.DataReplication 16 | import akka.contrib.datareplication.LWWMap 17 | import akka.contrib.datareplication.Replicator.GetReplicaCount 18 | import akka.contrib.datareplication.Replicator.ReplicaCount 19 | import akka.contrib.datareplication.STMultiNodeSpec 20 | import akka.remote.testconductor.RoleName 21 | import akka.remote.testkit.MultiNodeConfig 22 | import akka.remote.testkit.MultiNodeSpec 23 | import akka.testkit._ 24 | import com.typesafe.config.ConfigFactory 25 | 26 | object ReplicatedMetricsSpec extends MultiNodeConfig { 27 | val node1 = role("node-1") 28 | val node2 = role("node-2") 29 | val node3 = role("node-3") 30 | 31 | commonConfig(ConfigFactory.parseString(""" 32 | akka.loglevel = INFO 33 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 34 | akka.log-dead-letters-during-shutdown = off 35 | """)) 36 | 37 | } 38 | 39 | object ReplicatedMetrics { 40 | import akka.contrib.datareplication.Replicator._ 41 | 42 | def props(measureInterval: FiniteDuration, cleanupInterval: FiniteDuration): Props = 43 | Props(new ReplicatedMetrics(measureInterval, cleanupInterval)) 44 | 45 | def props: Props = props(1.second, 1.minute) 46 | 47 | private case object Tick 48 | private case object Cleanup 49 | 50 | case class UsedHeap(percentPerNode: Map[String, Double]) { 51 | override def toString = 52 | percentPerNode.toSeq.sortBy(_._1).map { 53 | case (key, value) => key + " --> " + value + " %" 54 | }.mkString("\n") 55 | } 56 | 57 | def nodeKey(address: Address): String = address.host.get + ":" + address.port.get 58 | 59 | } 60 | 61 | class ReplicatedMetrics(measureInterval: FiniteDuration, cleanupInterval: FiniteDuration) 62 | extends Actor with ActorLogging { 63 | import akka.contrib.datareplication.Replicator._ 64 | import ReplicatedMetrics._ 65 | 66 | val replicator = DataReplication(context.system).replicator 67 | implicit val cluster = Cluster(context.system) 68 | val node = nodeKey(cluster.selfAddress) 69 | 70 | val tickTask = context.system.scheduler.schedule(measureInterval, measureInterval, 71 | self, Tick)(context.dispatcher) 72 | val cleanupTask = context.system.scheduler.schedule(cleanupInterval, cleanupInterval, 73 | self, Cleanup)(context.dispatcher) 74 | val memoryMBean: MemoryMXBean = ManagementFactory.getMemoryMXBean 75 | 76 | replicator ! Subscribe("usedHeap", self) 77 | replicator ! Subscribe("maxHeap", self) 78 | 79 | cluster.subscribe(self, InitialStateAsEvents, classOf[MemberUp], classOf[MemberRemoved]) 80 | 81 | override def postStop(): Unit = { 82 | tickTask.cancel() 83 | cluster.unsubscribe(self) 84 | super.postStop() 85 | } 86 | 87 | var maxHeap = Map.empty[String, Long] 88 | var nodesInCluster = Set.empty[String] 89 | 90 | def receive = { 91 | case Tick => 92 | val heap = memoryMBean.getHeapMemoryUsage 93 | val used = heap.getUsed 94 | val max = heap.getMax 95 | replicator ! Update("usedHeap", LWWMap.empty[Long], WriteLocal)(_ + (node -> used)) 96 | replicator ! Update("maxHeap", LWWMap.empty[Long], WriteLocal) { data => 97 | data.get(node) match { 98 | case Some(`max`) => data // unchanged 99 | case _ => data + (node -> max) 100 | } 101 | } 102 | 103 | case Changed("maxHeap", data: LWWMap[Long] @unchecked) => 104 | maxHeap = data.entries 105 | 106 | case Changed("usedHeap", data: LWWMap[Long] @unchecked) => 107 | val usedHeapPercent = UsedHeap(data.entries.collect { 108 | case (key, value) if maxHeap.contains(key) => 109 | (key -> (value.toDouble / maxHeap(key)) * 100.0) 110 | }) 111 | log.debug("Node {} observed:\n{}", node, usedHeapPercent) 112 | context.system.eventStream.publish(usedHeapPercent) 113 | 114 | case _: UpdateResponse => // ok 115 | 116 | case MemberUp(m) => 117 | nodesInCluster += nodeKey(m.address) 118 | 119 | case MemberRemoved(m, _) => 120 | nodesInCluster -= nodeKey(m.address) 121 | 122 | case Cleanup => 123 | def cleanupRemoved(data: LWWMap[Long]): LWWMap[Long] = 124 | (data.entries.keySet -- nodesInCluster).foldLeft(data) { case (d, key) => d - key } 125 | 126 | replicator ! Update("usedHeap", LWWMap.empty[Long], WriteLocal)(cleanupRemoved) 127 | replicator ! Update("maxHeap", LWWMap.empty[Long], WriteLocal)(cleanupRemoved) 128 | } 129 | 130 | } 131 | 132 | class ReplicatedMetricsSpecMultiJvmNode1 extends ReplicatedMetricsSpec 133 | class ReplicatedMetricsSpecMultiJvmNode2 extends ReplicatedMetricsSpec 134 | class ReplicatedMetricsSpecMultiJvmNode3 extends ReplicatedMetricsSpec 135 | 136 | class ReplicatedMetricsSpec extends MultiNodeSpec(ReplicatedMetricsSpec) with STMultiNodeSpec with ImplicitSender { 137 | import ReplicatedMetricsSpec._ 138 | import ReplicatedMetrics._ 139 | 140 | override def initialParticipants = roles.size 141 | 142 | val cluster = Cluster(system) 143 | val replicatedMetrics = system.actorOf(ReplicatedMetrics.props(1.second, 3.seconds)) 144 | 145 | def join(from: RoleName, to: RoleName): Unit = { 146 | runOn(from) { 147 | cluster join node(to).address 148 | } 149 | enterBarrier(from.name + "-joined") 150 | } 151 | 152 | "Demo of a replicated metrics" must { 153 | "join cluster" in within(10.seconds) { 154 | join(node1, node1) 155 | join(node2, node1) 156 | join(node3, node1) 157 | 158 | awaitAssert { 159 | DataReplication(system).replicator ! GetReplicaCount 160 | expectMsg(ReplicaCount(roles.size)) 161 | } 162 | enterBarrier("after-1") 163 | } 164 | 165 | "replicate metrics" in within(10.seconds) { 166 | val probe = TestProbe() 167 | system.eventStream.subscribe(probe.ref, classOf[UsedHeap]) 168 | awaitAssert { 169 | probe.expectMsgType[UsedHeap].percentPerNode.size should be(3) 170 | } 171 | probe.expectMsgType[UsedHeap].percentPerNode.size should be(3) 172 | probe.expectMsgType[UsedHeap].percentPerNode.size should be(3) 173 | enterBarrier("after-2") 174 | } 175 | 176 | "cleanup removed node" in within(15.seconds) { 177 | val node3Address = node(node3).address 178 | runOn(node1) { 179 | cluster.leave(node3Address) 180 | } 181 | runOn(node1, node2) { 182 | val probe = TestProbe() 183 | system.eventStream.subscribe(probe.ref, classOf[UsedHeap]) 184 | awaitAssert { 185 | probe.expectMsgType[UsedHeap].percentPerNode.size should be(2) 186 | } 187 | probe.expectMsgType[UsedHeap].percentPerNode should not contain ( 188 | nodeKey(node3Address)) 189 | } 190 | enterBarrier("after-3") 191 | } 192 | 193 | } 194 | 195 | } 196 | 197 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/sample/datareplication/ReplicatedShoppingCartSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package sample.datareplication 5 | 6 | import scala.concurrent.duration._ 7 | import akka.actor.Actor 8 | import akka.actor.ActorRef 9 | import akka.actor.Props 10 | import akka.actor.Stash 11 | import akka.cluster.Cluster 12 | import akka.contrib.datareplication.DataReplication 13 | import akka.contrib.datareplication.LWWMap 14 | import akka.contrib.datareplication.Replicator.GetReplicaCount 15 | import akka.contrib.datareplication.Replicator.ReplicaCount 16 | import akka.contrib.datareplication.STMultiNodeSpec 17 | import akka.remote.testconductor.RoleName 18 | import akka.remote.testkit.MultiNodeConfig 19 | import akka.remote.testkit.MultiNodeSpec 20 | import akka.testkit._ 21 | import com.typesafe.config.ConfigFactory 22 | 23 | object ReplicatedShoppingCartSpec extends MultiNodeConfig { 24 | val node1 = role("node-1") 25 | val node2 = role("node-2") 26 | val node3 = role("node-3") 27 | 28 | commonConfig(ConfigFactory.parseString(""" 29 | akka.loglevel = INFO 30 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 31 | akka.log-dead-letters-during-shutdown = off 32 | """)) 33 | 34 | } 35 | 36 | object ShoppingCart { 37 | import akka.contrib.datareplication.Replicator._ 38 | 39 | def props(userId: String): Props = Props(new ShoppingCart(userId)) 40 | 41 | case object GetCart 42 | final case class AddItem(item: LineItem) 43 | final case class RemoveItem(productId: String) 44 | 45 | final case class Cart(items: Set[LineItem]) 46 | final case class LineItem(productId: String, title: String, quantity: Int) 47 | 48 | private val timeout = 3.seconds 49 | private val readQuorum = ReadQuorum(timeout) 50 | private val writeQuorum = WriteQuorum(timeout) 51 | 52 | } 53 | 54 | class ShoppingCart(userId: String) extends Actor with Stash { 55 | import ShoppingCart._ 56 | import akka.contrib.datareplication.Replicator._ 57 | 58 | val replicator = DataReplication(context.system).replicator 59 | implicit val cluster = Cluster(context.system) 60 | 61 | val DataKey = "cart-" + userId 62 | 63 | def receive = { 64 | 65 | case GetCart ⇒ 66 | replicator ! Get(DataKey, readQuorum, Some(sender())) 67 | 68 | case GetSuccess(DataKey, data: LWWMap[LineItem] @unchecked, Some(replyTo: ActorRef)) ⇒ 69 | val cart = Cart(data.entries.values.toSet) 70 | replyTo ! cart 71 | 72 | case NotFound(DataKey, Some(replyTo: ActorRef)) ⇒ 73 | replyTo ! Cart(Set.empty) 74 | 75 | case GetFailure(DataKey, Some(replyTo: ActorRef)) ⇒ 76 | // ReadQuorum failure, try again with local read 77 | replicator ! Get(DataKey, ReadLocal, Some(replyTo)) 78 | 79 | case cmd @ AddItem(item) ⇒ 80 | val update = Update(DataKey, LWWMap.empty[LineItem], readQuorum, writeQuorum, Some(cmd)) { 81 | cart => updateCart(cart, item) 82 | } 83 | replicator ! update 84 | 85 | case ReadFailure(DataKey, Some(AddItem(item))) => 86 | // ReadQuorum of Update failed, fall back to best effort local value 87 | replicator ! Update(DataKey, LWWMap.empty[LineItem], writeQuorum, None) { 88 | cart => updateCart(cart, item) 89 | } 90 | 91 | case cmd @ RemoveItem(productId) ⇒ 92 | val update = Update(DataKey, LWWMap(), readQuorum, writeQuorum, Some(cmd)) { 93 | _ - productId 94 | } 95 | replicator ! update 96 | 97 | case ReadFailure(DataKey, Some(RemoveItem(productId))) => 98 | // ReadQuorum of Update failed, fall back to best effort local value 99 | replicator ! Update(DataKey, LWWMap(), writeQuorum, None) { 100 | _ - productId 101 | } 102 | 103 | case _: UpdateSuccess | _: UpdateTimeout ⇒ 104 | // UpdateTimeout, will eventually be replicated 105 | } 106 | 107 | def updateCart(data: LWWMap[LineItem], item: LineItem): LWWMap[LineItem] = 108 | data.get(item.productId) match { 109 | case Some(LineItem(_, _, existingQuantity)) ⇒ 110 | data + (item.productId -> item.copy(quantity = existingQuantity + item.quantity)) 111 | case None ⇒ data + (item.productId -> item) 112 | } 113 | 114 | override def unhandled(msg: Any): Unit = msg match { 115 | case e: UpdateFailure ⇒ throw new IllegalStateException("Unexpected failure: " + e) 116 | case _ ⇒ super.unhandled(msg) 117 | } 118 | 119 | } 120 | 121 | class ReplicatedShoppingCartSpecMultiJvmNode1 extends ReplicatedShoppingCartSpec 122 | class ReplicatedShoppingCartSpecMultiJvmNode2 extends ReplicatedShoppingCartSpec 123 | class ReplicatedShoppingCartSpecMultiJvmNode3 extends ReplicatedShoppingCartSpec 124 | 125 | class ReplicatedShoppingCartSpec extends MultiNodeSpec(ReplicatedShoppingCartSpec) with STMultiNodeSpec with ImplicitSender { 126 | import ReplicatedShoppingCartSpec._ 127 | import ShoppingCart._ 128 | 129 | override def initialParticipants = roles.size 130 | 131 | val cluster = Cluster(system) 132 | val shoppingCart = system.actorOf(ShoppingCart.props("user-1")) 133 | 134 | def join(from: RoleName, to: RoleName): Unit = { 135 | runOn(from) { 136 | cluster join node(to).address 137 | } 138 | enterBarrier(from.name + "-joined") 139 | } 140 | 141 | "Demo of a replicated shopping cart" must { 142 | "join cluster" in within(10.seconds) { 143 | join(node1, node1) 144 | join(node2, node1) 145 | join(node3, node1) 146 | 147 | awaitAssert { 148 | DataReplication(system).replicator ! GetReplicaCount 149 | expectMsg(ReplicaCount(roles.size)) 150 | } 151 | enterBarrier("after-1") 152 | } 153 | 154 | "handle updates directly after start" in within(15.seconds) { 155 | runOn(node2) { 156 | shoppingCart ! ShoppingCart.AddItem(LineItem("1", "Apples", quantity = 2)) 157 | shoppingCart ! ShoppingCart.AddItem(LineItem("2", "Oranges", quantity = 3)) 158 | } 159 | enterBarrier("updates-done") 160 | 161 | awaitAssert { 162 | shoppingCart ! ShoppingCart.GetCart 163 | val cart = expectMsgType[Cart] 164 | cart.items should be(Set(LineItem("1", "Apples", quantity = 2), LineItem("2", "Oranges", quantity = 3))) 165 | } 166 | 167 | enterBarrier("after-2") 168 | } 169 | 170 | "handle updates from different nodes" in within(5.seconds) { 171 | runOn(node2) { 172 | shoppingCart ! ShoppingCart.AddItem(LineItem("1", "Apples", quantity = 5)) 173 | shoppingCart ! ShoppingCart.RemoveItem("2") 174 | } 175 | runOn(node3) { 176 | shoppingCart ! ShoppingCart.AddItem(LineItem("3", "Bananas", quantity = 4)) 177 | } 178 | enterBarrier("updates-done") 179 | 180 | awaitAssert { 181 | shoppingCart ! ShoppingCart.GetCart 182 | val cart = expectMsgType[Cart] 183 | cart.items should be(Set(LineItem("1", "Apples", quantity = 7), LineItem("3", "Bananas", quantity = 4))) 184 | } 185 | 186 | enterBarrier("after-3") 187 | } 188 | 189 | "read own updates" in within(5.seconds) { 190 | runOn(node2) { 191 | shoppingCart ! ShoppingCart.AddItem(LineItem("1", "Apples", quantity = 1)) 192 | shoppingCart ! ShoppingCart.RemoveItem("3") 193 | shoppingCart ! ShoppingCart.AddItem(LineItem("3", "Bananas", quantity = 5)) 194 | shoppingCart ! ShoppingCart.GetCart 195 | val cart = expectMsgType[Cart] 196 | cart.items should be(Set(LineItem("1", "Apples", quantity = 8), LineItem("3", "Bananas", quantity = 5))) 197 | } 198 | 199 | enterBarrier("after-4") 200 | } 201 | } 202 | 203 | } 204 | 205 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/akka/contrib/datareplication/ReplicatorChaosSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import scala.concurrent.duration._ 7 | import akka.remote.testconductor.RoleName 8 | import akka.remote.testkit.MultiNodeConfig 9 | import akka.remote.testkit.MultiNodeSpec 10 | import com.typesafe.config.ConfigFactory 11 | import akka.cluster.Cluster 12 | import akka.remote.transport.ThrottlerTransportAdapter.Direction 13 | import akka.testkit._ 14 | import akka.cluster.ClusterEvent.InitialStateAsEvents 15 | import akka.cluster.ClusterEvent.MemberUp 16 | 17 | object ReplicatorChaosSpec extends MultiNodeConfig { 18 | val first = role("first") 19 | val second = role("second") 20 | val third = role("third") 21 | val fourth = role("fourth") 22 | val fifth = role("fifth") 23 | 24 | commonConfig(ConfigFactory.parseString(""" 25 | akka.loglevel = INFO 26 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 27 | akka.cluster.roles = ["backend"] 28 | akka.log-dead-letters-during-shutdown = off 29 | """)) 30 | 31 | testTransport(on = true) 32 | } 33 | 34 | class ReplicatorChaosSpecMultiJvmNode1 extends ReplicatorChaosSpec 35 | class ReplicatorChaosSpecMultiJvmNode2 extends ReplicatorChaosSpec 36 | class ReplicatorChaosSpecMultiJvmNode3 extends ReplicatorChaosSpec 37 | class ReplicatorChaosSpecMultiJvmNode4 extends ReplicatorChaosSpec 38 | class ReplicatorChaosSpecMultiJvmNode5 extends ReplicatorChaosSpec 39 | 40 | class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMultiNodeSpec with ImplicitSender { 41 | import ReplicatorChaosSpec._ 42 | import Replicator._ 43 | 44 | override def initialParticipants = roles.size 45 | 46 | implicit val cluster = Cluster(system) 47 | val replicator = system.actorOf(Replicator.props( 48 | ReplicatorSettings(role = Some("backend"), gossipInterval = 1.second)), "replicator") 49 | val timeout = 3.seconds.dilated 50 | 51 | def join(from: RoleName, to: RoleName): Unit = { 52 | runOn(from) { 53 | cluster join node(to).address 54 | } 55 | enterBarrier(from.name + "-joined") 56 | } 57 | 58 | def assertValue(key: String, expected: Any): Unit = 59 | within(10.seconds) { 60 | awaitAssert { 61 | replicator ! Get(key, ReadLocal) 62 | val value = expectMsgPF() { 63 | case GetSuccess(`key`, c: GCounter, _) ⇒ c.value 64 | case GetSuccess(`key`, c: PNCounter, _) ⇒ c.value 65 | case GetSuccess(`key`, c: GSet[_], _) ⇒ c.elements 66 | case GetSuccess(`key`, c: ORSet[_], _) ⇒ c.elements 67 | } 68 | value should be(expected) 69 | } 70 | } 71 | 72 | def assertDeleted(key: String): Unit = 73 | within(5.seconds) { 74 | awaitAssert { 75 | replicator ! Get(key, ReadLocal) 76 | expectMsg(DataDeleted(key)) 77 | } 78 | } 79 | 80 | "Replicator in chaotic cluster" must { 81 | 82 | "replicate data in initial phase" in { 83 | join(first, first) 84 | join(second, first) 85 | join(third, first) 86 | join(fourth, first) 87 | join(fifth, first) 88 | 89 | within(10.seconds) { 90 | awaitAssert { 91 | replicator ! GetReplicaCount 92 | expectMsg(ReplicaCount(5)) 93 | } 94 | } 95 | 96 | runOn(first) { 97 | (0 until 5).foreach { i ⇒ 98 | replicator ! Update("A", GCounter(), WriteLocal)(_ + 1) 99 | replicator ! Update("B", PNCounter(), WriteLocal)(_ - 1) 100 | replicator ! Update("C", GCounter(), WriteAll(timeout))(_ + 1) 101 | } 102 | receiveN(15).map(_.getClass).toSet should be(Set(classOf[UpdateSuccess])) 103 | } 104 | 105 | runOn(second) { 106 | replicator ! Update("A", GCounter(), WriteLocal)(_ + 20) 107 | replicator ! Update("B", PNCounter(), WriteTo(2, timeout))(_ + 20) 108 | replicator ! Update("C", GCounter(), WriteAll(timeout))(_ + 20) 109 | receiveN(3).toSet should be(Set(UpdateSuccess("A", None), 110 | UpdateSuccess("B", None), UpdateSuccess("C", None))) 111 | 112 | replicator ! Update("E", GSet(), WriteLocal)(_ + "e1" + "e2") 113 | expectMsg(UpdateSuccess("E", None)) 114 | 115 | replicator ! Update("F", ORSet(), WriteLocal)(_ + "e1" + "e2") 116 | expectMsg(UpdateSuccess("F", None)) 117 | } 118 | 119 | runOn(fourth) { 120 | replicator ! Update("D", GCounter(), WriteLocal)(_ + 40) 121 | expectMsg(UpdateSuccess("D", None)) 122 | 123 | replicator ! Update("E", GSet(), WriteLocal)(_ + "e2" + "e3") 124 | expectMsg(UpdateSuccess("E", None)) 125 | 126 | replicator ! Update("F", ORSet(), WriteLocal)(_ + "e2" + "e3") 127 | expectMsg(UpdateSuccess("F", None)) 128 | } 129 | 130 | runOn(fifth) { 131 | replicator ! Update("X", GCounter(), WriteTo(2, timeout))(_ + 50) 132 | expectMsg(UpdateSuccess("X", None)) 133 | replicator ! Delete("X", WriteLocal) 134 | expectMsg(DeleteSuccess("X")) 135 | } 136 | 137 | enterBarrier("initial-updates-done") 138 | 139 | assertValue("A", 25) 140 | assertValue("B", 15) 141 | assertValue("C", 25) 142 | assertValue("D", 40) 143 | assertValue("E", Set("e1", "e2", "e3")) 144 | assertValue("F", Set("e1", "e2", "e3")) 145 | assertDeleted("X") 146 | 147 | enterBarrier("after-1") 148 | } 149 | 150 | "be available during network split" in { 151 | val side1 = Seq(first, second) 152 | val side2 = Seq(third, fourth, fifth) 153 | runOn(first) { 154 | for (a ← side1; b ← side2) 155 | testConductor.blackhole(a, b, Direction.Both).await 156 | } 157 | enterBarrier("split") 158 | 159 | runOn(first) { 160 | replicator ! Update("A", GCounter(), WriteTo(2, timeout))(_ + 1) 161 | expectMsg(UpdateSuccess("A", None)) 162 | } 163 | 164 | runOn(third) { 165 | replicator ! Update("A", GCounter(), WriteTo(2, timeout))(_ + 2) 166 | expectMsg(UpdateSuccess("A", None)) 167 | 168 | replicator ! Update("E", GSet(), WriteTo(2, timeout))(_ + "e4") 169 | expectMsg(UpdateSuccess("E", None)) 170 | 171 | replicator ! Update("F", ORSet(), WriteTo(2, timeout))(_ - "e2") 172 | expectMsg(UpdateSuccess("F", None)) 173 | } 174 | runOn(fourth) { 175 | replicator ! Update("D", GCounter(), WriteTo(2, timeout))(_ + 1) 176 | expectMsg(UpdateSuccess("D", None)) 177 | } 178 | enterBarrier("update-during-split") 179 | 180 | runOn(side1: _*) { 181 | assertValue("A", 26) 182 | assertValue("B", 15) 183 | assertValue("D", 40) 184 | assertValue("E", Set("e1", "e2", "e3")) 185 | assertValue("F", Set("e1", "e2", "e3")) 186 | } 187 | runOn(side2: _*) { 188 | assertValue("A", 27) 189 | assertValue("B", 15) 190 | assertValue("D", 41) 191 | assertValue("E", Set("e1", "e2", "e3", "e4")) 192 | assertValue("F", Set("e1", "e3")) 193 | } 194 | enterBarrier("update-during-split-verified") 195 | 196 | runOn(first) { 197 | testConductor.exit(fourth, 0).await 198 | } 199 | 200 | enterBarrier("after-2") 201 | } 202 | 203 | "converge after partition" in { 204 | val side1 = Seq(first, second) 205 | val side2 = Seq(third, fifth) // fourth was shutdown 206 | runOn(first) { 207 | for (a ← side1; b ← side2) 208 | testConductor.passThrough(a, b, Direction.Both).await 209 | } 210 | enterBarrier("split-repaired") 211 | 212 | assertValue("A", 28) 213 | assertValue("B", 15) 214 | assertValue("C", 25) 215 | assertValue("D", 41) 216 | assertValue("E", Set("e1", "e2", "e3", "e4")) 217 | assertValue("F", Set("e1", "e3")) 218 | assertDeleted("X") 219 | 220 | enterBarrier("after-3") 221 | } 222 | } 223 | 224 | } 225 | 226 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/VectorClockSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.BeforeAndAfterAll 8 | import org.scalatest.Matchers 9 | import org.scalatest.WordSpecLike 10 | import akka.actor.ActorSystem 11 | import akka.actor.Address 12 | import akka.cluster.UniqueAddress 13 | import akka.testkit.TestKit 14 | 15 | class VectorClockSpec extends TestKit(ActorSystem("VectorClockSpec")) 16 | with WordSpecLike with Matchers with BeforeAndAfterAll { 17 | 18 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 19 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 20 | val node3 = UniqueAddress(node1.address.copy(port = Some(2553)), 3) 21 | val node4 = UniqueAddress(node1.address.copy(port = Some(2554)), 4) 22 | 23 | override def afterAll { 24 | shutdown() 25 | } 26 | 27 | "A VectorClock" must { 28 | 29 | "have zero versions when created" in { 30 | val clock = VectorClock() 31 | clock.versions should be(Map()) 32 | } 33 | 34 | "not happen before itself" in { 35 | val clock1 = VectorClock() 36 | val clock2 = VectorClock() 37 | 38 | clock1 <> clock2 should be(false) 39 | } 40 | 41 | "pass misc comparison test 1" in { 42 | val clock1_1 = VectorClock() 43 | val clock2_1 = clock1_1 + node1 44 | val clock3_1 = clock2_1 + node2 45 | val clock4_1 = clock3_1 + node1 46 | 47 | val clock1_2 = VectorClock() 48 | val clock2_2 = clock1_2 + node1 49 | val clock3_2 = clock2_2 + node2 50 | val clock4_2 = clock3_2 + node1 51 | 52 | clock4_1 <> clock4_2 should be(false) 53 | } 54 | 55 | "pass misc comparison test 2" in { 56 | val clock1_1 = VectorClock() 57 | val clock2_1 = clock1_1 + node1 58 | val clock3_1 = clock2_1 + node2 59 | val clock4_1 = clock3_1 + node1 60 | 61 | val clock1_2 = VectorClock() 62 | val clock2_2 = clock1_2 + node1 63 | val clock3_2 = clock2_2 + node2 64 | val clock4_2 = clock3_2 + node1 65 | val clock5_2 = clock4_2 + node3 66 | 67 | clock4_1 < clock5_2 should be(true) 68 | } 69 | 70 | "pass misc comparison test 3" in { 71 | var clock1_1 = VectorClock() 72 | val clock2_1 = clock1_1 + node1 73 | 74 | val clock1_2 = VectorClock() 75 | val clock2_2 = clock1_2 + node2 76 | 77 | clock2_1 <> clock2_2 should be(true) 78 | } 79 | 80 | "pass misc comparison test 4" in { 81 | val clock1_3 = VectorClock() 82 | val clock2_3 = clock1_3 + node1 83 | val clock3_3 = clock2_3 + node2 84 | val clock4_3 = clock3_3 + node1 85 | 86 | val clock1_4 = VectorClock() 87 | val clock2_4 = clock1_4 + node1 88 | val clock3_4 = clock2_4 + node1 89 | val clock4_4 = clock3_4 + node3 90 | 91 | clock4_3 <> clock4_4 should be(true) 92 | } 93 | 94 | "pass misc comparison test 5" in { 95 | val clock1_1 = VectorClock() 96 | val clock2_1 = clock1_1 + node2 97 | val clock3_1 = clock2_1 + node2 98 | 99 | val clock1_2 = VectorClock() 100 | val clock2_2 = clock1_2 + node1 101 | val clock3_2 = clock2_2 + node2 102 | val clock4_2 = clock3_2 + node2 103 | val clock5_2 = clock4_2 + node3 104 | 105 | clock3_1 < clock5_2 should be(true) 106 | clock5_2 > clock3_1 should be(true) 107 | } 108 | 109 | "pass misc comparison test 6" in { 110 | val clock1_1 = VectorClock() 111 | val clock2_1 = clock1_1 + node1 112 | val clock3_1 = clock2_1 + node2 113 | 114 | val clock1_2 = VectorClock() 115 | val clock2_2 = clock1_2 + node1 116 | val clock3_2 = clock2_2 + node1 117 | 118 | clock3_1 <> clock3_2 should be(true) 119 | clock3_2 <> clock3_1 should be(true) 120 | } 121 | 122 | "pass misc comparison test 7" in { 123 | val clock1_1 = VectorClock() 124 | val clock2_1 = clock1_1 + node1 125 | val clock3_1 = clock2_1 + node2 126 | val clock4_1 = clock3_1 + node2 127 | val clock5_1 = clock4_1 + node3 128 | 129 | val clock1_2 = clock4_1 130 | val clock2_2 = clock1_2 + node2 131 | val clock3_2 = clock2_2 + node2 132 | 133 | clock5_1 <> clock3_2 should be(true) 134 | clock3_2 <> clock5_1 should be(true) 135 | } 136 | 137 | "pass misc comparison test 8" in { 138 | val clock1_1 = VectorClock() 139 | val clock2_1 = clock1_1 + node1 140 | val clock3_1 = clock2_1 + node3 141 | 142 | val clock1_2 = clock3_1 + node2 143 | 144 | val clock4_1 = clock3_1 + node3 145 | 146 | clock4_1 <> clock1_2 should be(true) 147 | clock1_2 <> clock4_1 should be(true) 148 | } 149 | 150 | "correctly merge two clocks" in { 151 | val clock1_1 = VectorClock() 152 | val clock2_1 = clock1_1 + node1 153 | val clock3_1 = clock2_1 + node2 154 | val clock4_1 = clock3_1 + node2 155 | val clock5_1 = clock4_1 + node3 156 | 157 | val clock1_2 = clock4_1 158 | val clock2_2 = clock1_2 + node2 159 | val clock3_2 = clock2_2 + node2 160 | 161 | val merged1 = clock3_2 merge clock5_1 162 | merged1.versions.size should be(3) 163 | merged1.versions.contains(node1) should be(true) 164 | merged1.versions.contains(node2) should be(true) 165 | merged1.versions.contains(node3) should be(true) 166 | 167 | val merged2 = clock5_1 merge clock3_2 168 | merged2.versions.size should be(3) 169 | merged2.versions.contains(node1) should be(true) 170 | merged2.versions.contains(node2) should be(true) 171 | merged2.versions.contains(node3) should be(true) 172 | 173 | clock3_2 < merged1 should be(true) 174 | clock5_1 < merged1 should be(true) 175 | 176 | clock3_2 < merged2 should be(true) 177 | clock5_1 < merged2 should be(true) 178 | 179 | merged1 == merged2 should be(true) 180 | } 181 | 182 | "correctly merge two disjoint vector clocks" in { 183 | 184 | val clock1_1 = VectorClock() 185 | val clock2_1 = clock1_1 + node1 186 | val clock3_1 = clock2_1 + node2 187 | val clock4_1 = clock3_1 + node2 188 | val clock5_1 = clock4_1 + node3 189 | 190 | val clock1_2 = VectorClock() 191 | val clock2_2 = clock1_2 + node4 192 | val clock3_2 = clock2_2 + node4 193 | 194 | val merged1 = clock3_2 merge clock5_1 195 | merged1.versions.size should be(4) 196 | merged1.versions.contains(node1) should be(true) 197 | merged1.versions.contains(node2) should be(true) 198 | merged1.versions.contains(node3) should be(true) 199 | merged1.versions.contains(node4) should be(true) 200 | 201 | val merged2 = clock5_1 merge clock3_2 202 | merged2.versions.size should be(4) 203 | merged2.versions.contains(node1) should be(true) 204 | merged2.versions.contains(node2) should be(true) 205 | merged2.versions.contains(node3) should be(true) 206 | merged2.versions.contains(node4) should be(true) 207 | 208 | clock3_2 < merged1 should be(true) 209 | clock5_1 < merged1 should be(true) 210 | 211 | clock3_2 < merged2 should be(true) 212 | clock5_1 < merged2 should be(true) 213 | 214 | merged1 == merged2 should be(true) 215 | } 216 | 217 | "pass blank clock incrementing" in { 218 | val v1 = VectorClock() 219 | val v2 = VectorClock() 220 | 221 | val vv1 = v1 + node1 222 | val vv2 = v2 + node2 223 | 224 | (vv1 > v1) should be(true) 225 | (vv2 > v2) should be(true) 226 | 227 | (vv1 > v2) should be(true) 228 | (vv2 > v1) should be(true) 229 | 230 | (vv2 > vv1) should be(false) 231 | (vv1 > vv2) should be(false) 232 | } 233 | 234 | "pass merging behavior" in { 235 | val a = VectorClock() 236 | val b = VectorClock() 237 | 238 | val a1 = a + node1 239 | val b1 = b + node2 240 | 241 | var a2 = a1 + node1 242 | var c = a2.merge(b1) 243 | var c1 = c + node3 244 | 245 | (c1 > a2) should be(true) 246 | (c1 > b1) should be(true) 247 | } 248 | } 249 | } 250 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/protobuf/ReplicatedDataSerializerSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication.protobuf 5 | 6 | import scala.concurrent.duration._ 7 | import org.scalatest.BeforeAndAfterAll 8 | import org.scalatest.Matchers 9 | import org.scalatest.WordSpecLike 10 | import akka.actor.ActorSystem 11 | import akka.actor.Address 12 | import akka.actor.ExtendedActorSystem 13 | import akka.contrib.datareplication.Flag 14 | import akka.contrib.datareplication.GCounter 15 | import akka.contrib.datareplication.GSet 16 | import akka.contrib.datareplication.LWWMap 17 | import akka.contrib.datareplication.LWWRegister 18 | import akka.contrib.datareplication.ORMap 19 | import akka.contrib.datareplication.ORMultiMap 20 | import akka.contrib.datareplication.ORSet 21 | import akka.contrib.datareplication.PNCounter 22 | import akka.contrib.datareplication.PNCounterMap 23 | import akka.contrib.datareplication.Replicator._ 24 | import akka.contrib.datareplication.Replicator.Internal._ 25 | import akka.contrib.datareplication.VectorClock 26 | import akka.testkit.TestKit 27 | import akka.cluster.UniqueAddress 28 | import com.typesafe.config.ConfigFactory 29 | 30 | class ReplicatedDataSerializerSpec extends TestKit(ActorSystem("ReplicatedDataSerializerSpec", 31 | ConfigFactory.parseString("akka.actor.provider=akka.cluster.ClusterActorRefProvider"))) 32 | with WordSpecLike with Matchers with BeforeAndAfterAll { 33 | 34 | val serializer = new ReplicatedDataSerializer(system.asInstanceOf[ExtendedActorSystem]) 35 | 36 | val address1 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4711), 1) 37 | val address2 = UniqueAddress(Address("akka.tcp", system.name, "other.host.org", 4711), 2) 38 | val address3 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4712), 3) 39 | 40 | override def afterAll { 41 | shutdown() 42 | } 43 | 44 | def checkSerialization(obj: AnyRef): Unit = { 45 | val blob = serializer.toBinary(obj) 46 | val ref = serializer.fromBinary(blob, obj.getClass) 47 | ref should be(obj) 48 | } 49 | 50 | def checkSameContent(a: AnyRef, b: AnyRef): Unit = { 51 | a should be(b) 52 | val blobA = serializer.toBinary(a) 53 | val blobB = serializer.toBinary(b) 54 | blobA.toSeq should be(blobB.toSeq) 55 | } 56 | 57 | "ReplicatedDataSerializer" must { 58 | 59 | "serialize GSet" in { 60 | checkSerialization(GSet()) 61 | checkSerialization(GSet() + "a") 62 | checkSerialization(GSet() + "a" + "b") 63 | 64 | checkSerialization(GSet() + 1 + 2 + 3) 65 | checkSerialization(GSet() + address1 + address2) 66 | 67 | checkSerialization(GSet() + 1L + "2" + 3 + address1) 68 | 69 | checkSameContent(GSet() + "a" + "b", GSet() + "a" + "b") 70 | checkSameContent(GSet() + "a" + "b", GSet() + "b" + "a") 71 | checkSameContent(GSet() + address1 + address2 + address3, GSet() + address2 + address1 + address3) 72 | checkSameContent(GSet() + address1 + address2 + address3, GSet() + address3 + address2 + address1) 73 | } 74 | 75 | "serialize ORSet" in { 76 | checkSerialization(ORSet()) 77 | checkSerialization(ORSet().add(address1, "a")) 78 | checkSerialization(ORSet().add(address1, "a").add(address2, "a")) 79 | checkSerialization(ORSet().add(address1, "a").remove(address2, "a")) 80 | checkSerialization(ORSet().add(address1, "a").add(address2, "b").remove(address1, "a")) 81 | checkSerialization(ORSet().add(address1, 1).add(address2, 2)) 82 | checkSerialization(ORSet().add(address1, 1L).add(address2, 2L)) 83 | checkSerialization(ORSet().add(address1, "a").add(address2, 2).add(address3, 3L).add(address3, address3)) 84 | 85 | val s1 = ORSet().add(address1, "a").add(address2, "b") 86 | val s2 = ORSet().add(address2, "b").add(address1, "a") 87 | checkSameContent(s1.merge(s2), s2.merge(s1)) 88 | 89 | val s3 = ORSet().add(address1, "a").add(address2, 17).remove(address3, 17) 90 | val s4 = ORSet().add(address2, 17).remove(address3, 17).add(address1, "a") 91 | checkSameContent(s3.merge(s4), s4.merge(s3)) 92 | } 93 | 94 | "serialize Flag" in { 95 | checkSerialization(Flag()) 96 | checkSerialization(Flag().switchOn) 97 | } 98 | 99 | "serialize LWWRegister" in { 100 | checkSerialization(LWWRegister(address1, "value1", LWWRegister.defaultClock)) 101 | checkSerialization(LWWRegister(address1, "value2", LWWRegister.defaultClock) 102 | .withValue(address2, "value3", LWWRegister.defaultClock)) 103 | } 104 | 105 | "serialize GCounter" in { 106 | checkSerialization(GCounter()) 107 | checkSerialization(GCounter().increment(address1, 3)) 108 | checkSerialization(GCounter().increment(address1, 2).increment(address2, 5)) 109 | 110 | checkSameContent( 111 | GCounter().increment(address1, 2).increment(address2, 5), 112 | GCounter().increment(address2, 5).increment(address1, 1).increment(address1, 1)) 113 | checkSameContent( 114 | GCounter().increment(address1, 2).increment(address3, 5), 115 | GCounter().increment(address3, 5).increment(address1, 2)) 116 | } 117 | 118 | "serialize PNCounter" in { 119 | checkSerialization(PNCounter()) 120 | checkSerialization(PNCounter().increment(address1, 3)) 121 | checkSerialization(PNCounter().increment(address1, 3).decrement(address1, 1)) 122 | checkSerialization(PNCounter().increment(address1, 2).increment(address2, 5)) 123 | checkSerialization(PNCounter().increment(address1, 2).increment(address2, 5).decrement(address1, 1)) 124 | 125 | checkSameContent( 126 | PNCounter().increment(address1, 2).increment(address2, 5), 127 | PNCounter().increment(address2, 5).increment(address1, 1).increment(address1, 1)) 128 | checkSameContent( 129 | PNCounter().increment(address1, 2).increment(address3, 5), 130 | PNCounter().increment(address3, 5).increment(address1, 2)) 131 | checkSameContent( 132 | PNCounter().increment(address1, 2).decrement(address1, 1).increment(address3, 5), 133 | PNCounter().increment(address3, 5).increment(address1, 2).decrement(address1, 1)) 134 | } 135 | 136 | "serialize ORMap" in { 137 | checkSerialization(ORMap()) 138 | checkSerialization(ORMap().put(address1, "a", GSet() + "A")) 139 | checkSerialization(ORMap().put(address1, "a", GSet() + "A").put(address2, "b", GSet() + "B")) 140 | } 141 | 142 | "serialize LWWMap" in { 143 | checkSerialization(LWWMap()) 144 | checkSerialization(LWWMap().put(address1, "a", "value1", LWWRegister.defaultClock)) 145 | checkSerialization(LWWMap().put(address1, "a", "value1", LWWRegister.defaultClock) 146 | .put(address2, "b", 17, LWWRegister.defaultClock)) 147 | } 148 | 149 | "serialize PNCounterMap" in { 150 | checkSerialization(PNCounterMap()) 151 | checkSerialization(PNCounterMap().increment(address1, "a", 3)) 152 | checkSerialization(PNCounterMap().increment(address1, "a", 3).decrement(address2, "a", 2). 153 | increment(address2, "b", 5)) 154 | } 155 | 156 | "serialize ORMultiMap" in { 157 | checkSerialization(ORMultiMap()) 158 | checkSerialization(ORMultiMap().addBinding(address1, "a", "A")) 159 | checkSerialization(ORMultiMap.empty[String] 160 | .addBinding(address1, "a", "A1") 161 | .put(address2, "b", Set("B1", "B2", "B3")) 162 | .addBinding(address2, "a", "A2")) 163 | 164 | val m1 = ORMultiMap.empty[String].addBinding(address1, "a", "A1").addBinding(address2, "a", "A2") 165 | val m2 = ORMultiMap.empty[String].put(address2, "b", Set("B1", "B2", "B3")) 166 | checkSameContent(m1.merge(m2), m2.merge(m1)) 167 | } 168 | 169 | "serialize DeletedData" in { 170 | checkSerialization(DeletedData) 171 | } 172 | 173 | "serialize VectorClock" in { 174 | checkSerialization(VectorClock()) 175 | checkSerialization(VectorClock().increment(address1)) 176 | checkSerialization(VectorClock().increment(address1).increment(address2)) 177 | 178 | val v1 = VectorClock().increment(address1).increment(address1) 179 | val v2 = VectorClock().increment(address2) 180 | checkSameContent(v1.merge(v2), v2.merge(v1)) 181 | } 182 | 183 | } 184 | } 185 | 186 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/akka/contrib/datareplication/PerformanceSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import scala.concurrent.duration._ 7 | import akka.remote.testconductor.RoleName 8 | import akka.remote.testkit.MultiNodeConfig 9 | import akka.remote.testkit.MultiNodeSpec 10 | import com.typesafe.config.ConfigFactory 11 | import akka.cluster.Cluster 12 | import akka.remote.transport.ThrottlerTransportAdapter.Direction 13 | import akka.testkit._ 14 | import scala.concurrent.forkjoin.ThreadLocalRandom 15 | import akka.actor.Actor 16 | import akka.actor.Props 17 | import scala.concurrent.Await 18 | import akka.actor.ActorRef 19 | 20 | object PerformanceSpec extends MultiNodeConfig { 21 | val n1 = role("n1") 22 | val n2 = role("n2") 23 | val n3 = role("n3") 24 | val n4 = role("n4") 25 | val n5 = role("n5") 26 | 27 | commonConfig(ConfigFactory.parseString(""" 28 | akka.loglevel = ERROR 29 | akka.stdout-loglevel = ERROR 30 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 31 | akka.log-dead-letters = off 32 | akka.log-dead-letters-during-shutdown = off 33 | akka.remote.log-remote-lifecycle-events = ERROR 34 | akka.remote.log-frame-size-exceeding=1000b 35 | akka.testconductor.barrier-timeout = 60 s 36 | akka.contrib.data-replication.gossip-interval = 1 s 37 | """)) 38 | 39 | def countDownProps(latch: TestLatch): Props = Props(new CountDown(latch)) 40 | 41 | class CountDown(latch: TestLatch) extends Actor { 42 | def receive = { 43 | case _ => 44 | latch.countDown() 45 | if (latch.isOpen) 46 | context.stop(self) 47 | } 48 | } 49 | 50 | } 51 | 52 | class PerformanceSpecMultiJvmNode1 extends PerformanceSpec 53 | class PerformanceSpecMultiJvmNode2 extends PerformanceSpec 54 | class PerformanceSpecMultiJvmNode3 extends PerformanceSpec 55 | class PerformanceSpecMultiJvmNode4 extends PerformanceSpec 56 | class PerformanceSpecMultiJvmNode5 extends PerformanceSpec 57 | 58 | class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpec with ImplicitSender { 59 | import PerformanceSpec._ 60 | import Replicator._ 61 | 62 | override def initialParticipants = roles.size 63 | 64 | implicit val cluster = Cluster(system) 65 | val replicator = DataReplication(system).replicator 66 | val timeout = 3.seconds.dilated 67 | val factor = 1 68 | val repeatCount = 3 // use at least 10 here for serious tuning 69 | 70 | def join(from: RoleName, to: RoleName): Unit = { 71 | runOn(from) { 72 | cluster join node(to).address 73 | } 74 | enterBarrier(from.name + "-joined") 75 | } 76 | 77 | def repeat(description: String, keys: Iterable[String], n: Int, expectedAfterReplication: Option[Set[Int]] = None)( 78 | block: (String, Int, ActorRef) => Unit, afterEachKey: String => Unit = _ => ()): Unit = { 79 | 80 | keys.foreach { key => 81 | val startTime = System.nanoTime() 82 | runOn(n1) { 83 | val latch = TestLatch(n) 84 | val replyTo = system.actorOf(countDownProps(latch)) 85 | 86 | for (i <- 0 until n) 87 | block(key, i, replyTo) 88 | Await.ready(latch, 5.seconds * factor) 89 | } 90 | expectedAfterReplication.foreach { expected => 91 | enterBarrier("repeat-" + key + "-before-awaitReplicated") 92 | awaitReplicated(key, expected) 93 | enterBarrier("repeat-" + key + "-after-awaitReplicated") 94 | } 95 | runOn(n1) { 96 | val endTime = System.nanoTime() 97 | val durationMs = (endTime - startTime).nanos.toMillis 98 | val tps = (n * 1000.0 / durationMs).toInt 99 | println(s"## $n $description took $durationMs ms, $tps TPS") 100 | } 101 | 102 | afterEachKey(key) 103 | enterBarrier("repeat-" + key + "-done") 104 | } 105 | } 106 | 107 | def awaitReplicated(keys: Iterable[String], expectedData: Set[Int]): Unit = 108 | keys.foreach { key => awaitReplicated(key, expectedData) } 109 | 110 | def awaitReplicated(key: String, expectedData: Set[Int]): Unit = { 111 | within(20.seconds) { 112 | awaitAssert { 113 | val readProbe = TestProbe() 114 | replicator.tell(Get(key, ReadLocal), readProbe.ref) 115 | val result = readProbe.expectMsgPF() { case GetSuccess(key, set: ORSet[Int] @unchecked, _) ⇒ set } 116 | result.elements should be(expectedData) 117 | } 118 | } 119 | } 120 | 121 | "Performance" must { 122 | 123 | "setup cluster" in { 124 | roles.foreach { join(_, n1) } 125 | 126 | within(10.seconds) { 127 | awaitAssert { 128 | replicator ! GetReplicaCount 129 | expectMsg(ReplicaCount(roles.size)) 130 | } 131 | } 132 | 133 | enterBarrier("after-setup") 134 | } 135 | 136 | "be great for ORSet Update WriteLocal" in { 137 | val keys = (1 to repeatCount).map("A" + _) 138 | val n = 500 * factor 139 | val expectedData = (0 until n).toSet 140 | repeat("ORSet Update WriteLocal", keys, n)({ (key, i, replyTo) => 141 | replicator.tell(Update(key, ORSet(), WriteLocal)(_ + i), replyTo) 142 | }, key => awaitReplicated(key, expectedData)) 143 | 144 | enterBarrier("after-1") 145 | } 146 | 147 | "be blazingly fast for ORSet Get ReadLocal" in { 148 | val keys = (1 to repeatCount).map("A" + _) 149 | repeat("Get ReadLocal", keys, 1000000 * factor) { (key, i, replyTo) => 150 | replicator.tell(Get(key, ReadLocal), replyTo) 151 | } 152 | enterBarrier("after-2") 153 | } 154 | 155 | "be good for ORSet Update WriteLocal and gossip replication" in { 156 | val keys = (1 to repeatCount).map("B" + _) 157 | val n = 500 * factor 158 | val expected = Some((0 until n).toSet) 159 | repeat("ORSet Update WriteLocal + gossip", keys, n, expected) { (key, i, replyTo) => 160 | replicator.tell(Update(key, ORSet(), WriteLocal)(_ + i), replyTo) 161 | } 162 | enterBarrier("after-3") 163 | } 164 | 165 | "be good for ORSet Update WriteLocal and gossip of existing keys" in { 166 | val keys = (1 to repeatCount).map("B" + _) 167 | val n = 500 * factor 168 | val expected = Some((0 until n).toSet ++ (0 until n).map(-_).toSet) 169 | repeat("ORSet Update WriteLocal existing + gossip", keys, n, expected) { (key, i, replyTo) => 170 | replicator.tell(Update(key, ORSet(), WriteLocal)(_ + (-i)), replyTo) 171 | } 172 | enterBarrier("after-4") 173 | } 174 | 175 | "be good for ORSet Update WriteTwo and gossip replication" in { 176 | val keys = (1 to repeatCount).map("C" + _) 177 | val n = 500 * factor 178 | val expected = Some((0 until n).toSet) 179 | val writeTwo = WriteTo(2, timeout) 180 | repeat("ORSet Update WriteTwo + gossip", keys, n, expected) { (key, i, replyTo) => 181 | replicator.tell(Update(key, ORSet(), writeTwo)(_ + i), replyTo) 182 | } 183 | enterBarrier("after-5") 184 | } 185 | 186 | "be awesome for GCounter Update WriteLocal" in { 187 | val startTime = System.nanoTime() 188 | val n = 100000 * factor 189 | val key = "D" 190 | runOn(n1, n2, n3) { 191 | val latch = TestLatch(n) 192 | val replyTo = system.actorOf(countDownProps(latch)) 193 | for (_ <- 0 until n) 194 | replicator.tell(Update(key, GCounter(), WriteLocal)(_ + 1), replyTo) 195 | Await.ready(latch, 5.seconds * factor) 196 | enterBarrier("update-done-6") 197 | runOn(n1) { 198 | val endTime = System.nanoTime() 199 | val durationMs = (endTime - startTime).nanos.toMillis 200 | val tps = (3 * n * 1000.0 / durationMs).toInt 201 | println(s"## ${3 * n} GCounter Update took $durationMs ms, $tps TPS") 202 | } 203 | } 204 | runOn(n4, n5) { 205 | enterBarrier("update-done-6") 206 | } 207 | 208 | within(20.seconds) { 209 | awaitAssert { 210 | val readProbe = TestProbe() 211 | replicator.tell(Get(key, ReadLocal), readProbe.ref) 212 | val result = readProbe.expectMsgPF() { case GetSuccess(key, c: GCounter, _) ⇒ c } 213 | result.value should be(3 * n) 214 | } 215 | } 216 | enterBarrier("replication-done-6") 217 | runOn(n1) { 218 | val endTime = System.nanoTime() 219 | val durationMs = (endTime - startTime).nanos.toMillis 220 | val tps = (n * 1000.0 / durationMs).toInt 221 | println(s"## $n GCounter Update + gossip took $durationMs ms, $tps TPS") 222 | } 223 | 224 | enterBarrier("after-6") 225 | } 226 | 227 | } 228 | 229 | } 230 | 231 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/ORMap.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.cluster.Cluster 7 | import akka.cluster.UniqueAddress 8 | import akka.util.HashCode 9 | 10 | object ORMap { 11 | private val _empty: ORMap[ReplicatedData] = new ORMap(ORSet.empty, Map.empty) 12 | def empty[A <: ReplicatedData]: ORMap[A] = _empty.asInstanceOf[ORMap[A]] 13 | def apply(): ORMap[ReplicatedData] = _empty 14 | /** 15 | * Java API 16 | */ 17 | def create[A <: ReplicatedData](): ORMap[A] = empty[A] 18 | 19 | /** 20 | * Extract the [[ORMap#entries]]. 21 | */ 22 | def unapply[A <: ReplicatedData](m: ORMap[A]): Option[Map[String, A]] = Some(m.entries) 23 | 24 | } 25 | 26 | /** 27 | * Implements a 'Observed Remove Map' CRDT, also called a 'OR-Map'. 28 | * 29 | * It has similar semantics as an [[ORSet]], but in case of concurrent updates 30 | * the values are merged, and must therefore be [[ReplicatedData]] types themselves. 31 | * 32 | * This class is immutable, i.e. "modifying" methods return a new instance. 33 | */ 34 | @SerialVersionUID(1L) 35 | final class ORMap[A <: ReplicatedData] private[akka] ( 36 | private[akka] val keys: ORSet[String], 37 | private[akka] val values: Map[String, A]) 38 | extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning { 39 | 40 | type T = ORMap[A] 41 | 42 | /** 43 | * Scala API: All entries of the map. 44 | */ 45 | def entries: Map[String, A] = values 46 | 47 | /** 48 | * Java API: All entries of the map. 49 | */ 50 | def getEntries(): java.util.Map[String, A] = { 51 | import scala.collection.JavaConverters._ 52 | entries.asJava 53 | } 54 | 55 | def get(key: String): Option[A] = values.get(key) 56 | 57 | /** 58 | * Scala API: Get the value associated with the key if there is one, 59 | * else return the given default. 60 | */ 61 | def getOrElse(key: String, default: => A): A = values.getOrElse(key, default) 62 | 63 | def contains(key: String): Boolean = values.contains(key) 64 | 65 | def isEmpty: Boolean = values.isEmpty 66 | 67 | /** 68 | * Adds an entry to the map 69 | * @see [[#put]] 70 | */ 71 | def +(entry: (String, A))(implicit node: Cluster): ORMap[A] = { 72 | val (key, value) = entry 73 | put(node, key, value) 74 | } 75 | 76 | /** 77 | * Adds an entry to the map. 78 | * Note that the new `value` will be merged with existing values 79 | * on other nodes and the outcome depends on what `ReplicatedData` 80 | * type that is used. 81 | * 82 | * Consider using [[#updated]] instead of `put` if you want modify 83 | * existing entry. 84 | * 85 | * `IllegalArgumentException` is thrown if you try to replace an existing `ORSet` 86 | * value, because important history can be lost when replacing the `ORSet` and 87 | * undesired effects of merging will occur. Use [[ORMultiMap]] or [[#updated]] instead. 88 | */ 89 | def put(node: Cluster, key: String, value: A): ORMap[A] = put(node.selfUniqueAddress, key, value) 90 | 91 | /** 92 | * INTERNAL API 93 | */ 94 | private[akka] def put(node: UniqueAddress, key: String, value: A): ORMap[A] = 95 | if (value.isInstanceOf[ORSet[_]] && values.contains(key)) 96 | throw new IllegalArgumentException( 97 | "`ORMap.put` must not be used to replace an existing `ORSet` " + 98 | "value, because important history can be lost when replacing the `ORSet` and " + 99 | "undesired effects of merging will occur. Use `ORMultiMap` or `ORMap.updated` instead.") 100 | else 101 | new ORMap(keys.add(node, key), values.updated(key, value)) 102 | 103 | /** 104 | * Scala API: Replace a value by applying the `modify` function on the existing value. 105 | * 106 | * If there is no current value for the `key` the `initial` value will be 107 | * passed to the `modify` function. 108 | */ 109 | def updated(node: Cluster, key: String, initial: A)(modify: A => A): ORMap[A] = 110 | updated(node.selfUniqueAddress, key, initial)(modify) 111 | 112 | /** 113 | * Java API: Replace a value by applying the `modify` function on the existing value. 114 | * 115 | * If there is no current value for the `key` the `initial` value will be 116 | * passed to the `modify` function. 117 | */ 118 | def updated(node: Cluster, key: String, initial: A, modify: akka.japi.Function[A, A]): ORMap[A] = 119 | updated(node, key, initial)(value => modify.apply(value)) 120 | 121 | /** 122 | * INTERNAL API 123 | */ 124 | private[akka] def updated(node: UniqueAddress, key: String, initial: A)(modify: A => A): ORMap[A] = { 125 | val newValue = values.get(key) match { 126 | case Some(old) => modify(old) 127 | case _ => modify(initial) 128 | } 129 | new ORMap(keys.add(node, key), values.updated(key, newValue)) 130 | } 131 | 132 | /** 133 | * Removes an entry from the map. 134 | * Note that if there is a conflicting update on another node the entry will 135 | * not be removed after merge. 136 | */ 137 | def -(key: String)(implicit node: Cluster): ORMap[A] = remove(node, key) 138 | 139 | /** 140 | * Removes an entry from the map. 141 | * Note that if there is a conflicting update on another node the entry will 142 | * not be removed after merge. 143 | */ 144 | def remove(node: Cluster, key: String): ORMap[A] = remove(node.selfUniqueAddress, key) 145 | 146 | /** 147 | * INTERNAL API 148 | */ 149 | private[akka] def remove(node: UniqueAddress, key: String): ORMap[A] = { 150 | new ORMap(keys.remove(node, key), values - key) 151 | } 152 | 153 | override def merge(that: ORMap[A]): ORMap[A] = { 154 | val mergedKeys = keys.merge(that.keys) 155 | var mergedValues = Map.empty[String, A] 156 | mergedKeys.elementsMap.keysIterator.foreach { 157 | case key: String ⇒ 158 | (this.values.get(key), that.values.get(key)) match { 159 | case (Some(thisValue), Some(thatValue)) ⇒ 160 | if (thisValue.getClass != thatValue.getClass) { 161 | val errMsg = s"Wrong type for merging [$key] in [${getClass.getName}], existing type " + 162 | s"[${thisValue.getClass.getName}], got [${thatValue.getClass.getName}]" 163 | throw new IllegalArgumentException(errMsg) 164 | } 165 | val mergedValue = thisValue.merge(thatValue.asInstanceOf[thisValue.T]).asInstanceOf[A] 166 | mergedValues = mergedValues.updated(key, mergedValue) 167 | case (Some(thisValue), None) ⇒ 168 | mergedValues = mergedValues.updated(key, thisValue) 169 | case (None, Some(thatValue)) ⇒ 170 | mergedValues = mergedValues.updated(key, thatValue) 171 | case (None, None) ⇒ throw new IllegalStateException(s"missing value for $key") 172 | } 173 | } 174 | 175 | new ORMap(mergedKeys, mergedValues) 176 | } 177 | 178 | override def needPruningFrom(removedNode: UniqueAddress): Boolean = { 179 | keys.needPruningFrom(removedNode) || values.exists { 180 | case (_, data: RemovedNodePruning) ⇒ data.needPruningFrom(removedNode) 181 | case _ ⇒ false 182 | } 183 | } 184 | 185 | override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): ORMap[A] = { 186 | val prunedKeys = keys.prune(removedNode, collapseInto) 187 | val prunedValues = values.foldLeft(values) { 188 | case (acc, (key, data: RemovedNodePruning)) if data.needPruningFrom(removedNode) ⇒ 189 | acc.updated(key, data.prune(removedNode, collapseInto).asInstanceOf[A]) 190 | case (acc, _) ⇒ acc 191 | } 192 | new ORMap(prunedKeys, prunedValues) 193 | } 194 | 195 | override def pruningCleanup(removedNode: UniqueAddress): ORMap[A] = { 196 | val pruningCleanupedKeys = keys.pruningCleanup(removedNode) 197 | val pruningCleanupedValues = values.foldLeft(values) { 198 | case (acc, (key, data: RemovedNodePruning)) if data.needPruningFrom(removedNode) ⇒ 199 | acc.updated(key, data.pruningCleanup(removedNode).asInstanceOf[A]) 200 | case (acc, _) ⇒ acc 201 | } 202 | new ORMap(pruningCleanupedKeys, pruningCleanupedValues) 203 | } 204 | 205 | // this class cannot be a `case class` because we need different `unapply` 206 | 207 | override def toString: String = s"OR$entries" 208 | 209 | override def equals(o: Any): Boolean = o match { 210 | case other: ORMap[_] => keys == other.keys && values == other.values 211 | case _ => false 212 | } 213 | 214 | override def hashCode: Int = { 215 | var result = HashCode.SEED 216 | result = HashCode.hash(result, keys) 217 | result = HashCode.hash(result, values) 218 | result 219 | } 220 | 221 | } 222 | 223 | -------------------------------------------------------------------------------- /src/test/scala/akka/contrib/datareplication/ORMapSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | 5 | package akka.contrib.datareplication 6 | 7 | import org.scalatest.WordSpec 8 | import org.scalatest.Matchers 9 | import akka.actor.Address 10 | import akka.cluster.UniqueAddress 11 | import akka.contrib.datareplication.Replicator.Changed 12 | 13 | class ORMapSpec extends WordSpec with Matchers { 14 | 15 | val node1 = UniqueAddress(Address("akka.tcp", "Sys", "localhost", 2551), 1) 16 | val node2 = UniqueAddress(node1.address.copy(port = Some(2552)), 2) 17 | 18 | "A ORMap" must { 19 | 20 | "be able to add entries" in { 21 | val m = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B") 22 | val GSet(a) = m.entries("a") 23 | a should be(Set("A")) 24 | val GSet(b) = m.entries("b") 25 | b should be(Set("B")) 26 | 27 | val m2 = m.put(node1, "a", GSet() + "C") 28 | val GSet(a2) = m2.entries("a") 29 | a2 should be(Set("C")) 30 | 31 | } 32 | 33 | "be able to remove entry" in { 34 | val m = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B").remove(node1, "a") 35 | m.entries.keySet should not contain ("a") 36 | m.entries.keySet should contain("b") 37 | } 38 | 39 | "be able to add removed" in { 40 | val m = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B").remove(node1, "a") 41 | m.entries.keySet should not contain ("a") 42 | m.entries.keySet should contain("b") 43 | val m2 = m.put(node1, "a", GSet() + "C") 44 | m2.entries.keySet should contain("a") 45 | m2.entries.keySet should contain("b") 46 | } 47 | 48 | "be able to have its entries correctly merged with another ORMap with other entries" in { 49 | val m1 = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B") 50 | val m2 = ORMap().put(node2, "c", GSet() + "C") 51 | 52 | // merge both ways 53 | val merged1 = m1 merge m2 54 | merged1.entries.keySet should contain("a") 55 | merged1.entries.keySet should contain("b") 56 | merged1.entries.keySet should contain("c") 57 | 58 | val merged2 = m2 merge m1 59 | merged2.entries.keySet should contain("a") 60 | merged2.entries.keySet should contain("b") 61 | merged2.entries.keySet should contain("c") 62 | } 63 | 64 | "be able to have its entries correctly merged with another ORMap with overlapping entries" in { 65 | val m1 = ORMap().put(node1, "a", GSet() + "A1").put(node1, "b", GSet() + "B1"). 66 | remove(node1, "a").put(node1, "d", GSet() + "D1") 67 | val m2 = ORMap().put(node2, "c", GSet() + "C2").put(node2, "a", GSet() + "A2"). 68 | put(node2, "b", GSet() + "B2").remove(node2, "b").put(node2, "d", GSet() + "D2") 69 | 70 | // merge both ways 71 | val merged1 = m1 merge m2 72 | merged1.entries.keySet should contain("a") 73 | val GSet(a1) = merged1.entries("a") 74 | a1 should be(Set("A2")) 75 | merged1.entries.keySet should contain("b") 76 | val GSet(b1) = merged1.entries("b") 77 | b1 should be(Set("B1")) 78 | merged1.entries.keySet should contain("c") 79 | merged1.entries.keySet should contain("d") 80 | val GSet(d1) = merged1.entries("d") 81 | d1 should be(Set("D1", "D2")) 82 | 83 | val merged2 = m2 merge m1 84 | merged2.entries.keySet should contain("a") 85 | val GSet(a2) = merged1.entries("a") 86 | a2 should be(Set("A2")) 87 | merged2.entries.keySet should contain("b") 88 | val GSet(b2) = merged2.entries("b") 89 | b2 should be(Set("B1")) 90 | merged2.entries.keySet should contain("c") 91 | merged2.entries.keySet should contain("d") 92 | val GSet(d2) = merged2.entries("d") 93 | d2 should be(Set("D1", "D2")) 94 | } 95 | 96 | "illustrate the danger of using remove+put to replace an entry" in { 97 | val m1 = ORMap.empty.put(node1, "a", GSet.empty + "A").put(node1, "b", GSet.empty + "B") 98 | val m2 = ORMap.empty.put(node2, "c", GSet.empty + "C") 99 | 100 | val merged1 = m1 merge m2 101 | 102 | val m3 = merged1.remove(node1, "b").put(node1, "b", GSet.empty + "B2") 103 | // same thing if only put is used 104 | // val m3 = merged1.put(node1, "b", GSet() + "B2") 105 | val merged2 = merged1 merge m3 106 | 107 | merged2.entries("a").elements should be(Set("A")) 108 | // note that B is included, because GSet("B") is merged with GSet("B2") 109 | merged2.entries("b").elements should be(Set("B", "B2")) 110 | merged2.entries("c").elements should be(Set("C")) 111 | } 112 | 113 | "not allow put for ORSet elements type" in { 114 | val m = ORMap().put(node1, "a", ORSet().add(node1, "A")) 115 | 116 | intercept[IllegalArgumentException] { 117 | m.put(node1, "a", ORSet().add(node1, "B")) 118 | } 119 | } 120 | 121 | "be able to update entry" in { 122 | val m1 = ORMap.empty[ORSet[String]].put(node1, "a", ORSet.empty.add(node1, "A")) 123 | .put(node1, "b", ORSet.empty.add(node1, "B01").add(node1, "B02").add(node1, "B03")) 124 | val m2 = ORMap.empty[ORSet[String]].put(node2, "c", ORSet.empty.add(node2, "C")) 125 | 126 | val merged1: ORMap[ORSet[String]] = m1 merge m2 127 | 128 | val m3 = merged1.updated(node1, "b", ORSet.empty[String])(_.clear(node1).add(node1, "B2")) 129 | 130 | val merged2 = merged1 merge m3 131 | merged2.entries("a").elements should be(Set("A")) 132 | merged2.entries("b").elements should be(Set("B2")) 133 | merged2.entries("c").elements should be(Set("C")) 134 | 135 | val m4 = merged1.updated(node2, "b", ORSet.empty[String])(_.add(node2, "B3")) 136 | val merged3 = m3 merge m4 137 | merged3.entries("a").elements should be(Set("A")) 138 | merged3.entries("b").elements should be(Set("B2", "B3")) 139 | merged3.entries("c").elements should be(Set("C")) 140 | } 141 | 142 | "be able to update ORSet entry with remove+put" in { 143 | val m1 = ORMap.empty[ORSet[String]].put(node1, "a", ORSet.empty.add(node1, "A01")) 144 | .updated(node1, "a", ORSet.empty[String])(_.add(node1, "A02")) 145 | .updated(node1, "a", ORSet.empty[String])(_.add(node1, "A03")) 146 | .put(node1, "b", ORSet.empty.add(node1, "B01").add(node1, "B02").add(node1, "B03")) 147 | val m2 = ORMap.empty[ORSet[String]].put(node2, "c", ORSet.empty.add(node2, "C")) 148 | 149 | val merged1 = m1 merge m2 150 | 151 | // note that remove + put work because the new VectorClock version is incremented 152 | // from a global counter 153 | val m3 = merged1.remove(node1, "b").put(node1, "b", ORSet.empty.add(node1, "B2")) 154 | 155 | val merged2 = merged1 merge m3 156 | merged2.entries("a").elements should be(Set("A01", "A02", "A03")) 157 | merged2.entries("b").elements should be(Set("B2")) 158 | merged2.entries("c").elements should be(Set("C")) 159 | 160 | val m4 = merged1.updated(node2, "b", ORSet.empty[String])(_.add(node2, "B3")) 161 | val merged3 = m3 merge m4 162 | merged3.entries("a").elements should be(Set("A01", "A02", "A03")) 163 | merged3.entries("b").elements should be(Set("B2", "B3")) 164 | merged3.entries("c").elements should be(Set("C")) 165 | } 166 | 167 | "be able to update ORSet entry with remove -> merge -> put" in { 168 | val m1 = ORMap.empty.put(node1, "a", ORSet.empty.add(node1, "A")) 169 | .put(node1, "b", ORSet.empty.add(node1, "B01").add(node1, "B02").add(node1, "B03")) 170 | val m2 = ORMap.empty.put(node2, "c", ORSet.empty.add(node2, "C")) 171 | 172 | val merged1 = m1 merge m2 173 | 174 | val m3 = merged1.remove(node1, "b") 175 | 176 | val merged2 = merged1 merge m3 177 | merged2.entries("a").elements should be(Set("A")) 178 | merged2.contains("b") should be(false) 179 | merged2.entries("c").elements should be(Set("C")) 180 | 181 | val m4 = merged2.put(node1, "b", ORSet.empty.add(node1, "B2")) 182 | val m5 = merged2.updated(node2, "c", ORSet.empty[String])(_.add(node2, "C2")) 183 | .put(node2, "b", ORSet.empty.add(node2, "B3")) 184 | 185 | val merged3 = m5 merge m4 186 | merged3.entries("a").elements should be(Set("A")) 187 | merged3.entries("b").elements should be(Set("B2", "B3")) 188 | merged3.entries("c").elements should be(Set("C", "C2")) 189 | } 190 | 191 | "have unapply extractor" in { 192 | val m1 = ORMap.empty.put(node1, "a", Flag(true)).put(node2, "b", Flag(false)) 193 | val m2: ORMap[Flag] = m1 194 | val ORMap(entries1) = m1 195 | val entries2: Map[String, Flag] = entries1 196 | Changed("key", m1) match { 197 | case Changed("key", ORMap(entries3)) => 198 | val entries4: Map[String, ReplicatedData] = entries3 199 | entries4 should be(Map("a" -> Flag(true), "b" -> Flag(false))) 200 | } 201 | } 202 | 203 | } 204 | } 205 | -------------------------------------------------------------------------------- /src/main/scala/akka/contrib/datareplication/VectorClock.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import akka.AkkaException 7 | import System.{ currentTimeMillis ⇒ newTimestamp } 8 | import java.security.MessageDigest 9 | import java.util.concurrent.atomic.AtomicLong 10 | import scala.collection.immutable.TreeMap 11 | import scala.annotation.tailrec 12 | import akka.cluster.Cluster 13 | import akka.cluster.UniqueAddress 14 | 15 | /** 16 | * VectorClock module with helper classes and methods. 17 | */ 18 | object VectorClock { 19 | 20 | val empty: VectorClock = new VectorClock(TreeMap.empty[UniqueAddress, Long]) 21 | def apply(): VectorClock = empty 22 | /** 23 | * Java API 24 | */ 25 | def create(): VectorClock = empty 26 | 27 | sealed trait Ordering 28 | case object After extends Ordering 29 | case object Before extends Ordering 30 | case object Same extends Ordering 31 | case object Concurrent extends Ordering 32 | /** 33 | * Marker to ensure that we do a full order comparison instead of bailing out early. 34 | */ 35 | private case object FullOrder extends Ordering 36 | 37 | /** 38 | * Java API: The [[After]] instance 39 | */ 40 | def AfterInstance = After 41 | 42 | /** 43 | * Java API: The [[Before]] instance 44 | */ 45 | def BeforeInstance = Before 46 | 47 | /** 48 | * Java API: The [[Same]] instance 49 | */ 50 | def SameInstance = Same 51 | 52 | /** 53 | * Java API: The [[Concurrent]] instance 54 | */ 55 | def ConcurrentInstance = Concurrent 56 | 57 | private object Timestamp { 58 | final val Zero = 0L 59 | final val EndMarker = Long.MinValue 60 | val counter = new AtomicLong(1L) 61 | } 62 | 63 | /** 64 | * Marker to signal that we have reached the end of a vector clock. 65 | */ 66 | private val cmpEndMarker = (null, Timestamp.EndMarker) 67 | 68 | } 69 | 70 | /** 71 | * Representation of a Vector-based clock (counting clock), inspired by Lamport logical clocks. 72 | * {{{ 73 | * Reference: 74 | * 1) Leslie Lamport (1978). "Time, clocks, and the ordering of events in a distributed system". Communications of the ACM 21 (7): 558-565. 75 | * 2) Friedemann Mattern (1988). "Virtual Time and Global States of Distributed Systems". Workshop on Parallel and Distributed Algorithms: pp. 215-226 76 | * }}} 77 | * 78 | * Based on code from `akka.cluster.VectorClock`. 79 | * 80 | * This class is immutable, i.e. "modifying" methods return a new instance. 81 | */ 82 | @SerialVersionUID(1L) 83 | final case class VectorClock private[akka] ( 84 | private[akka] val versions: TreeMap[UniqueAddress, Long]) 85 | extends ReplicatedData with ReplicatedDataSerialization with RemovedNodePruning { 86 | 87 | type T = VectorClock 88 | 89 | import VectorClock._ 90 | 91 | /** 92 | * Increment the version for the node passed as argument. Returns a new VectorClock. 93 | */ 94 | def +(node: Cluster): VectorClock = increment(node) 95 | 96 | /** 97 | * INTERNAL API 98 | * Increment the version for the node passed as argument. Returns a new VectorClock. 99 | */ 100 | private[akka] def +(node: UniqueAddress): VectorClock = increment(node) 101 | 102 | /** 103 | * Increment the version for the node passed as argument. Returns a new VectorClock. 104 | */ 105 | def increment(node: Cluster): VectorClock = increment(node.selfUniqueAddress) 106 | 107 | /** 108 | * INTERNAL API 109 | * Increment the version for the node passed as argument. Returns a new VectorClock. 110 | */ 111 | private[akka] def increment(node: UniqueAddress): VectorClock = 112 | copy(versions = versions.updated(node, Timestamp.counter.getAndIncrement())) 113 | 114 | /** 115 | * Returns true if this and that are concurrent else false. 116 | */ 117 | def <>(that: VectorClock): Boolean = compareOnlyTo(that, Concurrent) eq Concurrent 118 | 119 | /** 120 | * Returns true if this is before that else false. 121 | */ 122 | def <(that: VectorClock): Boolean = compareOnlyTo(that, Before) eq Before 123 | 124 | /** 125 | * Returns true if this is after that else false. 126 | */ 127 | def >(that: VectorClock): Boolean = compareOnlyTo(that, After) eq After 128 | 129 | /** 130 | * Returns true if this VectorClock has the same history as the 'that' VectorClock else false. 131 | */ 132 | def ==(that: VectorClock): Boolean = compareOnlyTo(that, Same) eq Same 133 | 134 | /** 135 | * Vector clock comparison according to the semantics described by compareTo, with the ability to bail 136 | * out early if the we can't reach the Ordering that we are looking for. 137 | * 138 | * The ordering always starts with Same and can then go to Same, Before or After 139 | * If we're on After we can only go to After or Concurrent 140 | * If we're on Before we can only go to Before or Concurrent 141 | * If we go to Concurrent we exit the loop immediately 142 | * 143 | * If you send in the ordering FullOrder, you will get a full comparison. 144 | */ 145 | private final def compareOnlyTo(that: VectorClock, order: Ordering): Ordering = { 146 | def nextOrElse[T](iter: Iterator[T], default: T): T = if (iter.hasNext) iter.next() else default 147 | 148 | def compare(i1: Iterator[(UniqueAddress, Long)], i2: Iterator[(UniqueAddress, Long)], requestedOrder: Ordering): Ordering = { 149 | @tailrec 150 | def compareNext(nt1: (UniqueAddress, Long), nt2: (UniqueAddress, Long), currentOrder: Ordering): Ordering = 151 | if ((requestedOrder ne FullOrder) && (currentOrder ne Same) && (currentOrder ne requestedOrder)) currentOrder 152 | else if ((nt1 eq cmpEndMarker) && (nt2 eq cmpEndMarker)) currentOrder 153 | // i1 is empty but i2 is not, so i1 can only be Before 154 | else if (nt1 eq cmpEndMarker) { if (currentOrder eq After) Concurrent else Before } 155 | // i2 is empty but i1 is not, so i1 can only be After 156 | else if (nt2 eq cmpEndMarker) { if (currentOrder eq Before) Concurrent else After } 157 | else { 158 | // compare the nodes 159 | val nc = nt1._1 compareTo nt2._1 160 | if (nc == 0) { 161 | // both nodes exist compare the timestamps 162 | // same timestamp so just continue with the next nodes 163 | if (nt1._2 == nt2._2) compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), currentOrder) 164 | else if (nt1._2 < nt2._2) { 165 | // t1 is less than t2, so i1 can only be Before 166 | if (currentOrder eq After) Concurrent 167 | else compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), Before) 168 | } else { 169 | // t2 is less than t1, so i1 can only be After 170 | if (currentOrder eq Before) Concurrent 171 | else compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), After) 172 | } 173 | } else if (nc < 0) { 174 | // this node only exists in i1 so i1 can only be After 175 | if (currentOrder eq Before) Concurrent 176 | else compareNext(nextOrElse(i1, cmpEndMarker), nt2, After) 177 | } else { 178 | // this node only exists in i2 so i1 can only be Before 179 | if (currentOrder eq After) Concurrent 180 | else compareNext(nt1, nextOrElse(i2, cmpEndMarker), Before) 181 | } 182 | } 183 | 184 | compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), Same) 185 | } 186 | 187 | if ((this eq that) || (this.versions eq that.versions)) Same 188 | else compare(this.versions.iterator, that.versions.iterator, if (order eq Concurrent) FullOrder else order) 189 | } 190 | 191 | /** 192 | * Compare two vector clocks. The outcome will be one of the following: 193 | *

194 | * {{{ 195 | * 1. Clock 1 is SAME (==) as Clock 2 iff for all i c1(i) == c2(i) 196 | * 2. Clock 1 is BEFORE (<) Clock 2 iff for all i c1(i) <= c2(i) and there exist a j such that c1(j) < c2(j) 197 | * 3. Clock 1 is AFTER (>) Clock 2 iff for all i c1(i) >= c2(i) and there exist a j such that c1(j) > c2(j). 198 | * 4. Clock 1 is CONCURRENT (<>) to Clock 2 otherwise. 199 | * }}} 200 | */ 201 | def compareTo(that: VectorClock): Ordering = { 202 | compareOnlyTo(that, FullOrder) 203 | } 204 | 205 | /** 206 | * Merges this VectorClock with another VectorClock. E.g. merges its versioned history. 207 | */ 208 | def merge(that: VectorClock): VectorClock = { 209 | var mergedVersions = that.versions 210 | for ((node, time) ← versions) { 211 | val mergedVersionsCurrentTime = mergedVersions.getOrElse(node, Timestamp.Zero) 212 | if (time > mergedVersionsCurrentTime) 213 | mergedVersions = mergedVersions.updated(node, time) 214 | } 215 | VectorClock(mergedVersions) 216 | } 217 | 218 | override def needPruningFrom(removedNode: UniqueAddress): Boolean = 219 | versions.contains(removedNode) 220 | 221 | override def prune(removedNode: UniqueAddress, collapseInto: UniqueAddress): VectorClock = 222 | copy(versions = versions - removedNode) + collapseInto 223 | 224 | override def pruningCleanup(removedNode: UniqueAddress): VectorClock = copy(versions = versions - removedNode) 225 | 226 | override def toString = versions.map { case ((n, t)) ⇒ n + " -> " + t }.mkString("VectorClock(", ", ", ")") 227 | } 228 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/sample/datareplication/ReplicatedServiceRegistrySpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package sample.datareplication 5 | 6 | import scala.concurrent.duration._ 7 | import akka.actor.Actor 8 | import akka.actor.ActorLogging 9 | import akka.actor.ActorRef 10 | import akka.actor.PoisonPill 11 | import akka.actor.Props 12 | import akka.actor.Terminated 13 | import akka.cluster.Cluster 14 | import akka.cluster.ClusterEvent 15 | import akka.cluster.ClusterEvent.LeaderChanged 16 | import akka.contrib.datareplication.DataReplication 17 | import akka.contrib.datareplication.GSet 18 | import akka.contrib.datareplication.ORSet 19 | import akka.contrib.datareplication.Replicator.GetReplicaCount 20 | import akka.contrib.datareplication.Replicator.ReplicaCount 21 | import akka.contrib.datareplication.STMultiNodeSpec 22 | import akka.remote.testconductor.RoleName 23 | import akka.remote.testkit.MultiNodeConfig 24 | import akka.remote.testkit.MultiNodeSpec 25 | import akka.testkit._ 26 | import com.typesafe.config.ConfigFactory 27 | 28 | object ReplicatedServiceRegistrySpec extends MultiNodeConfig { 29 | val node1 = role("node-1") 30 | val node2 = role("node-2") 31 | val node3 = role("node-3") 32 | 33 | commonConfig(ConfigFactory.parseString(""" 34 | akka.loglevel = INFO 35 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 36 | akka.log-dead-letters-during-shutdown = off 37 | """)) 38 | 39 | class Service extends Actor { 40 | def receive = { 41 | case s: String => sender() ! self.path.name + ": " + s 42 | } 43 | } 44 | 45 | } 46 | 47 | object ReplicatedServiceRegistry { 48 | import akka.contrib.datareplication.Replicator._ 49 | 50 | val props: Props = Props[ReplicatedServiceRegistry] 51 | 52 | /** 53 | * Register a `service` with a `name`. Several services 54 | * can be registered with the same `name`. 55 | * It will be removed when it is terminated. 56 | */ 57 | final case class Register(name: String, service: ActorRef) 58 | /** 59 | * Lookup services registered for a `name`. [[Bindings]] will 60 | * be sent to `sender()`. 61 | */ 62 | final case class Lookup(name: String) 63 | /** 64 | * Reply for [[Lookup]] 65 | */ 66 | final case class Bindings(name: String, services: Set[ActorRef]) 67 | /** 68 | * Published to `System.eventStream` when services are changed. 69 | */ 70 | final case class BindingChanged(name: String, services: Set[ActorRef]) 71 | 72 | private val KeysDataKey = "service-keys" 73 | } 74 | 75 | class ReplicatedServiceRegistry() extends Actor with ActorLogging { 76 | import akka.contrib.datareplication.Replicator._ 77 | import ReplicatedServiceRegistry._ 78 | 79 | val replicator = DataReplication(context.system).replicator 80 | implicit val cluster = Cluster(context.system) 81 | 82 | var keys = Set.empty[String] 83 | var services = Map.empty[String, Set[ActorRef]] 84 | var leader = false 85 | 86 | def dataKey(serviceName: String): String = 87 | "service:" + serviceName 88 | 89 | override def preStart(): Unit = { 90 | replicator ! Subscribe(KeysDataKey, self) 91 | cluster.subscribe(self, ClusterEvent.InitialStateAsEvents, classOf[ClusterEvent.LeaderChanged]) 92 | } 93 | 94 | override def postStop(): Unit = { 95 | cluster.unsubscribe(self) 96 | } 97 | 98 | def receive = { 99 | case Register(name, service) => 100 | val dKey = dataKey(name) 101 | // store the service names in a separate GSet to be able to 102 | // get notifications of new names 103 | if (!keys(dKey)) 104 | replicator ! Update(KeysDataKey, GSet(), WriteLocal)(_ + dKey) 105 | // add the service 106 | replicator ! Update(dKey, ORSet(), WriteLocal)(_ + service) 107 | 108 | case Lookup(key) => 109 | sender() ! Bindings(key, services.getOrElse(key, Set.empty)) 110 | 111 | case Changed(KeysDataKey, data: GSet[String] @unchecked) => 112 | val newKeys = data.elements 113 | log.debug("Services changed, added: {}, all: {}", (newKeys -- keys), newKeys) 114 | (newKeys -- keys).foreach { dKey => 115 | // subscribe to get notifications of when services with this name are added or removed 116 | replicator ! Subscribe(dKey, self) 117 | } 118 | keys = newKeys 119 | 120 | case Changed(dKey, data: ORSet[ActorRef] @unchecked) => 121 | val name = dKey.split(":").tail.mkString 122 | val newServices = data.elements 123 | log.debug("Services changed for name [{}]: {}", name, newServices) 124 | services = services.updated(name, newServices) 125 | context.system.eventStream.publish(BindingChanged(name, newServices)) 126 | if (leader) 127 | newServices.foreach(context.watch) // watch is idempotent 128 | 129 | case LeaderChanged(node) => 130 | // Let one node (the leader) be responsible for removal of terminated services 131 | // to avoid redundant work and too many death watch notifications. 132 | // It is not critical to only do it from one node. 133 | val wasLeader = leader 134 | leader = node.exists(_ == cluster.selfAddress) 135 | // when used with many (> 500) services you must increase the system message buffer 136 | // `akka.remote.system-message-buffer-size` 137 | if (!wasLeader && leader) 138 | for (refs <- services.valuesIterator; ref <- refs) 139 | context.watch(ref) 140 | else if (wasLeader && !leader) 141 | for (refs <- services.valuesIterator; ref <- refs) 142 | context.unwatch(ref) 143 | 144 | case Terminated(ref) => 145 | val names = services.collect { case (name, refs) if refs.contains(ref) => name } 146 | names.foreach { name => 147 | log.debug("Service with name [{}] terminated: {}", name, ref) 148 | replicator ! Update(dataKey(name), ORSet(), WriteLocal)(_ - ref) 149 | } 150 | 151 | case _: UpdateResponse => // ok 152 | } 153 | 154 | } 155 | 156 | class ReplicatedServiceRegistrySpecMultiJvmNode1 extends ReplicatedServiceRegistrySpec 157 | class ReplicatedServiceRegistrySpecMultiJvmNode2 extends ReplicatedServiceRegistrySpec 158 | class ReplicatedServiceRegistrySpecMultiJvmNode3 extends ReplicatedServiceRegistrySpec 159 | 160 | class ReplicatedServiceRegistrySpec extends MultiNodeSpec(ReplicatedServiceRegistrySpec) with STMultiNodeSpec with ImplicitSender { 161 | import ReplicatedServiceRegistrySpec._ 162 | import ReplicatedServiceRegistry._ 163 | 164 | override def initialParticipants = roles.size 165 | 166 | val cluster = Cluster(system) 167 | val registry = system.actorOf(ReplicatedServiceRegistry.props) 168 | 169 | def join(from: RoleName, to: RoleName): Unit = { 170 | runOn(from) { 171 | cluster join node(to).address 172 | } 173 | enterBarrier(from.name + "-joined") 174 | } 175 | 176 | "Demo of a replicated service registry" must { 177 | "join cluster" in within(10.seconds) { 178 | join(node1, node1) 179 | join(node2, node1) 180 | join(node3, node1) 181 | 182 | awaitAssert { 183 | DataReplication(system).replicator ! GetReplicaCount 184 | expectMsg(ReplicaCount(roles.size)) 185 | } 186 | enterBarrier("after-1") 187 | } 188 | 189 | "replicate service entry" in within(10.seconds) { 190 | runOn(node1) { 191 | val a1 = system.actorOf(Props[Service], name = "a1") 192 | registry ! Register("a", a1) 193 | } 194 | 195 | awaitAssert { 196 | val probe = TestProbe() 197 | registry.tell(Lookup("a"), probe.ref) 198 | probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1")) 199 | } 200 | 201 | enterBarrier("after-2") 202 | } 203 | 204 | "replicate many service entries" in within(10.seconds) { 205 | for (i <- 100 until 200) { 206 | val service = system.actorOf(Props[Service], name = myself.name + "_" + i) 207 | registry ! Register("a" + i, service) 208 | } 209 | 210 | awaitAssert { 211 | val probe = TestProbe() 212 | for (i <- 100 until 200) { 213 | registry.tell(Lookup("a" + i), probe.ref) 214 | probe.expectMsgType[Bindings].services.map(_.path.name) should be(roles.map(_.name + "_" + i).toSet) 215 | } 216 | } 217 | 218 | enterBarrier("after-3") 219 | } 220 | 221 | "replicate updated service entry, and publish to even bus" in { 222 | val probe = TestProbe() 223 | system.eventStream.subscribe(probe.ref, classOf[BindingChanged]) 224 | 225 | runOn(node2) { 226 | val a2 = system.actorOf(Props[Service], name = "a2") 227 | registry ! Register("a", a2) 228 | } 229 | 230 | probe.within(10.seconds) { 231 | probe.expectMsgType[BindingChanged].services.map(_.path.name) should be(Set("a1", "a2")) 232 | registry.tell(Lookup("a"), probe.ref) 233 | probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1", "a2")) 234 | } 235 | 236 | enterBarrier("after-4") 237 | } 238 | 239 | "remove terminated service" in { 240 | val probe = TestProbe() 241 | system.eventStream.subscribe(probe.ref, classOf[BindingChanged]) 242 | 243 | runOn(node2) { 244 | registry.tell(Lookup("a"), probe.ref) 245 | val a2 = probe.expectMsgType[Bindings].services.find(_.path.name == "a2").get 246 | a2 ! PoisonPill 247 | } 248 | 249 | probe.within(10.seconds) { 250 | probe.expectMsgType[BindingChanged].services.map(_.path.name) should be(Set("a1")) 251 | registry.tell(Lookup("a"), probe.ref) 252 | probe.expectMsgType[Bindings].services.map(_.path.name) should be(Set("a1")) 253 | } 254 | 255 | enterBarrier("after-5") 256 | } 257 | 258 | } 259 | 260 | } 261 | 262 | -------------------------------------------------------------------------------- /src/multi-jvm/scala/akka/contrib/datareplication/JepsenInspiredInsertSpec.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (C) 2009-2014 Typesafe Inc. 3 | */ 4 | package akka.contrib.datareplication 5 | 6 | import scala.concurrent.duration._ 7 | import akka.remote.testconductor.RoleName 8 | import akka.remote.testkit.MultiNodeConfig 9 | import akka.remote.testkit.MultiNodeSpec 10 | import com.typesafe.config.ConfigFactory 11 | import akka.cluster.Cluster 12 | import akka.remote.transport.ThrottlerTransportAdapter.Direction 13 | import akka.testkit._ 14 | import scala.concurrent.forkjoin.ThreadLocalRandom 15 | 16 | object JepsenInspiredInsertSpec extends MultiNodeConfig { 17 | val controller = role("controller") 18 | val n1 = role("n1") 19 | val n2 = role("n2") 20 | val n3 = role("n3") 21 | val n4 = role("n4") 22 | val n5 = role("n5") 23 | 24 | commonConfig(ConfigFactory.parseString(""" 25 | akka.loglevel = INFO 26 | akka.actor.provider = "akka.cluster.ClusterActorRefProvider" 27 | akka.log-dead-letters = off 28 | akka.log-dead-letters-during-shutdown = off 29 | akka.remote.log-remote-lifecycle-events = ERROR 30 | akka.testconductor.barrier-timeout = 60 s 31 | """)) 32 | 33 | testTransport(on = true) 34 | 35 | } 36 | 37 | class JepsenInspiredInsertSpecMultiJvmNode1 extends JepsenInspiredInsertSpec 38 | class JepsenInspiredInsertSpecMultiJvmNode2 extends JepsenInspiredInsertSpec 39 | class JepsenInspiredInsertSpecMultiJvmNode3 extends JepsenInspiredInsertSpec 40 | class JepsenInspiredInsertSpecMultiJvmNode4 extends JepsenInspiredInsertSpec 41 | class JepsenInspiredInsertSpecMultiJvmNode5 extends JepsenInspiredInsertSpec 42 | class JepsenInspiredInsertSpecMultiJvmNode6 extends JepsenInspiredInsertSpec 43 | 44 | class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) with STMultiNodeSpec with ImplicitSender { 45 | import JepsenInspiredInsertSpec._ 46 | import Replicator._ 47 | 48 | override def initialParticipants = roles.size 49 | 50 | implicit val cluster = Cluster(system) 51 | val replicator = DataReplication(system).replicator 52 | val nodes = roles.drop(1) // controller not part of active nodes 53 | val nodeCount = nodes.size 54 | val timeout = 3.seconds.dilated 55 | val delayMillis = 0 56 | val totalCount = 200 57 | // val delayMillis = 20 58 | // val totalCount = 2000 59 | val expectedData = (0 until totalCount).toSet 60 | val data: Map[RoleName, Seq[Int]] = { 61 | val nodeIndex = nodes.zipWithIndex.map { case (n, i) => i -> n }.toMap 62 | (0 until totalCount).groupBy(i => nodeIndex(i % nodeCount)) 63 | } 64 | lazy val myData: Seq[Int] = data(myself) 65 | 66 | def sleepDelay(): Unit = 67 | if (delayMillis != 0) { 68 | val rndDelay = ThreadLocalRandom.current().nextInt(delayMillis) 69 | if (rndDelay != 0) Thread.sleep(delayMillis) 70 | } 71 | 72 | def sleepBeforePartition(): Unit = { 73 | if (delayMillis != 0) 74 | Thread.sleep(delayMillis * totalCount / nodeCount / 10) 75 | } 76 | 77 | def sleepDuringPartition(): Unit = 78 | Thread.sleep(math.max(5000, delayMillis * totalCount / nodeCount / 2)) 79 | 80 | def join(from: RoleName, to: RoleName): Unit = { 81 | runOn(from) { 82 | cluster join node(to).address 83 | } 84 | enterBarrier(from.name + "-joined") 85 | } 86 | 87 | "Insert from 5 nodes" must { 88 | 89 | "setup cluster" in { 90 | runOn(nodes: _*) { 91 | nodes.foreach { join(_, n1) } 92 | 93 | within(10.seconds) { 94 | awaitAssert { 95 | replicator ! GetReplicaCount 96 | expectMsg(ReplicaCount(nodes.size)) 97 | } 98 | } 99 | } 100 | 101 | runOn(controller) { 102 | nodes.foreach { n => enterBarrier(n.name + "-joined") } 103 | } 104 | 105 | enterBarrier("after-setup") 106 | } 107 | } 108 | 109 | "replicate values when all nodes connected" in { 110 | val key = "A" 111 | runOn(nodes: _*) { 112 | val writeProbe = TestProbe() 113 | val writeAcks = myData.map { i => 114 | sleepDelay() 115 | replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ + i), writeProbe.ref) 116 | writeProbe.receiveOne(3.seconds) 117 | } 118 | val successWriteAcks = writeAcks.collect { case success: UpdateSuccess => success } 119 | val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure => fail } 120 | successWriteAcks.map(_.request.get).toSet should be(myData.toSet) 121 | successWriteAcks.size should be(myData.size) 122 | failureWriteAcks should be(Nil) 123 | (successWriteAcks.size + failureWriteAcks.size) should be(myData.size) 124 | 125 | // eventually all nodes will have the data 126 | within(15.seconds) { 127 | awaitAssert { 128 | val readProbe = TestProbe() 129 | replicator.tell(Get(key, ReadLocal), readProbe.ref) 130 | val result = readProbe.expectMsgPF() { case GetSuccess(key, set: ORSet[_], _) ⇒ set } 131 | result.elements should be(expectedData) 132 | } 133 | } 134 | 135 | } 136 | 137 | enterBarrier("after-test-1") 138 | } 139 | 140 | "write/read to quorum when all nodes connected" in { 141 | val key = "B" 142 | val readQuorum = ReadQuorum(timeout) 143 | val writeQuorum = WriteQuorum(timeout) 144 | runOn(nodes: _*) { 145 | val writeProbe = TestProbe() 146 | val writeAcks = myData.map { i => 147 | sleepDelay() 148 | replicator.tell(Update(key, ORSet(), writeQuorum, Some(i))(_ + i), writeProbe.ref) 149 | writeProbe.receiveOne(timeout + 1.second) 150 | } 151 | val successWriteAcks = writeAcks.collect { case success: UpdateSuccess => success } 152 | val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure => fail } 153 | successWriteAcks.map(_.request.get).toSet should be(myData.toSet) 154 | successWriteAcks.size should be(myData.size) 155 | failureWriteAcks should be(Nil) 156 | (successWriteAcks.size + failureWriteAcks.size) should be(myData.size) 157 | 158 | enterBarrier("data-written-2") 159 | 160 | // read from quorum of nodes, which is enough to retrieve all data 161 | val readProbe = TestProbe() 162 | replicator.tell(Get(key, readQuorum), readProbe.ref) 163 | val result = readProbe.expectMsgPF() { case GetSuccess(key, set: ORSet[_], _) ⇒ set } 164 | val survivors = result.elements.size 165 | result.elements should be(expectedData) 166 | 167 | } 168 | 169 | runOn(controller) { 170 | enterBarrier("data-written-2") 171 | } 172 | 173 | enterBarrier("after-test-2") 174 | } 175 | 176 | "replicate values after partition" in { 177 | val key = "C" 178 | runOn(controller) { 179 | sleepBeforePartition() 180 | for (a <- List(n1, n4, n5); b <- List(n2, n3)) 181 | testConductor.blackhole(a, b, Direction.Both).await 182 | sleepDuringPartition() 183 | for (a <- List(n1, n4, n5); b <- List(n2, n3)) 184 | testConductor.passThrough(a, b, Direction.Both).await 185 | enterBarrier("partition-healed-3") 186 | } 187 | 188 | runOn(nodes: _*) { 189 | val writeProbe = TestProbe() 190 | val writeAcks = myData.map { i => 191 | sleepDelay() 192 | replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ + i), writeProbe.ref) 193 | writeProbe.receiveOne(3.seconds) 194 | } 195 | val successWriteAcks = writeAcks.collect { case success: UpdateSuccess => success } 196 | val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure => fail } 197 | successWriteAcks.map(_.request.get).toSet should be(myData.toSet) 198 | successWriteAcks.size should be(myData.size) 199 | failureWriteAcks should be(Nil) 200 | (successWriteAcks.size + failureWriteAcks.size) should be(myData.size) 201 | 202 | enterBarrier("partition-healed-3") 203 | 204 | // eventually all nodes will have the data 205 | within(15.seconds) { 206 | awaitAssert { 207 | val readProbe = TestProbe() 208 | replicator.tell(Get(key, ReadLocal), readProbe.ref) 209 | val result = readProbe.expectMsgPF() { case GetSuccess(key, set: ORSet[_], _) ⇒ set } 210 | result.elements should be(expectedData) 211 | } 212 | } 213 | 214 | } 215 | 216 | enterBarrier("after-test-3") 217 | } 218 | 219 | "write to quorum during 3+2 partition and read from quorum after partition" in { 220 | val key = "D" 221 | val readQuorum = ReadQuorum(timeout) 222 | val writeQuorum = WriteQuorum(timeout) 223 | runOn(controller) { 224 | sleepBeforePartition() 225 | for (a <- List(n1, n4, n5); b <- List(n2, n3)) 226 | testConductor.blackhole(a, b, Direction.Both).await 227 | sleepDuringPartition() 228 | for (a <- List(n1, n4, n5); b <- List(n2, n3)) 229 | testConductor.passThrough(a, b, Direction.Both).await 230 | enterBarrier("partition-healed-4") 231 | } 232 | 233 | runOn(nodes: _*) { 234 | val writeProbe = TestProbe() 235 | val writeAcks = myData.map { i => 236 | sleepDelay() 237 | replicator.tell(Update(key, ORSet(), writeQuorum, Some(i))(_ + i), writeProbe.ref) 238 | writeProbe.receiveOne(timeout + 1.second) 239 | } 240 | val successWriteAcks = writeAcks.collect { case success: UpdateSuccess => success } 241 | val failureWriteAcks = writeAcks.collect { case fail: UpdateFailure => fail } 242 | runOn(n1, n4, n5) { 243 | successWriteAcks.map(_.request.get).toSet should be(myData.toSet) 244 | successWriteAcks.size should be(myData.size) 245 | failureWriteAcks should be(Nil) 246 | } 247 | runOn(n2, n3) { 248 | // without delays all could teoretically have been written before the blackhole 249 | if (delayMillis != 0) 250 | failureWriteAcks should not be (Nil) 251 | } 252 | (successWriteAcks.size + failureWriteAcks.size) should be(myData.size) 253 | 254 | enterBarrier("partition-healed-4") 255 | 256 | // on the 2 node side, read from quorum of nodes is enough to read all writes 257 | runOn(n2, n3) { 258 | val readProbe = TestProbe() 259 | replicator.tell(Get(key, readQuorum), readProbe.ref) 260 | val result = readProbe.expectMsgPF() { case GetSuccess(key, set: ORSet[_], _) ⇒ set } 261 | val survivors = result.elements.size 262 | result.elements should be(expectedData) 263 | } 264 | // but on the 3 node side, read from quorum doesn't mean that we are guaranteed to see 265 | // the writes from the other side, yet 266 | 267 | // eventually all nodes will have the data 268 | within(15.seconds) { 269 | awaitAssert { 270 | val readProbe = TestProbe() 271 | replicator.tell(Get(key, ReadLocal), readProbe.ref) 272 | val result = readProbe.expectMsgPF() { case GetSuccess(key, set: ORSet[_], _) ⇒ set } 273 | result.elements should be(expectedData) 274 | } 275 | } 276 | } 277 | 278 | enterBarrier("after-test-4") 279 | } 280 | 281 | } 282 | 283 | --------------------------------------------------------------------------------