2015-04-15 11:07:12 +02:00
|
|
|
/**
|
2018-01-04 17:26:29 +00:00
|
|
|
* Copyright (C) 2015-2018 Lightbend Inc. <https://www.lightbend.com>
|
2015-04-15 11:07:12 +02:00
|
|
|
*/
|
2018-04-24 16:03:55 +01:00
|
|
|
|
2015-04-27 14:48:28 +02:00
|
|
|
package akka.cluster.sharding
|
2015-04-15 11:07:12 +02:00
|
|
|
|
|
|
|
|
import java.io.File
|
|
|
|
|
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import akka.actor.Actor
|
|
|
|
|
import akka.actor.ActorIdentity
|
|
|
|
|
import akka.actor.ActorRef
|
|
|
|
|
import akka.actor.Identify
|
|
|
|
|
import akka.actor.Props
|
2017-12-04 14:23:55 +00:00
|
|
|
import akka.cluster.{ Cluster, MemberStatus, MultiNodeClusterSpec }
|
2015-04-15 11:07:12 +02:00
|
|
|
import akka.persistence.Persistence
|
|
|
|
|
import akka.persistence.journal.leveldb.SharedLeveldbJournal
|
|
|
|
|
import akka.persistence.journal.leveldb.SharedLeveldbStore
|
|
|
|
|
import akka.remote.testconductor.RoleName
|
|
|
|
|
import akka.remote.testkit.MultiNodeConfig
|
|
|
|
|
import akka.remote.testkit.MultiNodeSpec
|
|
|
|
|
import akka.remote.testkit.STMultiNodeSpec
|
|
|
|
|
import akka.testkit._
|
|
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
import org.apache.commons.io.FileUtils
|
|
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
object ClusterShardingLeavingSpec {
|
2015-04-15 11:07:12 +02:00
|
|
|
case class Ping(id: String)
|
|
|
|
|
|
|
|
|
|
class Entity extends Actor {
|
|
|
|
|
def receive = {
|
|
|
|
|
case Ping(_) ⇒ sender() ! self
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case object GetLocations
|
|
|
|
|
case class Locations(locations: Map[String, ActorRef])
|
|
|
|
|
|
|
|
|
|
class ShardLocations extends Actor {
|
|
|
|
|
var locations: Locations = _
|
|
|
|
|
def receive = {
|
|
|
|
|
case GetLocations ⇒ sender() ! locations
|
|
|
|
|
case l: Locations ⇒ locations = l
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-06-09 16:02:19 +02:00
|
|
|
val extractEntityId: ShardRegion.ExtractEntityId = {
|
2015-04-15 11:07:12 +02:00
|
|
|
case m @ Ping(id) ⇒ (id, m)
|
|
|
|
|
}
|
|
|
|
|
|
2015-06-09 16:02:19 +02:00
|
|
|
val extractShardId: ShardRegion.ExtractShardId = {
|
2015-04-15 11:07:12 +02:00
|
|
|
case Ping(id: String) ⇒ id.charAt(0).toString
|
|
|
|
|
}
|
2015-08-20 13:24:39 +03:00
|
|
|
}
|
2015-04-15 11:07:12 +02:00
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
abstract class ClusterShardingLeavingSpecConfig(val mode: String) extends MultiNodeConfig {
|
|
|
|
|
val first = role("first")
|
|
|
|
|
val second = role("second")
|
|
|
|
|
val third = role("third")
|
|
|
|
|
val fourth = role("fourth")
|
|
|
|
|
|
|
|
|
|
commonConfig(ConfigFactory.parseString(s"""
|
|
|
|
|
akka.loglevel = INFO
|
2016-06-10 15:04:13 +02:00
|
|
|
akka.actor.provider = "cluster"
|
2015-08-20 13:24:39 +03:00
|
|
|
akka.remote.log-remote-lifecycle-events = off
|
|
|
|
|
akka.cluster.auto-down-unreachable-after = 0s
|
|
|
|
|
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared"
|
|
|
|
|
akka.persistence.journal.leveldb-shared {
|
|
|
|
|
timeout = 5s
|
|
|
|
|
store {
|
|
|
|
|
native = off
|
2017-01-18 16:28:24 +01:00
|
|
|
dir = "target/ClusterShardingLeavingSpec/journal"
|
2015-08-20 13:24:39 +03:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
akka.persistence.snapshot-store.plugin = "akka.persistence.snapshot-store.local"
|
2017-01-18 16:28:24 +01:00
|
|
|
akka.persistence.snapshot-store.local.dir = "target/ClusterShardingLeavingSpec/snapshots"
|
2015-08-20 13:24:39 +03:00
|
|
|
akka.cluster.sharding.state-store-mode = "$mode"
|
2017-01-18 16:28:24 +01:00
|
|
|
akka.cluster.sharding.distributed-data.durable.lmdb {
|
|
|
|
|
dir = target/ClusterShardingLeavingSpec/sharding-ddata
|
|
|
|
|
map-size = 10 MiB
|
|
|
|
|
}
|
2017-12-04 14:23:55 +00:00
|
|
|
""").withFallback(MultiNodeClusterSpec.clusterConfig))
|
2015-04-15 11:07:12 +02:00
|
|
|
}
|
|
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
object PersistentClusterShardingLeavingSpecConfig extends ClusterShardingLeavingSpecConfig("persistence")
|
|
|
|
|
object DDataClusterShardingLeavingSpecConfig extends ClusterShardingLeavingSpecConfig("ddata")
|
|
|
|
|
|
|
|
|
|
class PersistentClusterShardingLeavingSpec extends ClusterShardingLeavingSpec(PersistentClusterShardingLeavingSpecConfig)
|
|
|
|
|
class DDataClusterShardingLeavingSpec extends ClusterShardingLeavingSpec(DDataClusterShardingLeavingSpecConfig)
|
|
|
|
|
|
|
|
|
|
class PersistentClusterShardingLeavingMultiJvmNode1 extends PersistentClusterShardingLeavingSpec
|
|
|
|
|
class PersistentClusterShardingLeavingMultiJvmNode2 extends PersistentClusterShardingLeavingSpec
|
|
|
|
|
class PersistentClusterShardingLeavingMultiJvmNode3 extends PersistentClusterShardingLeavingSpec
|
|
|
|
|
class PersistentClusterShardingLeavingMultiJvmNode4 extends PersistentClusterShardingLeavingSpec
|
|
|
|
|
|
|
|
|
|
class DDataClusterShardingLeavingMultiJvmNode1 extends DDataClusterShardingLeavingSpec
|
|
|
|
|
class DDataClusterShardingLeavingMultiJvmNode2 extends DDataClusterShardingLeavingSpec
|
|
|
|
|
class DDataClusterShardingLeavingMultiJvmNode3 extends DDataClusterShardingLeavingSpec
|
|
|
|
|
class DDataClusterShardingLeavingMultiJvmNode4 extends DDataClusterShardingLeavingSpec
|
2015-04-15 11:07:12 +02:00
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
abstract class ClusterShardingLeavingSpec(config: ClusterShardingLeavingSpecConfig) extends MultiNodeSpec(config) with STMultiNodeSpec with ImplicitSender {
|
2015-04-15 11:07:12 +02:00
|
|
|
import ClusterShardingLeavingSpec._
|
2015-08-20 13:24:39 +03:00
|
|
|
import config._
|
2015-04-15 11:07:12 +02:00
|
|
|
|
|
|
|
|
override def initialParticipants = roles.size
|
|
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
val storageLocations = List(new File(system.settings.config.getString(
|
|
|
|
|
"akka.cluster.sharding.distributed-data.durable.lmdb.dir")).getParentFile)
|
2015-04-15 11:07:12 +02:00
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
override protected def atStartup(): Unit = {
|
2017-01-18 16:28:24 +01:00
|
|
|
storageLocations.foreach(dir ⇒ if (dir.exists) FileUtils.deleteQuietly(dir))
|
|
|
|
|
enterBarrier("startup")
|
2015-04-15 11:07:12 +02:00
|
|
|
}
|
|
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
override protected def afterTermination(): Unit = {
|
2017-01-18 16:28:24 +01:00
|
|
|
storageLocations.foreach(dir ⇒ if (dir.exists) FileUtils.deleteQuietly(dir))
|
2015-04-15 11:07:12 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val cluster = Cluster(system)
|
|
|
|
|
|
|
|
|
|
def join(from: RoleName, to: RoleName): Unit = {
|
|
|
|
|
runOn(from) {
|
|
|
|
|
cluster join node(to).address
|
|
|
|
|
startSharding()
|
2015-09-09 14:36:03 +02:00
|
|
|
within(15.seconds) {
|
2015-04-15 11:07:12 +02:00
|
|
|
awaitAssert(cluster.state.members.exists { m ⇒
|
|
|
|
|
m.uniqueAddress == cluster.selfUniqueAddress && m.status == MemberStatus.Up
|
|
|
|
|
} should be(true))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
enterBarrier(from.name + "-joined")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def startSharding(): Unit = {
|
|
|
|
|
ClusterSharding(system).start(
|
|
|
|
|
typeName = "Entity",
|
2015-06-09 12:25:58 +02:00
|
|
|
entityProps = Props[Entity],
|
2015-06-07 14:49:38 +02:00
|
|
|
settings = ClusterShardingSettings(system),
|
2015-06-09 16:02:19 +02:00
|
|
|
extractEntityId = extractEntityId,
|
|
|
|
|
extractShardId = extractShardId)
|
2015-04-15 11:07:12 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
lazy val region = ClusterSharding(system).shardRegion("Entity")
|
|
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
def isDdataMode: Boolean = mode == ClusterShardingSettings.StateStoreModeDData
|
|
|
|
|
|
2015-08-20 13:24:39 +03:00
|
|
|
s"Cluster sharding ($mode) with leaving member" must {
|
2015-04-15 11:07:12 +02:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
if (!isDdataMode) {
|
|
|
|
|
"setup shared journal" in {
|
|
|
|
|
// start the Persistence extension
|
|
|
|
|
Persistence(system)
|
|
|
|
|
runOn(first) {
|
|
|
|
|
system.actorOf(Props[SharedLeveldbStore], "store")
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("peristence-started")
|
2015-04-15 11:07:12 +02:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
system.actorSelection(node(first) / "user" / "store") ! Identify(None)
|
|
|
|
|
val sharedStore = expectMsgType[ActorIdentity](10.seconds).ref.get
|
|
|
|
|
SharedLeveldbJournal.setStore(sharedStore, system)
|
2015-04-15 11:07:12 +02:00
|
|
|
|
2017-01-18 16:28:24 +01:00
|
|
|
enterBarrier("after-1")
|
|
|
|
|
}
|
2015-04-15 11:07:12 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"join cluster" in within(20.seconds) {
|
|
|
|
|
join(first, first)
|
|
|
|
|
join(second, first)
|
|
|
|
|
join(third, first)
|
|
|
|
|
join(fourth, first)
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-2")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"initialize shards" in {
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val shardLocations = system.actorOf(Props[ShardLocations], "shardLocations")
|
|
|
|
|
val locations = (for (n ← 1 to 10) yield {
|
|
|
|
|
val id = n.toString
|
|
|
|
|
region ! Ping(id)
|
2016-06-02 14:06:57 +02:00
|
|
|
id → expectMsgType[ActorRef]
|
2015-04-15 11:07:12 +02:00
|
|
|
}).toMap
|
|
|
|
|
shardLocations ! Locations(locations)
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("after-3")
|
|
|
|
|
}
|
|
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
"recover after leaving coordinator node" in {
|
|
|
|
|
system.actorSelection(node(first) / "user" / "shardLocations") ! GetLocations
|
|
|
|
|
val Locations(originalLocations) = expectMsgType[Locations]
|
|
|
|
|
val firstAddress = node(first).address
|
|
|
|
|
|
2015-04-15 11:07:12 +02:00
|
|
|
runOn(third) {
|
|
|
|
|
cluster.leave(node(first).address)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
|
|
|
|
watch(region)
|
2015-06-01 19:03:00 +02:00
|
|
|
expectTerminated(region, 15.seconds)
|
2015-04-15 11:07:12 +02:00
|
|
|
}
|
|
|
|
|
enterBarrier("stopped")
|
|
|
|
|
|
|
|
|
|
runOn(second, third, fourth) {
|
2016-12-01 18:49:38 +01:00
|
|
|
within(15.seconds) {
|
|
|
|
|
awaitAssert {
|
|
|
|
|
val probe = TestProbe()
|
|
|
|
|
originalLocations.foreach {
|
|
|
|
|
case (id, ref) ⇒
|
|
|
|
|
region.tell(Ping(id), probe.ref)
|
|
|
|
|
if (ref.path.address == firstAddress)
|
|
|
|
|
probe.expectMsgType[ActorRef](1.second) should not be (ref)
|
|
|
|
|
else
|
|
|
|
|
probe.expectMsg(1.second, ref) // should not move
|
|
|
|
|
}
|
2015-04-15 11:07:12 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-4")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|