Add common multi node cluster config to all cluster sharding tests (#25202)

This commit is contained in:
Christopher Batey 2018-06-05 06:58:17 +01:00 committed by GitHub
parent 2a979fe296
commit 01f90ad95d
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
11 changed files with 32 additions and 33 deletions

View file

@ -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)
}

View file

@ -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"]"""))

View file

@ -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"]"""))

View file

@ -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")

View file

@ -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")

View file

@ -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(
"""

View file

@ -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 {

View file

@ -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)

View file

@ -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

View file

@ -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)
}
}
}

View file

@ -15,7 +15,7 @@ object JoinConfigCompatCheckerSpec {
}
class JoinConfigCompatCheckerSpec extends AkkaSpec() with ClusterTestKit {
class JoinConfigCompatCheckerSpec extends AkkaSpec with ClusterTestKit {
"A Joining Node" must {