Update to a working version of Scalariform
This commit is contained in:
parent
cae070bd93
commit
c66ce62d63
616 changed files with 5966 additions and 5436 deletions
|
|
@ -20,7 +20,7 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
|
||||
"be able to set entries" in {
|
||||
val m = LWWMap.empty[Int].put(node1, "a", 1, defaultClock[Int]).put(node2, "b", 2, defaultClock[Int])
|
||||
m.entries should be(Map("a" -> 1, "b" -> 2))
|
||||
m.entries should be(Map("a" → 1, "b" → 2))
|
||||
}
|
||||
|
||||
"be able to have its entries correctly merged with another LWWMap with other entries" in {
|
||||
|
|
@ -28,7 +28,7 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
||||
|
||||
// merge both ways
|
||||
val expected = Map("a" -> 1, "b" -> 2, "c" -> 3)
|
||||
val expected = Map("a" → 1, "b" → 2, "c" → 3)
|
||||
(m1 merge m2).entries should be(expected)
|
||||
(m2 merge m1).entries should be(expected)
|
||||
}
|
||||
|
|
@ -40,11 +40,11 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
val merged1 = m1 merge m2
|
||||
|
||||
val m3 = merged1.remove(node1, "b")
|
||||
(merged1 merge m3).entries should be(Map("a" -> 1, "c" -> 3))
|
||||
(merged1 merge m3).entries should be(Map("a" → 1, "c" → 3))
|
||||
|
||||
// but if there is a conflicting update the entry is not removed
|
||||
val m4 = merged1.put(node2, "b", 22, defaultClock[Int])
|
||||
(m3 merge m4).entries should be(Map("a" -> 1, "b" -> 22, "c" -> 3))
|
||||
(m3 merge m4).entries should be(Map("a" → 1, "b" → 22, "c" → 3))
|
||||
}
|
||||
|
||||
"have unapply extractor" in {
|
||||
|
|
@ -55,7 +55,7 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
case c @ Changed(LWWMapKey("key")) ⇒
|
||||
val LWWMap(entries3) = c.dataValue
|
||||
val entries4: Map[String, Long] = entries3
|
||||
entries4 should be(Map("a" -> 1L))
|
||||
entries4 should be(Map("a" → 1L))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -41,7 +41,8 @@ class LocalConcurrencySpec(_system: ActorSystem) extends TestKit(_system)
|
|||
import LocalConcurrencySpec._
|
||||
|
||||
def this() {
|
||||
this(ActorSystem("LocalConcurrencySpec",
|
||||
this(ActorSystem(
|
||||
"LocalConcurrencySpec",
|
||||
ConfigFactory.parseString("""
|
||||
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
||||
akka.remote.netty.tcp.port=0
|
||||
|
|
|
|||
|
|
@ -197,7 +197,7 @@ class ORMapSpec extends WordSpec with Matchers {
|
|||
case c @ Changed(ORMapKey("key")) ⇒
|
||||
val ORMap(entries3) = c.dataValue
|
||||
val entries4: Map[String, ReplicatedData] = entries3
|
||||
entries4 should be(Map("a" -> Flag(true), "b" -> Flag(false)))
|
||||
entries4 should be(Map("a" → Flag(true), "b" → Flag(false)))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -17,20 +17,20 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
|||
|
||||
"be able to add entries" in {
|
||||
val m = ORMultiMap().addBinding(node1, "a", "A").addBinding(node1, "b", "B")
|
||||
m.entries should be(Map("a" -> Set("A"), "b" -> Set("B")))
|
||||
m.entries should be(Map("a" → Set("A"), "b" → Set("B")))
|
||||
|
||||
val m2 = m.addBinding(node1, "a", "C")
|
||||
m2.entries should be(Map("a" -> Set("A", "C"), "b" -> Set("B")))
|
||||
m2.entries should be(Map("a" → Set("A", "C"), "b" → Set("B")))
|
||||
}
|
||||
|
||||
"be able to remove entry" in {
|
||||
val m = ORMultiMap().addBinding(node1, "a", "A").addBinding(node1, "b", "B").removeBinding(node1, "a", "A")
|
||||
m.entries should be(Map("b" -> Set("B")))
|
||||
m.entries should be(Map("b" → Set("B")))
|
||||
}
|
||||
|
||||
"be able to replace an entry" in {
|
||||
val m = ORMultiMap().addBinding(node1, "a", "A").replaceBinding(node1, "a", "A", "B")
|
||||
m.entries should be(Map("a" -> Set("B")))
|
||||
m.entries should be(Map("a" → Set("B")))
|
||||
}
|
||||
|
||||
"be able to have its entries correctly merged with another ORMultiMap with other entries" in {
|
||||
|
|
@ -40,9 +40,9 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
|||
// merge both ways
|
||||
|
||||
val expectedMerge = Map(
|
||||
"a" -> Set("A"),
|
||||
"b" -> Set("B"),
|
||||
"c" -> Set("C"))
|
||||
"a" → Set("A"),
|
||||
"b" → Set("B"),
|
||||
"c" → Set("C"))
|
||||
|
||||
val merged1 = m1 merge m2
|
||||
merged1.entries should be(expectedMerge)
|
||||
|
|
@ -67,10 +67,10 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
|||
// merge both ways
|
||||
|
||||
val expectedMerged = Map(
|
||||
"a" -> Set("A2"),
|
||||
"b" -> Set("B1"),
|
||||
"c" -> Set("C2"),
|
||||
"d" -> Set("D1", "D2"))
|
||||
"a" → Set("A2"),
|
||||
"b" → Set("B1"),
|
||||
"c" → Set("C2"),
|
||||
"d" → Set("D1", "D2"))
|
||||
|
||||
val merged1 = m1 merge m2
|
||||
merged1.entries should be(expectedMerged)
|
||||
|
|
@ -89,8 +89,8 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
|||
val m2 = m.put(node1, "a", a - "A1")
|
||||
|
||||
val expectedMerged = Map(
|
||||
"a" -> Set("A2"),
|
||||
"b" -> Set("B1"))
|
||||
"a" → Set("A2"),
|
||||
"b" → Set("B1"))
|
||||
|
||||
m2.entries should be(expectedMerged)
|
||||
}
|
||||
|
|
@ -104,7 +104,7 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
|||
"remove all bindings for a given key" in {
|
||||
val m = ORMultiMap().addBinding(node1, "a", "A1").addBinding(node1, "a", "A2").addBinding(node1, "b", "B1")
|
||||
val m2 = m.remove(node1, "a")
|
||||
m2.entries should be(Map("b" -> Set("B1")))
|
||||
m2.entries should be(Map("b" → Set("B1")))
|
||||
}
|
||||
|
||||
"have unapply extractor" in {
|
||||
|
|
@ -116,7 +116,7 @@ class ORMultiMapSpec extends WordSpec with Matchers {
|
|||
case c @ Changed(ORMultiMapKey("key")) ⇒
|
||||
val ORMultiMap(entries3) = c.dataValue
|
||||
val entries4: Map[String, Set[Long]] = entries3
|
||||
entries4 should be(Map("a" -> Set(1L, 2L), "b" -> Set(3L)))
|
||||
entries4 should be(Map("a" → Set(1L, 2L), "b" → Set(3L)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -228,30 +228,30 @@ class ORSetSpec extends WordSpec with Matchers {
|
|||
|
||||
"ORSet unit test" must {
|
||||
"verify subtractDots" in {
|
||||
val dot = VersionVector(TreeMap(nodeA -> 3L, nodeB -> 2L, nodeD -> 14L, nodeG -> 22L))
|
||||
val vvector = VersionVector(TreeMap(nodeA -> 4L, nodeB -> 1L, nodeC -> 1L, nodeD -> 14L, nodeE -> 5L, nodeF -> 2L))
|
||||
val expected = VersionVector(TreeMap(nodeB -> 2L, nodeG -> 22L))
|
||||
val dot = VersionVector(TreeMap(nodeA → 3L, nodeB → 2L, nodeD → 14L, nodeG → 22L))
|
||||
val vvector = VersionVector(TreeMap(nodeA → 4L, nodeB → 1L, nodeC → 1L, nodeD → 14L, nodeE → 5L, nodeF → 2L))
|
||||
val expected = VersionVector(TreeMap(nodeB → 2L, nodeG → 22L))
|
||||
ORSet.subtractDots(dot, vvector) should be(expected)
|
||||
}
|
||||
|
||||
"verify mergeCommonKeys" in {
|
||||
val commonKeys: Set[String] = Set("K1", "K2")
|
||||
val thisDot1 = VersionVector(TreeMap(nodeA -> 3L, nodeD -> 7L))
|
||||
val thisDot2 = VersionVector(TreeMap(nodeB -> 5L, nodeC -> 2L))
|
||||
val thisVvector = VersionVector(TreeMap(nodeA -> 3L, nodeB -> 5L, nodeC -> 2L, nodeD -> 7L))
|
||||
val thisDot1 = VersionVector(TreeMap(nodeA → 3L, nodeD → 7L))
|
||||
val thisDot2 = VersionVector(TreeMap(nodeB → 5L, nodeC → 2L))
|
||||
val thisVvector = VersionVector(TreeMap(nodeA → 3L, nodeB → 5L, nodeC → 2L, nodeD → 7L))
|
||||
val thisSet = new ORSet(
|
||||
elementsMap = Map("K1" -> thisDot1, "K2" -> thisDot2),
|
||||
elementsMap = Map("K1" → thisDot1, "K2" → thisDot2),
|
||||
vvector = thisVvector)
|
||||
val thatDot1 = VersionVector(nodeA, 3L)
|
||||
val thatDot2 = VersionVector(nodeB, 6L)
|
||||
val thatVvector = VersionVector(TreeMap(nodeA -> 3L, nodeB -> 6L, nodeC -> 1L, nodeD -> 8L))
|
||||
val thatVvector = VersionVector(TreeMap(nodeA → 3L, nodeB → 6L, nodeC → 1L, nodeD → 8L))
|
||||
val thatSet = new ORSet(
|
||||
elementsMap = Map("K1" -> thatDot1, "K2" -> thatDot2),
|
||||
elementsMap = Map("K1" → thatDot1, "K2" → thatDot2),
|
||||
vvector = thatVvector)
|
||||
|
||||
val expectedDots = Map(
|
||||
"K1" -> VersionVector(nodeA, 3L),
|
||||
"K2" -> VersionVector(TreeMap(nodeB -> 6L, nodeC -> 2L)))
|
||||
"K1" → VersionVector(nodeA, 3L),
|
||||
"K2" → VersionVector(TreeMap(nodeB → 6L, nodeC → 2L)))
|
||||
|
||||
ORSet.mergeCommonKeys(commonKeys, thisSet, thatSet) should be(expectedDots)
|
||||
}
|
||||
|
|
@ -259,14 +259,14 @@ class ORSetSpec extends WordSpec with Matchers {
|
|||
"verify mergeDisjointKeys" in {
|
||||
val keys: Set[Any] = Set("K3", "K4", "K5")
|
||||
val elements: Map[Any, VersionVector] = Map(
|
||||
"K3" -> VersionVector(nodeA, 4L),
|
||||
"K4" -> VersionVector(TreeMap(nodeA -> 3L, nodeD -> 8L)),
|
||||
"K5" -> VersionVector(nodeA, 2L))
|
||||
val vvector = VersionVector(TreeMap(nodeA -> 3L, nodeD -> 7L))
|
||||
val acc: Map[Any, VersionVector] = Map("K1" -> VersionVector(nodeA, 3L))
|
||||
"K3" → VersionVector(nodeA, 4L),
|
||||
"K4" → VersionVector(TreeMap(nodeA → 3L, nodeD → 8L)),
|
||||
"K5" → VersionVector(nodeA, 2L))
|
||||
val vvector = VersionVector(TreeMap(nodeA → 3L, nodeD → 7L))
|
||||
val acc: Map[Any, VersionVector] = Map("K1" → VersionVector(nodeA, 3L))
|
||||
val expectedDots = acc ++ Map(
|
||||
"K3" -> VersionVector(nodeA, 4L),
|
||||
"K4" -> VersionVector(nodeD, 8L)) // "a" -> 3 removed, optimized to include only those unseen
|
||||
"K3" → VersionVector(nodeA, 4L),
|
||||
"K4" → VersionVector(nodeD, 8L)) // "a" -> 3 removed, optimized to include only those unseen
|
||||
|
||||
ORSet.mergeDisjointKeys(keys, elements, vvector, acc) should be(expectedDots)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ class PNCounterMapSpec extends WordSpec with Matchers {
|
|||
|
||||
"be able to increment and decrement entries" in {
|
||||
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))
|
||||
}
|
||||
|
||||
"be able to have its entries correctly merged with another ORMap with other entries" in {
|
||||
|
|
@ -27,7 +27,7 @@ class PNCounterMapSpec extends WordSpec with Matchers {
|
|||
val m2 = PNCounterMap().increment(node2, "c", 5)
|
||||
|
||||
// merge both ways
|
||||
val expected = Map("a" -> 1, "b" -> 3, "c" -> 7)
|
||||
val expected = Map("a" → 1, "b" → 3, "c" → 7)
|
||||
(m1 merge m2).entries should be(expected)
|
||||
(m2 merge m1).entries should be(expected)
|
||||
}
|
||||
|
|
@ -39,11 +39,11 @@ class PNCounterMapSpec extends WordSpec with Matchers {
|
|||
val merged1 = m1 merge m2
|
||||
|
||||
val m3 = merged1.remove(node1, "b")
|
||||
(merged1 merge m3).entries should be(Map("a" -> 1, "c" -> 7))
|
||||
(merged1 merge m3).entries should be(Map("a" → 1, "c" → 7))
|
||||
|
||||
// but if there is a conflicting update the entry is not removed
|
||||
val m4 = merged1.increment(node2, "b", 10)
|
||||
(m3 merge m4).entries should be(Map("a" -> 1, "b" -> 13, "c" -> 7))
|
||||
(m3 merge m4).entries should be(Map("a" → 1, "b" → 13, "c" → 7))
|
||||
}
|
||||
|
||||
"have unapply extractor" in {
|
||||
|
|
@ -54,7 +54,7 @@ class PNCounterMapSpec extends WordSpec with Matchers {
|
|||
case c @ Changed(PNCounterMapKey("key")) ⇒
|
||||
val PNCounterMap(entries3) = c.dataValue
|
||||
val entries4: Map[String, BigInt] = entries3
|
||||
entries4 should be(Map("a" -> 1L, "b" -> 2L))
|
||||
entries4 should be(Map("a" → 1L, "b" → 2L))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -68,7 +68,7 @@ class WriteAggregatorSpec extends AkkaSpec("""
|
|||
val writeMajority = WriteMajority(timeout)
|
||||
|
||||
def probes(probe: ActorRef): Map[Address, ActorRef] =
|
||||
nodes.toSeq.map(_ -> system.actorOf(WriteAggregatorSpec.writeAckAdapterProps(probe))).toMap
|
||||
nodes.toSeq.map(_ → system.actorOf(WriteAggregatorSpec.writeAckAdapterProps(probe))).toMap
|
||||
|
||||
"WriteAggregator" must {
|
||||
"send to at least N/2+1 replicas when WriteMajority" in {
|
||||
|
|
|
|||
|
|
@ -25,7 +25,8 @@ import akka.testkit.TestKit
|
|||
import akka.cluster.UniqueAddress
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
class ReplicatedDataSerializerSpec extends TestKit(ActorSystem("ReplicatedDataSerializerSpec",
|
||||
class ReplicatedDataSerializerSpec extends TestKit(ActorSystem(
|
||||
"ReplicatedDataSerializerSpec",
|
||||
ConfigFactory.parseString("""
|
||||
akka.actor.provider=akka.cluster.ClusterActorRefProvider
|
||||
akka.remote.netty.tcp.port=0
|
||||
|
|
|
|||
|
|
@ -23,7 +23,8 @@ import akka.util.ByteString
|
|||
import akka.cluster.UniqueAddress
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem("ReplicatorMessageSerializerSpec",
|
||||
class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem(
|
||||
"ReplicatorMessageSerializerSpec",
|
||||
ConfigFactory.parseString("""
|
||||
akka.actor.provider=akka.cluster.ClusterActorRefProvider
|
||||
akka.remote.netty.tcp.port=0
|
||||
|
|
@ -64,17 +65,19 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem("ReplicatorMes
|
|||
checkSerialization(Changed(keyA)(data1))
|
||||
checkSerialization(DataEnvelope(data1))
|
||||
checkSerialization(DataEnvelope(data1, pruning = Map(
|
||||
address1 -> PruningState(address2, PruningPerformed),
|
||||
address3 -> PruningState(address2, PruningInitialized(Set(address1.address))))))
|
||||
address1 → PruningState(address2, PruningPerformed),
|
||||
address3 → PruningState(address2, PruningInitialized(Set(address1.address))))))
|
||||
checkSerialization(Write("A", DataEnvelope(data1)))
|
||||
checkSerialization(WriteAck)
|
||||
checkSerialization(Read("A"))
|
||||
checkSerialization(ReadResult(Some(DataEnvelope(data1))))
|
||||
checkSerialization(ReadResult(None))
|
||||
checkSerialization(Status(Map("A" -> ByteString.fromString("a"),
|
||||
"B" -> ByteString.fromString("b")), chunk = 3, totChunks = 10))
|
||||
checkSerialization(Gossip(Map("A" -> DataEnvelope(data1),
|
||||
"B" -> DataEnvelope(GSet() + "b" + "c")), sendBack = true))
|
||||
checkSerialization(Status(Map(
|
||||
"A" → ByteString.fromString("a"),
|
||||
"B" → ByteString.fromString("b")), chunk = 3, totChunks = 10))
|
||||
checkSerialization(Gossip(Map(
|
||||
"A" → DataEnvelope(data1),
|
||||
"B" → DataEnvelope(GSet() + "b" + "c")), sendBack = true))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -141,7 +144,7 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem("ReplicatorMes
|
|||
"handle Int wrap around" ignore { // ignored because it takes 20 seconds (but it works)
|
||||
val cache = new SmallCache[Read, String](2, 5.seconds, _ ⇒ null)
|
||||
val a = Read("a")
|
||||
val x = a -> "A"
|
||||
val x = a → "A"
|
||||
var n = 0
|
||||
while (n <= Int.MaxValue - 3) {
|
||||
cache.add(x)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue