pekko/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala

237 lines
7 KiB
Scala
Raw Normal View History

/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.cluster.sharding
import scala.concurrent.duration._
import akka.actor._
import akka.cluster.Cluster
import akka.cluster.MemberStatus
import akka.cluster.MultiNodeClusterSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
Disable Java serialization by default, #22333 (#27285) * akka.actor.allow-java-serialization = off * Moved primitive (Long, Int, String, ByteString) serializers from akka-remote to akka-actor since they had no dependency and are useful also in local systems, e.g. persistence. * e.g. needed for persistence-tck * less allow-java-serialization=on in tests * CborSerializable in Jackson/test module for ease of use * JavaSerializable for Java serialization in tests, already in akka-testkit, but misconfigured * Made tests pass * allow-java-serialization=on in akka-persistence * allow-java-serialization=on in classic remoting tests * JavaSerializable and CborSerializable in other remoting tests * Added serialization for * Boolean * java.util.concurrent.TimeoutException, AskTimeoutException * support for testing serialization with the inmem journal * utility to verifySerialization, in SerializationTestKit * remove AccountExampleWithCommandHandlersInState becuase not possible to serialize State when it's not static * Effect() is factory in EventSourcedBehavior class * test the account examples * SharedLeveldbJournal.configToEnableJavaSerializationForTest * support for exceptions from remote deployed child actors * fallback to akka.remote.serialization.ThrowableNotSerializableException if exception is not serializable when wrapped in system messages from remote deployed child actors and Status.Failure messages * it's implemented in `WrappedPayloadSupport.payloadBuilder` * update reference documentation * serialize-messages=off in most places, separate ticket for improving or removing that feature * migration guide, including description of rolling update * fix 2.13 compiler error * minor review feedback
2019-07-11 14:04:24 +02:00
import akka.serialization.jackson.CborSerializable
import akka.testkit.TestDuration
import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
Disable Java serialization by default, #22333 (#27285) * akka.actor.allow-java-serialization = off * Moved primitive (Long, Int, String, ByteString) serializers from akka-remote to akka-actor since they had no dependency and are useful also in local systems, e.g. persistence. * e.g. needed for persistence-tck * less allow-java-serialization=on in tests * CborSerializable in Jackson/test module for ease of use * JavaSerializable for Java serialization in tests, already in akka-testkit, but misconfigured * Made tests pass * allow-java-serialization=on in akka-persistence * allow-java-serialization=on in classic remoting tests * JavaSerializable and CborSerializable in other remoting tests * Added serialization for * Boolean * java.util.concurrent.TimeoutException, AskTimeoutException * support for testing serialization with the inmem journal * utility to verifySerialization, in SerializationTestKit * remove AccountExampleWithCommandHandlersInState becuase not possible to serialize State when it's not static * Effect() is factory in EventSourcedBehavior class * test the account examples * SharedLeveldbJournal.configToEnableJavaSerializationForTest * support for exceptions from remote deployed child actors * fallback to akka.remote.serialization.ThrowableNotSerializableException if exception is not serializable when wrapped in system messages from remote deployed child actors and Status.Failure messages * it's implemented in `WrappedPayloadSupport.payloadBuilder` * update reference documentation * serialize-messages=off in most places, separate ticket for improving or removing that feature * migration guide, including description of rolling update * fix 2.13 compiler error * minor review feedback
2019-07-11 14:04:24 +02:00
object ClusterShardingGetStatsSpec {
Disable Java serialization by default, #22333 (#27285) * akka.actor.allow-java-serialization = off * Moved primitive (Long, Int, String, ByteString) serializers from akka-remote to akka-actor since they had no dependency and are useful also in local systems, e.g. persistence. * e.g. needed for persistence-tck * less allow-java-serialization=on in tests * CborSerializable in Jackson/test module for ease of use * JavaSerializable for Java serialization in tests, already in akka-testkit, but misconfigured * Made tests pass * allow-java-serialization=on in akka-persistence * allow-java-serialization=on in classic remoting tests * JavaSerializable and CborSerializable in other remoting tests * Added serialization for * Boolean * java.util.concurrent.TimeoutException, AskTimeoutException * support for testing serialization with the inmem journal * utility to verifySerialization, in SerializationTestKit * remove AccountExampleWithCommandHandlersInState becuase not possible to serialize State when it's not static * Effect() is factory in EventSourcedBehavior class * test the account examples * SharedLeveldbJournal.configToEnableJavaSerializationForTest * support for exceptions from remote deployed child actors * fallback to akka.remote.serialization.ThrowableNotSerializableException if exception is not serializable when wrapped in system messages from remote deployed child actors and Status.Failure messages * it's implemented in `WrappedPayloadSupport.payloadBuilder` * update reference documentation * serialize-messages=off in most places, separate ticket for improving or removing that feature * migration guide, including description of rolling update * fix 2.13 compiler error * minor review feedback
2019-07-11 14:04:24 +02:00
case object Stop extends CborSerializable
case class Ping(id: Long) extends CborSerializable
case object Pong extends CborSerializable
class ShardedActor extends Actor with ActorLogging {
log.info(s"entity started {}", self.path)
def receive = {
case Stop => context.stop(self)
case _: Ping => sender() ! Pong
}
}
val extractEntityId: ShardRegion.ExtractEntityId = {
case msg @ Ping(id) => (id.toString, msg)
}
val numberOfShards = 3
val extractShardId: ShardRegion.ExtractShardId = {
case Ping(id) => (id % numberOfShards).toString
}
val shardTypeName = "Ping"
}
object ClusterShardingGetStatsSpecConfig extends MultiNodeConfig {
val controller = role("controller")
val first = role("first")
val second = role("second")
val third = role("third")
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.classic.log-remote-lifecycle-events = off
akka.log-dead-letters-during-shutdown = off
akka.cluster.auto-down-unreachable-after = 0s
akka.cluster.sharding {
state-store-mode = "ddata"
updating-state-timeout = 2s
waiting-for-state-timeout = 2s
}
akka.cluster.sharding.distributed-data.durable.lmdb {
dir = target/ClusterShardingGetStatsSpec/sharding-ddata
map-size = 10 MiB
}
""").withFallback(MultiNodeClusterSpec.clusterConfig))
2019-03-11 10:38:24 +01:00
nodeConfig(first, second, third)(ConfigFactory.parseString("""akka.cluster.roles=["shard"]"""))
}
class ClusterShardingGetStatsSpecMultiJvmNode1 extends ClusterShardingGetStatsSpec
class ClusterShardingGetStatsSpecMultiJvmNode2 extends ClusterShardingGetStatsSpec
class ClusterShardingGetStatsSpecMultiJvmNode3 extends ClusterShardingGetStatsSpec
class ClusterShardingGetStatsSpecMultiJvmNode4 extends ClusterShardingGetStatsSpec
2019-03-11 10:38:24 +01:00
abstract class ClusterShardingGetStatsSpec
extends MultiNodeSpec(ClusterShardingGetStatsSpecConfig)
with STMultiNodeSpec {
import ClusterShardingGetStatsSpec._
import ClusterShardingGetStatsSpecConfig._
def initialParticipants = roles.size
def startShard(): ActorRef = {
2019-03-13 10:56:20 +01:00
ClusterSharding(system).start(
typeName = shardTypeName,
entityProps = Props(new ShardedActor),
settings = ClusterShardingSettings(system).withRole("shard"),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
}
def startProxy(): ActorRef = {
2019-03-13 10:56:20 +01:00
ClusterSharding(system).startProxy(
typeName = shardTypeName,
role = Some("shard"),
extractEntityId = extractEntityId,
extractShardId = extractShardId)
}
def join(from: RoleName): Unit = {
runOn(from) {
Cluster(system).join(node(controller).address)
}
enterBarrier(from.name + "-joined")
}
lazy val region = ClusterSharding(system).shardRegion(shardTypeName)
"Inspecting cluster sharding state" must {
"join cluster" in {
join(controller)
join(first)
join(second)
join(third)
// make sure all nodes are up
within(10.seconds) {
awaitAssert {
Cluster(system).state.members.count(_.status == MemberStatus.Up) should ===(4)
}
}
runOn(controller) {
startProxy()
}
runOn(first, second, third) {
startShard()
}
enterBarrier("sharding started")
}
"return empty state when no sharded actors has started" in {
within(10.seconds) {
awaitAssert {
val probe = TestProbe()
region.tell(ShardRegion.GetClusterShardingStats(10.seconds.dilated), probe.ref)
val shardStats = probe.expectMsgType[ShardRegion.ClusterShardingStats]
shardStats.regions.size should ===(3)
shardStats.regions.values.map(_.stats.size).sum should ===(0)
shardStats.regions.keys.forall(_.hasGlobalScope) should ===(true)
shardStats.regions.values.forall(_.failed.isEmpty) shouldBe true
}
}
enterBarrier("empty sharding")
}
"trigger sharded actors" in {
runOn(controller) {
within(10.seconds) {
awaitAssert {
val pingProbe = TestProbe()
// trigger starting of 2 entities on first and second node
// but leave third node without entities
List(1, 2, 4, 6).foreach(n => region.tell(Ping(n), pingProbe.ref))
pingProbe.receiveWhile(messages = 4) {
case Pong => ()
}
}
}
}
enterBarrier("sharded actors started")
}
"get shard stats" in {
within(10.seconds) {
awaitAssert {
val probe = TestProbe()
val region = ClusterSharding(system).shardRegion(shardTypeName)
region.tell(ShardRegion.GetClusterShardingStats(10.seconds.dilated), probe.ref)
val regions = probe.expectMsgType[ShardRegion.ClusterShardingStats].regions
regions.size shouldEqual 3
regions.values.flatMap(_.stats.values).sum shouldEqual 4
regions.values.forall(_.failed.isEmpty) shouldBe true
regions.keys.forall(_.hasGlobalScope) shouldBe true
}
}
enterBarrier("received shard stats")
}
"return stats after a node leaves" in {
runOn(controller) {
Cluster(system).leave(node(third).address)
}
runOn(controller, first, second) {
within(30.seconds) {
awaitAssert {
Cluster(system).state.members.size should ===(3)
}
}
}
enterBarrier("third node removed")
system.log.info("third node removed")
runOn(controller) {
within(10.seconds) {
awaitAssert {
val pingProbe = TestProbe()
// make sure we have the 4 entities still alive across the fewer nodes
List(1, 2, 4, 6).foreach(n => region.tell(Ping(n), pingProbe.ref))
pingProbe.receiveWhile(messages = 4) {
case Pong => ()
}
}
}
}
enterBarrier("shards revived")
runOn(controller) {
within(20.seconds) {
awaitAssert {
val probe = TestProbe()
region.tell(ShardRegion.GetClusterShardingStats(20.seconds.dilated), probe.ref)
val regions = probe.expectMsgType[ShardRegion.ClusterShardingStats].regions
regions.size === 2
regions.values.flatMap(_.stats.values).sum should ===(4)
}
}
}
enterBarrier("done")
}
}
}