pekko/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/DurableDataSpec.scala

343 lines
10 KiB
Scala
Raw Normal View History

/*
* Copyright (C) 2016-2020 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.cluster.ddata
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Props
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import org.scalatest.CancelAfterFailure
final case class DurableDataSpecConfig(writeBehind: Boolean) extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(ConfigFactory.parseString(s"""akka.loglevel = DEBUG
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.log-dead-letters-during-shutdown = off
akka.cluster.distributed-data.durable.keys = ["durable*"]
akka.cluster.distributed-data.durable.lmdb {
dir = target/DurableDataSpec-${System.currentTimeMillis}-ddata
map-size = 10 MiB
write-behind-interval = ${if (writeBehind) "200ms" else "off"}
}
# initialization of lmdb can be very slow in CI environment
akka.test.single-expect-default = 15s
"""))
}
object DurableDataSpec {
def testDurableStoreProps(failLoad: Boolean = false, failStore: Boolean = false): Props =
Props(new TestDurableStore(failLoad, failStore))
class TestDurableStore(failLoad: Boolean, failStore: Boolean) extends Actor {
import DurableStore._
def receive = {
case LoadAll =>
if (failLoad)
throw new LoadFailed("TestDurableStore: failed to load durable distributed-data") with NoStackTrace
else
sender() ! LoadAllCompleted
case Store(_, _, reply) =>
if (failStore) reply match {
case Some(StoreReply(_, failureMsg, replyTo)) => replyTo ! failureMsg
case None =>
2019-03-11 10:38:24 +01:00
} else
reply match {
case Some(StoreReply(successMsg, _, replyTo)) => replyTo ! successMsg
case None =>
}
}
}
}
class DurableDataSpecMultiJvmNode1 extends DurableDataSpec(DurableDataSpecConfig(writeBehind = false))
class DurableDataSpecMultiJvmNode2 extends DurableDataSpec(DurableDataSpecConfig(writeBehind = false))
class DurableDataWriteBehindSpecMultiJvmNode1 extends DurableDataSpec(DurableDataSpecConfig(writeBehind = true))
class DurableDataWriteBehindSpecMultiJvmNode2 extends DurableDataSpec(DurableDataSpecConfig(writeBehind = true))
abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
2019-03-11 10:38:24 +01:00
extends MultiNodeSpec(multiNodeConfig)
with STMultiNodeSpec
with ImplicitSender
with CancelAfterFailure {
import DurableDataSpec._
import Replicator._
import multiNodeConfig._
override def initialParticipants = roles.size
val cluster = Cluster(system)
implicit val selfUniqueAddress: SelfUniqueAddress = DistributedData(system).selfUniqueAddress
val timeout = 14.seconds.dilated // initialization of lmdb can be very slow in CI environment
val writeTwo = WriteTo(2, timeout)
val readTwo = ReadFrom(2, timeout)
val KeyA = GCounterKey("durable-A")
val KeyB = GCounterKey("durable-B")
val KeyC = ORSetKey[String]("durable-C")
var testStepCounter = 0
def enterBarrierAfterTestStep(): Unit = {
testStepCounter += 1
enterBarrier("after-" + testStepCounter)
}
2019-03-11 10:38:24 +01:00
def newReplicator(sys: ActorSystem = system) =
2019-03-13 10:56:20 +01:00
sys.actorOf(
Replicator.props(ReplicatorSettings(system).withGossipInterval(1.second)),
"replicator-" + testStepCounter)
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
2019-03-11 10:38:24 +01:00
cluster.join(node(to).address)
}
enterBarrier(from.name + "-joined")
}
"Durable CRDT" must {
"work in single node cluster" in {
join(first, first)
runOn(first) {
val r = newReplicator()
2016-12-21 16:44:39 +01:00
within(10.seconds) {
awaitAssert {
r ! GetReplicaCount
expectMsg(ReplicaCount(1))
}
}
r ! Get(KeyA, ReadLocal)
expectMsg(NotFound(KeyA, None))
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
expectMsg(UpdateSuccess(KeyA, None))
expectMsg(UpdateSuccess(KeyA, None))
expectMsg(UpdateSuccess(KeyA, None))
watch(r)
system.stop(r)
expectTerminated(r)
var r2: ActorRef = null
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
2018-11-22 16:18:10 +01:00
awaitAssert { r2 = newReplicator() } // try until name is free
// note that it will stash the commands until loading completed
r2 ! Get(KeyA, ReadLocal)
expectMsgType[GetSuccess[GCounter]].dataValue.value.toInt should be(3)
watch(r2)
system.stop(r2)
expectTerminated(r2)
}
enterBarrierAfterTestStep()
}
}
"work in multi node cluster" in {
join(second, first)
val r = newReplicator()
2016-12-21 16:44:39 +01:00
within(10.seconds) {
awaitAssert {
r ! GetReplicaCount
expectMsg(ReplicaCount(2))
}
}
enterBarrier("both-initialized")
r ! Update(KeyA, GCounter(), writeTwo)(_ :+ 1)
expectMsg(UpdateSuccess(KeyA, None))
r ! Update(KeyC, ORSet.empty[String], writeTwo)(_ :+ myself.name)
expectMsg(UpdateSuccess(KeyC, None))
enterBarrier("update-done-" + testStepCounter)
r ! Get(KeyA, readTwo)
expectMsgType[GetSuccess[GCounter]].dataValue.value.toInt should be(2)
r ! Get(KeyC, readTwo)
expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements should be(Set(first.name, second.name))
enterBarrier("values-verified-" + testStepCounter)
watch(r)
system.stop(r)
expectTerminated(r)
var r2: ActorRef = null
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
2018-11-22 16:18:10 +01:00
awaitAssert { r2 = newReplicator() } // try until name is free
awaitAssert {
r2 ! GetKeyIds
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
}
r2 ! Get(KeyA, ReadLocal)
expectMsgType[GetSuccess[GCounter]].dataValue.value.toInt should be(2)
r2 ! Get(KeyC, ReadLocal)
expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements should be(Set(first.name, second.name))
enterBarrierAfterTestStep()
}
"be durable after gossip update" in {
val r = newReplicator()
runOn(first) {
// FIXME
log.debug("sending message with sender: {}", implicitly[ActorRef])
r ! Update(KeyC, ORSet.empty[String], WriteLocal)(_ :+ myself.name)
expectMsg(UpdateSuccess(KeyC, None))
}
runOn(second) {
r ! Subscribe(KeyC, testActor)
expectMsgType[Changed[ORSet[String]]].dataValue.elements should be(Set(first.name))
// must do one more roundtrip to be sure that it keyB is stored, since Changed might have
// been sent out before storage
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
expectMsg(UpdateSuccess(KeyA, None))
watch(r)
system.stop(r)
expectTerminated(r)
val r2 = awaitAssert { newReplicator() } // try until name is free
awaitAssert {
r2 ! GetKeyIds
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
}
r2 ! Get(KeyC, ReadLocal)
expectMsgType[GetSuccess[ORSet[String]]].dataValue.elements should be(Set(first.name))
}
enterBarrierAfterTestStep()
}
"handle Update before load" in {
runOn(first) {
val sys1 = ActorSystem("AdditionalSys", system.settings.config)
val address = Cluster(sys1).selfAddress
try {
Cluster(sys1).join(address)
new TestKit(sys1) with ImplicitSender {
val r = newReplicator(sys1)
2016-12-21 16:44:39 +01:00
within(10.seconds) {
awaitAssert {
r ! GetReplicaCount
expectMsg(ReplicaCount(1))
}
}
r ! Get(KeyA, ReadLocal)
expectMsg(NotFound(KeyA, None))
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
r ! Update(KeyA, GCounter(), WriteLocal)(_ :+ 1)
r ! Update(KeyB, GCounter(), WriteLocal)(_ :+ 1)
expectMsg(UpdateSuccess(KeyA, None))
expectMsg(UpdateSuccess(KeyA, None))
expectMsg(UpdateSuccess(KeyA, None))
expectMsg(UpdateSuccess(KeyB, None))
watch(r)
system.stop(r)
expectTerminated(r)
}
} finally {
Await.ready(sys1.terminate(), 10.seconds)
}
2019-03-13 10:56:20 +01:00
val sys2 = ActorSystem(
"AdditionalSys",
// use the same port
ConfigFactory.parseString(s"""
akka.remote.artery.canonical.port = ${address.port.get}
akka.remote.classic.netty.tcp.port = ${address.port.get}
""").withFallback(system.settings.config))
try {
Cluster(sys2).join(address)
new TestKit(sys2) with ImplicitSender {
val r2: ActorRef = newReplicator(sys2)
// it should be possible to update while loading is in progress
r2 ! Update(KeyB, GCounter(), WriteLocal)(_ :+ 1)
expectMsg(UpdateSuccess(KeyB, None))
// wait until all loaded
awaitAssert {
r2 ! GetKeyIds
expectMsgType[GetKeyIdsResult].keyIds should ===(Set(KeyA.id, KeyB.id))
}
r2 ! Get(KeyA, ReadLocal)
expectMsgType[GetSuccess[GCounter]].dataValue.value.toInt should be(3)
r2 ! Get(KeyB, ReadLocal)
expectMsgType[GetSuccess[GCounter]].dataValue.value.toInt should be(2)
}
} finally {
Await.ready(sys1.terminate(), 10.seconds)
}
}
system.log.info("Setup complete")
enterBarrierAfterTestStep()
system.log.info("All setup complete")
}
"stop Replicator if Load fails" in {
runOn(first) {
val r = system.actorOf(
2019-03-11 10:38:24 +01:00
Replicator.props(ReplicatorSettings(system).withDurableStoreProps(testDurableStoreProps(failLoad = true))),
"replicator-" + testStepCounter)
watch(r)
expectTerminated(r)
}
enterBarrierAfterTestStep()
}
"reply with StoreFailure if store fails" in {
runOn(first) {
val r = system.actorOf(
2019-03-11 10:38:24 +01:00
Replicator.props(ReplicatorSettings(system).withDurableStoreProps(testDurableStoreProps(failStore = true))),
"replicator-" + testStepCounter)
r ! Update(KeyA, GCounter(), WriteLocal, request = Some("a"))(_ :+ 1)
expectMsg(StoreFailure(KeyA, Some("a")))
}
enterBarrierAfterTestStep()
}
}