Add common multi node cluster config to all cluster sharding tests (#25202)
This commit is contained in:
parent
2a979fe296
commit
01f90ad95d
11 changed files with 32 additions and 33 deletions
|
|
@ -5,13 +5,14 @@
|
|||
package akka.cluster.sharding
|
||||
|
||||
import java.io.File
|
||||
|
||||
import akka.cluster.sharding.ShardRegion.Passivate
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import org.apache.commons.io.FileUtils
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor._
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.{ Cluster, MemberStatus, MultiNodeClusterSpec }
|
||||
import akka.persistence.Persistence
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbJournal
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbStore
|
||||
|
|
@ -21,7 +22,6 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.remote.testkit.STMultiNodeSpec
|
||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.testkit._
|
||||
import akka.cluster.MemberStatus
|
||||
|
||||
object ClusterShardingFailureSpec {
|
||||
case class Get(id: String)
|
||||
|
|
@ -79,7 +79,7 @@ abstract class ClusterShardingFailureSpecConfig(val mode: String) extends MultiN
|
|||
dir = target/ClusterShardingFailureSpec/sharding-ddata
|
||||
map-size = 10 MiB
|
||||
}
|
||||
"""))
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
testTransport(on = true)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.cluster.sharding
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.{ Cluster, MultiNodeClusterSpec }
|
||||
import akka.cluster.ClusterEvent.CurrentClusterState
|
||||
import akka.remote.testconductor.RoleName
|
||||
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
|
||||
|
|
@ -59,7 +59,7 @@ object ClusterShardingGetStateSpecConfig extends MultiNodeConfig {
|
|||
dir = target/ClusterShardingGetStateSpec/sharding-ddata
|
||||
map-size = 10 MiB
|
||||
}
|
||||
"""))
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
nodeConfig(first, second)(ConfigFactory.parseString(
|
||||
"""akka.cluster.roles=["shard"]"""))
|
||||
|
|
|
|||
|
|
@ -5,11 +5,11 @@
|
|||
package akka.cluster.sharding
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster.{ MemberStatus, Cluster }
|
||||
import akka.cluster.{ Cluster, MemberStatus, MultiNodeClusterSpec }
|
||||
import akka.cluster.ClusterEvent.CurrentClusterState
|
||||
import akka.remote.testconductor.RoleName
|
||||
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
|
||||
import akka.testkit.{ TestProbe, TestDuration }
|
||||
import akka.testkit.{ TestDuration, TestProbe }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
|
@ -62,7 +62,7 @@ object ClusterShardingGetStatsSpecConfig extends MultiNodeConfig {
|
|||
map-size = 10 MiB
|
||||
}
|
||||
akka.actor.warn-about-java-serializer-usage=false
|
||||
"""))
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
nodeConfig(first, second, third)(ConfigFactory.parseString(
|
||||
"""akka.cluster.roles=["shard"]"""))
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import scala.concurrent.duration._
|
|||
import java.io.File
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.{ Cluster, MultiNodeClusterSpec }
|
||||
import akka.cluster.sharding.ShardRegion.GracefulShutdown
|
||||
import akka.persistence.Persistence
|
||||
import akka.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbStore }
|
||||
|
|
@ -64,7 +64,7 @@ abstract class ClusterShardingGracefulShutdownSpecConfig(val mode: String) exten
|
|||
dir = target/ClusterShardingGracefulShutdownSpec/sharding-ddata
|
||||
map-size = 10 MiB
|
||||
}
|
||||
"""))
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
object PersistentClusterShardingGracefulShutdownSpecConfig extends ClusterShardingGracefulShutdownSpecConfig("persistence")
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import scala.concurrent.duration._
|
|||
import java.io.File
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.{ Cluster, MemberStatus, MultiNodeClusterSpec }
|
||||
import akka.cluster.sharding.ShardRegion.GracefulShutdown
|
||||
import akka.persistence.Persistence
|
||||
import akka.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbStore }
|
||||
|
|
@ -21,7 +21,6 @@ import org.apache.commons.io.FileUtils
|
|||
import scala.concurrent.duration._
|
||||
import akka.cluster.sharding.ShardRegion.GetClusterShardingStats
|
||||
import akka.cluster.sharding.ShardRegion.ClusterShardingStats
|
||||
import akka.cluster.MemberStatus
|
||||
|
||||
object ClusterShardingMinMembersSpec {
|
||||
case object StopEntity
|
||||
|
|
@ -62,7 +61,7 @@ abstract class ClusterShardingMinMembersSpecConfig(val mode: String) extends Mul
|
|||
map-size = 10 MiB
|
||||
}
|
||||
akka.cluster.min-nr-of-members = 3
|
||||
"""))
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
}
|
||||
|
||||
object PersistentClusterShardingMinMembersSpecConfig extends ClusterShardingMinMembersSpecConfig("persistence")
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.cluster.sharding
|
|||
import java.io.File
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster.{ Cluster, MemberStatus }
|
||||
import akka.cluster.{ Cluster, MemberStatus, MultiNodeClusterSpec }
|
||||
import akka.persistence.Persistence
|
||||
import akka.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbStore }
|
||||
import akka.remote.testconductor.RoleName
|
||||
|
|
@ -76,7 +76,7 @@ abstract class ClusterShardingRememberEntitiesNewExtractorSpecConfig(val mode: S
|
|||
dir = target/ShardingRememberEntitiesNewExtractorSpec/sharding-ddata
|
||||
map-size = 10 MiB
|
||||
}
|
||||
"""))
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
val roleConfig = ConfigFactory.parseString(
|
||||
"""
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import scala.concurrent.duration._
|
|||
import java.io.File
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.{ Cluster, MemberStatus, MultiNodeClusterSpec }
|
||||
import akka.cluster.sharding.ShardRegion.GracefulShutdown
|
||||
import akka.persistence.Persistence
|
||||
import akka.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbStore }
|
||||
|
|
@ -21,7 +21,6 @@ import org.apache.commons.io.FileUtils
|
|||
import scala.concurrent.duration._
|
||||
import akka.cluster.sharding.ShardRegion.GetClusterShardingStats
|
||||
import akka.cluster.sharding.ShardRegion.ClusterShardingStats
|
||||
import akka.cluster.MemberStatus
|
||||
|
||||
object ClusterShardingRememberEntitiesSpec {
|
||||
|
||||
|
|
@ -73,7 +72,7 @@ abstract class ClusterShardingRememberEntitiesSpecConfig(val mode: String) exten
|
|||
dir = target/ShardingRememberEntitiesSpec/sharding-ddata
|
||||
map-size = 10 MiB
|
||||
}
|
||||
"""))
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
nodeConfig(third)(ConfigFactory.parseString(s"""
|
||||
akka.cluster.sharding.distributed-data.durable.lmdb {
|
||||
|
|
|
|||
|
|
@ -4,16 +4,17 @@
|
|||
|
||||
package akka.cluster.sharding
|
||||
|
||||
import akka.cluster.ddata.{ ReplicatorSettings, Replicator }
|
||||
import akka.cluster.sharding.ShardCoordinator.Internal.{ ShardStopped, HandOff }
|
||||
import akka.cluster.ddata.{ Replicator, ReplicatorSettings }
|
||||
import akka.cluster.sharding.ShardCoordinator.Internal.{ HandOff, ShardStopped }
|
||||
import akka.cluster.sharding.ShardRegion.Passivate
|
||||
import akka.cluster.sharding.ShardRegion.GetCurrentRegions
|
||||
import akka.cluster.sharding.ShardRegion.CurrentRegions
|
||||
|
||||
import language.postfixOps
|
||||
import scala.concurrent.duration._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor._
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.{ Cluster, MultiNodeClusterSpec }
|
||||
import akka.persistence.PersistentActor
|
||||
import akka.persistence.Persistence
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbJournal
|
||||
|
|
@ -25,6 +26,7 @@ import akka.remote.testkit.STMultiNodeSpec
|
|||
import akka.testkit._
|
||||
import akka.testkit.TestEvent.Mute
|
||||
import java.io.File
|
||||
|
||||
import org.apache.commons.io.FileUtils
|
||||
import akka.cluster.singleton.ClusterSingletonManager
|
||||
import akka.cluster.singleton.ClusterSingletonManagerSettings
|
||||
|
|
@ -165,7 +167,7 @@ abstract class ClusterShardingSpecConfig(
|
|||
}
|
||||
}
|
||||
akka.testconductor.barrier-timeout = 70s
|
||||
"""))
|
||||
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
nodeConfig(sixth) {
|
||||
ConfigFactory.parseString("""akka.cluster.roles = ["frontend"]""")
|
||||
}
|
||||
|
|
@ -252,12 +254,11 @@ class DDataClusterShardingWithEntityRecoveryMultiJvmNode5 extends DDataClusterSh
|
|||
class DDataClusterShardingWithEntityRecoveryMultiJvmNode6 extends DDataClusterShardingSpec
|
||||
class DDataClusterShardingWithEntityRecoveryMultiJvmNode7 extends DDataClusterShardingSpec
|
||||
|
||||
abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends MultiNodeSpec(config) with STMultiNodeSpec with ImplicitSender {
|
||||
abstract class ClusterShardingSpec(config: ClusterShardingSpecConfig) extends MultiNodeSpec(config) with MultiNodeClusterSpec
|
||||
with STMultiNodeSpec with ImplicitSender {
|
||||
import ClusterShardingSpec._
|
||||
import config._
|
||||
|
||||
override def initialParticipants = roles.size
|
||||
|
||||
val storageLocations = List(new File(system.settings.config.getString(
|
||||
"akka.cluster.sharding.distributed-data.durable.lmdb.dir")).getParentFile)
|
||||
|
||||
|
|
|
|||
|
|
@ -47,8 +47,11 @@ object MultiNodeClusterSpec {
|
|||
periodic-tasks-initial-delay = 300 ms
|
||||
publish-stats-interval = 0 s # always, when it happens
|
||||
failure-detector.heartbeat-interval = 500 ms
|
||||
|
||||
run-coordinated-shutdown-when-down = off
|
||||
|
||||
sharding {
|
||||
retry-interval = 200ms
|
||||
}
|
||||
}
|
||||
akka.loglevel = INFO
|
||||
akka.log-dead-letters = off
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ trait ClusterTestKit extends TestKitBase {
|
|||
/**
|
||||
* Register an [[ActorSystem]].
|
||||
*
|
||||
* The [ActorSystem]] will be prepended to list and be considered the first node
|
||||
* The [[ActorSystem]] will be prepended to list and be considered the first node
|
||||
*/
|
||||
def registerAsFirst(actorSystem: ActorSystem) = {
|
||||
actorSystems = actorSystem +: actorSystems
|
||||
|
|
@ -102,7 +102,7 @@ trait ClusterTestKit extends TestKitBase {
|
|||
* Force the passed [[ActorSystem]] to quit the cluster and shutdown.
|
||||
* Once original system is removed, a new [[ActorSystem]] is started using the same address.
|
||||
*/
|
||||
def quitAndRestart(actorSystem: ActorSystem, config: Config) = {
|
||||
def quitAndRestart(actorSystem: ActorSystem, config: Config): ActorSystem = {
|
||||
require(isRegistered(actorSystem), "Unknown actor system")
|
||||
|
||||
// is this first seed node?
|
||||
|
|
@ -113,7 +113,7 @@ trait ClusterTestKit extends TestKitBase {
|
|||
|
||||
// remove old before starting the new one
|
||||
cluster.leave(cluster.readView.selfAddress)
|
||||
awaitCond(cluster.readView.status == Removed, message = s"awaiting node [${cluster.readView.selfAddress}] to be 'Removed'")
|
||||
awaitCond(cluster.readView.status == Removed, message = s"awaiting node [${cluster.readView.selfAddress}] to be 'Removed'. Current status: [${cluster.readView.status}]")
|
||||
|
||||
shutdown(actorSystem)
|
||||
awaitCond(cluster.isTerminated)
|
||||
|
|
@ -130,9 +130,6 @@ trait ClusterTestKit extends TestKitBase {
|
|||
|
||||
if (firstSeedNode) newActorSystemAsFirst(newConfig)
|
||||
else newActorSystem(newConfig)
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ object JoinConfigCompatCheckerSpec {
|
|||
|
||||
}
|
||||
|
||||
class JoinConfigCompatCheckerSpec extends AkkaSpec() with ClusterTestKit {
|
||||
class JoinConfigCompatCheckerSpec extends AkkaSpec with ClusterTestKit {
|
||||
|
||||
"A Joining Node" must {
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue