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:
Patrik Nordwall 2017-03-30 13:26:35 +02:00
parent dc070e4e1c
commit 5293666a30
11 changed files with 136 additions and 85 deletions

View file

@ -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))
} }

View file

@ -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))
} }
/** /**

View file

@ -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._

View file

@ -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
} }

View file

@ -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
} }

View file

@ -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))

View file

@ -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")

View file

@ -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"))

View file

@ -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")

View file

@ -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))

View file

@ -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"))