Use full state for the ORSet and ORMap removals until #22648 is fixed
* some tests disabled, search for "FIXME use full state for removals, until issue #22648 is fixed" to find them
This commit is contained in:
parent
dc070e4e1c
commit
5293666a30
11 changed files with 136 additions and 85 deletions
|
|
@ -298,8 +298,11 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
@InternalApi private[akka] def remove(node: UniqueAddress, key: A): ORMap[A, B] = {
|
@InternalApi private[akka] def remove(node: UniqueAddress, key: A): ORMap[A, B] = {
|
||||||
// for removals the delta values map emitted will be empty
|
// for removals the delta values map emitted will be empty
|
||||||
val newKeys = keys.resetDelta.remove(node, key)
|
val newKeys = keys.resetDelta.remove(node, key)
|
||||||
val removeDeltaOp = RemoveDeltaOp(newKeys.delta.get, zeroTag)
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
new ORMap(newKeys, values - key, zeroTag, Some(newDelta(removeDeltaOp)))
|
// val removeDeltaOp = RemoveDeltaOp(newKeys.delta.get, zeroTag)
|
||||||
|
// new ORMap(newKeys, values - key, zeroTag, Some(newDelta(removeDeltaOp)))
|
||||||
|
new ORMap(newKeys, values - key, zeroTag, delta = None)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -309,8 +312,10 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] def removeKey(node: UniqueAddress, key: A): ORMap[A, B] = {
|
@InternalApi private[akka] def removeKey(node: UniqueAddress, key: A): ORMap[A, B] = {
|
||||||
val newKeys = keys.resetDelta.remove(node, key)
|
val newKeys = keys.resetDelta.remove(node, key)
|
||||||
val removeKeyDeltaOp = RemoveKeyDeltaOp(newKeys.delta.get, key, zeroTag)
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
new ORMap(newKeys, values, zeroTag, Some(newDelta(removeKeyDeltaOp)))
|
// val removeKeyDeltaOp = RemoveKeyDeltaOp(newKeys.delta.get, key, zeroTag)
|
||||||
|
// new ORMap(newKeys, values, zeroTag, Some(newDelta(removeKeyDeltaOp)))
|
||||||
|
new ORMap(newKeys, values, zeroTag, delta = None)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def dryMerge(that: ORMap[A, B], mergedKeys: ORSet[A], valueKeysIterator: Iterator[A]): ORMap[A, B] = {
|
private def dryMerge(that: ORMap[A, B], mergedKeys: ORSet[A], valueKeysIterator: Iterator[A]): ORMap[A, B] = {
|
||||||
|
|
@ -389,7 +394,7 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
||||||
updateOp.values.foreach {
|
updateOp.values.foreach {
|
||||||
case (key, value) ⇒
|
case (key, value) ⇒
|
||||||
if (thatValueDeltas.contains(key))
|
if (thatValueDeltas.contains(key))
|
||||||
thatValueDeltas = thatValueDeltas + (key → (thatValueDeltas(key) :+ (key, value)))
|
thatValueDeltas = thatValueDeltas + (key → (thatValueDeltas(key) :+ (key → value)))
|
||||||
else
|
else
|
||||||
thatValueDeltas += (key → ((key, value) :: Nil))
|
thatValueDeltas += (key → ((key, value) :: Nil))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -345,13 +345,15 @@ final class ORSet[A] private[akka] (
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] def remove(node: UniqueAddress, element: A): ORSet[A] = {
|
@InternalApi private[akka] def remove(node: UniqueAddress, element: A): ORSet[A] = {
|
||||||
val deltaDot = VersionVector(node, vvector.versionAt(node))
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
val rmOp = ORSet.RemoveDeltaOp(new ORSet(Map(element → deltaDot), vvector))
|
// val deltaDot = VersionVector(node, vvector.versionAt(node))
|
||||||
val newDelta = delta match {
|
// val rmOp = ORSet.RemoveDeltaOp(new ORSet(Map(element → deltaDot), vvector))
|
||||||
case None ⇒ rmOp
|
// val newDelta = delta match {
|
||||||
case Some(d) ⇒ d.merge(rmOp)
|
// case None ⇒ rmOp
|
||||||
}
|
// case Some(d) ⇒ d.merge(rmOp)
|
||||||
assignAncestor(copy(elementsMap = elementsMap - element, delta = Some(newDelta)))
|
// }
|
||||||
|
// assignAncestor(copy(elementsMap = elementsMap - element, delta = Some(newDelta)))
|
||||||
|
assignAncestor(copy(elementsMap = elementsMap - element, delta = None))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -123,29 +123,29 @@ object ReplicatorSettings {
|
||||||
* in the `Set`.
|
* in the `Set`.
|
||||||
*/
|
*/
|
||||||
final class ReplicatorSettings(
|
final class ReplicatorSettings(
|
||||||
val role: Option[String],
|
val role: Option[String],
|
||||||
val gossipInterval: FiniteDuration,
|
val gossipInterval: FiniteDuration,
|
||||||
val notifySubscribersInterval: FiniteDuration,
|
val notifySubscribersInterval: FiniteDuration,
|
||||||
val maxDeltaElements: Int,
|
val maxDeltaElements: Int,
|
||||||
val dispatcher: String,
|
val dispatcher: String,
|
||||||
val pruningInterval: FiniteDuration,
|
val pruningInterval: FiniteDuration,
|
||||||
val maxPruningDissemination: FiniteDuration,
|
val maxPruningDissemination: FiniteDuration,
|
||||||
val durableStoreProps: Either[(String, Config), Props],
|
val durableStoreProps: Either[(String, Config), Props],
|
||||||
val durableKeys: Set[KeyId],
|
val durableKeys: Set[KeyId],
|
||||||
val pruningMarkerTimeToLive: FiniteDuration,
|
val pruningMarkerTimeToLive: FiniteDuration,
|
||||||
val durablePruningMarkerTimeToLive: FiniteDuration,
|
val durablePruningMarkerTimeToLive: FiniteDuration,
|
||||||
val deltaCrdtEnabled: Boolean) {
|
val deltaCrdtEnabled: Boolean) {
|
||||||
|
|
||||||
// For backwards compatibility
|
// For backwards compatibility
|
||||||
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
|
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
|
||||||
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration) =
|
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration) =
|
||||||
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
|
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
|
||||||
maxPruningDissemination, Right(Props.empty), Set.empty, 6.hours, 10.days, true)
|
maxPruningDissemination, Right(Props.empty), Set.empty, 6.hours, 10.days, true)
|
||||||
|
|
||||||
// For backwards compatibility
|
// For backwards compatibility
|
||||||
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
|
def this(role: Option[String], gossipInterval: FiniteDuration, notifySubscribersInterval: FiniteDuration,
|
||||||
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration,
|
maxDeltaElements: Int, dispatcher: String, pruningInterval: FiniteDuration, maxPruningDissemination: FiniteDuration,
|
||||||
durableStoreProps: Either[(String, Config), Props], durableKeys: Set[String]) =
|
durableStoreProps: Either[(String, Config), Props], durableKeys: Set[String]) =
|
||||||
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
|
this(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher, pruningInterval,
|
||||||
maxPruningDissemination, durableStoreProps, durableKeys, 6.hours, 10.days, true)
|
maxPruningDissemination, durableStoreProps, durableKeys, 6.hours, 10.days, true)
|
||||||
|
|
||||||
|
|
@ -174,7 +174,7 @@ final class ReplicatorSettings(
|
||||||
copy(pruningInterval = pruningInterval, maxPruningDissemination = maxPruningDissemination)
|
copy(pruningInterval = pruningInterval, maxPruningDissemination = maxPruningDissemination)
|
||||||
|
|
||||||
def withPruningMarkerTimeToLive(
|
def withPruningMarkerTimeToLive(
|
||||||
pruningMarkerTimeToLive: FiniteDuration,
|
pruningMarkerTimeToLive: FiniteDuration,
|
||||||
durablePruningMarkerTimeToLive: FiniteDuration): ReplicatorSettings =
|
durablePruningMarkerTimeToLive: FiniteDuration): ReplicatorSettings =
|
||||||
copy(
|
copy(
|
||||||
pruningMarkerTimeToLive = pruningMarkerTimeToLive,
|
pruningMarkerTimeToLive = pruningMarkerTimeToLive,
|
||||||
|
|
@ -201,18 +201,18 @@ final class ReplicatorSettings(
|
||||||
copy(deltaCrdtEnabled = deltaCrdtEnabled)
|
copy(deltaCrdtEnabled = deltaCrdtEnabled)
|
||||||
|
|
||||||
private def copy(
|
private def copy(
|
||||||
role: Option[String] = role,
|
role: Option[String] = role,
|
||||||
gossipInterval: FiniteDuration = gossipInterval,
|
gossipInterval: FiniteDuration = gossipInterval,
|
||||||
notifySubscribersInterval: FiniteDuration = notifySubscribersInterval,
|
notifySubscribersInterval: FiniteDuration = notifySubscribersInterval,
|
||||||
maxDeltaElements: Int = maxDeltaElements,
|
maxDeltaElements: Int = maxDeltaElements,
|
||||||
dispatcher: String = dispatcher,
|
dispatcher: String = dispatcher,
|
||||||
pruningInterval: FiniteDuration = pruningInterval,
|
pruningInterval: FiniteDuration = pruningInterval,
|
||||||
maxPruningDissemination: FiniteDuration = maxPruningDissemination,
|
maxPruningDissemination: FiniteDuration = maxPruningDissemination,
|
||||||
durableStoreProps: Either[(String, Config), Props] = durableStoreProps,
|
durableStoreProps: Either[(String, Config), Props] = durableStoreProps,
|
||||||
durableKeys: Set[KeyId] = durableKeys,
|
durableKeys: Set[KeyId] = durableKeys,
|
||||||
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
|
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
|
||||||
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
|
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
|
||||||
deltaCrdtEnabled: Boolean = deltaCrdtEnabled): ReplicatorSettings =
|
deltaCrdtEnabled: Boolean = deltaCrdtEnabled): ReplicatorSettings =
|
||||||
new ReplicatorSettings(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher,
|
new ReplicatorSettings(role, gossipInterval, notifySubscribersInterval, maxDeltaElements, dispatcher,
|
||||||
pruningInterval, maxPruningDissemination, durableStoreProps, durableKeys,
|
pruningInterval, maxPruningDissemination, durableStoreProps, durableKeys,
|
||||||
pruningMarkerTimeToLive, durablePruningMarkerTimeToLive, deltaCrdtEnabled)
|
pruningMarkerTimeToLive, durablePruningMarkerTimeToLive, deltaCrdtEnabled)
|
||||||
|
|
@ -436,7 +436,7 @@ object Replicator {
|
||||||
* for example not access `sender()` reference of an enclosing actor.
|
* for example not access `sender()` reference of an enclosing actor.
|
||||||
*/
|
*/
|
||||||
final case class Update[A <: ReplicatedData](key: Key[A], writeConsistency: WriteConsistency,
|
final case class Update[A <: ReplicatedData](key: Key[A], writeConsistency: WriteConsistency,
|
||||||
request: Option[Any])(val modify: Option[A] ⇒ A)
|
request: Option[Any])(val modify: Option[A] ⇒ A)
|
||||||
extends Command[A] with NoSerializationVerificationNeeded {
|
extends Command[A] with NoSerializationVerificationNeeded {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -606,9 +606,9 @@ object Replicator {
|
||||||
* The `DataEnvelope` wraps a data entry and carries state of the pruning process for the entry.
|
* The `DataEnvelope` wraps a data entry and carries state of the pruning process for the entry.
|
||||||
*/
|
*/
|
||||||
final case class DataEnvelope(
|
final case class DataEnvelope(
|
||||||
data: ReplicatedData,
|
data: ReplicatedData,
|
||||||
pruning: Map[UniqueAddress, PruningState] = Map.empty,
|
pruning: Map[UniqueAddress, PruningState] = Map.empty,
|
||||||
deltaVersions: VersionVector = VersionVector.empty)
|
deltaVersions: VersionVector = VersionVector.empty)
|
||||||
extends ReplicatorMessage {
|
extends ReplicatorMessage {
|
||||||
|
|
||||||
import PruningState._
|
import PruningState._
|
||||||
|
|
@ -1227,7 +1227,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
def isLocalSender(): Boolean = !replyTo.path.address.hasGlobalScope
|
def isLocalSender(): Boolean = !replyTo.path.address.hasGlobalScope
|
||||||
|
|
||||||
def receiveUpdate(key: KeyR, modify: Option[ReplicatedData] ⇒ ReplicatedData,
|
def receiveUpdate(key: KeyR, modify: Option[ReplicatedData] ⇒ ReplicatedData,
|
||||||
writeConsistency: WriteConsistency, req: Option[Any]): Unit = {
|
writeConsistency: WriteConsistency, req: Option[Any]): Unit = {
|
||||||
val localValue = getData(key.id)
|
val localValue = getData(key.id)
|
||||||
|
|
||||||
def deltaOrPlaceholder(d: DeltaReplicatedData): Option[ReplicatedDelta] = {
|
def deltaOrPlaceholder(d: DeltaReplicatedData): Option[ReplicatedDelta] = {
|
||||||
|
|
@ -1880,15 +1880,15 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] object WriteAggregator {
|
@InternalApi private[akka] object WriteAggregator {
|
||||||
def props(
|
def props(
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
envelope: Replicator.Internal.DataEnvelope,
|
envelope: Replicator.Internal.DataEnvelope,
|
||||||
delta: Option[Replicator.Internal.Delta],
|
delta: Option[Replicator.Internal.Delta],
|
||||||
consistency: Replicator.WriteConsistency,
|
consistency: Replicator.WriteConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
nodes: Set[Address],
|
nodes: Set[Address],
|
||||||
unreachable: Set[Address],
|
unreachable: Set[Address],
|
||||||
replyTo: ActorRef,
|
replyTo: ActorRef,
|
||||||
durable: Boolean): Props =
|
durable: Boolean): Props =
|
||||||
Props(new WriteAggregator(key, envelope, delta, consistency, req, nodes, unreachable, replyTo, durable))
|
Props(new WriteAggregator(key, envelope, delta, consistency, req, nodes, unreachable, replyTo, durable))
|
||||||
.withDeploy(Deploy.local)
|
.withDeploy(Deploy.local)
|
||||||
}
|
}
|
||||||
|
|
@ -1897,15 +1897,15 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] class WriteAggregator(
|
@InternalApi private[akka] class WriteAggregator(
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
envelope: Replicator.Internal.DataEnvelope,
|
envelope: Replicator.Internal.DataEnvelope,
|
||||||
delta: Option[Replicator.Internal.Delta],
|
delta: Option[Replicator.Internal.Delta],
|
||||||
consistency: Replicator.WriteConsistency,
|
consistency: Replicator.WriteConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
override val nodes: Set[Address],
|
override val nodes: Set[Address],
|
||||||
override val unreachable: Set[Address],
|
override val unreachable: Set[Address],
|
||||||
replyTo: ActorRef,
|
replyTo: ActorRef,
|
||||||
durable: Boolean) extends ReadWriteAggregator {
|
durable: Boolean) extends ReadWriteAggregator {
|
||||||
|
|
||||||
import Replicator._
|
import Replicator._
|
||||||
import Replicator.Internal._
|
import Replicator.Internal._
|
||||||
|
|
@ -2009,13 +2009,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] object ReadAggregator {
|
@InternalApi private[akka] object ReadAggregator {
|
||||||
def props(
|
def props(
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
consistency: Replicator.ReadConsistency,
|
consistency: Replicator.ReadConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
nodes: Set[Address],
|
nodes: Set[Address],
|
||||||
unreachable: Set[Address],
|
unreachable: Set[Address],
|
||||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||||
replyTo: ActorRef): Props =
|
replyTo: ActorRef): Props =
|
||||||
Props(new ReadAggregator(key, consistency, req, nodes, unreachable, localValue, replyTo))
|
Props(new ReadAggregator(key, consistency, req, nodes, unreachable, localValue, replyTo))
|
||||||
.withDeploy(Deploy.local)
|
.withDeploy(Deploy.local)
|
||||||
|
|
||||||
|
|
@ -2025,13 +2025,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
@InternalApi private[akka] class ReadAggregator(
|
@InternalApi private[akka] class ReadAggregator(
|
||||||
key: KeyR,
|
key: KeyR,
|
||||||
consistency: Replicator.ReadConsistency,
|
consistency: Replicator.ReadConsistency,
|
||||||
req: Option[Any],
|
req: Option[Any],
|
||||||
override val nodes: Set[Address],
|
override val nodes: Set[Address],
|
||||||
override val unreachable: Set[Address],
|
override val unreachable: Set[Address],
|
||||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||||
replyTo: ActorRef) extends ReadWriteAggregator {
|
replyTo: ActorRef) extends ReadWriteAggregator {
|
||||||
|
|
||||||
import Replicator._
|
import Replicator._
|
||||||
import Replicator.Internal._
|
import Replicator.Internal._
|
||||||
|
|
|
||||||
|
|
@ -118,11 +118,12 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
|
||||||
case 3 ⇒
|
case 3 ⇒
|
||||||
// ORSet
|
// ORSet
|
||||||
val key = rndOrSetkey()
|
val key = rndOrSetkey()
|
||||||
// only removals for KeyF on node first
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
if (key == KeyF && onNode == first && rnd.nextBoolean())
|
// // only removals for KeyF on node first
|
||||||
Remove(key, rndRemoveElement(), consistency())
|
// if (key == KeyF && onNode == first && rnd.nextBoolean())
|
||||||
else
|
// Remove(key, rndRemoveElement(), consistency())
|
||||||
Add(key, rndAddElement(), consistency())
|
// else
|
||||||
|
Add(key, rndAddElement(), consistency())
|
||||||
}
|
}
|
||||||
}.toVector
|
}.toVector
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -105,11 +105,12 @@ object ReplicatorMapDeltaSpec extends MultiNodeConfig {
|
||||||
case 3 ⇒
|
case 3 ⇒
|
||||||
// ORSet
|
// ORSet
|
||||||
val key = rndOrSetkey()
|
val key = rndOrSetkey()
|
||||||
// only removals for KeyF on node first
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
if (key == KeyF && onNode == first && rnd.nextBoolean())
|
// // only removals for KeyF on node first
|
||||||
Remove(key, rndRemoveElement(), consistency())
|
// if (key == KeyF && onNode == first && rnd.nextBoolean())
|
||||||
else
|
// Remove(key, rndRemoveElement(), consistency())
|
||||||
Add(key, rndAddElement(), consistency())
|
// else
|
||||||
|
Add(key, rndAddElement(), consistency())
|
||||||
}
|
}
|
||||||
}.toVector
|
}.toVector
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -60,6 +60,9 @@ class LWWMapSpec extends WordSpec with Matchers {
|
||||||
|
|
||||||
val merged1 = m1 merge m2
|
val merged1 = m1 merge m2
|
||||||
|
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val m3 = merged1.resetDelta.remove(node1, "b")
|
val m3 = merged1.resetDelta.remove(node1, "b")
|
||||||
(merged1 mergeDelta m3.delta.get).entries should be(Map("a" → 1, "c" → 3))
|
(merged1 mergeDelta m3.delta.get).entries should be(Map("a" → 1, "c" → 3))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -55,6 +55,9 @@ class ORMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to remove entry using a delta" in {
|
"be able to remove entry using a delta" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val m = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B")
|
val m = ORMap().put(node1, "a", GSet() + "A").put(node1, "b", GSet() + "B")
|
||||||
val addDelta = m.delta.get
|
val addDelta = m.delta.get
|
||||||
|
|
||||||
|
|
@ -325,6 +328,9 @@ class ORMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"not have anomalies for remove+updated scenario and deltas 8" in {
|
"not have anomalies for remove+updated scenario and deltas 8" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val m1 = ORMap.empty.put(node1, "a", GSet.empty + "A")
|
val m1 = ORMap.empty.put(node1, "a", GSet.empty + "A")
|
||||||
.put(node1, "b", GSet.empty + "B").put(node2, "b", GSet.empty + "B")
|
.put(node1, "b", GSet.empty + "B").put(node2, "b", GSet.empty + "B")
|
||||||
val m2 = ORMap.empty.put(node2, "c", GSet.empty + "C")
|
val m2 = ORMap.empty.put(node2, "c", GSet.empty + "C")
|
||||||
|
|
@ -348,6 +354,9 @@ class ORMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"not have anomalies for remove+updated scenario and deltas 9" in {
|
"not have anomalies for remove+updated scenario and deltas 9" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val m1 = ORMap.empty.put(node1, "a", GSet.empty + "A")
|
val m1 = ORMap.empty.put(node1, "a", GSet.empty + "A")
|
||||||
.put(node1, "b", GSet.empty + "B").put(node2, "b", GSet.empty + "B")
|
.put(node1, "b", GSet.empty + "B").put(node2, "b", GSet.empty + "B")
|
||||||
val m2 = ORMap.empty.put(node2, "c", GSet.empty + "C")
|
val m2 = ORMap.empty.put(node2, "c", GSet.empty + "C")
|
||||||
|
|
|
||||||
|
|
@ -78,6 +78,9 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
||||||
val merged2 = m2 merge m1
|
val merged2 = m2 merge m1
|
||||||
merged2.entries should be(expectedMerged)
|
merged2.entries should be(expectedMerged)
|
||||||
|
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val merged3 = m1 mergeDelta m2.delta.get
|
val merged3 = m1 mergeDelta m2.delta.get
|
||||||
merged3.entries should be(expectedMerged)
|
merged3.entries should be(expectedMerged)
|
||||||
|
|
||||||
|
|
@ -114,6 +117,9 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"not have usual anomalies for remove+addBinding scenario and delta-deltas" in {
|
"not have usual anomalies for remove+addBinding scenario and delta-deltas" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val m1 = ORMultiMap.emptyWithValueDeltas[String, String].put(node1, "a", Set("A")).put(node1, "b", Set("B"))
|
val m1 = ORMultiMap.emptyWithValueDeltas[String, String].put(node1, "a", Set("A")).put(node1, "b", Set("B"))
|
||||||
val m2 = ORMultiMap.emptyWithValueDeltas[String, String].put(node2, "c", Set("C"))
|
val m2 = ORMultiMap.emptyWithValueDeltas[String, String].put(node2, "c", Set("C"))
|
||||||
|
|
||||||
|
|
@ -142,6 +148,9 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"not have usual anomalies for remove+addBinding scenario and delta-deltas 2" in {
|
"not have usual anomalies for remove+addBinding scenario and delta-deltas 2" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
// the new delta-delta ORMultiMap is free from this anomaly
|
// the new delta-delta ORMultiMap is free from this anomaly
|
||||||
val m1 = ORMultiMap.emptyWithValueDeltas[String, String].put(node1, "a", Set("A")).put(node1, "b", Set("B"))
|
val m1 = ORMultiMap.emptyWithValueDeltas[String, String].put(node1, "a", Set("A")).put(node1, "b", Set("B"))
|
||||||
val m2 = ORMultiMap.emptyWithValueDeltas[String, String].put(node2, "c", Set("C"))
|
val m2 = ORMultiMap.emptyWithValueDeltas[String, String].put(node2, "c", Set("C"))
|
||||||
|
|
|
||||||
|
|
@ -294,6 +294,9 @@ class ORSetSpec extends WordSpec with Matchers {
|
||||||
s1.mergeDelta(d4) should ===(s3)
|
s1.mergeDelta(d4) should ===(s3)
|
||||||
s2.mergeDelta(d4) should ===(s3)
|
s2.mergeDelta(d4) should ===(s3)
|
||||||
|
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val s5 = s3.resetDelta.remove(node1, "b")
|
val s5 = s3.resetDelta.remove(node1, "b")
|
||||||
val d5 = s5.delta.get
|
val d5 = s5.delta.get
|
||||||
val d6 = (d4 merge d5).asInstanceOf[ORSet.DeltaGroup[String]]
|
val d6 = (d4 merge d5).asInstanceOf[ORSet.DeltaGroup[String]]
|
||||||
|
|
@ -312,6 +315,9 @@ class ORSetSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"work for removals" in {
|
"work for removals" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val s1 = ORSet.empty[String]
|
val s1 = ORSet.empty[String]
|
||||||
val s2 = s1.add(node1, "a").add(node1, "b").resetDelta
|
val s2 = s1.add(node1, "a").add(node1, "b").resetDelta
|
||||||
val s3 = s2.remove(node1, "b")
|
val s3 = s2.remove(node1, "b")
|
||||||
|
|
@ -357,6 +363,9 @@ class ORSetSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"handle a mixed add/remove scenario" in {
|
"handle a mixed add/remove scenario" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val s1 = ORSet.empty[String]
|
val s1 = ORSet.empty[String]
|
||||||
val s2 = s1.resetDelta.remove(node1, "e")
|
val s2 = s1.resetDelta.remove(node1, "e")
|
||||||
val s3 = s2.resetDelta.add(node1, "b")
|
val s3 = s2.resetDelta.add(node1, "b")
|
||||||
|
|
@ -376,6 +385,9 @@ class ORSetSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"handle a mixed add/remove scenario 2" in {
|
"handle a mixed add/remove scenario 2" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val s1 = ORSet.empty[String]
|
val s1 = ORSet.empty[String]
|
||||||
val s2 = s1.resetDelta.add(node1, "a")
|
val s2 = s1.resetDelta.add(node1, "a")
|
||||||
val s3 = s2.resetDelta.add(node1, "b")
|
val s3 = s2.resetDelta.add(node1, "b")
|
||||||
|
|
@ -398,6 +410,9 @@ class ORSetSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"handle a mixed add/remove scenario 3" in {
|
"handle a mixed add/remove scenario 3" in {
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val s1 = ORSet.empty[String]
|
val s1 = ORSet.empty[String]
|
||||||
val s2 = s1.resetDelta.add(node1, "a")
|
val s2 = s1.resetDelta.add(node1, "a")
|
||||||
val s3 = s2.resetDelta.add(node1, "b")
|
val s3 = s2.resetDelta.add(node1, "b")
|
||||||
|
|
|
||||||
|
|
@ -56,6 +56,9 @@ class PNCounterMapSpec extends WordSpec with Matchers {
|
||||||
|
|
||||||
val merged1 = m1 merge m2
|
val merged1 = m1 merge m2
|
||||||
|
|
||||||
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
pending
|
||||||
|
|
||||||
val m3 = merged1.resetDelta.remove(node1, "b")
|
val m3 = merged1.resetDelta.remove(node1, "b")
|
||||||
(merged1 mergeDelta m3.delta.get).entries should be(Map("a" → 1, "c" → 7))
|
(merged1 mergeDelta m3.delta.get).entries should be(Map("a" → 1, "c" → 7))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -113,8 +113,9 @@ class ReplicatedDataSerializerSpec extends TestKit(ActorSystem(
|
||||||
|
|
||||||
"serialize ORSet delta" in {
|
"serialize ORSet delta" in {
|
||||||
checkSerialization(ORSet().add(address1, "a").delta.get)
|
checkSerialization(ORSet().add(address1, "a").delta.get)
|
||||||
checkSerialization(ORSet().add(address1, "a").resetDelta.remove(address2, "a").delta.get)
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
checkSerialization(ORSet().add(address1, "a").remove(address2, "a").delta.get)
|
//checkSerialization(ORSet().add(address1, "a").resetDelta.remove(address2, "a").delta.get)
|
||||||
|
// checkSerialization(ORSet().add(address1, "a").remove(address2, "a").delta.get)
|
||||||
checkSerialization(ORSet().add(address1, "a").resetDelta.clear(address2).delta.get)
|
checkSerialization(ORSet().add(address1, "a").resetDelta.clear(address2).delta.get)
|
||||||
checkSerialization(ORSet().add(address1, "a").clear(address2).delta.get)
|
checkSerialization(ORSet().add(address1, "a").clear(address2).delta.get)
|
||||||
}
|
}
|
||||||
|
|
@ -196,8 +197,9 @@ class ReplicatedDataSerializerSpec extends TestKit(ActorSystem(
|
||||||
|
|
||||||
"serialize ORMap delta" in {
|
"serialize ORMap delta" in {
|
||||||
checkSerialization(ORMap().put(address1, "a", GSet() + "A").put(address2, "b", GSet() + "B").delta.get)
|
checkSerialization(ORMap().put(address1, "a", GSet() + "A").put(address2, "b", GSet() + "B").delta.get)
|
||||||
checkSerialization(ORMap().put(address1, "a", GSet() + "A").resetDelta.remove(address2, "a").delta.get)
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
checkSerialization(ORMap().put(address1, "a", GSet() + "A").remove(address2, "a").delta.get)
|
// checkSerialization(ORMap().put(address1, "a", GSet() + "A").resetDelta.remove(address2, "a").delta.get)
|
||||||
|
// checkSerialization(ORMap().put(address1, "a", GSet() + "A").remove(address2, "a").delta.get)
|
||||||
checkSerialization(ORMap().put(address1, 1, GSet() + "A").delta.get)
|
checkSerialization(ORMap().put(address1, 1, GSet() + "A").delta.get)
|
||||||
checkSerialization(ORMap().put(address1, 1L, GSet() + "A").delta.get)
|
checkSerialization(ORMap().put(address1, 1L, GSet() + "A").delta.get)
|
||||||
checkSerialization(ORMap.empty[String, ORSet[String]]
|
checkSerialization(ORMap.empty[String, ORSet[String]]
|
||||||
|
|
@ -281,7 +283,8 @@ class ReplicatedDataSerializerSpec extends TestKit(ActorSystem(
|
||||||
checkSerialization(ORMultiMap._emptyWithValueDeltas.addBinding(address1, 1, "A"))
|
checkSerialization(ORMultiMap._emptyWithValueDeltas.addBinding(address1, 1, "A"))
|
||||||
checkSerialization(ORMultiMap._emptyWithValueDeltas.addBinding(address1, 1L, "A"))
|
checkSerialization(ORMultiMap._emptyWithValueDeltas.addBinding(address1, 1L, "A"))
|
||||||
checkSerialization(ORMultiMap._emptyWithValueDeltas.addBinding(address1, Flag(), "A"))
|
checkSerialization(ORMultiMap._emptyWithValueDeltas.addBinding(address1, Flag(), "A"))
|
||||||
checkSerialization(ORMultiMap.emptyWithValueDeltas[String, String].addBinding(address1, "a", "A").remove(address1, "a").delta.get)
|
// FIXME use full state for removals, until issue #22648 is fixed
|
||||||
|
// checkSerialization(ORMultiMap.emptyWithValueDeltas[String, String].addBinding(address1, "a", "A").remove(address1, "a").delta.get)
|
||||||
checkSerialization(ORMultiMap.emptyWithValueDeltas[String, String]
|
checkSerialization(ORMultiMap.emptyWithValueDeltas[String, String]
|
||||||
.addBinding(address1, "a", "A1")
|
.addBinding(address1, "a", "A1")
|
||||||
.put(address2, "b", Set("B1", "B2", "B3"))
|
.put(address2, "b", Set("B1", "B2", "B3"))
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue