parent
8ba9fda183
commit
6fe2f66adc
21 changed files with 615 additions and 1444 deletions
|
|
@ -4,27 +4,16 @@
|
|||
|
||||
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, MemberStatus, MultiNodeClusterSpec }
|
||||
import akka.persistence.Persistence
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbJournal
|
||||
import akka.persistence.journal.leveldb.SharedLeveldbStore
|
||||
import akka.cluster.sharding.ShardRegion.Passivate
|
||||
import akka.remote.testconductor.RoleName
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.remote.testkit.STMultiNodeSpec
|
||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.serialization.jackson.CborSerializable
|
||||
import akka.testkit._
|
||||
import akka.util.ccompat._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
@ccompatUsedUntil213
|
||||
object ClusterShardingFailureSpec {
|
||||
case class Get(id: String) extends CborSerializable
|
||||
|
|
@ -49,56 +38,36 @@ object ClusterShardingFailureSpec {
|
|||
case Get(id) => id.charAt(0).toString
|
||||
case Add(id, _) => id.charAt(0).toString
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
abstract class ClusterShardingFailureSpecConfig(val mode: String) extends MultiNodeConfig {
|
||||
abstract class ClusterShardingFailureSpecConfig(override val mode: String)
|
||||
extends MultiNodeClusterShardingConfig(
|
||||
mode,
|
||||
additionalConfig = s"""
|
||||
akka.cluster.roles = ["backend"]
|
||||
akka.cluster.sharding {
|
||||
coordinator-failure-backoff = 3s
|
||||
shard-failure-backoff = 3s
|
||||
}
|
||||
akka.persistence.journal.leveldb-shared.store.native = off
|
||||
# using Java serialization for these messages because test is sending them
|
||||
# to other nodes, which isn't normal usage.
|
||||
akka.actor.serialization-bindings {
|
||||
"${classOf[ShardRegion.Passivate].getName}" = java-test
|
||||
}
|
||||
""") {
|
||||
|
||||
val controller = role("controller")
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
|
||||
commonConfig(
|
||||
ConfigFactory
|
||||
.parseString(s"""
|
||||
akka.loglevel = INFO
|
||||
akka.actor.provider = "cluster"
|
||||
akka.remote.classic.log-remote-lifecycle-events = off
|
||||
akka.cluster.downing-provider-class = akka.cluster.testkit.AutoDowning
|
||||
akka.cluster.testkit.auto-down-unreachable-after = 0s
|
||||
akka.cluster.roles = ["backend"]
|
||||
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared"
|
||||
akka.persistence.journal.leveldb-shared {
|
||||
timeout = 5s
|
||||
store {
|
||||
native = off
|
||||
dir = "target/ClusterShardingFailureSpec/journal"
|
||||
}
|
||||
}
|
||||
akka.persistence.snapshot-store.plugin = "akka.persistence.snapshot-store.local"
|
||||
akka.persistence.snapshot-store.local.dir = "target/ClusterShardingFailureSpec/snapshots"
|
||||
akka.cluster.sharding {
|
||||
coordinator-failure-backoff = 3s
|
||||
shard-failure-backoff = 3s
|
||||
state-store-mode = "$mode"
|
||||
}
|
||||
akka.cluster.sharding.distributed-data.durable.lmdb {
|
||||
dir = target/ClusterShardingFailureSpec/sharding-ddata
|
||||
map-size = 10 MiB
|
||||
}
|
||||
# using Java serialization for these messages because test is sending them
|
||||
# to other nodes, which isn't normal usage.
|
||||
akka.actor.serialization-bindings {
|
||||
"${classOf[ShardRegion.Passivate].getName}" = java-test
|
||||
}
|
||||
""")
|
||||
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
|
||||
.withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
testTransport(on = true)
|
||||
}
|
||||
|
||||
object PersistentClusterShardingFailureSpecConfig extends ClusterShardingFailureSpecConfig("persistence")
|
||||
object DDataClusterShardingFailureSpecConfig extends ClusterShardingFailureSpecConfig("ddata")
|
||||
object PersistentClusterShardingFailureSpecConfig
|
||||
extends ClusterShardingFailureSpecConfig(ClusterShardingSettings.StateStoreModePersistence)
|
||||
object DDataClusterShardingFailureSpecConfig
|
||||
extends ClusterShardingFailureSpecConfig(ClusterShardingSettings.StateStoreModeDData)
|
||||
|
||||
class PersistentClusterShardingFailureSpec
|
||||
extends ClusterShardingFailureSpec(PersistentClusterShardingFailureSpecConfig)
|
||||
|
|
@ -112,79 +81,31 @@ class DDataClusterShardingFailureMultiJvmNode1 extends DDataClusterShardingFailu
|
|||
class DDataClusterShardingFailureMultiJvmNode2 extends DDataClusterShardingFailureSpec
|
||||
class DDataClusterShardingFailureMultiJvmNode3 extends DDataClusterShardingFailureSpec
|
||||
|
||||
abstract class ClusterShardingFailureSpec(config: ClusterShardingFailureSpecConfig)
|
||||
extends MultiNodeSpec(config)
|
||||
with STMultiNodeSpec
|
||||
abstract class ClusterShardingFailureSpec(multiNodeConfig: ClusterShardingFailureSpecConfig)
|
||||
extends MultiNodeClusterShardingSpec(multiNodeConfig)
|
||||
with ImplicitSender {
|
||||
import ClusterShardingFailureSpec._
|
||||
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)
|
||||
|
||||
override protected def atStartup(): Unit = {
|
||||
storageLocations.foreach(dir => if (dir.exists) FileUtils.deleteQuietly(dir))
|
||||
enterBarrier("startup")
|
||||
}
|
||||
|
||||
override protected def afterTermination(): Unit = {
|
||||
storageLocations.foreach(dir => if (dir.exists) FileUtils.deleteQuietly(dir))
|
||||
}
|
||||
|
||||
val cluster = Cluster(system)
|
||||
import multiNodeConfig._
|
||||
|
||||
def join(from: RoleName, to: RoleName): Unit = {
|
||||
runOn(from) {
|
||||
cluster.join(node(to).address)
|
||||
startSharding()
|
||||
|
||||
within(remaining) {
|
||||
awaitAssert {
|
||||
cluster.state.members.unsorted.map(_.uniqueAddress) should contain(cluster.selfUniqueAddress)
|
||||
cluster.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
}
|
||||
}
|
||||
}
|
||||
enterBarrier(from.name + "-joined")
|
||||
}
|
||||
|
||||
def startSharding(): Unit = {
|
||||
ClusterSharding(system).start(
|
||||
typeName = "Entity",
|
||||
entityProps = Props[Entity],
|
||||
settings = ClusterShardingSettings(system).withRememberEntities(true),
|
||||
extractEntityId = extractEntityId,
|
||||
extractShardId = extractShardId)
|
||||
join(
|
||||
from,
|
||||
to,
|
||||
startSharding(
|
||||
system,
|
||||
typeName = "Entity",
|
||||
entityProps = Props[Entity],
|
||||
extractEntityId = extractEntityId,
|
||||
extractShardId = extractShardId))
|
||||
}
|
||||
|
||||
lazy val region = ClusterSharding(system).shardRegion("Entity")
|
||||
|
||||
def isDdataMode: Boolean = mode == ClusterShardingSettings.StateStoreModeDData
|
||||
|
||||
s"Cluster sharding ($mode) with flaky journal/network" must {
|
||||
|
||||
if (!isDdataMode) {
|
||||
"setup shared journal" in {
|
||||
// start the Persistence extension
|
||||
Persistence(system)
|
||||
runOn(controller) {
|
||||
system.actorOf(Props[SharedLeveldbStore], "store")
|
||||
}
|
||||
enterBarrier("persistence-started")
|
||||
|
||||
runOn(first, second) {
|
||||
system.actorSelection(node(controller) / "user" / "store") ! Identify(None)
|
||||
val sharedStore = expectMsgType[ActorIdentity](10.seconds).ref.get
|
||||
SharedLeveldbJournal.setStore(sharedStore, system)
|
||||
}
|
||||
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
}
|
||||
|
||||
"join cluster" in within(20.seconds) {
|
||||
startPersistenceIfNotDdataMode(startOn = controller, setStoreOn = Seq(first, second))
|
||||
|
||||
join(first, first)
|
||||
join(second, first)
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue