Merge pull request #28483 from helena/failure-joinconfig-checker-sharding
Failed: JoinConfigCompatCheckerClusterShardingSpec #28477
This commit is contained in:
commit
a9750b3c37
1 changed files with 24 additions and 39 deletions
|
|
@ -4,21 +4,19 @@
|
|||
|
||||
package akka.cluster.sharding.typed
|
||||
|
||||
import akka.actor.testkit.typed.scaladsl.ActorTestKit
|
||||
import akka.actor.testkit.typed.scaladsl.LogCapturing
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import scala.collection.{ immutable => im }
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.actor.CoordinatedShutdown
|
||||
import akka.actor.testkit.typed.scaladsl.{ ActorTestKit, LogCapturing, ScalaTestWithActorTestKit }
|
||||
import akka.actor.typed.ActorSystem
|
||||
import akka.cluster.{ Cluster => ClassicCluster }
|
||||
import akka.testkit.LongRunningTest
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
|
||||
import scala.collection.{ immutable => im }
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.Try
|
||||
import org.scalatest.matchers.should.Matchers
|
||||
import org.scalatest.wordspec.AnyWordSpecLike
|
||||
|
||||
object JoinConfig {
|
||||
object JoinConfigCompatCheckerClusterShardingSpec {
|
||||
|
||||
val Shards = 2
|
||||
|
||||
|
|
@ -30,26 +28,34 @@ object JoinConfig {
|
|||
akka.cluster.sharding.state-store-mode = "persistence"
|
||||
akka.cluster.configuration-compatibility-check.enforce-on-join = on
|
||||
akka.cluster.jmx.enabled = off
|
||||
akka.coordinated-shutdown.terminate-actor-system = on
|
||||
akka.remote.classic.netty.tcp.port = 0
|
||||
akka.remote.artery.canonical.port = 0
|
||||
""")
|
||||
|
||||
def clusterConfig: Config =
|
||||
joinConfig(Shards)
|
||||
|
||||
def joinConfig(configured: Int): Config =
|
||||
ConfigFactory.parseString(s"$Key = $configured").withFallback(baseConfig)
|
||||
}
|
||||
|
||||
abstract class JoinConfigCompatCheckerClusterShardingSpec
|
||||
extends ScalaTestWithActorTestKit(JoinConfig.joinConfig(JoinConfig.Shards))
|
||||
class JoinConfigCompatCheckerClusterShardingSpec
|
||||
extends ScalaTestWithActorTestKit(JoinConfigCompatCheckerClusterShardingSpec.clusterConfig)
|
||||
with AnyWordSpecLike
|
||||
with Matchers
|
||||
with LogCapturing {
|
||||
|
||||
protected val clusterWaitDuration = 5.seconds
|
||||
import JoinConfigCompatCheckerClusterShardingSpec._
|
||||
import CoordinatedShutdown.IncompatibleConfigurationDetectedReason
|
||||
|
||||
protected def join(sys: ActorSystem[_]): ClassicCluster = {
|
||||
private val clusterWaitDuration = 5.seconds
|
||||
|
||||
private def configured(system: ActorSystem[_]): Int =
|
||||
system.settings.config.getInt(Key)
|
||||
|
||||
private def join(sys: ActorSystem[_]): ClassicCluster = {
|
||||
if (sys eq system) {
|
||||
configured(system) should ===(JoinConfig.Shards)
|
||||
configured(system) should ===(Shards)
|
||||
val seedNode = ClassicCluster(system)
|
||||
seedNode.join(seedNode.selfAddress)
|
||||
val probe = createTestProbe()
|
||||
|
|
@ -62,39 +68,18 @@ abstract class JoinConfigCompatCheckerClusterShardingSpec
|
|||
}
|
||||
}
|
||||
|
||||
protected def configured(system: ActorSystem[_]): Int =
|
||||
Try(system.settings.config.getInt(JoinConfig.Key)).getOrElse(0)
|
||||
|
||||
}
|
||||
|
||||
class JoinConfigIncompatibilitySpec extends JoinConfigCompatCheckerClusterShardingSpec {
|
||||
"A Joining Node" must {
|
||||
|
||||
s"not be allowed to join a cluster with different '${JoinConfig.Key}'" taggedAs LongRunningTest in {
|
||||
s"not be allowed to join a cluster with different '${Key}'" taggedAs LongRunningTest in {
|
||||
join(system)
|
||||
val joining = ActorTestKit(system.name, JoinConfig.joinConfig(JoinConfig.Shards + 1)) // different
|
||||
val joining = ActorTestKit(system.name, joinConfig(Shards + 1)) // different
|
||||
configured(joining.system) should ===(configured(system) + 1)
|
||||
|
||||
val joiningNode = join(joining.system)
|
||||
val probe = createTestProbe()
|
||||
probe.awaitAssert(joiningNode.readView.isTerminated should ===(true), clusterWaitDuration)
|
||||
joining.shutdownTestKit()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class JoinConfigCompatibilitySpec extends JoinConfigCompatCheckerClusterShardingSpec {
|
||||
"A Joining Node" must {
|
||||
|
||||
s"be allowed to join a cluster with the same '${JoinConfig.Key}'" taggedAs LongRunningTest in {
|
||||
val seedNode = join(system)
|
||||
val joining = ActorTestKit(system.name, JoinConfig.joinConfig(JoinConfig.Shards)) // same
|
||||
val joinConfig = configured(joining.system)
|
||||
configured(system) should ===(joinConfig)
|
||||
join(joining.system)
|
||||
|
||||
val probe = createTestProbe()
|
||||
probe.awaitAssert(seedNode.readView.members.size should ===(2), clusterWaitDuration)
|
||||
CoordinatedShutdown(joining.system).shutdownReason() should ===(Some(IncompatibleConfigurationDetectedReason))
|
||||
|
||||
joining.shutdownTestKit()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue