Typed Distributed Data requires untyped Cluster [#25746](https://github.com/akka/akka/issues/25746)
This commit is contained in:
parent
2c145cd3c3
commit
8a44fca087
44 changed files with 656 additions and 368 deletions
|
|
@ -11,6 +11,7 @@ import akka.actor.typed.ActorRef
|
||||||
import akka.actor.typed.ExtensionSetup
|
import akka.actor.typed.ExtensionSetup
|
||||||
import akka.annotation.DoNotInherit
|
import akka.annotation.DoNotInherit
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
import akka.cluster.ddata.SelfUniqueAddress
|
||||||
|
|
||||||
object DistributedData extends ExtensionId[DistributedData] {
|
object DistributedData extends ExtensionId[DistributedData] {
|
||||||
def get(system: ActorSystem[_]): DistributedData = apply(system)
|
def get(system: ActorSystem[_]): DistributedData = apply(system)
|
||||||
|
|
@ -38,6 +39,8 @@ abstract class DistributedData extends Extension {
|
||||||
* `ActorRef` of the [[Replicator]] .
|
* `ActorRef` of the [[Replicator]] .
|
||||||
*/
|
*/
|
||||||
def replicator: ActorRef[Replicator.Command]
|
def replicator: ActorRef[Replicator.Command]
|
||||||
|
|
||||||
|
def selfUniqueAddress: SelfUniqueAddress
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -48,6 +51,9 @@ abstract class DistributedData extends Extension {
|
||||||
override val replicator: ActorRef[Replicator.Command] =
|
override val replicator: ActorRef[Replicator.Command] =
|
||||||
akka.cluster.ddata.typed.scaladsl.DistributedData(system).replicator.narrow[Replicator.Command]
|
akka.cluster.ddata.typed.scaladsl.DistributedData(system).replicator.narrow[Replicator.Command]
|
||||||
|
|
||||||
|
override val selfUniqueAddress: SelfUniqueAddress =
|
||||||
|
akka.cluster.ddata.typed.scaladsl.DistributedData(system).selfUniqueAddress
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
object DistributedDataSetup {
|
object DistributedDataSetup {
|
||||||
|
|
|
||||||
|
|
@ -10,6 +10,9 @@ import akka.actor.typed.ExtensionId
|
||||||
import akka.actor.typed.ActorRef
|
import akka.actor.typed.ActorRef
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.actor.typed.Props
|
import akka.actor.typed.Props
|
||||||
|
import akka.cluster.{ ddata ⇒ dd }
|
||||||
|
import akka.cluster.typed.{ Cluster ⇒ ClusterT }
|
||||||
|
import akka.cluster.ddata.SelfUniqueAddress
|
||||||
|
|
||||||
object DistributedData extends ExtensionId[DistributedData] {
|
object DistributedData extends ExtensionId[DistributedData] {
|
||||||
def get(system: ActorSystem[_]): DistributedData = apply(system)
|
def get(system: ActorSystem[_]): DistributedData = apply(system)
|
||||||
|
|
@ -30,22 +33,30 @@ object DistributedData extends ExtensionId[DistributedData] {
|
||||||
class DistributedData(system: ActorSystem[_]) extends Extension {
|
class DistributedData(system: ActorSystem[_]) extends Extension {
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
|
|
||||||
|
private val settings: ReplicatorSettings = ReplicatorSettings(system)
|
||||||
|
|
||||||
private val untypedSystem = system.toUntyped.asInstanceOf[ExtendedActorSystem]
|
private val untypedSystem = system.toUntyped.asInstanceOf[ExtendedActorSystem]
|
||||||
private val config = system.settings.config.getConfig("akka.cluster.distributed-data")
|
|
||||||
private val settings = ReplicatorSettings(config)
|
implicit val selfUniqueAddress: SelfUniqueAddress = dd.DistributedData(untypedSystem).selfUniqueAddress
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* `ActorRef` of the [[Replicator]] .
|
* `ActorRef` of the [[Replicator]] .
|
||||||
*/
|
*/
|
||||||
val replicator: ActorRef[Replicator.Command] = {
|
val replicator: ActorRef[Replicator.Command] =
|
||||||
val configuredName = config.getString("name")
|
if (isTerminated) {
|
||||||
val name = "typed" + configuredName.take(1).toUpperCase + configuredName.drop(1)
|
system.log.warning("Replicator points to dead letters: Make sure the cluster node is not terminated and has the proper role!")
|
||||||
|
system.deadLetters
|
||||||
|
} else {
|
||||||
|
val underlyingReplicator = dd.DistributedData(untypedSystem).replicator
|
||||||
|
val replicatorBehavior = Replicator.behavior(settings, underlyingReplicator)
|
||||||
|
|
||||||
val underlyingReplicator = akka.cluster.ddata.DistributedData(untypedSystem).replicator
|
system.internalSystemActorOf(replicatorBehavior, ReplicatorSettings.name(system), Props.empty)
|
||||||
val replicatorBehavior = Replicator.behavior(settings, underlyingReplicator)
|
}
|
||||||
|
|
||||||
system.internalSystemActorOf(replicatorBehavior, name, Props.empty)
|
/**
|
||||||
}
|
* Returns true if this member is not tagged with the role configured for the replicas.
|
||||||
|
*/
|
||||||
|
private def isTerminated: Boolean = dd.DistributedData(system.toUntyped).isTerminated
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,7 @@ package akka.cluster.ddata.typed.scaladsl
|
||||||
import akka.cluster.{ ddata ⇒ dd }
|
import akka.cluster.{ ddata ⇒ dd }
|
||||||
import akka.actor.typed.ActorSystem
|
import akka.actor.typed.ActorSystem
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
|
import akka.annotation.InternalApi
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -26,4 +27,11 @@ object ReplicatorSettings {
|
||||||
*/
|
*/
|
||||||
def apply(config: Config): ReplicatorSettings =
|
def apply(config: Config): ReplicatorSettings =
|
||||||
dd.ReplicatorSettings(config)
|
dd.ReplicatorSettings(config)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
* The name of the actor used in DistributedData extensions.
|
||||||
|
*/
|
||||||
|
@InternalApi private[akka] def name(system: ActorSystem[_]): String =
|
||||||
|
dd.ReplicatorSettings.name(system.toUntyped, Some("typed"))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -10,10 +10,10 @@ import akka.cluster.ClusterEvent.{ ClusterDomainEvent, CurrentClusterState }
|
||||||
import akka.cluster._
|
import akka.cluster._
|
||||||
import akka.japi.Util
|
import akka.japi.Util
|
||||||
import akka.actor.typed.{ ActorRef, ActorSystem, Extension, ExtensionId }
|
import akka.actor.typed.{ ActorRef, ActorSystem, Extension, ExtensionId }
|
||||||
import akka.cluster.typed.internal.AdapterClusterImpl
|
|
||||||
import scala.collection.immutable
|
|
||||||
|
|
||||||
import akka.actor.typed.ExtensionSetup
|
import akka.actor.typed.ExtensionSetup
|
||||||
|
import akka.cluster.typed.internal.AdapterClusterImpl
|
||||||
|
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Messages for subscribing to changes in the cluster state
|
* Messages for subscribing to changes in the cluster state
|
||||||
|
|
|
||||||
|
|
@ -7,13 +7,12 @@ package akka.cluster.typed.internal
|
||||||
import akka.actor.typed.Props
|
import akka.actor.typed.Props
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import akka.cluster.ClusterEvent.MemberEvent
|
import akka.cluster.ClusterEvent.MemberEvent
|
||||||
import akka.cluster.{ ClusterEvent, MemberStatus }
|
import akka.cluster.{ ClusterEvent, Member, MemberStatus, UniqueAddress }
|
||||||
import akka.actor.typed.{ ActorRef, ActorSystem, Terminated }
|
import akka.actor.typed.{ ActorRef, ActorSystem, Terminated }
|
||||||
import akka.cluster.typed._
|
import akka.cluster.typed._
|
||||||
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
||||||
import akka.actor.typed.scaladsl.Behaviors
|
import akka.actor.typed.scaladsl.Behaviors
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import akka.cluster.Member
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API:
|
* INTERNAL API:
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,9 @@ import akka.actor.typed.scaladsl.{ ActorContext, Behaviors }
|
||||||
import akka.actor.typed.{ ActorRef, Behavior, Terminated }
|
import akka.actor.typed.{ ActorRef, Behavior, Terminated }
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import akka.cluster.ClusterEvent.MemberRemoved
|
import akka.cluster.ClusterEvent.MemberRemoved
|
||||||
import akka.cluster.ddata.{ DistributedData, ORMultiMap, ORMultiMapKey, Replicator }
|
import akka.cluster.ddata.typed.scaladsl.DistributedData
|
||||||
|
import akka.cluster.{ ddata ⇒ dd }
|
||||||
|
import akka.cluster.ddata.{ ORMultiMap, ORMultiMapKey, Replicator }
|
||||||
import akka.cluster.{ Cluster, ClusterEvent, UniqueAddress }
|
import akka.cluster.{ Cluster, ClusterEvent, UniqueAddress }
|
||||||
import akka.remote.AddressUidExtension
|
import akka.remote.AddressUidExtension
|
||||||
import akka.util.TypedMultiMap
|
import akka.util.TypedMultiMap
|
||||||
|
|
@ -52,13 +54,14 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
final class Setup(ctx: ActorContext[Command]) {
|
final class Setup(ctx: ActorContext[Command]) {
|
||||||
val untypedSystem = ctx.system.toUntyped
|
val untypedSystem = ctx.system.toUntyped
|
||||||
val settings = ClusterReceptionistSettings(ctx.system)
|
val settings = ClusterReceptionistSettings(ctx.system)
|
||||||
val replicator = DistributedData(untypedSystem).replicator
|
val replicator = dd.DistributedData(untypedSystem).replicator
|
||||||
val selfSystemUid = AddressUidExtension(untypedSystem).longAddressUid
|
val selfSystemUid = AddressUidExtension(untypedSystem).longAddressUid
|
||||||
lazy val keepTombstonesFor = cluster.settings.PruneGossipTombstonesAfter match {
|
lazy val keepTombstonesFor = cluster.settings.PruneGossipTombstonesAfter match {
|
||||||
case f: FiniteDuration ⇒ f
|
case f: FiniteDuration ⇒ f
|
||||||
case _ ⇒ throw new IllegalStateException("Cannot actually happen")
|
case _ ⇒ throw new IllegalStateException("Cannot actually happen")
|
||||||
}
|
}
|
||||||
implicit val cluster = Cluster(untypedSystem)
|
val cluster = Cluster(untypedSystem)
|
||||||
|
implicit val selfNodeAddress = DistributedData(ctx.system).selfUniqueAddress
|
||||||
def newTombstoneDeadline() = Deadline(keepTombstonesFor)
|
def newTombstoneDeadline() = Deadline(keepTombstonesFor)
|
||||||
def selfUniqueAddress: UniqueAddress = cluster.selfUniqueAddress
|
def selfUniqueAddress: UniqueAddress = cluster.selfUniqueAddress
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -8,8 +8,8 @@ import akka.actor.typed.ActorRef
|
||||||
import akka.actor.typed.internal.receptionist.AbstractServiceKey
|
import akka.actor.typed.internal.receptionist.AbstractServiceKey
|
||||||
import akka.actor.typed.receptionist.ServiceKey
|
import akka.actor.typed.receptionist.ServiceKey
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import akka.cluster.{ Cluster, UniqueAddress }
|
import akka.cluster.UniqueAddress
|
||||||
import akka.cluster.ddata.{ ORMultiMap, ORMultiMapKey }
|
import akka.cluster.ddata.{ ORMultiMap, ORMultiMapKey, SelfUniqueAddress }
|
||||||
import akka.cluster.typed.internal.receptionist.ClusterReceptionist.{ DDataKey, EmptyORMultiMap, Entry }
|
import akka.cluster.typed.internal.receptionist.ClusterReceptionist.{ DDataKey, EmptyORMultiMap, Entry }
|
||||||
|
|
||||||
import scala.concurrent.duration.Deadline
|
import scala.concurrent.duration.Deadline
|
||||||
|
|
@ -109,13 +109,13 @@ import scala.concurrent.duration.Deadline
|
||||||
def entriesFor(key: AbstractServiceKey): Set[Entry] =
|
def entriesFor(key: AbstractServiceKey): Set[Entry] =
|
||||||
entries.getOrElse(key.asServiceKey, Set.empty[Entry])
|
entries.getOrElse(key.asServiceKey, Set.empty[Entry])
|
||||||
|
|
||||||
def addBinding[T](key: ServiceKey[T], value: Entry)(implicit cluster: Cluster): ServiceRegistry =
|
def addBinding[T](key: ServiceKey[T], value: Entry)(implicit node: SelfUniqueAddress): ServiceRegistry =
|
||||||
copy(entries = entries.addBinding(key, value))
|
copy(entries = entries.addBinding(node, key, value))
|
||||||
|
|
||||||
def removeBinding[T](key: ServiceKey[T], value: Entry)(implicit cluster: Cluster): ServiceRegistry =
|
def removeBinding[T](key: ServiceKey[T], value: Entry)(implicit node: SelfUniqueAddress): ServiceRegistry =
|
||||||
copy(entries = entries.removeBinding(key, value))
|
copy(entries = entries.removeBinding(node, key, value))
|
||||||
|
|
||||||
def removeAll(entries: Map[AbstractServiceKey, Set[Entry]])(implicit cluster: Cluster): ServiceRegistry = {
|
def removeAll(entries: Map[AbstractServiceKey, Set[Entry]])(implicit node: SelfUniqueAddress): ServiceRegistry = {
|
||||||
entries.foldLeft(this) {
|
entries.foldLeft(this) {
|
||||||
case (acc, (key, entries)) ⇒
|
case (acc, (key, entries)) ⇒
|
||||||
entries.foldLeft(acc) {
|
entries.foldLeft(acc) {
|
||||||
|
|
|
||||||
|
|
@ -4,6 +4,7 @@
|
||||||
|
|
||||||
package akka.cluster.ddata.typed.javadsl;
|
package akka.cluster.ddata.typed.javadsl;
|
||||||
|
|
||||||
|
import akka.cluster.ddata.*;
|
||||||
import com.typesafe.config.Config;
|
import com.typesafe.config.Config;
|
||||||
import com.typesafe.config.ConfigFactory;
|
import com.typesafe.config.ConfigFactory;
|
||||||
import org.junit.ClassRule;
|
import org.junit.ClassRule;
|
||||||
|
|
@ -13,11 +14,6 @@ import org.scalatest.junit.JUnitSuite;
|
||||||
// #sample
|
// #sample
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import akka.actor.typed.ActorSystem;
|
import akka.actor.typed.ActorSystem;
|
||||||
import akka.cluster.Cluster;
|
|
||||||
import akka.cluster.ddata.GCounter;
|
|
||||||
import akka.cluster.ddata.GCounterKey;
|
|
||||||
import akka.cluster.ddata.Key;
|
|
||||||
import akka.cluster.ddata.ReplicatedData;
|
|
||||||
import akka.testkit.AkkaJUnitActorSystemResource;
|
import akka.testkit.AkkaJUnitActorSystemResource;
|
||||||
import akka.testkit.javadsl.TestKit;
|
import akka.testkit.javadsl.TestKit;
|
||||||
import akka.actor.typed.ActorRef;
|
import akka.actor.typed.ActorRef;
|
||||||
|
|
@ -86,14 +82,14 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
|
|
||||||
static class Counter extends AbstractBehavior<ClientCommand> {
|
static class Counter extends AbstractBehavior<ClientCommand> {
|
||||||
private final ActorRef<Replicator.Command> replicator;
|
private final ActorRef<Replicator.Command> replicator;
|
||||||
private final Cluster node;
|
private final SelfUniqueAddress node;
|
||||||
final ActorRef<Replicator.UpdateResponse<GCounter>> updateResponseAdapter;
|
final ActorRef<Replicator.UpdateResponse<GCounter>> updateResponseAdapter;
|
||||||
final ActorRef<Replicator.GetResponse<GCounter>> getResponseAdapter;
|
final ActorRef<Replicator.GetResponse<GCounter>> getResponseAdapter;
|
||||||
final ActorRef<Replicator.Changed<GCounter>> changedAdapter;
|
final ActorRef<Replicator.Changed<GCounter>> changedAdapter;
|
||||||
|
|
||||||
private int cachedValue = 0;
|
private int cachedValue = 0;
|
||||||
|
|
||||||
Counter(ActorRef<Command> replicator, Cluster node, ActorContext<ClientCommand> ctx) {
|
Counter(ActorRef<Command> replicator, SelfUniqueAddress node, ActorContext<ClientCommand> ctx) {
|
||||||
this.replicator = replicator;
|
this.replicator = replicator;
|
||||||
this.node = node;
|
this.node = node;
|
||||||
|
|
||||||
|
|
@ -116,9 +112,7 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
|
|
||||||
public static Behavior<ClientCommand> create() {
|
public static Behavior<ClientCommand> create() {
|
||||||
return Behaviors.setup((ctx) -> {
|
return Behaviors.setup((ctx) -> {
|
||||||
// The distributed data types still need the implicit untyped Cluster.
|
SelfUniqueAddress node = DistributedData.get(ctx.getSystem()).selfUniqueAddress();
|
||||||
// We will look into another solution for that.
|
|
||||||
Cluster node = Cluster.get(Adapter.toUntyped(ctx.getSystem()));
|
|
||||||
ActorRef<Replicator.Command> replicator = DistributedData.get(ctx.getSystem()).replicator();
|
ActorRef<Replicator.Command> replicator = DistributedData.get(ctx.getSystem()).replicator();
|
||||||
|
|
||||||
return new Counter(replicator, node, ctx);
|
return new Counter(replicator, node, ctx);
|
||||||
|
|
@ -127,7 +121,7 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
|
|
||||||
// #sample
|
// #sample
|
||||||
// omitted from sample, needed for tests, factory above is for the docs sample
|
// omitted from sample, needed for tests, factory above is for the docs sample
|
||||||
public static Behavior<ClientCommand> create(ActorRef<Command> replicator, Cluster node) {
|
public static Behavior<ClientCommand> create(ActorRef<Command> replicator, SelfUniqueAddress node) {
|
||||||
return Behaviors.setup(ctx -> new Counter(replicator, node, ctx));
|
return Behaviors.setup(ctx -> new Counter(replicator, node, ctx));
|
||||||
}
|
}
|
||||||
// #sample
|
// #sample
|
||||||
|
|
@ -213,7 +207,7 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
ActorRef<Replicator.Command> replicator =
|
ActorRef<Replicator.Command> replicator =
|
||||||
Adapter.spawnAnonymous(system, Replicator.behavior(settings));
|
Adapter.spawnAnonymous(system, Replicator.behavior(settings));
|
||||||
ActorRef<ClientCommand> client =
|
ActorRef<ClientCommand> client =
|
||||||
Adapter.spawnAnonymous(system, Counter.create(replicator, Cluster.get(system)));
|
Adapter.spawnAnonymous(system, Counter.create(replicator, DistributedData.get(typedSystem()).selfUniqueAddress()));
|
||||||
|
|
||||||
client.tell(Increment.INSTANCE);
|
client.tell(Increment.INSTANCE);
|
||||||
client.tell(new GetValue(Adapter.toTyped(probe.getRef())));
|
client.tell(new GetValue(Adapter.toTyped(probe.getRef())));
|
||||||
|
|
@ -227,7 +221,7 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
ActorRef<Replicator.Command> replicator =
|
ActorRef<Replicator.Command> replicator =
|
||||||
Adapter.spawnAnonymous(system, Replicator.behavior(settings));
|
Adapter.spawnAnonymous(system, Replicator.behavior(settings));
|
||||||
ActorRef<ClientCommand> client =
|
ActorRef<ClientCommand> client =
|
||||||
Adapter.spawnAnonymous(system, Counter.create(replicator, Cluster.get(system)));
|
Adapter.spawnAnonymous(system, Counter.create(replicator, DistributedData.get(typedSystem()).selfUniqueAddress()));
|
||||||
|
|
||||||
client.tell(Increment.INSTANCE);
|
client.tell(Increment.INSTANCE);
|
||||||
client.tell(Increment.INSTANCE);
|
client.tell(Increment.INSTANCE);
|
||||||
|
|
|
||||||
|
|
@ -6,13 +6,13 @@ package akka.cluster.ddata.typed.scaladsl
|
||||||
|
|
||||||
import org.scalatest.WordSpecLike
|
import org.scalatest.WordSpecLike
|
||||||
import akka.actor.testkit.typed.TestKitSettings
|
import akka.actor.testkit.typed.TestKitSettings
|
||||||
|
import akka.cluster.ddata.SelfUniqueAddress
|
||||||
|
|
||||||
// #sample
|
// #sample
|
||||||
import akka.actor.Scheduler
|
import akka.actor.Scheduler
|
||||||
import akka.actor.typed.{ ActorRef, Behavior }
|
import akka.actor.typed.{ ActorRef, Behavior }
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
import akka.actor.typed.scaladsl.AskPattern._
|
||||||
import akka.actor.typed.scaladsl.Behaviors
|
import akka.actor.typed.scaladsl.Behaviors
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.ddata.typed.scaladsl.Replicator._
|
import akka.cluster.ddata.typed.scaladsl.Replicator._
|
||||||
import akka.cluster.ddata.{ GCounter, GCounterKey }
|
import akka.cluster.ddata.{ GCounter, GCounterKey }
|
||||||
|
|
@ -47,12 +47,9 @@ object ReplicatorSpec {
|
||||||
|
|
||||||
val Key = GCounterKey("counter")
|
val Key = GCounterKey("counter")
|
||||||
|
|
||||||
def client(replicator: ActorRef[Replicator.Command])(implicit cluster: Cluster): Behavior[ClientCommand] =
|
def client(replicator: ActorRef[Replicator.Command])(implicit node: SelfUniqueAddress): Behavior[ClientCommand] =
|
||||||
Behaviors.setup[ClientCommand] { ctx ⇒
|
Behaviors.setup[ClientCommand] { ctx ⇒
|
||||||
|
|
||||||
// The distributed data types still need the implicit untyped Cluster.
|
|
||||||
// We will look into another solution for that.
|
|
||||||
|
|
||||||
val updateResponseAdapter: ActorRef[Replicator.UpdateResponse[GCounter]] =
|
val updateResponseAdapter: ActorRef[Replicator.UpdateResponse[GCounter]] =
|
||||||
ctx.messageAdapter(InternalUpdateResponse.apply)
|
ctx.messageAdapter(InternalUpdateResponse.apply)
|
||||||
|
|
||||||
|
|
@ -68,7 +65,7 @@ object ReplicatorSpec {
|
||||||
Behaviors.receive[ClientCommand] { (ctx, msg) ⇒
|
Behaviors.receive[ClientCommand] { (ctx, msg) ⇒
|
||||||
msg match {
|
msg match {
|
||||||
case Increment ⇒
|
case Increment ⇒
|
||||||
replicator ! Replicator.Update(Key, GCounter.empty, Replicator.WriteLocal, updateResponseAdapter)(_ + 1)
|
replicator ! Replicator.Update(Key, GCounter.empty, Replicator.WriteLocal, updateResponseAdapter)(_ :+ 1)
|
||||||
Behaviors.same
|
Behaviors.same
|
||||||
|
|
||||||
case GetValue(replyTo) ⇒
|
case GetValue(replyTo) ⇒
|
||||||
|
|
@ -131,7 +128,7 @@ class ReplicatorSpec extends ScalaTestWithActorTestKit(ReplicatorSpec.config) wi
|
||||||
|
|
||||||
implicit val testSettings = TestKitSettings(system)
|
implicit val testSettings = TestKitSettings(system)
|
||||||
val settings = ReplicatorSettings(system)
|
val settings = ReplicatorSettings(system)
|
||||||
implicit val cluster = Cluster(system.toUntyped)
|
implicit val selfNodeAddress = DistributedData(system).selfUniqueAddress
|
||||||
|
|
||||||
"Replicator" must {
|
"Replicator" must {
|
||||||
|
|
||||||
|
|
@ -172,6 +169,10 @@ class ReplicatorSpec extends ScalaTestWithActorTestKit(ReplicatorSpec.config) wi
|
||||||
c ! GetValue(probe.ref)
|
c ! GetValue(probe.ref)
|
||||||
probe.expectMessage(1)
|
probe.expectMessage(1)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
"have the prefixed replicator name" in {
|
||||||
|
ReplicatorSettings.name(system) should ===("typedDdataReplicator")
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -10,7 +10,7 @@ import akka.actor.ExtendedActorSystem
|
||||||
import akka.actor.Extension
|
import akka.actor.Extension
|
||||||
import akka.actor.ExtensionId
|
import akka.actor.ExtensionId
|
||||||
import akka.actor.ExtensionIdProvider
|
import akka.actor.ExtensionIdProvider
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.{ Cluster, UniqueAddress }
|
||||||
|
|
||||||
object DistributedData extends ExtensionId[DistributedData] with ExtensionIdProvider {
|
object DistributedData extends ExtensionId[DistributedData] with ExtensionIdProvider {
|
||||||
override def get(system: ActorSystem): DistributedData = super.get(system)
|
override def get(system: ActorSystem): DistributedData = super.get(system)
|
||||||
|
|
@ -28,14 +28,9 @@ object DistributedData extends ExtensionId[DistributedData] with ExtensionIdProv
|
||||||
*/
|
*/
|
||||||
class DistributedData(system: ExtendedActorSystem) extends Extension {
|
class DistributedData(system: ExtendedActorSystem) extends Extension {
|
||||||
|
|
||||||
private val config = system.settings.config.getConfig("akka.cluster.distributed-data")
|
private val settings = ReplicatorSettings(system)
|
||||||
private val settings = ReplicatorSettings(config)
|
|
||||||
|
|
||||||
/**
|
implicit val selfUniqueAddress: SelfUniqueAddress = SelfUniqueAddress(Cluster(system).selfUniqueAddress)
|
||||||
* Returns true if this member is not tagged with the role configured for the
|
|
||||||
* replicas.
|
|
||||||
*/
|
|
||||||
def isTerminated: Boolean = Cluster(system).isTerminated || !settings.roles.subsetOf(Cluster(system).selfRoles)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* `ActorRef` of the [[Replicator]] .
|
* `ActorRef` of the [[Replicator]] .
|
||||||
|
|
@ -45,7 +40,20 @@ class DistributedData(system: ExtendedActorSystem) extends Extension {
|
||||||
system.log.warning("Replicator points to dead letters: Make sure the cluster node is not terminated and has the proper role!")
|
system.log.warning("Replicator points to dead letters: Make sure the cluster node is not terminated and has the proper role!")
|
||||||
system.deadLetters
|
system.deadLetters
|
||||||
} else {
|
} else {
|
||||||
val name = config.getString("name")
|
system.systemActorOf(Replicator.props(settings), ReplicatorSettings.name(system, None))
|
||||||
system.systemActorOf(Replicator.props(settings), name)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if this member is not tagged with the role configured for the
|
||||||
|
* replicas.
|
||||||
|
*/
|
||||||
|
def isTerminated: Boolean =
|
||||||
|
Cluster(system).isTerminated || !settings.roles.subsetOf(Cluster(system).selfRoles)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Cluster non-specific (typed vs untyped) wrapper for [[akka.cluster.UniqueAddress]].
|
||||||
|
*/
|
||||||
|
@SerialVersionUID(1L)
|
||||||
|
final case class SelfUniqueAddress(uniqueAddress: UniqueAddress)
|
||||||
|
|
|
||||||
|
|
@ -65,14 +65,19 @@ final class GCounter private[akka] (
|
||||||
* Increment the counter with the delta `n` specified.
|
* Increment the counter with the delta `n` specified.
|
||||||
* The delta must be zero or positive.
|
* The delta must be zero or positive.
|
||||||
*/
|
*/
|
||||||
def +(n: Long)(implicit node: Cluster): GCounter = increment(node, n)
|
def :+(n: Long)(implicit node: SelfUniqueAddress): GCounter = increment(node.uniqueAddress, n)
|
||||||
|
|
||||||
|
@deprecated("Use `:+` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def +(n: Long)(implicit node: Cluster): GCounter = increment(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Increment the counter with the delta `n` specified.
|
* Increment the counter with the delta `n` specified.
|
||||||
* The delta `n` must be zero or positive.
|
* The delta `n` must be zero or positive.
|
||||||
*/
|
*/
|
||||||
def increment(node: Cluster, n: Long = 1): GCounter =
|
def increment(node: SelfUniqueAddress, n: Long): GCounter = increment(node.uniqueAddress, n)
|
||||||
increment(node.selfUniqueAddress, n)
|
|
||||||
|
@deprecated("Use `increment` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def increment(node: Cluster, n: Long = 1): GCounter = increment(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
|
||||||
|
|
@ -4,9 +4,9 @@
|
||||||
|
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
|
import akka.annotation.InternalApi
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import akka.annotation.InternalApi
|
|
||||||
import akka.cluster.ddata.ORMap.ZeroTag
|
import akka.cluster.ddata.ORMap.ZeroTag
|
||||||
|
|
||||||
object LWWMap {
|
object LWWMap {
|
||||||
|
|
@ -87,6 +87,12 @@ final class LWWMap[A, B] private[akka] (
|
||||||
/**
|
/**
|
||||||
* Adds an entry to the map
|
* Adds an entry to the map
|
||||||
*/
|
*/
|
||||||
|
def :+(entry: (A, B))(implicit node: SelfUniqueAddress): LWWMap[A, B] = {
|
||||||
|
val (key, value) = entry
|
||||||
|
put(node, key, value)
|
||||||
|
}
|
||||||
|
|
||||||
|
@deprecated("Use `:+` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def +(entry: (A, B))(implicit node: Cluster): LWWMap[A, B] = {
|
def +(entry: (A, B))(implicit node: Cluster): LWWMap[A, B] = {
|
||||||
val (key, value) = entry
|
val (key, value) = entry
|
||||||
put(node, key, value)
|
put(node, key, value)
|
||||||
|
|
@ -95,8 +101,12 @@ final class LWWMap[A, B] private[akka] (
|
||||||
/**
|
/**
|
||||||
* Adds an entry to the map
|
* Adds an entry to the map
|
||||||
*/
|
*/
|
||||||
|
def put(node: SelfUniqueAddress, key: A, value: B): LWWMap[A, B] =
|
||||||
|
put(node.uniqueAddress, key, value, defaultClock[B])
|
||||||
|
|
||||||
|
@deprecated("Use `put` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def put(node: Cluster, key: A, value: B): LWWMap[A, B] =
|
def put(node: Cluster, key: A, value: B): LWWMap[A, B] =
|
||||||
put(node, key, value, defaultClock[B])
|
put(node.selfUniqueAddress, key, value, defaultClock[B])
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds an entry to the map.
|
* Adds an entry to the map.
|
||||||
|
|
@ -106,6 +116,10 @@ final class LWWMap[A, B] private[akka] (
|
||||||
* increasing version number from a database record that is used for optimistic
|
* increasing version number from a database record that is used for optimistic
|
||||||
* concurrency control.
|
* concurrency control.
|
||||||
*/
|
*/
|
||||||
|
def put(node: SelfUniqueAddress, key: A, value: B, clock: Clock[B]): LWWMap[A, B] =
|
||||||
|
put(node.uniqueAddress, key, value, clock)
|
||||||
|
|
||||||
|
@deprecated("Use `put` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def put(node: Cluster, key: A, value: B, clock: Clock[B]): LWWMap[A, B] =
|
def put(node: Cluster, key: A, value: B, clock: Clock[B]): LWWMap[A, B] =
|
||||||
put(node.selfUniqueAddress, key, value, clock)
|
put(node.selfUniqueAddress, key, value, clock)
|
||||||
|
|
||||||
|
|
@ -117,8 +131,9 @@ final class LWWMap[A, B] private[akka] (
|
||||||
* increasing version number from a database record that is used for optimistic
|
* increasing version number from a database record that is used for optimistic
|
||||||
* concurrency control.
|
* concurrency control.
|
||||||
*/
|
*/
|
||||||
|
@deprecated("Use `put` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def put(key: A, value: B)(implicit node: Cluster, clock: Clock[B] = defaultClock[B]): LWWMap[A, B] =
|
def put(key: A, value: B)(implicit node: Cluster, clock: Clock[B] = defaultClock[B]): LWWMap[A, B] =
|
||||||
put(node, key, value, clock)
|
put(node.selfUniqueAddress, key, value, clock)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -136,6 +151,7 @@ final class LWWMap[A, B] private[akka] (
|
||||||
* Note that if there is a conflicting update on another node the entry will
|
* Note that if there is a conflicting update on another node the entry will
|
||||||
* not be removed after merge.
|
* not be removed after merge.
|
||||||
*/
|
*/
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def -(key: A)(implicit node: Cluster): LWWMap[A, B] = remove(node, key)
|
def -(key: A)(implicit node: Cluster): LWWMap[A, B] = remove(node, key)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -143,6 +159,10 @@ final class LWWMap[A, B] private[akka] (
|
||||||
* Note that if there is a conflicting update on another node the entry will
|
* Note that if there is a conflicting update on another node the entry will
|
||||||
* not be removed after merge.
|
* not be removed after merge.
|
||||||
*/
|
*/
|
||||||
|
def remove(node: SelfUniqueAddress, key: A): LWWMap[A, B] =
|
||||||
|
remove(node.uniqueAddress, key)
|
||||||
|
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def remove(node: Cluster, key: A): LWWMap[A, B] =
|
def remove(node: Cluster, key: A): LWWMap[A, B] =
|
||||||
remove(node.selfUniqueAddress, key)
|
remove(node.selfUniqueAddress, key)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,10 +4,10 @@
|
||||||
|
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
|
import akka.annotation.InternalApi
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import akka.util.HashCode
|
import akka.util.HashCode
|
||||||
import akka.annotation.InternalApi
|
|
||||||
|
|
||||||
object LWWRegister {
|
object LWWRegister {
|
||||||
|
|
||||||
|
|
@ -48,20 +48,48 @@ object LWWRegister {
|
||||||
@InternalApi private[akka] def apply[A](node: UniqueAddress, initialValue: A, clock: Clock[A]): LWWRegister[A] =
|
@InternalApi private[akka] def apply[A](node: UniqueAddress, initialValue: A, clock: Clock[A]): LWWRegister[A] =
|
||||||
new LWWRegister(node, initialValue, clock(0L, initialValue))
|
new LWWRegister(node, initialValue, clock(0L, initialValue))
|
||||||
|
|
||||||
|
def apply[A](node: SelfUniqueAddress, initialValue: A): LWWRegister[A] =
|
||||||
|
apply(node.uniqueAddress, initialValue, defaultClock[A])
|
||||||
|
|
||||||
|
def apply[A](node: SelfUniqueAddress, initialValue: A, clock: Clock[A]): LWWRegister[A] =
|
||||||
|
apply(node.uniqueAddress, initialValue, clock)
|
||||||
|
|
||||||
|
@deprecated("Use `apply` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def apply[A](initialValue: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
|
def apply[A](initialValue: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
|
||||||
apply(node.selfUniqueAddress, initialValue, clock)
|
apply(node.selfUniqueAddress, initialValue, clock)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Scala API
|
||||||
|
* Creates a `LWWRegister` with implicits, given deprecated `apply` functions using Cluster constrain overloading.
|
||||||
|
*/
|
||||||
|
def create[A](initialValue: A)(implicit node: SelfUniqueAddress, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
|
||||||
|
apply(node.uniqueAddress, initialValue, clock)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API
|
* Java API
|
||||||
*/
|
*/
|
||||||
|
@deprecated("Use `create` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def create[A](node: Cluster, initialValue: A): LWWRegister[A] =
|
def create[A](node: Cluster, initialValue: A): LWWRegister[A] =
|
||||||
apply(initialValue)(node)
|
apply(initialValue)(node)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API
|
* Java API
|
||||||
*/
|
*/
|
||||||
|
@deprecated("Use `create` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def create[A](node: Cluster, initialValue: A, clock: Clock[A]): LWWRegister[A] =
|
def create[A](node: Cluster, initialValue: A, clock: Clock[A]): LWWRegister[A] =
|
||||||
apply(initialValue)(node, clock)
|
apply(node.selfUniqueAddress, initialValue, clock)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API
|
||||||
|
*/
|
||||||
|
def create[A](node: SelfUniqueAddress, initialValue: A, clock: Clock[A]): LWWRegister[A] =
|
||||||
|
apply(node.uniqueAddress, initialValue, clock)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API
|
||||||
|
*/
|
||||||
|
def create[A](node: SelfUniqueAddress, initialValue: A): LWWRegister[A] =
|
||||||
|
apply(node.uniqueAddress, initialValue, defaultClock[A])
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Extract the [[LWWRegister#value]].
|
* Extract the [[LWWRegister#value]].
|
||||||
|
|
@ -122,13 +150,13 @@ final class LWWRegister[A] private[akka] (
|
||||||
* increasing version number from a database record that is used for optimistic
|
* increasing version number from a database record that is used for optimistic
|
||||||
* concurrency control.
|
* concurrency control.
|
||||||
*/
|
*/
|
||||||
def withValue(value: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
|
def withValue(node: SelfUniqueAddress, value: A, clock: Clock[A]): LWWRegister[A] =
|
||||||
withValue(node, value, clock)
|
withValue(node.uniqueAddress, value, clock)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Change the value of the register.
|
* Change the value of the register.
|
||||||
*/
|
*/
|
||||||
def withValue(node: Cluster, value: A): LWWRegister[A] =
|
def withValue(node: SelfUniqueAddress, value: A): LWWRegister[A] =
|
||||||
withValue(node, value, defaultClock[A])
|
withValue(node, value, defaultClock[A])
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -139,6 +167,18 @@ final class LWWRegister[A] private[akka] (
|
||||||
* increasing version number from a database record that is used for optimistic
|
* increasing version number from a database record that is used for optimistic
|
||||||
* concurrency control.
|
* concurrency control.
|
||||||
*/
|
*/
|
||||||
|
def withValueOf(value: A)(implicit node: SelfUniqueAddress, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
|
||||||
|
withValue(node, value, clock)
|
||||||
|
|
||||||
|
@deprecated("Use `withValueOf` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def withValue(value: A)(implicit node: Cluster, clock: Clock[A] = defaultClock[A]): LWWRegister[A] =
|
||||||
|
withValue(node, value, clock)
|
||||||
|
|
||||||
|
@deprecated("Use `withValue` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def withValue(node: Cluster, value: A): LWWRegister[A] =
|
||||||
|
withValue(node, value, defaultClock[A])
|
||||||
|
|
||||||
|
@deprecated("Use `withValue` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def withValue(node: Cluster, value: A, clock: Clock[A]): LWWRegister[A] =
|
def withValue(node: Cluster, value: A, clock: Clock[A]): LWWRegister[A] =
|
||||||
withValue(node.selfUniqueAddress, value, clock)
|
withValue(node.selfUniqueAddress, value, clock)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -199,9 +199,15 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
* Adds an entry to the map
|
* Adds an entry to the map
|
||||||
* @see [[#put]]
|
* @see [[#put]]
|
||||||
*/
|
*/
|
||||||
|
def :+(entry: (A, B))(implicit node: SelfUniqueAddress): ORMap[A, B] = {
|
||||||
|
val (key, value) = entry
|
||||||
|
put(node.uniqueAddress, key, value)
|
||||||
|
}
|
||||||
|
|
||||||
|
@deprecated("Use `:+` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def +(entry: (A, B))(implicit node: Cluster): ORMap[A, B] = {
|
def +(entry: (A, B))(implicit node: Cluster): ORMap[A, B] = {
|
||||||
val (key, value) = entry
|
val (key, value) = entry
|
||||||
put(node, key, value)
|
put(node.selfUniqueAddress, key, value)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -217,6 +223,9 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
* value, because important history can be lost when replacing the `ORSet` and
|
* value, because important history can be lost when replacing the `ORSet` and
|
||||||
* undesired effects of merging will occur. Use [[ORMultiMap]] or [[#updated]] instead.
|
* undesired effects of merging will occur. Use [[ORMultiMap]] or [[#updated]] instead.
|
||||||
*/
|
*/
|
||||||
|
def put(node: SelfUniqueAddress, key: A, value: B): ORMap[A, B] = put(node.uniqueAddress, key, value)
|
||||||
|
|
||||||
|
@deprecated("Use `put` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def put(node: Cluster, key: A, value: B): ORMap[A, B] = put(node.selfUniqueAddress, key, value)
|
def put(node: Cluster, key: A, value: B): ORMap[A, B] = put(node.selfUniqueAddress, key, value)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -241,6 +250,10 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
* If there is no current value for the `key` the `initial` value will be
|
* If there is no current value for the `key` the `initial` value will be
|
||||||
* passed to the `modify` function.
|
* passed to the `modify` function.
|
||||||
*/
|
*/
|
||||||
|
def updated(node: SelfUniqueAddress, key: A, initial: B)(modify: B ⇒ B): ORMap[A, B] =
|
||||||
|
updated(node.uniqueAddress, key, initial)(modify)
|
||||||
|
|
||||||
|
@deprecated("Use `updated` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def updated(node: Cluster, key: A, initial: B)(modify: B ⇒ B): ORMap[A, B] =
|
def updated(node: Cluster, key: A, initial: B)(modify: B ⇒ B): ORMap[A, B] =
|
||||||
updated(node.selfUniqueAddress, key, initial)(modify)
|
updated(node.selfUniqueAddress, key, initial)(modify)
|
||||||
|
|
||||||
|
|
@ -251,9 +264,9 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
* passed to the `modify` function.
|
* passed to the `modify` function.
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
@deprecated("use update for the Java API as updated is ambiguous with the Scala API", "2.5.19")
|
@deprecated("use update for the Java API as updated is ambiguous with the Scala API", "2.5.20")
|
||||||
def updated(node: Cluster, key: A, initial: B, modify: java.util.function.Function[B, B]): ORMap[A, B] =
|
def updated(node: Cluster, key: A, initial: B, modify: java.util.function.Function[B, B]): ORMap[A, B] =
|
||||||
updated(node, key, initial)(value ⇒ modify.apply(value))
|
updated(node.selfUniqueAddress, key, initial)(value ⇒ modify.apply(value))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Replace a value by applying the `modify` function on the existing value.
|
* Java API: Replace a value by applying the `modify` function on the existing value.
|
||||||
|
|
@ -261,6 +274,11 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
* If there is no current value for the `key` the `initial` value will be
|
* If there is no current value for the `key` the `initial` value will be
|
||||||
* passed to the `modify` function.
|
* passed to the `modify` function.
|
||||||
*/
|
*/
|
||||||
|
def update(node: SelfUniqueAddress, key: A, initial: B, modify: java.util.function.Function[B, B]): ORMap[A, B] =
|
||||||
|
updated(node.uniqueAddress, key, initial)(value ⇒ modify.apply(value))
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
@deprecated("Use `update` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def update(node: Cluster, key: A, initial: B, modify: java.util.function.Function[B, B]): ORMap[A, B] =
|
def update(node: Cluster, key: A, initial: B, modify: java.util.function.Function[B, B]): ORMap[A, B] =
|
||||||
updated(node, key, initial)(value ⇒ modify.apply(value))
|
updated(node, key, initial)(value ⇒ modify.apply(value))
|
||||||
|
|
||||||
|
|
@ -295,17 +313,25 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Scala API
|
||||||
* Removes an entry from the map.
|
* Removes an entry from the map.
|
||||||
* Note that if there is a conflicting update on another node the entry will
|
* Note that if there is a conflicting update on another node the entry will
|
||||||
* not be removed after merge.
|
* not be removed after merge.
|
||||||
*/
|
*/
|
||||||
def -(key: A)(implicit node: Cluster): ORMap[A, B] = remove(node, key)
|
def remove(key: A)(implicit node: SelfUniqueAddress): ORMap[A, B] = remove(node.uniqueAddress, key)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Java API
|
||||||
* Removes an entry from the map.
|
* Removes an entry from the map.
|
||||||
* Note that if there is a conflicting update on another node the entry will
|
* Note that if there is a conflicting update on another node the entry will
|
||||||
* not be removed after merge.
|
* not be removed after merge.
|
||||||
*/
|
*/
|
||||||
|
def remove(node: SelfUniqueAddress, key: A): ORMap[A, B] = remove(node.uniqueAddress, key)
|
||||||
|
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def -(key: A)(implicit node: Cluster): ORMap[A, B] = remove(node.selfUniqueAddress, key)
|
||||||
|
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def remove(node: Cluster, key: A): ORMap[A, B] = remove(node.selfUniqueAddress, key)
|
def remove(node: Cluster, key: A): ORMap[A, B] = remove(node.selfUniqueAddress, key)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -125,18 +125,28 @@ final class ORMultiMap[A, B] private[akka] (
|
||||||
def size: Int = underlying.keys.elements.size
|
def size: Int = underlying.keys.elements.size
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Convenience for put. Requires an implicit Cluster.
|
* Convenience for put. Requires an implicit SelfUniqueAddress.
|
||||||
* @see [[#put]]
|
* @see [[#put]]
|
||||||
*/
|
*/
|
||||||
|
def :+(entry: (A, Set[B]))(implicit node: SelfUniqueAddress): ORMultiMap[A, B] = {
|
||||||
|
val (key, value) = entry
|
||||||
|
put(node.uniqueAddress, key, value)
|
||||||
|
}
|
||||||
|
|
||||||
|
@deprecated("Use `:+` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def +(entry: (A, Set[B]))(implicit node: Cluster): ORMultiMap[A, B] = {
|
def +(entry: (A, Set[B]))(implicit node: Cluster): ORMultiMap[A, B] = {
|
||||||
val (key, value) = entry
|
val (key, value) = entry
|
||||||
put(node, key, value)
|
put(node.selfUniqueAddress, key, value)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Associate an entire set with the key while retaining the history of the previous
|
* Scala API: Associate an entire set with the key while retaining the history of the previous
|
||||||
* replicated data set.
|
* replicated data set.
|
||||||
*/
|
*/
|
||||||
|
def put(node: SelfUniqueAddress, key: A, value: Set[B]): ORMultiMap[A, B] =
|
||||||
|
put(node.uniqueAddress, key, value)
|
||||||
|
|
||||||
|
@deprecated("Use `put` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def put(node: Cluster, key: A, value: Set[B]): ORMultiMap[A, B] =
|
def put(node: Cluster, key: A, value: Set[B]): ORMultiMap[A, B] =
|
||||||
put(node.selfUniqueAddress, key, value)
|
put(node.selfUniqueAddress, key, value)
|
||||||
|
|
||||||
|
|
@ -144,9 +154,16 @@ final class ORMultiMap[A, B] private[akka] (
|
||||||
* Java API: Associate an entire set with the key while retaining the history of the previous
|
* Java API: Associate an entire set with the key while retaining the history of the previous
|
||||||
* replicated data set.
|
* replicated data set.
|
||||||
*/
|
*/
|
||||||
|
def put(node: SelfUniqueAddress, key: A, value: java.util.Set[B]): ORMultiMap[A, B] = {
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
put(node.uniqueAddress, key, value.asScala.toSet)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
@deprecated("Use `put` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def put(node: Cluster, key: A, value: java.util.Set[B]): ORMultiMap[A, B] = {
|
def put(node: Cluster, key: A, value: java.util.Set[B]): ORMultiMap[A, B] = {
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
put(node, key, value.asScala.toSet)
|
put(node.selfUniqueAddress, key, value.asScala.toSet)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -159,18 +176,27 @@ final class ORMultiMap[A, B] private[akka] (
|
||||||
new ORMultiMap(newUnderlying, withValueDeltas)
|
new ORMultiMap(newUnderlying, withValueDeltas)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Scala API
|
||||||
|
* Remove an entire set associated with the key.
|
||||||
|
*/
|
||||||
|
def remove(key: A)(implicit node: SelfUniqueAddress): ORMultiMap[A, B] = remove(node.uniqueAddress, key)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Convenience for remove. Requires an implicit Cluster.
|
* Convenience for remove. Requires an implicit Cluster.
|
||||||
* @see [[#remove]]
|
* @see [[#remove]]
|
||||||
*/
|
*/
|
||||||
def -(key: A)(implicit node: Cluster): ORMultiMap[A, B] =
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
remove(node, key)
|
def -(key: A)(implicit node: Cluster): ORMultiMap[A, B] = remove(node.selfUniqueAddress, key)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Java API
|
||||||
* Remove an entire set associated with the key.
|
* Remove an entire set associated with the key.
|
||||||
*/
|
*/
|
||||||
def remove(node: Cluster, key: A): ORMultiMap[A, B] =
|
def remove(node: SelfUniqueAddress, key: A): ORMultiMap[A, B] = remove(node.uniqueAddress, key)
|
||||||
remove(node.selfUniqueAddress, key)
|
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def remove(node: Cluster, key: A): ORMultiMap[A, B] = remove(node.selfUniqueAddress, key)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -185,16 +211,22 @@ final class ORMultiMap[A, B] private[akka] (
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Add an element to a set associated with a key. If there is no existing set then one will be initialised.
|
* Add an element to a set associated with a key. If there is no existing set then one will be initialised.
|
||||||
|
* TODO add implicit after deprecated is EOL.
|
||||||
*/
|
*/
|
||||||
|
def addBinding(node: SelfUniqueAddress, key: A, element: B): ORMultiMap[A, B] =
|
||||||
|
addBinding(node.uniqueAddress, key, element)
|
||||||
|
|
||||||
|
def addBindingBy(key: A, element: B)(implicit node: SelfUniqueAddress): ORMultiMap[A, B] =
|
||||||
|
addBinding(node, key, element)
|
||||||
|
|
||||||
|
@deprecated("Use `addBinding` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def addBinding(key: A, element: B)(implicit node: Cluster): ORMultiMap[A, B] =
|
def addBinding(key: A, element: B)(implicit node: Cluster): ORMultiMap[A, B] =
|
||||||
addBinding(node.selfUniqueAddress, key, element)
|
addBinding(node.selfUniqueAddress, key, element)
|
||||||
|
|
||||||
/**
|
@deprecated("Use `addBinding` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
* Java API: Add an element to a set associated with a key. If there is no existing set then one will be initialised.
|
|
||||||
*/
|
|
||||||
def addBinding(node: Cluster, key: A, element: B): ORMultiMap[A, B] =
|
def addBinding(node: Cluster, key: A, element: B): ORMultiMap[A, B] =
|
||||||
addBinding(key, element)(node)
|
addBinding(node.selfUniqueAddress, key, element)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -205,18 +237,24 @@ final class ORMultiMap[A, B] private[akka] (
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Remove an element of a set associated with a key. If there are no more elements in the set then the
|
* Remove an element of a set associated with a key. If there are no more elements in the set then the
|
||||||
* entire set will be removed.
|
* entire set will be removed.
|
||||||
|
* TODO add implicit after deprecated is EOL.
|
||||||
*/
|
*/
|
||||||
|
def removeBinding(node: SelfUniqueAddress, key: A, element: B): ORMultiMap[A, B] =
|
||||||
|
removeBinding(node.uniqueAddress, key, element)
|
||||||
|
|
||||||
|
def removeBindingBy(key: A, element: B)(implicit node: SelfUniqueAddress): ORMultiMap[A, B] =
|
||||||
|
removeBinding(node, key, element)
|
||||||
|
|
||||||
|
@deprecated("Use `removeBinding` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def removeBinding(key: A, element: B)(implicit node: Cluster): ORMultiMap[A, B] =
|
def removeBinding(key: A, element: B)(implicit node: Cluster): ORMultiMap[A, B] =
|
||||||
removeBinding(node.selfUniqueAddress, key, element)
|
removeBinding(node.selfUniqueAddress, key, element)
|
||||||
|
|
||||||
/**
|
@Deprecated
|
||||||
* Java API: Remove an element of a set associated with a key. If there are no more elements in the set then the
|
@deprecated("Use `removeBinding` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
* entire set will be removed.
|
|
||||||
*/
|
|
||||||
def removeBinding(node: Cluster, key: A, element: B): ORMultiMap[A, B] =
|
def removeBinding(node: Cluster, key: A, element: B): ORMultiMap[A, B] =
|
||||||
removeBinding(key, element)(node)
|
removeBinding(node.selfUniqueAddress, key, element)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -241,6 +279,13 @@ final class ORMultiMap[A, B] private[akka] (
|
||||||
* and another one is added within the same Update. The order of addition and removal is important in order
|
* and another one is added within the same Update. The order of addition and removal is important in order
|
||||||
* to retain history for replicated data.
|
* to retain history for replicated data.
|
||||||
*/
|
*/
|
||||||
|
def replaceBinding(node: SelfUniqueAddress, key: A, oldElement: B, newElement: B): ORMultiMap[A, B] =
|
||||||
|
replaceBinding(node.uniqueAddress, key, oldElement, newElement)
|
||||||
|
|
||||||
|
def replaceBindingBy(key: A, oldElement: B, newElement: B)(implicit node: SelfUniqueAddress): ORMultiMap[A, B] =
|
||||||
|
replaceBinding(node, key, oldElement, newElement)
|
||||||
|
|
||||||
|
@deprecated("Use `replaceBinding` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def replaceBinding(key: A, oldElement: B, newElement: B)(implicit node: Cluster): ORMultiMap[A, B] =
|
def replaceBinding(key: A, oldElement: B, newElement: B)(implicit node: Cluster): ORMultiMap[A, B] =
|
||||||
replaceBinding(node.selfUniqueAddress, key, oldElement, newElement)
|
replaceBinding(node.selfUniqueAddress, key, oldElement, newElement)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -307,14 +307,17 @@ final class ORSet[A] private[akka] (
|
||||||
|
|
||||||
def size: Int = elementsMap.size
|
def size: Int = elementsMap.size
|
||||||
|
|
||||||
/**
|
/** Adds an element to the set. */
|
||||||
* Adds an element to the set
|
def :+(element: A)(implicit node: SelfUniqueAddress): ORSet[A] = add(node, element)
|
||||||
*/
|
|
||||||
def +(element: A)(implicit node: Cluster): ORSet[A] = add(node, element)
|
|
||||||
|
|
||||||
/**
|
@deprecated("Use `:+` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
* Adds an element to the set
|
def +(element: A)(implicit node: Cluster): ORSet[A] = add(node.selfUniqueAddress, element)
|
||||||
*/
|
|
||||||
|
/** Adds an element to the set. */
|
||||||
|
def add(node: SelfUniqueAddress, element: A): ORSet[A] = add(node.uniqueAddress, element)
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
@deprecated("Use `add` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def add(node: Cluster, element: A): ORSet[A] = add(node.selfUniqueAddress, element)
|
def add(node: Cluster, element: A): ORSet[A] = add(node.selfUniqueAddress, element)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -335,13 +338,27 @@ final class ORSet[A] private[akka] (
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Scala API
|
||||||
* Removes an element from the set.
|
* Removes an element from the set.
|
||||||
*/
|
*/
|
||||||
def -(element: A)(implicit node: Cluster): ORSet[A] = remove(node, element)
|
def remove(element: A)(implicit node: SelfUniqueAddress): ORSet[A] = remove(node.uniqueAddress, element)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API
|
||||||
|
* Removes an element from the set.
|
||||||
|
*/
|
||||||
|
def remove(node: SelfUniqueAddress, element: A): ORSet[A] = remove(node.uniqueAddress, element)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Removes an element from the set.
|
* Removes an element from the set.
|
||||||
*/
|
*/
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def -(element: A)(implicit node: Cluster): ORSet[A] = remove(node.selfUniqueAddress, element)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes an element from the set.
|
||||||
|
*/
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def remove(node: Cluster, element: A): ORSet[A] = remove(node.selfUniqueAddress, element)
|
def remove(node: Cluster, element: A): ORSet[A] = remove(node.selfUniqueAddress, element)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -362,6 +379,9 @@ final class ORSet[A] private[akka] (
|
||||||
* This has the same result as using [[#remove]] for each
|
* This has the same result as using [[#remove]] for each
|
||||||
* element, but it is more efficient.
|
* element, but it is more efficient.
|
||||||
*/
|
*/
|
||||||
|
def clear(node: SelfUniqueAddress): ORSet[A] = clear(node.uniqueAddress)
|
||||||
|
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def clear(node: Cluster): ORSet[A] = clear(node.selfUniqueAddress)
|
def clear(node: Cluster): ORSet[A] = clear(node.selfUniqueAddress)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -62,67 +62,98 @@ final class PNCounter private[akka] (
|
||||||
* Increment the counter with the delta `n` specified.
|
* Increment the counter with the delta `n` specified.
|
||||||
* If the delta is negative then it will decrement instead of increment.
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
*/
|
*/
|
||||||
def +(n: Long)(implicit node: Cluster): PNCounter = increment(node, n)
|
def :+(n: Long)(implicit node: SelfUniqueAddress): PNCounter = increment(node.uniqueAddress, n)
|
||||||
|
|
||||||
|
@deprecated("Use `:+` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def +(n: Long)(implicit node: Cluster): PNCounter = increment(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Increment the counter with the delta `n` specified.
|
* Increment the counter with the delta `n` specified.
|
||||||
* If the delta is negative then it will decrement instead of increment.
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
*/
|
*/
|
||||||
def +(n: BigInt)(implicit node: Cluster): PNCounter = increment(node, n)
|
def :+(n: BigInt)(implicit node: SelfUniqueAddress): PNCounter = increment(node.uniqueAddress, n)
|
||||||
|
|
||||||
/**
|
@deprecated("Use `:+` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
* Increment the counter with the delta `n` specified.
|
def +(n: BigInt)(implicit node: Cluster): PNCounter = increment(node.selfUniqueAddress, n)
|
||||||
* If the delta is negative then it will decrement instead of increment.
|
|
||||||
*/
|
|
||||||
def increment(node: Cluster, n: Long = 1): PNCounter =
|
|
||||||
increment(node.selfUniqueAddress, n)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Increment the counter with the delta `n` specified.
|
* Scala API: Increment the counter with the delta `n` specified.
|
||||||
* If the delta is negative then it will decrement instead of increment.
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
*/
|
*/
|
||||||
def increment(node: Cluster, n: BigInt): PNCounter =
|
def increment(n: Long)(implicit node: SelfUniqueAddress): PNCounter = increment(node.uniqueAddress, n)
|
||||||
increment(node.selfUniqueAddress, n)
|
|
||||||
|
@deprecated("Use `increment` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def increment(node: Cluster, n: Long = 1): PNCounter = increment(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment the counter with the delta `n` specified.
|
||||||
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
|
*/
|
||||||
|
def increment(n: BigInt)(implicit node: SelfUniqueAddress): PNCounter = increment(node.uniqueAddress, n)
|
||||||
|
|
||||||
|
@deprecated("Use `increment` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def increment(node: Cluster, n: BigInt): PNCounter = increment(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Increment the counter with the delta `n` specified.
|
* Java API: Increment the counter with the delta `n` specified.
|
||||||
* If the delta is negative then it will decrement instead of increment.
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
*/
|
*/
|
||||||
def increment(node: Cluster, n: java.math.BigInteger): PNCounter =
|
def increment(node: SelfUniqueAddress, n: java.math.BigInteger): PNCounter = increment(node.uniqueAddress, n)
|
||||||
increment(node.selfUniqueAddress, n)
|
|
||||||
|
/**
|
||||||
|
* Java API: Increment the counter with the delta `n` specified.
|
||||||
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
|
*/
|
||||||
|
def increment(node: SelfUniqueAddress, n: Long): PNCounter = increment(node.uniqueAddress, n)
|
||||||
|
|
||||||
|
@deprecated("Use `increment` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def increment(node: Cluster, n: java.math.BigInteger): PNCounter = increment(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Decrement the counter with the delta `n` specified.
|
* Decrement the counter with the delta `n` specified.
|
||||||
* If the delta is negative then it will increment instead of decrement.
|
* If the delta is negative then it will increment instead of decrement.
|
||||||
*/
|
*/
|
||||||
def -(n: Long)(implicit node: Cluster): PNCounter = decrement(node, n)
|
def decrement(n: Long)(implicit node: SelfUniqueAddress): PNCounter = decrement(node.uniqueAddress, n)
|
||||||
|
|
||||||
|
@deprecated("Use `decrement` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def -(n: Long)(implicit node: Cluster): PNCounter = decrement(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Decrement the counter with the delta `n` specified.
|
* Decrement the counter with the delta `n` specified.
|
||||||
* If the delta is negative then it will increment instead of decrement.
|
* If the delta is negative then it will increment instead of decrement.
|
||||||
*/
|
*/
|
||||||
def -(n: BigInt)(implicit node: Cluster): PNCounter = decrement(node, n)
|
def decrement(n: BigInt)(implicit node: SelfUniqueAddress): PNCounter = decrement(node.uniqueAddress, n)
|
||||||
|
|
||||||
|
@deprecated("Use `decrement` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def -(n: BigInt)(implicit node: Cluster): PNCounter = decrement(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Decrement the counter with the delta `n` specified.
|
* Decrement the counter with the delta `n` specified.
|
||||||
* If the delta `n` is negative then it will increment instead of decrement.
|
* If the delta `n` is negative then it will increment instead of decrement.
|
||||||
*/
|
*/
|
||||||
def decrement(node: Cluster, n: Long = 1): PNCounter =
|
def decrement(node: SelfUniqueAddress, n: Long): PNCounter = decrement(node.uniqueAddress, n)
|
||||||
decrement(node.selfUniqueAddress, n)
|
|
||||||
|
@deprecated("Use `decrement` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def decrement(node: Cluster, n: Long = 1): PNCounter = decrement(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Decrement the counter with the delta `n` specified.
|
* Scala API: Decrement the counter with the delta `n` specified.
|
||||||
* If the delta `n` is negative then it will increment instead of decrement.
|
* If the delta `n` is negative then it will increment instead of decrement.
|
||||||
*/
|
*/
|
||||||
def decrement(node: Cluster, n: BigInt): PNCounter =
|
def decrement(node: SelfUniqueAddress, n: BigInt): PNCounter = decrement(node.uniqueAddress, n)
|
||||||
decrement(node.selfUniqueAddress, n)
|
|
||||||
|
@deprecated("Use `decrement` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def decrement(node: Cluster, n: BigInt): PNCounter = decrement(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Decrement the counter with the delta `n` specified.
|
* Java API: Decrement the counter with the delta `n` specified.
|
||||||
* If the delta `n` is negative then it will increment instead of decrement.
|
* If the delta `n` is negative then it will increment instead of decrement.
|
||||||
*/
|
*/
|
||||||
def decrement(node: Cluster, n: java.math.BigInteger): PNCounter =
|
def decrement(node: SelfUniqueAddress, n: java.math.BigInteger): PNCounter = decrement(node.uniqueAddress, n)
|
||||||
decrement(node.selfUniqueAddress, n)
|
|
||||||
|
@Deprecated
|
||||||
|
@deprecated("Use `decrement` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def decrement(node: Cluster, n: java.math.BigInteger): PNCounter = decrement(node.selfUniqueAddress, n)
|
||||||
|
|
||||||
/** Internal API */
|
/** Internal API */
|
||||||
@InternalApi private[akka] def increment(key: UniqueAddress, n: BigInt): PNCounter = change(key, n)
|
@InternalApi private[akka] def increment(key: UniqueAddress, n: BigInt): PNCounter = change(key, n)
|
||||||
|
|
|
||||||
|
|
@ -4,11 +4,11 @@
|
||||||
|
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import akka.cluster.Cluster
|
|
||||||
import akka.cluster.UniqueAddress
|
|
||||||
import java.math.BigInteger
|
import java.math.BigInteger
|
||||||
|
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
import akka.cluster.Cluster
|
||||||
|
import akka.cluster.UniqueAddress
|
||||||
import akka.cluster.ddata.ORMap._
|
import akka.cluster.ddata.ORMap._
|
||||||
|
|
||||||
object PNCounterMap {
|
object PNCounterMap {
|
||||||
|
|
@ -75,13 +75,24 @@ final class PNCounterMap[A] private[akka] (
|
||||||
* Increment the counter with the delta specified.
|
* Increment the counter with the delta specified.
|
||||||
* If the delta is negative then it will decrement instead of increment.
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
*/
|
*/
|
||||||
def increment(key: A, delta: Long = 1)(implicit node: Cluster): PNCounterMap[A] =
|
def incrementBy(key: A, delta: Long)(implicit node: SelfUniqueAddress): PNCounterMap[A] =
|
||||||
increment(node, key, delta)
|
increment(node.uniqueAddress, key, delta)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Increment the counter with the delta specified.
|
* Increment the counter with the delta specified.
|
||||||
* If the delta is negative then it will decrement instead of increment.
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
*/
|
*/
|
||||||
|
def increment(key: A, delta: Long = 1)(implicit node: Cluster): PNCounterMap[A] =
|
||||||
|
increment(node.selfUniqueAddress, key, delta)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment the counter with the delta specified.
|
||||||
|
* If the delta is negative then it will decrement instead of increment.
|
||||||
|
*/
|
||||||
|
def increment(node: SelfUniqueAddress, key: A, delta: Long): PNCounterMap[A] =
|
||||||
|
increment(node.uniqueAddress, key, delta)
|
||||||
|
|
||||||
|
@deprecated("Use `increment` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def increment(node: Cluster, key: A, delta: Long): PNCounterMap[A] =
|
def increment(node: Cluster, key: A, delta: Long): PNCounterMap[A] =
|
||||||
increment(node.selfUniqueAddress, key, delta)
|
increment(node.selfUniqueAddress, key, delta)
|
||||||
|
|
||||||
|
|
@ -94,14 +105,28 @@ final class PNCounterMap[A] private[akka] (
|
||||||
/**
|
/**
|
||||||
* Decrement the counter with the delta specified.
|
* Decrement the counter with the delta specified.
|
||||||
* If the delta is negative then it will increment instead of decrement.
|
* If the delta is negative then it will increment instead of decrement.
|
||||||
|
* TODO add implicit after deprecated is EOL.
|
||||||
*/
|
*/
|
||||||
def decrement(key: A, delta: Long = 1)(implicit node: Cluster): PNCounterMap[A] =
|
def decrementBy(key: A, delta: Long = 1)(implicit node: SelfUniqueAddress): PNCounterMap[A] =
|
||||||
decrement(node, key, delta)
|
decrement(node, key, delta)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Decrement the counter with the delta specified.
|
* Decrement the counter with the delta specified.
|
||||||
* If the delta is negative then it will increment instead of decrement.
|
* If the delta is negative then it will increment instead of decrement.
|
||||||
|
* TODO add implicit after deprecated is EOL.
|
||||||
*/
|
*/
|
||||||
|
def decrement(node: SelfUniqueAddress, key: A, delta: Long): PNCounterMap[A] =
|
||||||
|
decrement(node.uniqueAddress, key, delta)
|
||||||
|
|
||||||
|
@deprecated("Use `decrement` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def decrement(key: A, delta: Long = 1)(implicit node: Cluster): PNCounterMap[A] =
|
||||||
|
decrement(node.selfUniqueAddress, key, delta)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Decrement the counter with the delta specified.
|
||||||
|
* If the delta is negative then it will increment instead of decrement.
|
||||||
|
*/
|
||||||
|
@deprecated("Use `decrement` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def decrement(node: Cluster, key: A, delta: Long): PNCounterMap[A] =
|
def decrement(node: Cluster, key: A, delta: Long): PNCounterMap[A] =
|
||||||
decrement(node.selfUniqueAddress, key, delta)
|
decrement(node.selfUniqueAddress, key, delta)
|
||||||
|
|
||||||
|
|
@ -117,16 +142,16 @@ final class PNCounterMap[A] private[akka] (
|
||||||
* Note that if there is a conflicting update on another node the entry will
|
* Note that if there is a conflicting update on another node the entry will
|
||||||
* not be removed after merge.
|
* not be removed after merge.
|
||||||
*/
|
*/
|
||||||
def -(key: A)(implicit node: Cluster): PNCounterMap[A] = remove(node, key)
|
def remove(key: A)(implicit node: SelfUniqueAddress): PNCounterMap[A] =
|
||||||
|
remove(node.uniqueAddress, key)
|
||||||
|
|
||||||
/**
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
* Removes an entry from the map.
|
|
||||||
* Note that if there is a conflicting update on another node the entry will
|
|
||||||
* not be removed after merge.
|
|
||||||
*/
|
|
||||||
def remove(node: Cluster, key: A): PNCounterMap[A] =
|
def remove(node: Cluster, key: A): PNCounterMap[A] =
|
||||||
remove(node.selfUniqueAddress, key)
|
remove(node.selfUniqueAddress, key)
|
||||||
|
|
||||||
|
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def -(key: A)(implicit node: Cluster): PNCounterMap[A] = remove(node, key)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -100,6 +100,15 @@ object ReplicatorSettings {
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] def roleOption(role: String): Option[String] =
|
@InternalApi private[akka] def roleOption(role: String): Option[String] =
|
||||||
if (role == "") None else Option(role)
|
if (role == "") None else Option(role)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
* The name of the actor used in DistributedData extensions.
|
||||||
|
*/
|
||||||
|
@InternalApi private[akka] def name(system: ActorSystem, modifier: Option[String]): String = {
|
||||||
|
val name = system.settings.config.getString("akka.cluster.distributed-data.name")
|
||||||
|
modifier.map(s ⇒ s + name.take(1).toUpperCase + name.drop(1)).getOrElse(name)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,6 @@ import scala.annotation.tailrec
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable.TreeMap
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
import akka.cluster.UniqueAddress
|
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -107,7 +106,10 @@ sealed abstract class VersionVector
|
||||||
/**
|
/**
|
||||||
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
||||||
*/
|
*/
|
||||||
def +(node: Cluster): VersionVector = increment(node)
|
def :+(node: SelfUniqueAddress): VersionVector = increment(node)
|
||||||
|
|
||||||
|
@deprecated("Use `:+` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
|
def +(node: Cluster): VersionVector = increment(node.selfUniqueAddress)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -118,6 +120,9 @@ sealed abstract class VersionVector
|
||||||
/**
|
/**
|
||||||
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
* Increment the version for the node passed as argument. Returns a new VersionVector.
|
||||||
*/
|
*/
|
||||||
|
def increment(node: SelfUniqueAddress): VersionVector = increment(node.uniqueAddress)
|
||||||
|
|
||||||
|
@deprecated("Use `increment` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||||
def increment(node: Cluster): VersionVector = increment(node.selfUniqueAddress)
|
def increment(node: Cluster): VersionVector = increment(node.selfUniqueAddress)
|
||||||
|
|
||||||
def isEmpty: Boolean
|
def isEmpty: Boolean
|
||||||
|
|
|
||||||
|
|
@ -50,7 +50,7 @@ object DurableDataSpec {
|
||||||
else
|
else
|
||||||
sender() ! LoadAllCompleted
|
sender() ! LoadAllCompleted
|
||||||
|
|
||||||
case Store(key, data, reply) ⇒
|
case Store(_, _, reply) ⇒
|
||||||
if (failStore) reply match {
|
if (failStore) reply match {
|
||||||
case Some(StoreReply(_, failureMsg, replyTo)) ⇒ replyTo ! failureMsg
|
case Some(StoreReply(_, failureMsg, replyTo)) ⇒ replyTo ! failureMsg
|
||||||
case None ⇒
|
case None ⇒
|
||||||
|
|
@ -79,8 +79,8 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val timeout = 14.seconds.dilated // initialization of lmdb can be very slow in CI environment
|
val timeout = 14.seconds.dilated // initialization of lmdb can be very slow in CI environment
|
||||||
val writeTwo = WriteTo(2, timeout)
|
val writeTwo = WriteTo(2, timeout)
|
||||||
val readTwo = ReadFrom(2, timeout)
|
val readTwo = ReadFrom(2, timeout)
|
||||||
|
|
@ -123,9 +123,9 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
r ! Get(KeyA, ReadLocal)
|
r ! Get(KeyA, ReadLocal)
|
||||||
expectMsg(NotFound(KeyA, None))
|
expectMsg(NotFound(KeyA, None))
|
||||||
|
|
||||||
r ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
|
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
r ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
|
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
r ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
|
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
|
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
|
@ -163,10 +163,10 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
}
|
}
|
||||||
enterBarrier("both-initialized")
|
enterBarrier("both-initialized")
|
||||||
|
|
||||||
r ! Update(KeyA, GCounter(), writeTwo)(_ + 1)
|
r ! Update(KeyA, GCounter(), writeTwo)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
|
||||||
r ! Update(KeyC, ORSet.empty[String], writeTwo)(_ + myself.name)
|
r ! Update(KeyC, ORSet.empty[String], writeTwo)(_ :+ myself.name)
|
||||||
expectMsg(UpdateSuccess(KeyC, None))
|
expectMsg(UpdateSuccess(KeyC, None))
|
||||||
|
|
||||||
enterBarrier("update-done-" + testStepCounter)
|
enterBarrier("update-done-" + testStepCounter)
|
||||||
|
|
@ -203,7 +203,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
val r = newReplicator()
|
val r = newReplicator()
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
r ! Update(KeyC, ORSet.empty[String], WriteLocal)(_ + myself.name)
|
r ! Update(KeyC, ORSet.empty[String], WriteLocal)(_ :+ myself.name)
|
||||||
expectMsg(UpdateSuccess(KeyC, None))
|
expectMsg(UpdateSuccess(KeyC, None))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -213,7 +213,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
|
|
||||||
// must do one more roundtrip to be sure that it keyB is stored, since Changed might have
|
// must do one more roundtrip to be sure that it keyB is stored, since Changed might have
|
||||||
// been sent out before storage
|
// been sent out before storage
|
||||||
r ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
|
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
|
||||||
watch(r)
|
watch(r)
|
||||||
|
|
@ -254,10 +254,10 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
r ! Get(KeyA, ReadLocal)
|
r ! Get(KeyA, ReadLocal)
|
||||||
expectMsg(NotFound(KeyA, None))
|
expectMsg(NotFound(KeyA, None))
|
||||||
|
|
||||||
r ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
|
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
r ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
|
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
r ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
|
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
r ! Update(KeyB, GCounter(), WriteLocal)(_ + 1)
|
r ! Update(KeyB, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
|
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
|
@ -286,7 +286,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
val r2: ActorRef = newReplicator(sys2)
|
val r2: ActorRef = newReplicator(sys2)
|
||||||
|
|
||||||
// it should be possible to update while loading is in progress
|
// it should be possible to update while loading is in progress
|
||||||
r2 ! Update(KeyB, GCounter(), WriteLocal)(_ + 1)
|
r2 ! Update(KeyB, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyB, None))
|
expectMsg(UpdateSuccess(KeyB, None))
|
||||||
|
|
||||||
// wait until all loaded
|
// wait until all loaded
|
||||||
|
|
@ -325,7 +325,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
Replicator.props(
|
Replicator.props(
|
||||||
ReplicatorSettings(system).withDurableStoreProps(testDurableStoreProps(failStore = true))),
|
ReplicatorSettings(system).withDurableStoreProps(testDurableStoreProps(failStore = true))),
|
||||||
"replicator-" + testStepCounter)
|
"replicator-" + testStepCounter)
|
||||||
r ! Update(KeyA, GCounter(), WriteLocal, request = Some("a"))(_ + 1)
|
r ! Update(KeyA, GCounter(), WriteLocal, request = Some("a"))(_ :+ 1)
|
||||||
expectMsg(StoreFailure(KeyA, Some("a")))
|
expectMsg(StoreFailure(KeyA, Some("a")))
|
||||||
}
|
}
|
||||||
enterBarrierAfterTestStep()
|
enterBarrierAfterTestStep()
|
||||||
|
|
|
||||||
|
|
@ -43,7 +43,8 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val maxPruningDissemination = 3.seconds
|
val maxPruningDissemination = 3.seconds
|
||||||
|
|
||||||
def startReplicator(sys: ActorSystem): ActorRef =
|
def startReplicator(sys: ActorSystem): ActorRef =
|
||||||
|
|
@ -90,7 +91,7 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 3)
|
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 3)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
|
||||||
replicator2.tell(Update(KeyA, GCounter(), WriteLocal)(_.increment(cluster2, 2)), probe2.ref)
|
replicator2.tell(Update(KeyA, GCounter(), WriteLocal)(_.increment(cluster2, 2)), probe2.ref)
|
||||||
|
|
|
||||||
|
|
@ -49,7 +49,8 @@ class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) w
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val replicator = DistributedData(system).replicator
|
val replicator = DistributedData(system).replicator
|
||||||
val nodes = roles.drop(1) // controller not part of active nodes
|
val nodes = roles.drop(1) // controller not part of active nodes
|
||||||
val nodeCount = nodes.size
|
val nodeCount = nodes.size
|
||||||
|
|
@ -114,7 +115,7 @@ class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) w
|
||||||
val writeProbe = TestProbe()
|
val writeProbe = TestProbe()
|
||||||
val writeAcks = myData.map { i ⇒
|
val writeAcks = myData.map { i ⇒
|
||||||
sleepDelay()
|
sleepDelay()
|
||||||
replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ + i), writeProbe.ref)
|
replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ :+ i), writeProbe.ref)
|
||||||
writeProbe.receiveOne(3.seconds)
|
writeProbe.receiveOne(3.seconds)
|
||||||
}
|
}
|
||||||
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] ⇒ success }
|
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] ⇒ success }
|
||||||
|
|
@ -147,7 +148,7 @@ class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) w
|
||||||
val writeProbe = TestProbe()
|
val writeProbe = TestProbe()
|
||||||
val writeAcks = myData.map { i ⇒
|
val writeAcks = myData.map { i ⇒
|
||||||
sleepDelay()
|
sleepDelay()
|
||||||
replicator.tell(Update(key, ORSet(), writeMajority, Some(i))(_ + i), writeProbe.ref)
|
replicator.tell(Update(key, ORSet(), writeMajority, Some(i))(_ :+ i), writeProbe.ref)
|
||||||
writeProbe.receiveOne(timeout + 1.second)
|
writeProbe.receiveOne(timeout + 1.second)
|
||||||
}
|
}
|
||||||
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] ⇒ success }
|
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] ⇒ success }
|
||||||
|
|
@ -163,7 +164,7 @@ class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) w
|
||||||
val readProbe = TestProbe()
|
val readProbe = TestProbe()
|
||||||
replicator.tell(Get(key, readMajority), readProbe.ref)
|
replicator.tell(Get(key, readMajority), readProbe.ref)
|
||||||
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) ⇒ g.get(key) }
|
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) ⇒ g.get(key) }
|
||||||
val survivors = result.elements.size
|
//val survivors = result.elements.size
|
||||||
result.elements should be(expectedData)
|
result.elements should be(expectedData)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -191,7 +192,7 @@ class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) w
|
||||||
val writeProbe = TestProbe()
|
val writeProbe = TestProbe()
|
||||||
val writeAcks = myData.map { i ⇒
|
val writeAcks = myData.map { i ⇒
|
||||||
sleepDelay()
|
sleepDelay()
|
||||||
replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ + i), writeProbe.ref)
|
replicator.tell(Update(key, ORSet(), WriteLocal, Some(i))(_ :+ i), writeProbe.ref)
|
||||||
writeProbe.receiveOne(3.seconds)
|
writeProbe.receiveOne(3.seconds)
|
||||||
}
|
}
|
||||||
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] ⇒ success }
|
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] ⇒ success }
|
||||||
|
|
@ -236,7 +237,7 @@ class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) w
|
||||||
val writeProbe = TestProbe()
|
val writeProbe = TestProbe()
|
||||||
val writeAcks = myData.map { i ⇒
|
val writeAcks = myData.map { i ⇒
|
||||||
sleepDelay()
|
sleepDelay()
|
||||||
replicator.tell(Update(key, ORSet(), writeMajority, Some(i))(_ + i), writeProbe.ref)
|
replicator.tell(Update(key, ORSet(), writeMajority, Some(i))(_ :+ i), writeProbe.ref)
|
||||||
writeProbe.receiveOne(timeout + 1.second)
|
writeProbe.receiveOne(timeout + 1.second)
|
||||||
}
|
}
|
||||||
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] ⇒ success }
|
val successWriteAcks = writeAcks.collect { case success: UpdateSuccess[_] ⇒ success }
|
||||||
|
|
@ -260,7 +261,7 @@ class JepsenInspiredInsertSpec extends MultiNodeSpec(JepsenInspiredInsertSpec) w
|
||||||
val readProbe = TestProbe()
|
val readProbe = TestProbe()
|
||||||
replicator.tell(Get(key, readMajority), readProbe.ref)
|
replicator.tell(Get(key, readMajority), readProbe.ref)
|
||||||
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) ⇒ g.get(key) }
|
val result = readProbe.expectMsgPF() { case g @ GetSuccess(`key`, _) ⇒ g.get(key) }
|
||||||
val survivors = result.elements.size
|
//val survivors = result.elements.size
|
||||||
result.elements should be(expectedData)
|
result.elements should be(expectedData)
|
||||||
}
|
}
|
||||||
// but on the 3 node side, read from majority doesn't mean that we are guaranteed to see
|
// but on the 3 node side, read from majority doesn't mean that we are guaranteed to see
|
||||||
|
|
|
||||||
|
|
@ -69,7 +69,8 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val replicator = DistributedData(system).replicator
|
val replicator = DistributedData(system).replicator
|
||||||
val timeout = 3.seconds.dilated
|
val timeout = 3.seconds.dilated
|
||||||
val factor = 1 // use 3 here for serious tuning
|
val factor = 1 // use 3 here for serious tuning
|
||||||
|
|
@ -156,7 +157,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
val n = 1000 * factor
|
val n = 1000 * factor
|
||||||
val expectedData = (0 until n).toSet
|
val expectedData = (0 until n).toSet
|
||||||
repeat("ORSet Update WriteLocal", keys, n)({ (key, i, replyTo) ⇒
|
repeat("ORSet Update WriteLocal", keys, n)({ (key, i, replyTo) ⇒
|
||||||
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + i), replyTo)
|
replicator.tell(Update(key, ORSet(), WriteLocal)(_ :+ i), replyTo)
|
||||||
}, key ⇒ awaitReplicated(key, expectedData))
|
}, key ⇒ awaitReplicated(key, expectedData))
|
||||||
|
|
||||||
enterBarrier("after-1")
|
enterBarrier("after-1")
|
||||||
|
|
@ -164,7 +165,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
|
|
||||||
"be blazingly fast for ORSet Get ReadLocal" taggedAs PerformanceTest in {
|
"be blazingly fast for ORSet Get ReadLocal" taggedAs PerformanceTest in {
|
||||||
val keys = (1 to repeatCount).map(n ⇒ ORSetKey[Int]("A" + n))
|
val keys = (1 to repeatCount).map(n ⇒ ORSetKey[Int]("A" + n))
|
||||||
repeat("Get ReadLocal", keys, 100000 * factor) { (key, i, replyTo) ⇒
|
repeat("Get ReadLocal", keys, 100000 * factor) { (key, _, replyTo) ⇒
|
||||||
replicator.tell(Get(key, ReadLocal), replyTo)
|
replicator.tell(Get(key, ReadLocal), replyTo)
|
||||||
}
|
}
|
||||||
enterBarrier("after-2")
|
enterBarrier("after-2")
|
||||||
|
|
@ -175,7 +176,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
val n = 200 * factor
|
val n = 200 * factor
|
||||||
val expected = Some((0 until n).toSet)
|
val expected = Some((0 until n).toSet)
|
||||||
repeat("ORSet Update WriteLocal + gossip", keys, n, expected) { (key, i, replyTo) ⇒
|
repeat("ORSet Update WriteLocal + gossip", keys, n, expected) { (key, i, replyTo) ⇒
|
||||||
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + i), replyTo)
|
replicator.tell(Update(key, ORSet(), WriteLocal)(_ :+ i), replyTo)
|
||||||
}
|
}
|
||||||
enterBarrier("after-3")
|
enterBarrier("after-3")
|
||||||
}
|
}
|
||||||
|
|
@ -185,7 +186,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
val n = 200 * factor
|
val n = 200 * factor
|
||||||
val expected = Some((0 until n).toSet ++ (0 until n).map(-_).toSet)
|
val expected = Some((0 until n).toSet ++ (0 until n).map(-_).toSet)
|
||||||
repeat("ORSet Update WriteLocal existing + gossip", keys, n, expected) { (key, i, replyTo) ⇒
|
repeat("ORSet Update WriteLocal existing + gossip", keys, n, expected) { (key, i, replyTo) ⇒
|
||||||
replicator.tell(Update(key, ORSet(), WriteLocal)(_ + (-i)), replyTo)
|
replicator.tell(Update(key, ORSet(), WriteLocal)(_ :+ (-i)), replyTo)
|
||||||
}
|
}
|
||||||
enterBarrier("after-4")
|
enterBarrier("after-4")
|
||||||
}
|
}
|
||||||
|
|
@ -196,7 +197,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
val expected = Some((0 until n).toSet)
|
val expected = Some((0 until n).toSet)
|
||||||
val writeTwo = WriteTo(2, timeout)
|
val writeTwo = WriteTo(2, timeout)
|
||||||
repeat("ORSet Update WriteTwo + gossip", keys, n, expected) { (key, i, replyTo) ⇒
|
repeat("ORSet Update WriteTwo + gossip", keys, n, expected) { (key, i, replyTo) ⇒
|
||||||
replicator.tell(Update(key, ORSet(), writeTwo)(_ + i), replyTo)
|
replicator.tell(Update(key, ORSet(), writeTwo)(_ :+ i), replyTo)
|
||||||
}
|
}
|
||||||
enterBarrier("after-5")
|
enterBarrier("after-5")
|
||||||
}
|
}
|
||||||
|
|
@ -209,7 +210,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
val latch = TestLatch(n)
|
val latch = TestLatch(n)
|
||||||
val replyTo = system.actorOf(countDownProps(latch))
|
val replyTo = system.actorOf(countDownProps(latch))
|
||||||
for (_ ← 0 until n)
|
for (_ ← 0 until n)
|
||||||
replicator.tell(Update(key, GCounter(), WriteLocal)(_ + 1), replyTo)
|
replicator.tell(Update(key, GCounter(), WriteLocal)(_ :+ 1), replyTo)
|
||||||
Await.ready(latch, 5.seconds + (1.second * factor))
|
Await.ready(latch, 5.seconds + (1.second * factor))
|
||||||
enterBarrier("update-done-6")
|
enterBarrier("update-done-6")
|
||||||
runOn(n1) {
|
runOn(n1) {
|
||||||
|
|
@ -247,7 +248,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
val n = 300 * factor
|
val n = 300 * factor
|
||||||
val writeMajority = WriteMajority(timeout)
|
val writeMajority = WriteMajority(timeout)
|
||||||
repeat("ORSet Update one-by-one deltas", keys, n, oneByOne = true) { (key, i, replyTo) ⇒
|
repeat("ORSet Update one-by-one deltas", keys, n, oneByOne = true) { (key, i, replyTo) ⇒
|
||||||
replicator.tell(Update(key, ORSet(), writeMajority)(_ + i), replyTo)
|
replicator.tell(Update(key, ORSet(), writeMajority)(_ :+ i), replyTo)
|
||||||
}
|
}
|
||||||
enterBarrier("after-7")
|
enterBarrier("after-7")
|
||||||
}
|
}
|
||||||
|
|
@ -257,7 +258,7 @@ class PerformanceSpec extends MultiNodeSpec(PerformanceSpec) with STMultiNodeSpe
|
||||||
val n = 200 * factor
|
val n = 200 * factor
|
||||||
val writeMajority = WriteMajority(timeout)
|
val writeMajority = WriteMajority(timeout)
|
||||||
repeat("ORSet Update deltas", keys, n, oneByOne = false) { (key, i, replyTo) ⇒
|
repeat("ORSet Update deltas", keys, n, oneByOne = false) { (key, i, replyTo) ⇒
|
||||||
replicator.tell(Update(key, ORSet(), writeMajority)(_ + i), replyTo)
|
replicator.tell(Update(key, ORSet(), writeMajority)(_ :+ i), replyTo)
|
||||||
}
|
}
|
||||||
enterBarrier("after-8")
|
enterBarrier("after-8")
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -43,7 +43,8 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val replicator = system.actorOf(Replicator.props(
|
val replicator = system.actorOf(Replicator.props(
|
||||||
ReplicatorSettings(system).withRole("backend").withGossipInterval(1.second)), "replicator")
|
ReplicatorSettings(system).withRole("backend").withGossipInterval(1.second)), "replicator")
|
||||||
val timeout = 3.seconds.dilated
|
val timeout = 3.seconds.dilated
|
||||||
|
|
@ -104,18 +105,18 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
(0 until 5).foreach { i ⇒
|
for (_ ← 0 until 5) {
|
||||||
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 1)
|
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
replicator ! Update(KeyB, PNCounter(), WriteLocal)(_ - 1)
|
replicator ! Update(KeyB, PNCounter(), WriteLocal)(_ decrement 1)
|
||||||
replicator ! Update(KeyC, GCounter(), WriteAll(timeout))(_ + 1)
|
replicator ! Update(KeyC, GCounter(), WriteAll(timeout))(_ :+ 1)
|
||||||
}
|
}
|
||||||
receiveN(15).map(_.getClass).toSet should be(Set(classOf[UpdateSuccess[_]]))
|
receiveN(15).map(_.getClass).toSet should be(Set(classOf[UpdateSuccess[_]]))
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 20)
|
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 20)
|
||||||
replicator ! Update(KeyB, PNCounter(), WriteTo(2, timeout))(_ + 20)
|
replicator ! Update(KeyB, PNCounter(), WriteTo(2, timeout))(_ :+ 20)
|
||||||
replicator ! Update(KeyC, GCounter(), WriteAll(timeout))(_ + 20)
|
replicator ! Update(KeyC, GCounter(), WriteAll(timeout))(_ :+ 20)
|
||||||
receiveN(3).toSet should be(Set(
|
receiveN(3).toSet should be(Set(
|
||||||
UpdateSuccess(KeyA, None),
|
UpdateSuccess(KeyA, None),
|
||||||
UpdateSuccess(KeyB, None), UpdateSuccess(KeyC, None)))
|
UpdateSuccess(KeyB, None), UpdateSuccess(KeyC, None)))
|
||||||
|
|
@ -123,23 +124,23 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
|
||||||
replicator ! Update(KeyE, GSet(), WriteLocal)(_ + "e1" + "e2")
|
replicator ! Update(KeyE, GSet(), WriteLocal)(_ + "e1" + "e2")
|
||||||
expectMsg(UpdateSuccess(KeyE, None))
|
expectMsg(UpdateSuccess(KeyE, None))
|
||||||
|
|
||||||
replicator ! Update(KeyF, ORSet(), WriteLocal)(_ + "e1" + "e2")
|
replicator ! Update(KeyF, ORSet(), WriteLocal)(_ :+ "e1" :+ "e2")
|
||||||
expectMsg(UpdateSuccess(KeyF, None))
|
expectMsg(UpdateSuccess(KeyF, None))
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(fourth) {
|
runOn(fourth) {
|
||||||
replicator ! Update(KeyD, GCounter(), WriteLocal)(_ + 40)
|
replicator ! Update(KeyD, GCounter(), WriteLocal)(_ :+ 40)
|
||||||
expectMsg(UpdateSuccess(KeyD, None))
|
expectMsg(UpdateSuccess(KeyD, None))
|
||||||
|
|
||||||
replicator ! Update(KeyE, GSet(), WriteLocal)(_ + "e2" + "e3")
|
replicator ! Update(KeyE, GSet(), WriteLocal)(_ + "e2" + "e3")
|
||||||
expectMsg(UpdateSuccess(KeyE, None))
|
expectMsg(UpdateSuccess(KeyE, None))
|
||||||
|
|
||||||
replicator ! Update(KeyF, ORSet(), WriteLocal)(_ + "e2" + "e3")
|
replicator ! Update(KeyF, ORSet(), WriteLocal)(_ :+ "e2" :+ "e3")
|
||||||
expectMsg(UpdateSuccess(KeyF, None))
|
expectMsg(UpdateSuccess(KeyF, None))
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(fifth) {
|
runOn(fifth) {
|
||||||
replicator ! Update(KeyX, GCounter(), WriteTo(2, timeout))(_ + 50)
|
replicator ! Update(KeyX, GCounter(), WriteTo(2, timeout))(_ :+ 50)
|
||||||
expectMsg(UpdateSuccess(KeyX, None))
|
expectMsg(UpdateSuccess(KeyX, None))
|
||||||
replicator ! Delete(KeyX, WriteLocal)
|
replicator ! Delete(KeyX, WriteLocal)
|
||||||
expectMsg(DeleteSuccess(KeyX, None))
|
expectMsg(DeleteSuccess(KeyX, None))
|
||||||
|
|
@ -168,22 +169,22 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
|
||||||
enterBarrier("split")
|
enterBarrier("split")
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
replicator ! Update(KeyA, GCounter(), WriteTo(2, timeout))(_ + 1)
|
replicator ! Update(KeyA, GCounter(), WriteTo(2, timeout))(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(third) {
|
runOn(third) {
|
||||||
replicator ! Update(KeyA, GCounter(), WriteTo(2, timeout))(_ + 2)
|
replicator ! Update(KeyA, GCounter(), WriteTo(2, timeout))(_ :+ 2)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
|
||||||
replicator ! Update(KeyE, GSet(), WriteTo(2, timeout))(_ + "e4")
|
replicator ! Update(KeyE, GSet(), WriteTo(2, timeout))(_ + "e4")
|
||||||
expectMsg(UpdateSuccess(KeyE, None))
|
expectMsg(UpdateSuccess(KeyE, None))
|
||||||
|
|
||||||
replicator ! Update(KeyF, ORSet(), WriteTo(2, timeout))(_ - "e2")
|
replicator ! Update(KeyF, ORSet(), WriteTo(2, timeout))(_ remove "e2")
|
||||||
expectMsg(UpdateSuccess(KeyF, None))
|
expectMsg(UpdateSuccess(KeyF, None))
|
||||||
}
|
}
|
||||||
runOn(fourth) {
|
runOn(fourth) {
|
||||||
replicator ! Update(KeyD, GCounter(), WriteTo(2, timeout))(_ + 1)
|
replicator ! Update(KeyD, GCounter(), WriteTo(2, timeout))(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyD, None))
|
expectMsg(UpdateSuccess(KeyD, None))
|
||||||
}
|
}
|
||||||
enterBarrier("update-during-split")
|
enterBarrier("update-during-split")
|
||||||
|
|
|
||||||
|
|
@ -141,7 +141,8 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val fullStateReplicator = system.actorOf(Replicator.props(
|
val fullStateReplicator = system.actorOf(Replicator.props(
|
||||||
ReplicatorSettings(system).withGossipInterval(1.second).withDeltaCrdtEnabled(false)), "fullStateReplicator")
|
ReplicatorSettings(system).withGossipInterval(1.second).withDeltaCrdtEnabled(false)), "fullStateReplicator")
|
||||||
val deltaReplicator = {
|
val deltaReplicator = {
|
||||||
|
|
@ -199,12 +200,12 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
// by setting something for each key we don't have to worry about NotFound
|
// by setting something for each key we don't have to worry about NotFound
|
||||||
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
||||||
fullStateReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ + 1)
|
fullStateReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ :+ 1)
|
||||||
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ + 1)
|
deltaReplicator ! Update(key, PNCounter.empty, WriteLocal)(_ :+ 1)
|
||||||
}
|
}
|
||||||
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||||
fullStateReplicator ! Update(key, ORSet.empty[String], WriteLocal)(_ + "a")
|
fullStateReplicator ! Update(key, ORSet.empty[String], WriteLocal)(_ :+ "a")
|
||||||
deltaReplicator ! Update(key, ORSet.empty[String], WriteLocal)(_ + "a")
|
deltaReplicator ! Update(key, ORSet.empty[String], WriteLocal)(_ :+ "a")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
enterBarrier("updated-1")
|
enterBarrier("updated-1")
|
||||||
|
|
@ -232,8 +233,8 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
"work with write consistency" in {
|
"work with write consistency" in {
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
val p1 = TestProbe()
|
val p1 = TestProbe()
|
||||||
fullStateReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ + "A"), p1.ref)
|
fullStateReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ :+ "A"), p1.ref)
|
||||||
deltaReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ + "A"), p1.ref)
|
deltaReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ :+ "A"), p1.ref)
|
||||||
p1.expectMsgType[UpdateSuccess[_]]
|
p1.expectMsgType[UpdateSuccess[_]]
|
||||||
p1.expectMsgType[UpdateSuccess[_]]
|
p1.expectMsgType[UpdateSuccess[_]]
|
||||||
}
|
}
|
||||||
|
|
@ -248,9 +249,9 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
// retry with full state to sort it out
|
// retry with full state to sort it out
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
val p1 = TestProbe()
|
val p1 = TestProbe()
|
||||||
deltaReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ + "B"), p1.ref)
|
deltaReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ :+ "B"), p1.ref)
|
||||||
deltaReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ + "C"), p1.ref)
|
deltaReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ :+ "C"), p1.ref)
|
||||||
deltaReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ + "D"), p1.ref)
|
deltaReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ :+ "D"), p1.ref)
|
||||||
p1.expectMsgType[UpdateSuccess[_]]
|
p1.expectMsgType[UpdateSuccess[_]]
|
||||||
p1.expectMsgType[UpdateSuccess[_]]
|
p1.expectMsgType[UpdateSuccess[_]]
|
||||||
p1.expectMsgType[UpdateSuccess[_]]
|
p1.expectMsgType[UpdateSuccess[_]]
|
||||||
|
|
@ -262,7 +263,7 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
// add same to the fullStateReplicator so they are in sync
|
// add same to the fullStateReplicator so they are in sync
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
val p1 = TestProbe()
|
val p1 = TestProbe()
|
||||||
fullStateReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ + "A" + "B" + "C" + "D"), p1.ref)
|
fullStateReplicator.tell(Update(KeyD, ORSet.empty[String], writeAll)(_ :+ "A" :+ "B" :+ "C" :+ "D"), p1.ref)
|
||||||
p1.expectMsgType[UpdateSuccess[_]]
|
p1.expectMsgType[UpdateSuccess[_]]
|
||||||
}
|
}
|
||||||
enterBarrier("write-3")
|
enterBarrier("write-3")
|
||||||
|
|
@ -366,22 +367,22 @@ class ReplicatorDeltaSpec extends MultiNodeSpec(ReplicatorDeltaSpec) with STMult
|
||||||
op match {
|
op match {
|
||||||
case Delay(d) ⇒ Thread.sleep(d)
|
case Delay(d) ⇒ Thread.sleep(d)
|
||||||
case Incr(key, n, consistency) ⇒
|
case Incr(key, n, consistency) ⇒
|
||||||
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ + n)
|
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ :+ n)
|
||||||
deltaReplicator ! Update(key, PNCounter.empty, consistency)(_ + n)
|
deltaReplicator ! Update(key, PNCounter.empty, consistency)(_ :+ n)
|
||||||
case Decr(key, n, consistency) ⇒
|
case Decr(key, n, consistency) ⇒
|
||||||
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ - n)
|
fullStateReplicator ! Update(key, PNCounter.empty, consistency)(_ decrement n)
|
||||||
deltaReplicator ! Update(key, PNCounter.empty, consistency)(_ - n)
|
deltaReplicator ! Update(key, PNCounter.empty, consistency)(_ decrement n)
|
||||||
case Add(key, elem, consistency) ⇒
|
case Add(key, elem, consistency) ⇒
|
||||||
// to have an deterministic result when mixing add/remove we can only perform
|
// to have an deterministic result when mixing add/remove we can only perform
|
||||||
// the ORSet operations from one node
|
// the ORSet operations from one node
|
||||||
runOn((if (key == KeyF) List(first) else List(first, second, third)): _*) {
|
runOn((if (key == KeyF) List(first) else List(first, second, third)): _*) {
|
||||||
fullStateReplicator ! Update(key, ORSet.empty[String], consistency)(_ + elem)
|
fullStateReplicator ! Update(key, ORSet.empty[String], consistency)(_ :+ elem)
|
||||||
deltaReplicator ! Update(key, ORSet.empty[String], consistency)(_ + elem)
|
deltaReplicator ! Update(key, ORSet.empty[String], consistency)(_ :+ elem)
|
||||||
}
|
}
|
||||||
case Remove(key, elem, consistency) ⇒
|
case Remove(key, elem, consistency) ⇒
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
fullStateReplicator ! Update(key, ORSet.empty[String], consistency)(_ - elem)
|
fullStateReplicator ! Update(key, ORSet.empty[String], consistency)(_ remove elem)
|
||||||
deltaReplicator ! Update(key, ORSet.empty[String], consistency)(_ - elem)
|
deltaReplicator ! Update(key, ORSet.empty[String], consistency)(_ remove elem)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -164,11 +164,11 @@ object ReplicatorMapDeltaSpec extends MultiNodeConfig {
|
||||||
}.toVector
|
}.toVector
|
||||||
}
|
}
|
||||||
|
|
||||||
def addElementToORMap(om: ORMap[String, ORSet[String]], key: String, element: String)(implicit node: Cluster) =
|
def addElementToORMap(om: ORMap[String, ORSet[String]], key: String, element: String)(implicit node: SelfUniqueAddress) =
|
||||||
om.updated(node, key, ORSet.empty[String])(_.add(node, element))
|
om.updated(node, key, ORSet.empty[String])(_ :+ element)
|
||||||
|
|
||||||
def removeElementFromORMap(om: ORMap[String, ORSet[String]], key: String, element: String)(implicit node: Cluster) =
|
def removeElementFromORMap(om: ORMap[String, ORSet[String]], key: String, element: String)(implicit node: SelfUniqueAddress) =
|
||||||
om.updated(node, key, ORSet.empty[String])(_.remove(node, element))
|
om.updated(node, key, ORSet.empty[String])(_.remove(element))
|
||||||
}
|
}
|
||||||
|
|
||||||
class ReplicatorMapDeltaSpecMultiJvmNode1 extends ReplicatorMapDeltaSpec
|
class ReplicatorMapDeltaSpecMultiJvmNode1 extends ReplicatorMapDeltaSpec
|
||||||
|
|
@ -182,7 +182,8 @@ class ReplicatorMapDeltaSpec extends MultiNodeSpec(ReplicatorMapDeltaSpec) with
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val fullStateReplicator = system.actorOf(Replicator.props(
|
val fullStateReplicator = system.actorOf(Replicator.props(
|
||||||
ReplicatorSettings(system).withGossipInterval(1.second).withDeltaCrdtEnabled(false)), "fullStateReplicator")
|
ReplicatorSettings(system).withGossipInterval(1.second).withDeltaCrdtEnabled(false)), "fullStateReplicator")
|
||||||
val deltaReplicator = {
|
val deltaReplicator = {
|
||||||
|
|
@ -241,20 +242,20 @@ class ReplicatorMapDeltaSpec extends MultiNodeSpec(ReplicatorMapDeltaSpec) with
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
// by setting something for each key we don't have to worry about NotFound
|
// by setting something for each key we don't have to worry about NotFound
|
||||||
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
List(KeyA, KeyB, KeyC).foreach { key ⇒
|
||||||
fullStateReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ increment key._2)
|
fullStateReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_.incrementBy(key._2, 1))
|
||||||
deltaReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ increment key._2)
|
deltaReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_.incrementBy(key._2, 1))
|
||||||
}
|
}
|
||||||
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||||
fullStateReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ + (key._2 → Set("a")))
|
fullStateReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ :+ (key._2 → Set("a")))
|
||||||
deltaReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ + (key._2 → Set("a")))
|
deltaReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ :+ (key._2 → Set("a")))
|
||||||
}
|
}
|
||||||
List(KeyG, KeyH, KeyI).foreach { key ⇒
|
List(KeyG, KeyH, KeyI).foreach { key ⇒
|
||||||
fullStateReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ + (key._2 → Set("a")))
|
fullStateReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ :+ (key._2 → Set("a")))
|
||||||
deltaReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ + (key._2 → Set("a")))
|
deltaReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ :+ (key._2 → Set("a")))
|
||||||
}
|
}
|
||||||
List(KeyJ, KeyK, KeyL).foreach { key ⇒
|
List(KeyJ, KeyK, KeyL).foreach { key ⇒
|
||||||
fullStateReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(_ + (key._2 → (ORSet.empty + "a")))
|
fullStateReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(_ :+ (key._2 → (ORSet.empty :+ "a")))
|
||||||
deltaReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(_ + (key._2 → (ORSet.empty + "a")))
|
deltaReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(_ :+ (key._2 → (ORSet.empty :+ "a")))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
enterBarrier("updated-1")
|
enterBarrier("updated-1")
|
||||||
|
|
@ -271,7 +272,7 @@ class ReplicatorMapDeltaSpec extends MultiNodeSpec(ReplicatorMapDeltaSpec) with
|
||||||
val p = TestProbe()
|
val p = TestProbe()
|
||||||
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
List(KeyD, KeyE, KeyF).foreach { key ⇒
|
||||||
fullStateReplicator.tell(Get(key._1, ReadLocal), p.ref)
|
fullStateReplicator.tell(Get(key._1, ReadLocal), p.ref)
|
||||||
val res = p.expectMsgType[GetSuccess[ORMultiMap[String, String]]].dataValue.get(key._2) should ===(Some(Set("a")))
|
p.expectMsgType[GetSuccess[ORMultiMap[String, String]]].dataValue.get(key._2) should ===(Some(Set("a")))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
|
|
@ -300,7 +301,7 @@ class ReplicatorMapDeltaSpec extends MultiNodeSpec(ReplicatorMapDeltaSpec) with
|
||||||
system.eventStream.subscribe(errorLogProbe.ref, classOf[Error])
|
system.eventStream.subscribe(errorLogProbe.ref, classOf[Error])
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
for (_ ← 1 to N; key ← List(KeyA, KeyB)) {
|
for (_ ← 1 to N; key ← List(KeyA, KeyB)) {
|
||||||
ordinaryReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ increment key._2)
|
ordinaryReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_.incrementBy(key._2, 1))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
enterBarrier("updated-2")
|
enterBarrier("updated-2")
|
||||||
|
|
@ -333,44 +334,44 @@ class ReplicatorMapDeltaSpec extends MultiNodeSpec(ReplicatorMapDeltaSpec) with
|
||||||
log.debug("operation: {}", op)
|
log.debug("operation: {}", op)
|
||||||
op match {
|
op match {
|
||||||
case Delay(d) ⇒ Thread.sleep(d)
|
case Delay(d) ⇒ Thread.sleep(d)
|
||||||
case Incr(key, n, consistency) ⇒
|
case Incr(key, n, _) ⇒
|
||||||
fullStateReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ increment (key._2, n))
|
fullStateReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ incrementBy (key._2, n))
|
||||||
deltaReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ increment (key._2, n))
|
deltaReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ incrementBy (key._2, n))
|
||||||
case Decr(key, n, consistency) ⇒
|
case Decr(key, n, _) ⇒
|
||||||
fullStateReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ decrement (key._2, n))
|
fullStateReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ decrementBy (key._2, n))
|
||||||
deltaReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ decrement (key._2, n))
|
deltaReplicator ! Update(key._1, PNCounterMap.empty[String], WriteLocal)(_ decrementBy (key._2, n))
|
||||||
case AddVD(key, elem, consistency) ⇒
|
case AddVD(key, elem, _) ⇒
|
||||||
// to have an deterministic result when mixing add/remove we can only perform
|
// to have an deterministic result when mixing add/remove we can only perform
|
||||||
// the ORSet operations from one node
|
// the ORSet operations from one node
|
||||||
runOn((if (key == KeyF) List(first) else List(first, second, third)): _*) {
|
runOn((if (key == KeyF) List(first) else List(first, second, third)): _*) {
|
||||||
fullStateReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ addBinding (key._2, elem))
|
fullStateReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ addBindingBy (key._2, elem))
|
||||||
deltaReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ addBinding (key._2, elem))
|
deltaReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ addBindingBy (key._2, elem))
|
||||||
}
|
}
|
||||||
case RemoveVD(key, elem, consistency) ⇒
|
case RemoveVD(key, elem, _) ⇒
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
fullStateReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ removeBinding (key._2, elem))
|
fullStateReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ removeBindingBy (key._2, elem))
|
||||||
deltaReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ removeBinding (key._2, elem))
|
deltaReplicator ! Update(key._1, ORMultiMap.emptyWithValueDeltas[String, String], WriteLocal)(_ removeBindingBy (key._2, elem))
|
||||||
}
|
}
|
||||||
case AddNoVD(key, elem, consistency) ⇒
|
case AddNoVD(key, elem, _) ⇒
|
||||||
// to have an deterministic result when mixing add/remove we can only perform
|
// to have an deterministic result when mixing add/remove we can only perform
|
||||||
// the ORSet operations from one node
|
// the ORSet operations from one node
|
||||||
runOn((if (key == KeyI) List(first) else List(first, second, third)): _*) {
|
runOn((if (key == KeyI) List(first) else List(first, second, third)): _*) {
|
||||||
fullStateReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ addBinding (key._2, elem))
|
fullStateReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ addBindingBy (key._2, elem))
|
||||||
deltaReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ addBinding (key._2, elem))
|
deltaReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ addBindingBy (key._2, elem))
|
||||||
}
|
}
|
||||||
case RemoveNoVD(key, elem, consistency) ⇒
|
case RemoveNoVD(key, elem, _) ⇒
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
fullStateReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ removeBinding (key._2, elem))
|
fullStateReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ removeBindingBy (key._2, elem))
|
||||||
deltaReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ removeBinding (key._2, elem))
|
deltaReplicator ! Update(key._1, ORMultiMap.empty[String, String], WriteLocal)(_ removeBindingBy (key._2, elem))
|
||||||
}
|
}
|
||||||
case AddOM(key, elem, consistency) ⇒
|
case AddOM(key, elem, _) ⇒
|
||||||
// to have an deterministic result when mixing add/remove we can only perform
|
// to have an deterministic result when mixing add/remove we can only perform
|
||||||
// the ORSet operations from one node
|
// the ORSet operations from one node
|
||||||
runOn((if (key == KeyL) List(first) else List(first, second, third)): _*) {
|
runOn((if (key == KeyL) List(first) else List(first, second, third)): _*) {
|
||||||
fullStateReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(om ⇒ addElementToORMap(om, key._2, elem))
|
fullStateReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(om ⇒ addElementToORMap(om, key._2, elem))
|
||||||
deltaReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(om ⇒ addElementToORMap(om, key._2, elem))
|
deltaReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(om ⇒ addElementToORMap(om, key._2, elem))
|
||||||
}
|
}
|
||||||
case RemoveOM(key, elem, consistency) ⇒
|
case RemoveOM(key, elem, _) ⇒
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
fullStateReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(om ⇒ removeElementFromORMap(om, key._2, elem))
|
fullStateReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(om ⇒ removeElementFromORMap(om, key._2, elem))
|
||||||
deltaReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(om ⇒ removeElementFromORMap(om, key._2, elem))
|
deltaReplicator ! Update(key._1, ORMap.empty[String, ORSet[String]], WriteLocal)(om ⇒ removeElementFromORMap(om, key._2, elem))
|
||||||
|
|
|
||||||
|
|
@ -42,7 +42,8 @@ class ReplicatorORSetDeltaSpec extends MultiNodeSpec(ReplicatorORSetDeltaSpec) w
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val replicator = system.actorOf(Replicator.props(
|
val replicator = system.actorOf(Replicator.props(
|
||||||
ReplicatorSettings(system).withGossipInterval(1.second)), "replicator")
|
ReplicatorSettings(system).withGossipInterval(1.second)), "replicator")
|
||||||
val timeout = 3.seconds.dilated
|
val timeout = 3.seconds.dilated
|
||||||
|
|
@ -88,7 +89,7 @@ class ReplicatorORSetDeltaSpec extends MultiNodeSpec(ReplicatorORSetDeltaSpec) w
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "a")
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ :+ "a")
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -108,11 +109,11 @@ class ReplicatorORSetDeltaSpec extends MultiNodeSpec(ReplicatorORSetDeltaSpec) w
|
||||||
enterBarrier("split")
|
enterBarrier("split")
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "b")
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ :+ "b")
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
}
|
}
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "d")
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ :+ "d")
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
}
|
}
|
||||||
runOn(first, second) {
|
runOn(first, second) {
|
||||||
|
|
@ -129,7 +130,7 @@ class ReplicatorORSetDeltaSpec extends MultiNodeSpec(ReplicatorORSetDeltaSpec) w
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
// delta for "c" will be sent to third, but it has not received the previous delta for "b"
|
// delta for "c" will be sent to third, but it has not received the previous delta for "b"
|
||||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "c")
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ :+ "c")
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
// let the delta be propagated (will not fail if it takes longer)
|
// let the delta be propagated (will not fail if it takes longer)
|
||||||
Thread.sleep(1000)
|
Thread.sleep(1000)
|
||||||
|
|
@ -154,7 +155,7 @@ class ReplicatorORSetDeltaSpec extends MultiNodeSpec(ReplicatorORSetDeltaSpec) w
|
||||||
|
|
||||||
// and now the delta seqNr should be in sync again
|
// and now the delta seqNr should be in sync again
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ + "e")
|
replicator ! Update(KeyA, ORSet.empty[String], WriteLocal)(_ :+ "e")
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
}
|
}
|
||||||
assertValue(KeyA, Set("a", "b", "c", "d", "e"))
|
assertValue(KeyA, Set("a", "b", "c", "d", "e"))
|
||||||
|
|
|
||||||
|
|
@ -38,7 +38,8 @@ class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with ST
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val maxPruningDissemination = 3.seconds
|
val maxPruningDissemination = 3.seconds
|
||||||
val replicator = system.actorOf(Replicator.props(
|
val replicator = system.actorOf(Replicator.props(
|
||||||
ReplicatorSettings(system).withGossipInterval(1.second)
|
ReplicatorSettings(system).withGossipInterval(1.second)
|
||||||
|
|
@ -83,19 +84,19 @@ class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with ST
|
||||||
member.uniqueAddress
|
member.uniqueAddress
|
||||||
}
|
}
|
||||||
|
|
||||||
replicator ! Update(KeyA, GCounter(), WriteAll(timeout))(_ + 3)
|
replicator ! Update(KeyA, GCounter(), WriteAll(timeout))(_ :+ 3)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
|
|
||||||
replicator ! Update(KeyB, ORSet(), WriteAll(timeout))(_ + "a" + "b" + "c")
|
replicator ! Update(KeyB, ORSet(), WriteAll(timeout))(_ :+ "a" :+ "b" :+ "c")
|
||||||
expectMsg(UpdateSuccess(KeyB, None))
|
expectMsg(UpdateSuccess(KeyB, None))
|
||||||
|
|
||||||
replicator ! Update(KeyC, PNCounterMap.empty[String], WriteAll(timeout)) { _ increment "x" increment "y" }
|
replicator ! Update(KeyC, PNCounterMap.empty[String], WriteAll(timeout)) { _.incrementBy("x", 1).incrementBy("y", 1) }
|
||||||
expectMsg(UpdateSuccess(KeyC, None))
|
expectMsg(UpdateSuccess(KeyC, None))
|
||||||
|
|
||||||
replicator ! Update(KeyD, ORMultiMap.empty[String, String], WriteAll(timeout)) { _ + ("a" → Set("A")) }
|
replicator ! Update(KeyD, ORMultiMap.empty[String, String], WriteAll(timeout)) { _ :+ ("a" → Set("A")) }
|
||||||
expectMsg(UpdateSuccess(KeyD, None))
|
expectMsg(UpdateSuccess(KeyD, None))
|
||||||
|
|
||||||
replicator ! Update(KeyE, ORMap.empty[String, GSet[String]], WriteAll(timeout)) { _ + ("a" → GSet.empty[String].add("A")) }
|
replicator ! Update(KeyE, ORMap.empty[String, GSet[String]], WriteAll(timeout)) { _ :+ ("a" → GSet.empty[String].add("A")) }
|
||||||
expectMsg(UpdateSuccess(KeyE, None))
|
expectMsg(UpdateSuccess(KeyE, None))
|
||||||
|
|
||||||
enterBarrier("updates-done")
|
enterBarrier("updates-done")
|
||||||
|
|
@ -125,7 +126,7 @@ class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with ST
|
||||||
enterBarrier("get-old")
|
enterBarrier("get-old")
|
||||||
|
|
||||||
runOn(third) {
|
runOn(third) {
|
||||||
replicator ! Update(KeyE, ORMap.empty[String, GSet[String]], WriteLocal) { _ - "a" }
|
replicator ! Update(KeyE, ORMap.empty[String, GSet[String]], WriteLocal) { _ remove "a" }
|
||||||
expectMsg(UpdateSuccess(KeyE, None))
|
expectMsg(UpdateSuccess(KeyE, None))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -206,7 +207,7 @@ class ReplicatorPruningSpec extends MultiNodeSpec(ReplicatorPruningSpec) with ST
|
||||||
def updateAfterPruning(expectedValue: Int): Unit = {
|
def updateAfterPruning(expectedValue: Int): Unit = {
|
||||||
replicator ! Update(KeyA, GCounter(), WriteAll(timeout), None) { existing ⇒
|
replicator ! Update(KeyA, GCounter(), WriteAll(timeout), None) { existing ⇒
|
||||||
// inject data from removed node to simulate bad data
|
// inject data from removed node to simulate bad data
|
||||||
existing.merge(oldCounter) + 1
|
existing.merge(oldCounter) :+ 1
|
||||||
}
|
}
|
||||||
expectMsgPF() {
|
expectMsgPF() {
|
||||||
case UpdateSuccess(KeyA, _) ⇒
|
case UpdateSuccess(KeyA, _) ⇒
|
||||||
|
|
|
||||||
|
|
@ -40,7 +40,8 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
|
|
||||||
override def initialParticipants = roles.size
|
override def initialParticipants = roles.size
|
||||||
|
|
||||||
implicit val cluster = Cluster(system)
|
val cluster = Cluster(system)
|
||||||
|
implicit val selfUniqueAddress = DistributedData(system).selfUniqueAddress
|
||||||
val replicator = system.actorOf(Replicator.props(
|
val replicator = system.actorOf(Replicator.props(
|
||||||
ReplicatorSettings(system).withGossipInterval(1.second).withMaxDeltaElements(10)), "replicator")
|
ReplicatorSettings(system).withGossipInterval(1.second).withMaxDeltaElements(10)), "replicator")
|
||||||
val timeout = 3.seconds.dilated
|
val timeout = 3.seconds.dilated
|
||||||
|
|
@ -100,8 +101,8 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
replicator ! Get(KeyA, ReadLocal)
|
replicator ! Get(KeyA, ReadLocal)
|
||||||
expectMsg(NotFound(KeyA, None))
|
expectMsg(NotFound(KeyA, None))
|
||||||
|
|
||||||
val c3 = GCounter() + 3
|
val c3 = GCounter() :+ 3
|
||||||
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ + 3)
|
replicator ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 3)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
replicator ! Get(KeyA, ReadLocal)
|
replicator ! Get(KeyA, ReadLocal)
|
||||||
expectMsg(GetSuccess(KeyA, None)(c3)).dataValue should be(c3)
|
expectMsg(GetSuccess(KeyA, None)(c3)).dataValue should be(c3)
|
||||||
|
|
@ -111,31 +112,31 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
replicator ! Subscribe(KeyA, changedProbe2.ref)
|
replicator ! Subscribe(KeyA, changedProbe2.ref)
|
||||||
changedProbe2.expectMsg(Changed(KeyA)(c3)).dataValue should be(c3)
|
changedProbe2.expectMsg(Changed(KeyA)(c3)).dataValue should be(c3)
|
||||||
|
|
||||||
val c4 = c3 + 1
|
val c4 = c3 :+ 1
|
||||||
// too strong consistency level
|
// too strong consistency level
|
||||||
replicator ! Update(KeyA, GCounter(), writeTwo)(_ + 1)
|
replicator ! Update(KeyA, GCounter(), writeTwo)(_ :+ 1)
|
||||||
expectMsg(timeout + 1.second, UpdateTimeout(KeyA, None))
|
expectMsg(timeout + 1.second, UpdateTimeout(KeyA, None))
|
||||||
replicator ! Get(KeyA, ReadLocal)
|
replicator ! Get(KeyA, ReadLocal)
|
||||||
expectMsg(GetSuccess(KeyA, None)(c4)).dataValue should be(c4)
|
expectMsg(GetSuccess(KeyA, None)(c4)).dataValue should be(c4)
|
||||||
changedProbe.expectMsg(Changed(KeyA)(c4)).dataValue should be(c4)
|
changedProbe.expectMsg(Changed(KeyA)(c4)).dataValue should be(c4)
|
||||||
|
|
||||||
val c5 = c4 + 1
|
val c5 = c4 :+ 1
|
||||||
// too strong consistency level
|
// too strong consistency level
|
||||||
replicator ! Update(KeyA, GCounter(), writeMajority)(_ + 1)
|
replicator ! Update(KeyA, GCounter(), writeMajority)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
replicator ! Get(KeyA, readMajority)
|
replicator ! Get(KeyA, readMajority)
|
||||||
expectMsg(GetSuccess(KeyA, None)(c5)).dataValue should be(c5)
|
expectMsg(GetSuccess(KeyA, None)(c5)).dataValue should be(c5)
|
||||||
changedProbe.expectMsg(Changed(KeyA)(c5)).dataValue should be(c5)
|
changedProbe.expectMsg(Changed(KeyA)(c5)).dataValue should be(c5)
|
||||||
|
|
||||||
val c6 = c5 + 1
|
val c6 = c5 :+ 1
|
||||||
replicator ! Update(KeyA, GCounter(), writeAll)(_ + 1)
|
replicator ! Update(KeyA, GCounter(), writeAll)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyA, None))
|
expectMsg(UpdateSuccess(KeyA, None))
|
||||||
replicator ! Get(KeyA, readAll)
|
replicator ! Get(KeyA, readAll)
|
||||||
expectMsg(GetSuccess(KeyA, None)(c6)).dataValue should be(c6)
|
expectMsg(GetSuccess(KeyA, None)(c6)).dataValue should be(c6)
|
||||||
changedProbe.expectMsg(Changed(KeyA)(c6)).dataValue should be(c6)
|
changedProbe.expectMsg(Changed(KeyA)(c6)).dataValue should be(c6)
|
||||||
|
|
||||||
val c9 = GCounter() + 9
|
val c9 = GCounter() :+ 9
|
||||||
replicator ! Update(KeyX, GCounter(), WriteLocal)(_ + 9)
|
replicator ! Update(KeyX, GCounter(), WriteLocal)(_ :+ 9)
|
||||||
expectMsg(UpdateSuccess(KeyX, None))
|
expectMsg(UpdateSuccess(KeyX, None))
|
||||||
changedProbe.expectMsg(Changed(KeyX)(c9)).dataValue should be(c9)
|
changedProbe.expectMsg(Changed(KeyX)(c9)).dataValue should be(c9)
|
||||||
replicator ! Delete(KeyX, WriteLocal, Some(777))
|
replicator ! Delete(KeyX, WriteLocal, Some(777))
|
||||||
|
|
@ -145,7 +146,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
expectMsg(DataDeleted(KeyX, Some(789)))
|
expectMsg(DataDeleted(KeyX, Some(789)))
|
||||||
replicator ! Get(KeyX, readAll, Some(456))
|
replicator ! Get(KeyX, readAll, Some(456))
|
||||||
expectMsg(DataDeleted(KeyX, Some(456)))
|
expectMsg(DataDeleted(KeyX, Some(456)))
|
||||||
replicator ! Update(KeyX, GCounter(), WriteLocal, Some(123))(_ + 1)
|
replicator ! Update(KeyX, GCounter(), WriteLocal, Some(123))(_ :+ 1)
|
||||||
expectMsg(DataDeleted(KeyX, Some(123)))
|
expectMsg(DataDeleted(KeyX, Some(123)))
|
||||||
replicator ! Delete(KeyX, WriteLocal, Some(555))
|
replicator ! Delete(KeyX, WriteLocal, Some(555))
|
||||||
expectMsg(DataDeleted(KeyX, Some(555)))
|
expectMsg(DataDeleted(KeyX, Some(555)))
|
||||||
|
|
@ -214,11 +215,11 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
|
|
||||||
runOn(first, second) {
|
runOn(first, second) {
|
||||||
// start with 20 on both nodes
|
// start with 20 on both nodes
|
||||||
replicator ! Update(KeyB, GCounter(), WriteLocal)(_ + 20)
|
replicator ! Update(KeyB, GCounter(), WriteLocal)(_ :+ 20)
|
||||||
expectMsg(UpdateSuccess(KeyB, None))
|
expectMsg(UpdateSuccess(KeyB, None))
|
||||||
|
|
||||||
// add 1 on both nodes using WriteTwo
|
// add 1 on both nodes using WriteTwo
|
||||||
replicator ! Update(KeyB, GCounter(), writeTwo)(_ + 1)
|
replicator ! Update(KeyB, GCounter(), writeTwo)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyB, None))
|
expectMsg(UpdateSuccess(KeyB, None))
|
||||||
|
|
||||||
// the total, after replication should be 42
|
// the total, after replication should be 42
|
||||||
|
|
@ -232,7 +233,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
|
|
||||||
runOn(first, second) {
|
runOn(first, second) {
|
||||||
// add 1 on both nodes using WriteAll
|
// add 1 on both nodes using WriteAll
|
||||||
replicator ! Update(KeyB, GCounter(), writeAll)(_ + 1)
|
replicator ! Update(KeyB, GCounter(), writeAll)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyB, None))
|
expectMsg(UpdateSuccess(KeyB, None))
|
||||||
|
|
||||||
// the total, after replication should be 44
|
// the total, after replication should be 44
|
||||||
|
|
@ -246,7 +247,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
|
|
||||||
runOn(first, second) {
|
runOn(first, second) {
|
||||||
// add 1 on both nodes using WriteMajority
|
// add 1 on both nodes using WriteMajority
|
||||||
replicator ! Update(KeyB, GCounter(), writeMajority)(_ + 1)
|
replicator ! Update(KeyB, GCounter(), writeMajority)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyB, None))
|
expectMsg(UpdateSuccess(KeyB, None))
|
||||||
|
|
||||||
// the total, after replication should be 46
|
// the total, after replication should be 46
|
||||||
|
|
@ -267,14 +268,14 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
replicator ! Update(KeyC, GCounter(), writeTwo)(_ + 30)
|
replicator ! Update(KeyC, GCounter(), writeTwo)(_ :+ 30)
|
||||||
expectMsg(UpdateSuccess(KeyC, None))
|
expectMsg(UpdateSuccess(KeyC, None))
|
||||||
changedProbe.expectMsgPF() { case c @ Changed(KeyC) ⇒ c.get(KeyC).value } should be(30)
|
changedProbe.expectMsgPF() { case c @ Changed(KeyC) ⇒ c.get(KeyC).value } should be(30)
|
||||||
|
|
||||||
replicator ! Update(KeyY, GCounter(), writeTwo)(_ + 30)
|
replicator ! Update(KeyY, GCounter(), writeTwo)(_ :+ 30)
|
||||||
expectMsg(UpdateSuccess(KeyY, None))
|
expectMsg(UpdateSuccess(KeyY, None))
|
||||||
|
|
||||||
replicator ! Update(KeyZ, GCounter(), writeMajority)(_ + 30)
|
replicator ! Update(KeyZ, GCounter(), writeMajority)(_ :+ 30)
|
||||||
expectMsg(UpdateSuccess(KeyZ, None))
|
expectMsg(UpdateSuccess(KeyZ, None))
|
||||||
}
|
}
|
||||||
enterBarrier("update-c30")
|
enterBarrier("update-c30")
|
||||||
|
|
@ -286,7 +287,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
changedProbe.expectMsgPF() { case c @ Changed(KeyC) ⇒ c.get(KeyC).value } should be(30)
|
changedProbe.expectMsgPF() { case c @ Changed(KeyC) ⇒ c.get(KeyC).value } should be(30)
|
||||||
|
|
||||||
// replicate with gossip after WriteLocal
|
// replicate with gossip after WriteLocal
|
||||||
replicator ! Update(KeyC, GCounter(), WriteLocal)(_ + 1)
|
replicator ! Update(KeyC, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyC, None))
|
expectMsg(UpdateSuccess(KeyC, None))
|
||||||
changedProbe.expectMsgPF() { case c @ Changed(KeyC) ⇒ c.get(KeyC).value } should be(31)
|
changedProbe.expectMsgPF() { case c @ Changed(KeyC) ⇒ c.get(KeyC).value } should be(31)
|
||||||
|
|
||||||
|
|
@ -320,7 +321,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
val c31 = expectMsgPF() { case g @ GetSuccess(KeyC, _) ⇒ g.get(KeyC) }
|
val c31 = expectMsgPF() { case g @ GetSuccess(KeyC, _) ⇒ g.get(KeyC) }
|
||||||
c31.value should be(31)
|
c31.value should be(31)
|
||||||
|
|
||||||
replicator ! Update(KeyC, GCounter(), WriteLocal)(_ + 1)
|
replicator ! Update(KeyC, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyC, None))
|
expectMsg(UpdateSuccess(KeyC, None))
|
||||||
|
|
||||||
within(5.seconds) {
|
within(5.seconds) {
|
||||||
|
|
@ -337,7 +338,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
|
|
||||||
"converge after partition" in {
|
"converge after partition" in {
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
replicator ! Update(KeyD, GCounter(), writeTwo)(_ + 40)
|
replicator ! Update(KeyD, GCounter(), writeTwo)(_ :+ 40)
|
||||||
expectMsg(UpdateSuccess(KeyD, None))
|
expectMsg(UpdateSuccess(KeyD, None))
|
||||||
|
|
||||||
testConductor.blackhole(first, second, Direction.Both).await
|
testConductor.blackhole(first, second, Direction.Both).await
|
||||||
|
|
@ -348,15 +349,15 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
replicator ! Get(KeyD, ReadLocal)
|
replicator ! Get(KeyD, ReadLocal)
|
||||||
val c40 = expectMsgPF() { case g @ GetSuccess(KeyD, _) ⇒ g.get(KeyD) }
|
val c40 = expectMsgPF() { case g @ GetSuccess(KeyD, _) ⇒ g.get(KeyD) }
|
||||||
c40.value should be(40)
|
c40.value should be(40)
|
||||||
replicator ! Update(KeyD, GCounter() + 1, writeTwo)(_ + 1)
|
replicator ! Update(KeyD, GCounter() :+ 1, writeTwo)(_ :+ 1)
|
||||||
expectMsg(timeout + 1.second, UpdateTimeout(KeyD, None))
|
expectMsg(timeout + 1.second, UpdateTimeout(KeyD, None))
|
||||||
replicator ! Update(KeyD, GCounter(), writeTwo)(_ + 1)
|
replicator ! Update(KeyD, GCounter(), writeTwo)(_ :+ 1)
|
||||||
expectMsg(timeout + 1.second, UpdateTimeout(KeyD, None))
|
expectMsg(timeout + 1.second, UpdateTimeout(KeyD, None))
|
||||||
}
|
}
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
for (n ← 1 to 30) {
|
for (n ← 1 to 30) {
|
||||||
val KeyDn = GCounterKey("D" + n)
|
val KeyDn = GCounterKey("D" + n)
|
||||||
replicator ! Update(KeyDn, GCounter(), WriteLocal)(_ + n)
|
replicator ! Update(KeyDn, GCounter(), WriteLocal)(_ :+ n)
|
||||||
expectMsg(UpdateSuccess(KeyDn, None))
|
expectMsg(UpdateSuccess(KeyDn, None))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -400,7 +401,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
enterBarrier("3-nodes")
|
enterBarrier("3-nodes")
|
||||||
|
|
||||||
runOn(first, second, third) {
|
runOn(first, second, third) {
|
||||||
replicator ! Update(KeyE, GCounter(), writeMajority)(_ + 50)
|
replicator ! Update(KeyE, GCounter(), writeMajority)(_ :+ 50)
|
||||||
expectMsg(UpdateSuccess(KeyE, None))
|
expectMsg(UpdateSuccess(KeyE, None))
|
||||||
}
|
}
|
||||||
enterBarrier("write-initial-majority")
|
enterBarrier("write-initial-majority")
|
||||||
|
|
@ -419,7 +420,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
enterBarrier("blackhole-third")
|
enterBarrier("blackhole-third")
|
||||||
|
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
replicator ! Update(KeyE, GCounter(), WriteLocal)(_ + 1)
|
replicator ! Update(KeyE, GCounter(), WriteLocal)(_ :+ 1)
|
||||||
expectMsg(UpdateSuccess(KeyE, None))
|
expectMsg(UpdateSuccess(KeyE, None))
|
||||||
}
|
}
|
||||||
enterBarrier("local-update-from-second")
|
enterBarrier("local-update-from-second")
|
||||||
|
|
@ -432,7 +433,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
probe2.expectMsgType[GetSuccess[_]]
|
probe2.expectMsgType[GetSuccess[_]]
|
||||||
replicator.tell(Update(KeyE, GCounter(), writeMajority, None) { data ⇒
|
replicator.tell(Update(KeyE, GCounter(), writeMajority, None) { data ⇒
|
||||||
probe1.ref ! data.value
|
probe1.ref ! data.value
|
||||||
data + 1
|
data :+ 1
|
||||||
}, probe2.ref)
|
}, probe2.ref)
|
||||||
// verify read your own writes, without waiting for the UpdateSuccess reply
|
// verify read your own writes, without waiting for the UpdateSuccess reply
|
||||||
// note that the order of the replies are not defined, and therefore we use separate probes
|
// note that the order of the replies are not defined, and therefore we use separate probes
|
||||||
|
|
@ -449,13 +450,13 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
val probe1 = TestProbe()
|
val probe1 = TestProbe()
|
||||||
replicator.tell(Get(KeyE, readMajority), probe1.ref)
|
replicator.tell(Get(KeyE, readMajority), probe1.ref)
|
||||||
probe1.expectMsgType[GetSuccess[_]]
|
probe1.expectMsgType[GetSuccess[_]]
|
||||||
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(153))(_ + 1), probe1.ref)
|
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(153))(_ :+ 1), probe1.ref)
|
||||||
// verify read your own writes, without waiting for the UpdateSuccess reply
|
// verify read your own writes, without waiting for the UpdateSuccess reply
|
||||||
// note that the order of the replies are not defined, and therefore we use separate probes
|
// note that the order of the replies are not defined, and therefore we use separate probes
|
||||||
val probe2 = TestProbe()
|
val probe2 = TestProbe()
|
||||||
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(154))(_ + 1), probe2.ref)
|
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(154))(_ :+ 1), probe2.ref)
|
||||||
val probe3 = TestProbe()
|
val probe3 = TestProbe()
|
||||||
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(155))(_ + 1), probe3.ref)
|
replicator.tell(Update(KeyE, GCounter(), writeMajority, Some(155))(_ :+ 1), probe3.ref)
|
||||||
val probe5 = TestProbe()
|
val probe5 = TestProbe()
|
||||||
replicator.tell(Get(KeyE, readMajority), probe5.ref)
|
replicator.tell(Get(KeyE, readMajority), probe5.ref)
|
||||||
probe1.expectMsg(UpdateSuccess(KeyE, Some(153)))
|
probe1.expectMsg(UpdateSuccess(KeyE, Some(153)))
|
||||||
|
|
@ -492,9 +493,9 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
"converge after many concurrent updates" in within(10.seconds) {
|
"converge after many concurrent updates" in within(10.seconds) {
|
||||||
runOn(first, second, third) {
|
runOn(first, second, third) {
|
||||||
var c = GCounter()
|
var c = GCounter()
|
||||||
for (i ← 0 until 100) {
|
for (_ ← 0 until 100) {
|
||||||
c += 1
|
c :+= 1
|
||||||
replicator ! Update(KeyF, GCounter(), writeTwo)(_ + 1)
|
replicator ! Update(KeyF, GCounter(), writeTwo)(_ :+ 1)
|
||||||
}
|
}
|
||||||
val results = receiveN(100)
|
val results = receiveN(100)
|
||||||
results.map(_.getClass).toSet should be(Set(classOf[UpdateSuccess[_]]))
|
results.map(_.getClass).toSet should be(Set(classOf[UpdateSuccess[_]]))
|
||||||
|
|
@ -510,7 +511,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
|
|
||||||
"read-repair happens before GetSuccess" in {
|
"read-repair happens before GetSuccess" in {
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
replicator ! Update(KeyG, ORSet(), writeTwo)(_ + "a" + "b")
|
replicator ! Update(KeyG, ORSet(), writeTwo)(_ :+ "a" :+ "b")
|
||||||
expectMsgType[UpdateSuccess[_]]
|
expectMsgType[UpdateSuccess[_]]
|
||||||
}
|
}
|
||||||
enterBarrier("a-b-added-to-G")
|
enterBarrier("a-b-added-to-G")
|
||||||
|
|
@ -528,20 +529,20 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
|
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
replicator ! Subscribe(KeyH, changedProbe.ref)
|
replicator ! Subscribe(KeyH, changedProbe.ref)
|
||||||
replicator ! Update(KeyH, ORMap.empty[String, Flag], writeTwo)(_ + ("a" → Flag.Disabled))
|
replicator ! Update(KeyH, ORMap.empty[String, Flag], writeTwo)(_ :+ ("a" → Flag.Disabled))
|
||||||
changedProbe.expectMsgPF() { case c @ Changed(KeyH) ⇒ c.get(KeyH).entries } should be(Map("a" → Flag.Disabled))
|
changedProbe.expectMsgPF() { case c @ Changed(KeyH) ⇒ c.get(KeyH).entries } should be(Map("a" → Flag.Disabled))
|
||||||
}
|
}
|
||||||
|
|
||||||
enterBarrier("update-h1")
|
enterBarrier("update-h1")
|
||||||
|
|
||||||
runOn(first) {
|
runOn(first) {
|
||||||
replicator ! Update(KeyH, ORMap.empty[String, Flag], writeTwo)(_ + ("a" → Flag.Enabled))
|
replicator ! Update(KeyH, ORMap.empty[String, Flag], writeTwo)(_ :+ ("a" → Flag.Enabled))
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
changedProbe.expectMsgPF() { case c @ Changed(KeyH) ⇒ c.get(KeyH).entries } should be(Map("a" → Flag.Enabled))
|
changedProbe.expectMsgPF() { case c @ Changed(KeyH) ⇒ c.get(KeyH).entries } should be(Map("a" → Flag.Enabled))
|
||||||
|
|
||||||
replicator ! Update(KeyH, ORMap.empty[String, Flag], writeTwo)(_ + ("b" → Flag.Enabled))
|
replicator ! Update(KeyH, ORMap.empty[String, Flag], writeTwo)(_ :+ ("b" → Flag.Enabled))
|
||||||
changedProbe.expectMsgPF() { case c @ Changed(KeyH) ⇒ c.get(KeyH).entries } should be(
|
changedProbe.expectMsgPF() { case c @ Changed(KeyH) ⇒ c.get(KeyH).entries } should be(
|
||||||
Map("a" → Flag.Enabled, "b" → Flag.Enabled))
|
Map("a" → Flag.Enabled, "b" → Flag.Enabled))
|
||||||
}
|
}
|
||||||
|
|
@ -568,7 +569,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
|
||||||
replicator ! Update(KeyI, GSet.empty[String], writeTwo)(_ + "a")
|
replicator ! Update(KeyI, GSet.empty[String], writeTwo)(_ + "a")
|
||||||
}
|
}
|
||||||
|
|
||||||
changedProbe.expectNoMsg(1.second)
|
changedProbe.expectNoMessage(1.second)
|
||||||
|
|
||||||
enterBarrierAfterTestStep()
|
enterBarrierAfterTestStep()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,13 +4,11 @@
|
||||||
|
|
||||||
package akka.cluster.ddata;
|
package akka.cluster.ddata;
|
||||||
|
|
||||||
import akka.cluster.Cluster;
|
|
||||||
|
|
||||||
public class ORMapTest {
|
public class ORMapTest {
|
||||||
|
|
||||||
public void compileOnlyORMapTest() {
|
public void compileOnlyORMapTest() {
|
||||||
// primarily to check API accessibility with overloads/types
|
// primarily to check API accessibility with overloads/types
|
||||||
Cluster node1 = null;
|
SelfUniqueAddress node1 = null;
|
||||||
|
|
||||||
ORMap<String, PNCounterMap<String>> orMap = ORMap.create();
|
ORMap<String, PNCounterMap<String>> orMap = ORMap.create();
|
||||||
// updated needs a cast
|
// updated needs a cast
|
||||||
|
|
|
||||||
|
|
@ -4,13 +4,11 @@
|
||||||
|
|
||||||
package akka.cluster.ddata;
|
package akka.cluster.ddata;
|
||||||
|
|
||||||
import akka.cluster.Cluster;
|
|
||||||
|
|
||||||
public class ORMultiMapTest {
|
public class ORMultiMapTest {
|
||||||
|
|
||||||
public void compileOnlyORMultiMapTest() {
|
public void compileOnlyORMultiMapTest() {
|
||||||
// primarily to check API accessibility with overloads/types
|
// primarily to check API accessibility with overloads/types
|
||||||
Cluster node = null;
|
SelfUniqueAddress node = null;
|
||||||
ORMultiMap<String, String> orMultiMap = ORMultiMap.create();
|
ORMultiMap<String, String> orMultiMap = ORMultiMap.create();
|
||||||
orMultiMap.addBinding(node, "a", "1");
|
orMultiMap.addBinding(node, "a", "1");
|
||||||
orMultiMap.removeBinding(node, "a", "1");
|
orMultiMap.removeBinding(node, "a", "1");
|
||||||
|
|
|
||||||
|
|
@ -4,8 +4,6 @@
|
||||||
|
|
||||||
package akka.cluster.ddata;
|
package akka.cluster.ddata;
|
||||||
|
|
||||||
import akka.cluster.Cluster;
|
|
||||||
|
|
||||||
import java.math.BigInteger;
|
import java.math.BigInteger;
|
||||||
|
|
||||||
public class PNCounterTest {
|
public class PNCounterTest {
|
||||||
|
|
@ -13,8 +11,8 @@ public class PNCounterTest {
|
||||||
|
|
||||||
public void compileOnlyPNCounterApiTest() {
|
public void compileOnlyPNCounterApiTest() {
|
||||||
// primarily to check API accessibility with overloads/types
|
// primarily to check API accessibility with overloads/types
|
||||||
Cluster node1 = null;
|
SelfUniqueAddress node1 = null;
|
||||||
Cluster node2 = null;
|
SelfUniqueAddress node2 = null;
|
||||||
|
|
||||||
PNCounter c1 = PNCounter.create();
|
PNCounter c1 = PNCounter.create();
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -56,17 +56,19 @@ class LWWRegisterSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"use monotonically increasing defaultClock" in {
|
"use monotonically increasing defaultClock" in {
|
||||||
(1 to 100).foldLeft(LWWRegister(node1, 0, defaultClock)) {
|
implicit val node = SelfUniqueAddress(node1)
|
||||||
|
|
||||||
|
(1 to 100).foldLeft(LWWRegister.create(0)) {
|
||||||
case (r, n) ⇒
|
case (r, n) ⇒
|
||||||
r.value should be(n - 1)
|
r.value should be(n - 1)
|
||||||
val r2 = r.withValue(node1, n, defaultClock[Int])
|
val r2 = r.withValueOf(n)
|
||||||
r2.timestamp should be > r.timestamp
|
r2.timestamp should be > r.timestamp
|
||||||
r2
|
r2
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"have unapply extractor" in {
|
"have unapply extractor" in {
|
||||||
val r1 = LWWRegister(node1, "a", defaultClock)
|
val r1 = LWWRegister(node1, "a", defaultClock[String])
|
||||||
val LWWRegister(value1) = r1
|
val LWWRegister(value1) = r1
|
||||||
val value2: String = value1
|
val value2: String = value1
|
||||||
Changed(LWWRegisterKey[String]("key"))(r1) match {
|
Changed(LWWRegisterKey[String]("key"))(r1) match {
|
||||||
|
|
|
||||||
|
|
@ -8,7 +8,6 @@ import akka.actor.Actor
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
import akka.actor.Stash
|
import akka.actor.Stash
|
||||||
import akka.cluster.Cluster
|
|
||||||
import akka.testkit.ImplicitSender
|
import akka.testkit.ImplicitSender
|
||||||
import akka.testkit.TestKit
|
import akka.testkit.TestKit
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
@ -25,12 +24,14 @@ object LocalConcurrencySpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
class Updater extends Actor with Stash {
|
class Updater extends Actor with Stash {
|
||||||
implicit val cluster = Cluster(context.system)
|
|
||||||
|
implicit val selfUniqueAddress = DistributedData(context.system).selfUniqueAddress
|
||||||
|
|
||||||
val replicator = DistributedData(context.system).replicator
|
val replicator = DistributedData(context.system).replicator
|
||||||
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case s: String ⇒
|
case s: String ⇒
|
||||||
val update = Replicator.Update(Updater.key, ORSet.empty[String], Replicator.WriteLocal)(_ + s)
|
val update = Replicator.Update(Updater.key, ORSet.empty[String], Replicator.WriteLocal)(_ :+ s)
|
||||||
replicator ! update
|
replicator ! update
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,20 +4,14 @@
|
||||||
|
|
||||||
package akka.cluster.ddata
|
package akka.cluster.ddata
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import java.util.concurrent.ThreadLocalRandom
|
import java.util.concurrent.ThreadLocalRandom
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
import akka.actor.Actor
|
import akka.actor.Actor
|
||||||
import akka.actor.ActorLogging
|
import akka.actor.ActorLogging
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
import akka.cluster.Cluster
|
|
||||||
import akka.cluster.ddata.Replicator.Changed
|
|
||||||
import akka.cluster.ddata.Replicator.GetKeyIds
|
|
||||||
import akka.cluster.ddata.Replicator.GetKeyIdsResult
|
|
||||||
import akka.cluster.ddata.Replicator.Subscribe
|
|
||||||
import akka.cluster.ddata.Replicator.Update
|
|
||||||
import akka.cluster.ddata.Replicator.UpdateResponse
|
|
||||||
import akka.cluster.ddata.Replicator.WriteLocal
|
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -75,8 +69,8 @@ class LotsOfDataBot extends Actor with ActorLogging {
|
||||||
import LotsOfDataBot._
|
import LotsOfDataBot._
|
||||||
import Replicator._
|
import Replicator._
|
||||||
|
|
||||||
|
implicit val selfUniqueAddress = DistributedData(context.system).selfUniqueAddress
|
||||||
val replicator = DistributedData(context.system).replicator
|
val replicator = DistributedData(context.system).replicator
|
||||||
implicit val cluster = Cluster(context.system)
|
|
||||||
|
|
||||||
import context.dispatcher
|
import context.dispatcher
|
||||||
val isPassive = context.system.settings.config.getBoolean("passive")
|
val isPassive = context.system.settings.config.getBoolean("passive")
|
||||||
|
|
@ -110,10 +104,10 @@ class LotsOfDataBot extends Actor with ActorLogging {
|
||||||
val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString
|
val s = ThreadLocalRandom.current().nextInt(97, 123).toChar.toString
|
||||||
if (count <= maxEntries || ThreadLocalRandom.current().nextBoolean()) {
|
if (count <= maxEntries || ThreadLocalRandom.current().nextBoolean()) {
|
||||||
// add
|
// add
|
||||||
replicator ! Update(key, ORSet(), WriteLocal)(_ + s)
|
replicator ! Update(key, ORSet(), WriteLocal)(_ :+ s)
|
||||||
} else {
|
} else {
|
||||||
// remove
|
// remove
|
||||||
replicator ! Update(key, ORSet(), WriteLocal)(_ - s)
|
replicator ! Update(key, ORSet(), WriteLocal)(_ remove s)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,11 @@ class PNCounterMapSpec extends WordSpec with Matchers {
|
||||||
|
|
||||||
"A PNCounterMap" must {
|
"A PNCounterMap" must {
|
||||||
|
|
||||||
|
"be able to increment and decrement entries with implicit SelfUniqueAddress" in {
|
||||||
|
implicit val node = SelfUniqueAddress(node1)
|
||||||
|
PNCounterMap().incrementBy("a", 2).incrementBy("b", 1).incrementBy("b", 2).decrementBy("a", 1).entries should be(Map("a" → 1, "b" → 3))
|
||||||
|
}
|
||||||
|
|
||||||
"be able to increment and decrement entries" in {
|
"be able to increment and decrement entries" in {
|
||||||
val m = PNCounterMap().increment(node1, "a", 2).increment(node1, "b", 3).decrement(node2, "a", 1)
|
val m = PNCounterMap().increment(node1, "a", 2).increment(node1, "b", 3).decrement(node2, "a", 1)
|
||||||
m.entries should be(Map("a" → 1, "b" → 3))
|
m.entries should be(Map("a" → 1, "b" → 3))
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,31 @@
|
||||||
|
/*
|
||||||
|
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.cluster.ddata
|
||||||
|
|
||||||
|
import akka.testkit.AkkaSpec
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
import org.scalatest.{ BeforeAndAfterAll, WordSpecLike }
|
||||||
|
|
||||||
|
object ReplicatorSettingsSpec {
|
||||||
|
|
||||||
|
val config = ConfigFactory.parseString("""
|
||||||
|
akka.actor.provider = "cluster"
|
||||||
|
akka.remote.netty.tcp.port = 0
|
||||||
|
akka.remote.artery.canonical.port = 0
|
||||||
|
akka.remote.artery.canonical.hostname = 127.0.0.1""")
|
||||||
|
}
|
||||||
|
|
||||||
|
class ReplicatorSettingsSpec extends AkkaSpec(ReplicatorSettingsSpec.config)
|
||||||
|
with WordSpecLike with BeforeAndAfterAll {
|
||||||
|
|
||||||
|
"DistributedData" must {
|
||||||
|
"have the default replicator name" in {
|
||||||
|
ReplicatorSettings.name(system, None) should ===("ddataReplicator")
|
||||||
|
}
|
||||||
|
"have the prefixed replicator name" in {
|
||||||
|
ReplicatorSettings.name(system, Some("other")) should ===("otherDdataReplicator")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -186,7 +186,7 @@ class ReplicatedDataSerializerSpec extends TestKit(ActorSystem(
|
||||||
}
|
}
|
||||||
|
|
||||||
"serialize LWWRegister" in {
|
"serialize LWWRegister" in {
|
||||||
checkSerialization(LWWRegister(address1, "value1", LWWRegister.defaultClock))
|
checkSerialization(LWWRegister(address1, "value1", LWWRegister.defaultClock[String]))
|
||||||
checkSerialization(LWWRegister(address1, "value2", LWWRegister.defaultClock[String])
|
checkSerialization(LWWRegister(address1, "value2", LWWRegister.defaultClock[String])
|
||||||
.withValue(address2, "value3", LWWRegister.defaultClock[String]))
|
.withValue(address2, "value3", LWWRegister.defaultClock[String]))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -39,8 +39,8 @@ actor provides the API for interacting with the data and is accessed through the
|
||||||
|
|
||||||
The messages for the replicator, such as `Replicator.Update` are defined in @scala[`akka.cluster.ddata.typed.scaladsl.Replicator`]
|
The messages for the replicator, such as `Replicator.Update` are defined in @scala[`akka.cluster.ddata.typed.scaladsl.Replicator`]
|
||||||
@java[`akka.cluster.ddata.typed.scaladsl.Replicator`] but the actual CRDTs are the
|
@java[`akka.cluster.ddata.typed.scaladsl.Replicator`] but the actual CRDTs are the
|
||||||
same as in untyped, for example `akka.cluster.ddata.GCounter`. This will require an @scala[implicit] untyped `Cluster`
|
same as in untyped, for example `akka.cluster.ddata.GCounter`. This will require a @scala[implicit] `akka.cluster.ddata.SelfUniqueAddress.SelfUniqueAddress`,
|
||||||
for now, we hope to improve this in the future ([issue #25746](https://github.com/akka/akka/issues/25746)).
|
available from @scala[`implicit val node = DistributedData(system).selfUniqueAddress`]@java[SelfUniqueAddress node = DistributedData.get(system).selfUniqueAddress();].
|
||||||
|
|
||||||
The replicator can contain multiple entries each containing a replicated data type, we therefore need to create a
|
The replicator can contain multiple entries each containing a replicated data type, we therefore need to create a
|
||||||
key identifying the entry and helping us know what type it has, and then use that key for every interaction with
|
key identifying the entry and helping us know what type it has, and then use that key for every interaction with
|
||||||
|
|
|
||||||
|
|
@ -50,7 +50,7 @@ public class DistributedDataDocTest extends AbstractJavaTest {
|
||||||
static
|
static
|
||||||
//#update
|
//#update
|
||||||
class DemonstrateUpdate extends AbstractActor {
|
class DemonstrateUpdate extends AbstractActor {
|
||||||
final Cluster node = Cluster.get(getContext().getSystem());
|
final SelfUniqueAddress node = DistributedData.get(getContext().getSystem()).selfUniqueAddress();
|
||||||
final ActorRef replicator =
|
final ActorRef replicator =
|
||||||
DistributedData.get(getContext().getSystem()).replicator();
|
DistributedData.get(getContext().getSystem()).replicator();
|
||||||
|
|
||||||
|
|
@ -305,7 +305,7 @@ public class DistributedDataDocTest extends AbstractJavaTest {
|
||||||
|
|
||||||
public void demonstratePNCounter() {
|
public void demonstratePNCounter() {
|
||||||
//#pncounter
|
//#pncounter
|
||||||
final Cluster node = Cluster.get(system);
|
final SelfUniqueAddress node = DistributedData.get(system).selfUniqueAddress();
|
||||||
final PNCounter c0 = PNCounter.create();
|
final PNCounter c0 = PNCounter.create();
|
||||||
final PNCounter c1 = c0.increment(node, 1);
|
final PNCounter c1 = c0.increment(node, 1);
|
||||||
final PNCounter c2 = c1.increment(node, 7);
|
final PNCounter c2 = c1.increment(node, 7);
|
||||||
|
|
@ -316,7 +316,7 @@ public class DistributedDataDocTest extends AbstractJavaTest {
|
||||||
|
|
||||||
public void demonstratePNCounterMap() {
|
public void demonstratePNCounterMap() {
|
||||||
//#pncountermap
|
//#pncountermap
|
||||||
final Cluster node = Cluster.get(system);
|
final SelfUniqueAddress node = DistributedData.get(system).selfUniqueAddress();
|
||||||
final PNCounterMap<String> m0 = PNCounterMap.create();
|
final PNCounterMap<String> m0 = PNCounterMap.create();
|
||||||
final PNCounterMap<String> m1 = m0.increment(node, "a", 7);
|
final PNCounterMap<String> m1 = m0.increment(node, "a", 7);
|
||||||
final PNCounterMap<String> m2 = m1.decrement(node, "a", 2);
|
final PNCounterMap<String> m2 = m1.decrement(node, "a", 2);
|
||||||
|
|
@ -349,7 +349,7 @@ public class DistributedDataDocTest extends AbstractJavaTest {
|
||||||
|
|
||||||
public void demonstrateORMultiMap() {
|
public void demonstrateORMultiMap() {
|
||||||
//#ormultimap
|
//#ormultimap
|
||||||
final Cluster node = Cluster.get(system);
|
final SelfUniqueAddress node = DistributedData.get(system).selfUniqueAddress();
|
||||||
final ORMultiMap<String, Integer> m0 = ORMultiMap.create();
|
final ORMultiMap<String, Integer> m0 = ORMultiMap.create();
|
||||||
final ORMultiMap<String, Integer> m1 = m0.put(node, "a",
|
final ORMultiMap<String, Integer> m1 = m0.put(node, "a",
|
||||||
new HashSet<>(Arrays.asList(1, 2, 3)));
|
new HashSet<>(Arrays.asList(1, 2, 3)));
|
||||||
|
|
@ -371,7 +371,7 @@ public class DistributedDataDocTest extends AbstractJavaTest {
|
||||||
@Test
|
@Test
|
||||||
public void demonstrateLWWRegister() {
|
public void demonstrateLWWRegister() {
|
||||||
//#lwwregister
|
//#lwwregister
|
||||||
final Cluster node = Cluster.get(system);
|
final SelfUniqueAddress node = DistributedData.get(system).selfUniqueAddress();
|
||||||
final LWWRegister<String> r1 = LWWRegister.create(node, "Hello");
|
final LWWRegister<String> r1 = LWWRegister.create(node, "Hello");
|
||||||
final LWWRegister<String> r2 = r1.withValue(node, "Hi");
|
final LWWRegister<String> r2 = r1.withValue(node, "Hi");
|
||||||
System.out.println(r1.value() + " by " + r1.updatedBy() + " at " + r1.timestamp());
|
System.out.println(r1.value() + " by " + r1.updatedBy() + " at " + r1.timestamp());
|
||||||
|
|
@ -398,7 +398,7 @@ public class DistributedDataDocTest extends AbstractJavaTest {
|
||||||
public void demonstrateLWWRegisterWithCustomClock() {
|
public void demonstrateLWWRegisterWithCustomClock() {
|
||||||
//#lwwregister-custom-clock
|
//#lwwregister-custom-clock
|
||||||
|
|
||||||
final Cluster node = Cluster.get(system);
|
final SelfUniqueAddress node = DistributedData.get(system).selfUniqueAddress();
|
||||||
final LWWRegister.Clock<Record> recordClock = new LWWRegister.Clock<Record>() {
|
final LWWRegister.Clock<Record> recordClock = new LWWRegister.Clock<Record>() {
|
||||||
@Override
|
@Override
|
||||||
public long apply(long currentTimestamp, Record value) {
|
public long apply(long currentTimestamp, Record value) {
|
||||||
|
|
|
||||||
|
|
@ -13,12 +13,7 @@ import java.time.Duration;
|
||||||
import akka.actor.AbstractActor;
|
import akka.actor.AbstractActor;
|
||||||
import akka.actor.ActorRef;
|
import akka.actor.ActorRef;
|
||||||
import akka.actor.Props;
|
import akka.actor.Props;
|
||||||
import akka.cluster.Cluster;
|
import akka.cluster.ddata.*;
|
||||||
import akka.cluster.ddata.DistributedData;
|
|
||||||
import akka.cluster.ddata.Key;
|
|
||||||
import akka.cluster.ddata.LWWMap;
|
|
||||||
import akka.cluster.ddata.LWWMapKey;
|
|
||||||
import akka.cluster.ddata.Replicator;
|
|
||||||
import akka.cluster.ddata.Replicator.GetFailure;
|
import akka.cluster.ddata.Replicator.GetFailure;
|
||||||
import akka.cluster.ddata.Replicator.GetResponse;
|
import akka.cluster.ddata.Replicator.GetResponse;
|
||||||
import akka.cluster.ddata.Replicator.GetSuccess;
|
import akka.cluster.ddata.Replicator.GetSuccess;
|
||||||
|
|
@ -126,7 +121,7 @@ public class ShoppingCart extends AbstractActor {
|
||||||
}
|
}
|
||||||
|
|
||||||
private final ActorRef replicator = DistributedData.get(getContext().getSystem()).replicator();
|
private final ActorRef replicator = DistributedData.get(getContext().getSystem()).replicator();
|
||||||
private final Cluster node = Cluster.get(getContext().getSystem());
|
private final SelfUniqueAddress node = DistributedData.get(getContext().getSystem()).selfUniqueAddress();
|
||||||
|
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
private final String userId;
|
private final String userId;
|
||||||
|
|
|
||||||
|
|
@ -5,8 +5,8 @@
|
||||||
package docs.ddata
|
package docs.ddata
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
import akka.actor.Actor
|
import akka.actor.Actor
|
||||||
import akka.cluster.Cluster
|
|
||||||
import akka.cluster.ddata._
|
import akka.cluster.ddata._
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import akka.testkit.TestProbe
|
import akka.testkit.TestProbe
|
||||||
|
|
@ -63,7 +63,7 @@ object DistributedDataDocSpec {
|
||||||
import DataBot._
|
import DataBot._
|
||||||
|
|
||||||
val replicator = DistributedData(context.system).replicator
|
val replicator = DistributedData(context.system).replicator
|
||||||
implicit val node = Cluster(context.system)
|
implicit val node = DistributedData(context.system).selfUniqueAddress
|
||||||
|
|
||||||
import context.dispatcher
|
import context.dispatcher
|
||||||
val tickTask = context.system.scheduler.schedule(5.seconds, 5.seconds, self, Tick)
|
val tickTask = context.system.scheduler.schedule(5.seconds, 5.seconds, self, Tick)
|
||||||
|
|
@ -78,11 +78,11 @@ object DistributedDataDocSpec {
|
||||||
if (ThreadLocalRandom.current().nextBoolean()) {
|
if (ThreadLocalRandom.current().nextBoolean()) {
|
||||||
// add
|
// add
|
||||||
log.info("Adding: {}", s)
|
log.info("Adding: {}", s)
|
||||||
replicator ! Update(DataKey, ORSet.empty[String], WriteLocal)(_ + s)
|
replicator ! Update(DataKey, ORSet.empty[String], WriteLocal)(_ :+ s)
|
||||||
} else {
|
} else {
|
||||||
// remove
|
// remove
|
||||||
log.info("Removing: {}", s)
|
log.info("Removing: {}", s)
|
||||||
replicator ! Update(DataKey, ORSet.empty[String], WriteLocal)(_ - s)
|
replicator ! Update(DataKey, ORSet.empty[String], WriteLocal)(_ remove s)
|
||||||
}
|
}
|
||||||
|
|
||||||
case _: UpdateResponse[_] ⇒ // ignore
|
case _: UpdateResponse[_] ⇒ // ignore
|
||||||
|
|
@ -107,7 +107,7 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
implicit val self = probe.ref
|
implicit val self = probe.ref
|
||||||
|
|
||||||
//#update
|
//#update
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
val replicator = DistributedData(system).replicator
|
val replicator = DistributedData(system).replicator
|
||||||
|
|
||||||
val Counter1Key = PNCounterKey("counter1")
|
val Counter1Key = PNCounterKey("counter1")
|
||||||
|
|
@ -115,13 +115,13 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
val Set2Key = ORSetKey[String]("set2")
|
val Set2Key = ORSetKey[String]("set2")
|
||||||
val ActiveFlagKey = FlagKey("active")
|
val ActiveFlagKey = FlagKey("active")
|
||||||
|
|
||||||
replicator ! Update(Counter1Key, PNCounter(), WriteLocal)(_ + 1)
|
replicator ! Update(Counter1Key, PNCounter(), WriteLocal)(_ :+ 1)
|
||||||
|
|
||||||
val writeTo3 = WriteTo(n = 3, timeout = 1.second)
|
val writeTo3 = WriteTo(n = 3, timeout = 1.second)
|
||||||
replicator ! Update(Set1Key, GSet.empty[String], writeTo3)(_ + "hello")
|
replicator ! Update(Set1Key, GSet.empty[String], writeTo3)(_ + "hello")
|
||||||
|
|
||||||
val writeMajority = WriteMajority(timeout = 5.seconds)
|
val writeMajority = WriteMajority(timeout = 5.seconds)
|
||||||
replicator ! Update(Set2Key, ORSet.empty[String], writeMajority)(_ + "hello")
|
replicator ! Update(Set2Key, ORSet.empty[String], writeMajority)(_ :+ "hello")
|
||||||
|
|
||||||
val writeAll = WriteAll(timeout = 5.seconds)
|
val writeAll = WriteAll(timeout = 5.seconds)
|
||||||
replicator ! Update(ActiveFlagKey, Flag.Disabled, writeAll)(_.switchOn)
|
replicator ! Update(ActiveFlagKey, Flag.Disabled, writeAll)(_.switchOn)
|
||||||
|
|
@ -152,7 +152,7 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
def sender() = self
|
def sender() = self
|
||||||
|
|
||||||
//#update-request-context
|
//#update-request-context
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
val replicator = DistributedData(system).replicator
|
val replicator = DistributedData(system).replicator
|
||||||
val writeTwo = WriteTo(n = 2, timeout = 3.second)
|
val writeTwo = WriteTo(n = 2, timeout = 3.second)
|
||||||
val Counter1Key = PNCounterKey("counter1")
|
val Counter1Key = PNCounterKey("counter1")
|
||||||
|
|
@ -160,7 +160,7 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
def receive: Receive = {
|
def receive: Receive = {
|
||||||
case "increment" ⇒
|
case "increment" ⇒
|
||||||
// incoming command to increase the counter
|
// incoming command to increase the counter
|
||||||
val upd = Update(Counter1Key, PNCounter(), writeTwo, request = Some(sender()))(_ + 1)
|
val upd = Update(Counter1Key, PNCounter(), writeTwo, request = Some(sender()))(_ :+ 1)
|
||||||
replicator ! upd
|
replicator ! upd
|
||||||
|
|
||||||
case UpdateSuccess(Counter1Key, Some(replyTo: ActorRef)) ⇒
|
case UpdateSuccess(Counter1Key, Some(replyTo: ActorRef)) ⇒
|
||||||
|
|
@ -224,7 +224,7 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
def sender() = self
|
def sender() = self
|
||||||
|
|
||||||
//#get-request-context
|
//#get-request-context
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
val replicator = DistributedData(system).replicator
|
val replicator = DistributedData(system).replicator
|
||||||
val readTwo = ReadFrom(n = 2, timeout = 3.second)
|
val readTwo = ReadFrom(n = 2, timeout = 3.second)
|
||||||
val Counter1Key = PNCounterKey("counter1")
|
val Counter1Key = PNCounterKey("counter1")
|
||||||
|
|
@ -287,11 +287,12 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
"demonstrate PNCounter" in {
|
"demonstrate PNCounter" in {
|
||||||
def println(o: Any): Unit = ()
|
def println(o: Any): Unit = ()
|
||||||
//#pncounter
|
//#pncounter
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
|
|
||||||
val c0 = PNCounter.empty
|
val c0 = PNCounter.empty
|
||||||
val c1 = c0 + 1
|
val c1 = c0 :+ 1
|
||||||
val c2 = c1 + 7
|
val c2 = c1 :+ 7
|
||||||
val c3: PNCounter = c2 - 2
|
val c3: PNCounter = c2 decrement 2
|
||||||
println(c3.value) // 6
|
println(c3.value) // 6
|
||||||
//#pncounter
|
//#pncounter
|
||||||
}
|
}
|
||||||
|
|
@ -299,11 +300,11 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
"demonstrate PNCounterMap" in {
|
"demonstrate PNCounterMap" in {
|
||||||
def println(o: Any): Unit = ()
|
def println(o: Any): Unit = ()
|
||||||
//#pncountermap
|
//#pncountermap
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
val m0 = PNCounterMap.empty[String]
|
val m0 = PNCounterMap.empty[String]
|
||||||
val m1 = m0.increment("a", 7)
|
val m1 = m0.increment(node, "a", 7)
|
||||||
val m2 = m1.decrement("a", 2)
|
val m2 = m1.decrement(node, "a", 2)
|
||||||
val m3 = m2.increment("b", 1)
|
val m3 = m2.increment(node, "b", 1)
|
||||||
println(m3.get("a")) // 5
|
println(m3.get("a")) // 5
|
||||||
m3.entries.foreach { case (key, value) ⇒ println(s"$key -> $value") }
|
m3.entries.foreach { case (key, value) ⇒ println(s"$key -> $value") }
|
||||||
//#pncountermap
|
//#pncountermap
|
||||||
|
|
@ -323,11 +324,11 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
"demonstrate ORSet" in {
|
"demonstrate ORSet" in {
|
||||||
def println(o: Any): Unit = ()
|
def println(o: Any): Unit = ()
|
||||||
//#orset
|
//#orset
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
val s0 = ORSet.empty[String]
|
val s0 = ORSet.empty[String]
|
||||||
val s1 = s0 + "a"
|
val s1 = s0 :+ "a"
|
||||||
val s2 = s1 + "b"
|
val s2 = s1 :+ "b"
|
||||||
val s3 = s2 - "a"
|
val s3 = s2 remove "a"
|
||||||
println(s3.elements) // b
|
println(s3.elements) // b
|
||||||
//#orset
|
//#orset
|
||||||
}
|
}
|
||||||
|
|
@ -335,12 +336,12 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
"demonstrate ORMultiMap" in {
|
"demonstrate ORMultiMap" in {
|
||||||
def println(o: Any): Unit = ()
|
def println(o: Any): Unit = ()
|
||||||
//#ormultimap
|
//#ormultimap
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
val m0 = ORMultiMap.empty[String, Int]
|
val m0 = ORMultiMap.empty[String, Int]
|
||||||
val m1 = m0 + ("a" -> Set(1, 2, 3))
|
val m1 = m0 :+ ("a" -> Set(1, 2, 3))
|
||||||
val m2 = m1.addBinding("a", 4)
|
val m2 = m1.addBinding(node, "a", 4)
|
||||||
val m3 = m2.removeBinding("a", 2)
|
val m3 = m2.removeBinding(node, "a", 2)
|
||||||
val m4 = m3.addBinding("b", 1)
|
val m4 = m3.addBinding(node, "b", 1)
|
||||||
println(m4.entries)
|
println(m4.entries)
|
||||||
//#ormultimap
|
//#ormultimap
|
||||||
}
|
}
|
||||||
|
|
@ -357,9 +358,9 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
"demonstrate LWWRegister" in {
|
"demonstrate LWWRegister" in {
|
||||||
def println(o: Any): Unit = ()
|
def println(o: Any): Unit = ()
|
||||||
//#lwwregister
|
//#lwwregister
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
val r1 = LWWRegister("Hello")
|
val r1 = LWWRegister.create("Hello")
|
||||||
val r2 = r1.withValue("Hi")
|
val r2 = r1.withValueOf("Hi")
|
||||||
println(s"${r1.value} by ${r1.updatedBy} at ${r1.timestamp}")
|
println(s"${r1.value} by ${r1.updatedBy} at ${r1.timestamp}")
|
||||||
//#lwwregister
|
//#lwwregister
|
||||||
r2.value should be("Hi")
|
r2.value should be("Hi")
|
||||||
|
|
@ -370,17 +371,17 @@ class DistributedDataDocSpec extends AkkaSpec(DistributedDataDocSpec.config) {
|
||||||
//#lwwregister-custom-clock
|
//#lwwregister-custom-clock
|
||||||
case class Record(version: Int, name: String, address: String)
|
case class Record(version: Int, name: String, address: String)
|
||||||
|
|
||||||
implicit val node = Cluster(system)
|
implicit val node = DistributedData(system).selfUniqueAddress
|
||||||
implicit val recordClock = new LWWRegister.Clock[Record] {
|
implicit val recordClock = new LWWRegister.Clock[Record] {
|
||||||
override def apply(currentTimestamp: Long, value: Record): Long =
|
override def apply(currentTimestamp: Long, value: Record): Long =
|
||||||
value.version
|
value.version
|
||||||
}
|
}
|
||||||
|
|
||||||
val record1 = Record(version = 1, "Alice", "Union Square")
|
val record1 = Record(version = 1, "Alice", "Union Square")
|
||||||
val r1 = LWWRegister(record1)
|
val r1 = LWWRegister(node, record1, recordClock)
|
||||||
|
|
||||||
val record2 = Record(version = 2, "Alice", "Madison Square")
|
val record2 = Record(version = 2, "Alice", "Madison Square")
|
||||||
val r2 = LWWRegister(record2)
|
val r2 = LWWRegister(node, record2, recordClock)
|
||||||
|
|
||||||
val r3 = r1.merge(r2)
|
val r3 = r1.merge(r2)
|
||||||
println(r3.value)
|
println(r3.value)
|
||||||
|
|
|
||||||
|
|
@ -5,10 +5,10 @@
|
||||||
package scala.docs.ddata
|
package scala.docs.ddata
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
import akka.actor.Actor
|
import akka.actor.Actor
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
import akka.cluster.Cluster
|
|
||||||
import akka.cluster.ddata.DistributedData
|
import akka.cluster.ddata.DistributedData
|
||||||
import akka.cluster.ddata.LWWMap
|
import akka.cluster.ddata.LWWMap
|
||||||
import akka.cluster.ddata.LWWMapKey
|
import akka.cluster.ddata.LWWMapKey
|
||||||
|
|
@ -38,7 +38,7 @@ class ShoppingCart(userId: String) extends Actor {
|
||||||
import akka.cluster.ddata.Replicator._
|
import akka.cluster.ddata.Replicator._
|
||||||
|
|
||||||
val replicator = DistributedData(context.system).replicator
|
val replicator = DistributedData(context.system).replicator
|
||||||
implicit val cluster = Cluster(context.system)
|
implicit val node = DistributedData(context.system).selfUniqueAddress
|
||||||
|
|
||||||
val DataKey = LWWMapKey[String, LineItem]("cart-" + userId)
|
val DataKey = LWWMapKey[String, LineItem]("cart-" + userId)
|
||||||
|
|
||||||
|
|
@ -79,8 +79,8 @@ class ShoppingCart(userId: String) extends Actor {
|
||||||
def updateCart(data: LWWMap[String, LineItem], item: LineItem): LWWMap[String, LineItem] =
|
def updateCart(data: LWWMap[String, LineItem], item: LineItem): LWWMap[String, LineItem] =
|
||||||
data.get(item.productId) match {
|
data.get(item.productId) match {
|
||||||
case Some(LineItem(_, _, existingQuantity)) ⇒
|
case Some(LineItem(_, _, existingQuantity)) ⇒
|
||||||
data + (item.productId -> item.copy(quantity = existingQuantity + item.quantity))
|
data :+ (item.productId -> item.copy(quantity = existingQuantity + item.quantity))
|
||||||
case None ⇒ data + (item.productId -> item)
|
case None ⇒ data :+ (item.productId -> item)
|
||||||
}
|
}
|
||||||
|
|
||||||
//#remove-item
|
//#remove-item
|
||||||
|
|
@ -92,13 +92,13 @@ class ShoppingCart(userId: String) extends Actor {
|
||||||
|
|
||||||
case GetSuccess(DataKey, Some(RemoveItem(productId))) ⇒
|
case GetSuccess(DataKey, Some(RemoveItem(productId))) ⇒
|
||||||
replicator ! Update(DataKey, LWWMap(), writeMajority, None) {
|
replicator ! Update(DataKey, LWWMap(), writeMajority, None) {
|
||||||
_ - productId
|
_.remove(node, productId)
|
||||||
}
|
}
|
||||||
|
|
||||||
case GetFailure(DataKey, Some(RemoveItem(productId))) ⇒
|
case GetFailure(DataKey, Some(RemoveItem(productId))) ⇒
|
||||||
// ReadMajority failed, fall back to best effort local value
|
// ReadMajority failed, fall back to best effort local value
|
||||||
replicator ! Update(DataKey, LWWMap(), writeMajority, None) {
|
replicator ! Update(DataKey, LWWMap(), writeMajority, None) {
|
||||||
_ - productId
|
_.remove(node, productId)
|
||||||
}
|
}
|
||||||
|
|
||||||
case NotFound(DataKey, Some(RemoveItem(productId))) ⇒
|
case NotFound(DataKey, Some(RemoveItem(productId))) ⇒
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue