Various scala-2.13.0-M5 fixes
fix akka-actor-tests compile errors some tests still fail though Fix test failures in akka-actor-test Manually work arround missing implicit Factory[Nothing, Seq[Nothing]] see https://github.com/scala/scala-collection-compat/issues/137 akka-remote scalafix changes Fix shutdownAll compile error test:akka-remote scalafix changes akka-multi-node-testkit scalafix Fix akka-remote-tests multi-jvm compile errors akka-stream-tests/test:scalafix Fix test:akka-stream-tests Crude implementation of ByteString.map scalafix akka-actor-typed, akka-actor-typed-tests akka-actor-typed-tests compile and succeed scalafix akka-camel scalafix akka-cluster akka-cluster compile & test scalafix akka-cluster-metrics Fix akka-cluster-metrics scalafix akka-cluster-tools akka-cluster-tools compile and test scalafix akka-distributed-data akka-distributed-data fixes scalafix akka-persistence scalafix akka-cluster-sharding fix akka-cluster-sharding scalafix akka-contrib Fix akka-cluster-sharding-typed test scalafix akka-docs Use scala-stm 0.9 (released for M5) akka-docs Remove dependency on collections-compat Cherry-pick the relevant constructs to our own private utils Shorten 'scala.collections.immutable' by importing it Duplicate 'immutable' imports Use 'foreach' on futures Replace MapLike with regular Map Internal API markers Simplify ccompat by moving PackageShared into object Since we don't currently need to differentiate between 2.11 and Avoid relying on 'union' (and ++) being left-biased Fix akka-actor/doc by removing -Ywarn-unused Make more things more private Copyright headers Use 'unsorted' to go from SortedSet to Set Duplicate import Use onComplete rather than failed.foreach Clarify why we partly duplicate scala-collection-compat
This commit is contained in:
parent
3bff646218
commit
d274e039f9
141 changed files with 596 additions and 468 deletions
|
|
@ -11,6 +11,7 @@ import akka.annotation.InternalApi
|
|||
import akka.cluster.ddata.Key.KeyId
|
||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* INTERNAL API: Used by the Replicator actor.
|
||||
|
|
@ -144,7 +145,7 @@ import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholde
|
|||
}
|
||||
|
||||
private def deltaEntriesAfter(entries: TreeMap[Long, ReplicatedData], version: Long): TreeMap[Long, ReplicatedData] =
|
||||
entries.from(version) match {
|
||||
entries.rangeFrom(version) match {
|
||||
case ntrs if ntrs.isEmpty ⇒ ntrs
|
||||
case ntrs if ntrs.firstKey == version ⇒ ntrs.tail // exclude first, i.e. version j that was already sent
|
||||
case ntrs ⇒ ntrs
|
||||
|
|
|
|||
|
|
@ -53,6 +53,7 @@ import scala.collection.immutable.TreeSet
|
|||
import akka.cluster.MemberStatus
|
||||
import scala.annotation.varargs
|
||||
import akka.util.JavaDurationConverters._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object ReplicatorSettings {
|
||||
|
||||
|
|
@ -1109,13 +1110,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation = {
|
||||
// Important to include the pruning state in the deltas. For example if the delta is based
|
||||
// on an entry that has been pruned but that has not yet been performed on the target node.
|
||||
DeltaPropagation(selfUniqueAddress, reply = false, deltas.collect {
|
||||
DeltaPropagation(selfUniqueAddress, reply = false, deltas.iterator.collect {
|
||||
case (key, (d, fromSeqNr, toSeqNr)) if d != NoDeltaPlaceholder ⇒
|
||||
getData(key) match {
|
||||
case Some(envelope) ⇒ key → Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
|
||||
case None ⇒ key → Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
||||
}
|
||||
}(collection.breakOut))
|
||||
}.toMap)
|
||||
}
|
||||
}
|
||||
val deltaPropagationTask: Option[Cancellable] =
|
||||
|
|
@ -1461,9 +1462,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
|
||||
def receiveGetKeyIds(): Unit = {
|
||||
val keys: Set[KeyId] = dataEntries.collect {
|
||||
val keys: Set[KeyId] = dataEntries.iterator.collect {
|
||||
case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData ⇒ key
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.Set)
|
||||
replyTo ! GetKeyIdsResult(keys)
|
||||
}
|
||||
|
||||
|
|
@ -1700,14 +1701,14 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
if (keys.nonEmpty) {
|
||||
if (log.isDebugEnabled)
|
||||
log.debug("Sending gossip to [{}], containing [{}]", replyTo.path.address, keys.mkString(", "))
|
||||
val g = Gossip(keys.map(k ⇒ k → getData(k).get)(collection.breakOut), sendBack = otherDifferentKeys.nonEmpty)
|
||||
val g = Gossip(keys.iterator.map(k ⇒ k → getData(k).get).toMap, sendBack = otherDifferentKeys.nonEmpty)
|
||||
replyTo ! g
|
||||
}
|
||||
val myMissingKeys = otherKeys diff myKeys
|
||||
if (myMissingKeys.nonEmpty) {
|
||||
if (log.isDebugEnabled)
|
||||
log.debug("Sending gossip status to [{}], requesting missing [{}]", replyTo.path.address, myMissingKeys.mkString(", "))
|
||||
val status = Status(myMissingKeys.map(k ⇒ k → NotFoundDigest)(collection.breakOut), chunk, totChunks)
|
||||
val status = Status(myMissingKeys.iterator.map(k ⇒ k → NotFoundDigest).toMap, chunk, totChunks)
|
||||
replyTo ! status
|
||||
}
|
||||
}
|
||||
|
|
@ -1846,9 +1847,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
def initRemovedNodePruning(): Unit = {
|
||||
// initiate pruning for removed nodes
|
||||
val removedSet: Set[UniqueAddress] = removedNodes.collect {
|
||||
val removedSet: Set[UniqueAddress] = removedNodes.iterator.collect {
|
||||
case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) ⇒ r
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.Set)
|
||||
|
||||
if (removedSet.nonEmpty) {
|
||||
for ((key, (envelope, _)) ← dataEntries; removed ← removedSet) {
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import java.util.TreeSet
|
|||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.breakOut
|
||||
import scala.collection.immutable
|
||||
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.cluster.ddata._
|
||||
|
|
@ -28,6 +28,7 @@ import java.io.NotSerializableException
|
|||
import akka.actor.ActorRef
|
||||
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
|
||||
import akka.serialization.Serialization
|
||||
import akka.util.ccompat._
|
||||
|
||||
private object ReplicatedDataSerializer {
|
||||
/*
|
||||
|
|
@ -472,7 +473,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = {
|
||||
val deltaGroup = rd.ORSetDeltaGroup.parseFrom(bytes)
|
||||
val ops: Vector[ORSet.DeltaOp] =
|
||||
deltaGroup.getEntriesList.asScala.map { entry ⇒
|
||||
deltaGroup.getEntriesList.asScala.iterator.map { entry ⇒
|
||||
if (entry.getOperation == rd.ORSetDeltaOp.Add)
|
||||
ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||
else if (entry.getOperation == rd.ORSetDeltaOp.Remove)
|
||||
|
|
@ -481,7 +482,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||
else
|
||||
throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}")
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.Vector)
|
||||
ORSet.DeltaGroup(ops)
|
||||
}
|
||||
|
||||
|
|
@ -538,8 +539,8 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
gcounterFromProto(rd.GCounter.parseFrom(bytes))
|
||||
|
||||
def gcounterFromProto(gcounter: rd.GCounter): GCounter = {
|
||||
new GCounter(state = gcounter.getEntriesList.asScala.map(entry ⇒
|
||||
uniqueAddressFromProto(entry.getNode) → BigInt(entry.getValue.toByteArray))(breakOut))
|
||||
new GCounter(state = gcounter.getEntriesList.asScala.iterator.map(entry ⇒
|
||||
uniqueAddressFromProto(entry.getNode) → BigInt(entry.getValue.toByteArray)).toMap)
|
||||
}
|
||||
|
||||
def pncounterToProto(pncounter: PNCounter): rd.PNCounter =
|
||||
|
|
@ -670,7 +671,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
private def ormapDeltaGroupOpsFromBinary(bytes: Array[Byte]): scala.collection.immutable.IndexedSeq[ORMap.DeltaOp] = {
|
||||
val deltaGroup = rd.ORMapDeltaGroup.parseFrom(bytes)
|
||||
val ops: Vector[ORMap.DeltaOp] =
|
||||
deltaGroup.getEntriesList.asScala.map { entry ⇒
|
||||
deltaGroup.getEntriesList.asScala.iterator.map { entry ⇒
|
||||
if (entry.getOperation == rd.ORMapDeltaOp.ORMapPut) {
|
||||
val map = singleMapEntryFromProto(entry.getEntryDataList, (v: dm.OtherMessage) ⇒ otherMessageFromProto(v).asInstanceOf[ReplicatedData])
|
||||
ORMap.PutDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map.head, zeroTagFromCode(entry.getZeroTag))
|
||||
|
|
@ -684,31 +685,31 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
ORMap.UpdateDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map, zeroTagFromCode(entry.getZeroTag))
|
||||
} else
|
||||
throw new NotSerializableException(s"Unknown ORMap delta operation ${entry.getOperation}")
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.Vector)
|
||||
ops
|
||||
}
|
||||
|
||||
private def ormapPutToProto(deltaOp: ORMap.PutDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
}
|
||||
|
||||
private def ormapRemoveToProto(deltaOp: ORMap.RemoveDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
}
|
||||
|
||||
private def ormapRemoveKeyToProto(deltaOp: ORMap.RemoveKeyDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
}
|
||||
|
||||
private def ormapUpdateToProto(deltaOp: ORMap.UpdateDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
}
|
||||
|
||||
private def ormapDeltaGroupToProto(deltaGroup: ORMap.DeltaGroup[_, _]): rd.ORMapDeltaGroup = {
|
||||
ormapDeltaGroupOpsToProto(deltaGroup.ops)
|
||||
}
|
||||
|
||||
private def ormapDeltaGroupOpsToProto(deltaGroupOps: scala.collection.immutable.IndexedSeq[ORMap.DeltaOp]): rd.ORMapDeltaGroup = {
|
||||
private def ormapDeltaGroupOpsToProto(deltaGroupOps: immutable.IndexedSeq[ORMap.DeltaOp]): rd.ORMapDeltaGroup = {
|
||||
def createEntry(opType: rd.ORMapDeltaOp, u: ORSet[_], m: Map[_, _], zt: Int) = {
|
||||
if (m.size > 1 && opType != rd.ORMapDeltaOp.ORMapUpdate)
|
||||
throw new IllegalArgumentException("Invalid size of ORMap delta map")
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.cluster.ddata.protobuf
|
|||
import scala.concurrent.duration._
|
||||
import java.util.concurrent.TimeUnit
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.breakOut
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.Duration
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.cluster.Member
|
||||
|
|
@ -32,6 +32,7 @@ import akka.actor.Address
|
|||
import akka.cluster.ddata.VersionVector
|
||||
import akka.annotation.InternalApi
|
||||
import akka.cluster.ddata.PruningState.PruningPerformed
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -268,8 +269,8 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
private def statusFromBinary(bytes: Array[Byte]): Status = {
|
||||
val status = dm.Status.parseFrom(bytes)
|
||||
Status(
|
||||
status.getEntriesList.asScala.map(e ⇒
|
||||
e.getKey → AkkaByteString(e.getDigest.toByteArray()))(breakOut),
|
||||
status.getEntriesList.asScala.iterator.map(e ⇒
|
||||
e.getKey → AkkaByteString(e.getDigest.toByteArray())).toMap,
|
||||
status.getChunk, status.getTotChunks)
|
||||
}
|
||||
|
||||
|
|
@ -287,8 +288,8 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
|
||||
val gossip = dm.Gossip.parseFrom(decompress(bytes))
|
||||
Gossip(
|
||||
gossip.getEntriesList.asScala.map(e ⇒
|
||||
e.getKey → dataEnvelopeFromProto(e.getEnvelope))(breakOut),
|
||||
gossip.getEntriesList.asScala.iterator.map(e ⇒
|
||||
e.getKey → dataEnvelopeFromProto(e.getEnvelope)).toMap,
|
||||
sendBack = gossip.getSendBack)
|
||||
}
|
||||
|
||||
|
|
@ -316,11 +317,11 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
DeltaPropagation(
|
||||
uniqueAddressFromProto(deltaPropagation.getFromNode),
|
||||
reply,
|
||||
deltaPropagation.getEntriesList.asScala.map { e ⇒
|
||||
deltaPropagation.getEntriesList.asScala.iterator.map { e ⇒
|
||||
val fromSeqNr = e.getFromSeqNr
|
||||
val toSeqNr = if (e.hasToSeqNr) e.getToSeqNr else fromSeqNr
|
||||
e.getKey → Delta(dataEnvelopeFromProto(e.getEnvelope), fromSeqNr, toSeqNr)
|
||||
}(breakOut))
|
||||
}.toMap)
|
||||
}
|
||||
|
||||
private def getToProto(get: Get[_]): dm.Get = {
|
||||
|
|
@ -482,7 +483,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
if (pruningEntries.isEmpty)
|
||||
Map.empty
|
||||
else
|
||||
pruningEntries.asScala.map { pruningEntry ⇒
|
||||
pruningEntries.asScala.iterator.map { pruningEntry ⇒
|
||||
val state =
|
||||
if (pruningEntry.getPerformed) {
|
||||
// for wire compatibility with Akka 2.4.x
|
||||
|
|
@ -491,10 +492,10 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
} else
|
||||
PruningState.PruningInitialized(
|
||||
uniqueAddressFromProto(pruningEntry.getOwnerAddress),
|
||||
pruningEntry.getSeenList.asScala.map(addressFromProto)(breakOut))
|
||||
pruningEntry.getSeenList.asScala.iterator.map(addressFromProto).to(immutable.Set))
|
||||
val removed = uniqueAddressFromProto(pruningEntry.getRemovedAddress)
|
||||
removed → state
|
||||
}(breakOut)
|
||||
}.toMap
|
||||
}
|
||||
|
||||
private def writeToProto(write: Write): dm.Write =
|
||||
|
|
|
|||
|
|
@ -11,7 +11,6 @@ import java.util.zip.GZIPOutputStream
|
|||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable.TreeMap
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.breakOut
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
import akka.actor.ExtendedActorSystem
|
||||
|
|
@ -21,6 +20,7 @@ import akka.serialization._
|
|||
import akka.protobuf.ByteString
|
||||
import akka.protobuf.MessageLite
|
||||
import akka.cluster.ddata.VersionVector
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* Some useful serialization helper methods.
|
||||
|
|
@ -124,8 +124,8 @@ trait SerializationSupport {
|
|||
else if (entries.size == 1)
|
||||
VersionVector(uniqueAddressFromProto(entries.get(0).getNode), entries.get(0).getVersion)
|
||||
else {
|
||||
val versions: TreeMap[UniqueAddress, Long] = versionVector.getEntriesList.asScala.map(entry ⇒
|
||||
uniqueAddressFromProto(entry.getNode) → entry.getVersion)(breakOut)
|
||||
val versions: TreeMap[UniqueAddress, Long] = scala.collection.immutable.TreeMap.from(versionVector.getEntriesList.asScala.iterator.map(entry ⇒
|
||||
uniqueAddressFromProto(entry.getNode) → entry.getVersion))
|
||||
VersionVector(versions)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -136,7 +136,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
|||
expectTerminated(r)
|
||||
|
||||
var r2: ActorRef = null
|
||||
awaitAssert(r2 = newReplicator()) // try until name is free
|
||||
awaitAssert { r2 = newReplicator() } // try until name is free
|
||||
|
||||
// note that it will stash the commands until loading completed
|
||||
r2 ! Get(KeyA, ReadLocal)
|
||||
|
|
@ -184,7 +184,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
|||
expectTerminated(r)
|
||||
|
||||
var r2: ActorRef = null
|
||||
awaitAssert(r2 = newReplicator()) // try until name is free
|
||||
awaitAssert { r2 = newReplicator() } // try until name is free
|
||||
awaitAssert {
|
||||
r2 ! GetKeyIds
|
||||
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
|
||||
|
|
@ -221,7 +221,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
|||
expectTerminated(r)
|
||||
|
||||
var r2: ActorRef = null
|
||||
awaitAssert(r2 = newReplicator()) // try until name is free
|
||||
awaitAssert { r2 = newReplicator() } // try until name is free
|
||||
awaitAssert {
|
||||
r2 ! GetKeyIds
|
||||
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import akka.actor.ActorSystem
|
|||
import akka.actor.ActorRef
|
||||
import scala.concurrent.Await
|
||||
import akka.cluster.MemberStatus
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object DurablePruningSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -76,9 +77,9 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
|||
Cluster(sys2).join(node(first).address)
|
||||
awaitAssert({
|
||||
Cluster(system).state.members.size should ===(4)
|
||||
Cluster(system).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
Cluster(system).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
Cluster(sys2).state.members.size should ===(4)
|
||||
Cluster(sys2).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
Cluster(sys2).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
}, 10.seconds)
|
||||
enterBarrier("joined")
|
||||
|
||||
|
|
|
|||
|
|
@ -21,8 +21,8 @@ object DeltaPropagationSelectorSpec {
|
|||
override val allNodes: Vector[Address]) extends DeltaPropagationSelector {
|
||||
override val gossipIntervalDivisor = 5
|
||||
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
|
||||
DeltaPropagation(selfUniqueAddress, false, deltas.mapValues {
|
||||
case (d, fromSeqNr, toSeqNr) ⇒ Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
||||
DeltaPropagation(selfUniqueAddress, false, deltas.map {
|
||||
case (key, (d, fromSeqNr, toSeqNr)) ⇒ (key, Delta(DataEnvelope(d), fromSeqNr, toSeqNr))
|
||||
})
|
||||
override def maxDeltaSize: Int = 10
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,8 +24,8 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
}
|
||||
|
||||
"be able to have its entries correctly merged with another LWWMap with other entries" in {
|
||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
||||
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||
|
||||
// merge both ways
|
||||
val expected = Map("a" → 1, "b" → 2, "c" → 3)
|
||||
|
|
@ -34,8 +34,8 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
}
|
||||
|
||||
"be able to remove entry" in {
|
||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
||||
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||
|
||||
val merged1 = m1 merge m2
|
||||
|
||||
|
|
@ -48,8 +48,8 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
}
|
||||
|
||||
"be able to work with deltas" in {
|
||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
||||
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||
|
||||
val expected = Map("a" → 1, "b" → 2, "c" → 3)
|
||||
(m1 merge m2).entries should be(expected)
|
||||
|
|
@ -69,7 +69,7 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
}
|
||||
|
||||
"have unapply extractor" in {
|
||||
val m1 = LWWMap.empty.put(node1, "a", 1L, defaultClock[Long])
|
||||
val m1 = LWWMap.empty[String, Long].put(node1, "a", 1L, defaultClock[Long])
|
||||
val LWWMap(entries1) = m1
|
||||
val entries2: Map[String, Long] = entries1
|
||||
Changed(LWWMapKey[String, Long]("key"))(m1) match {
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue